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 2015/12/11 22:22:35 UTC

[01/50] [abbrv] incubator-geode git commit: GEODE-616: failure detection ports not set in new membership view

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-217 3733ae71b -> 0e62bc04b


GEODE-616: failure detection ports not set in new membership view

Added a method to NetView to copy failure detection ports from an old
view to the new view.  I also added a JUnit test for NetView to make
sure we have adequate code-coverage for this class.


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

Branch: refs/heads/feature/GEODE-217
Commit: e4448adc3ce1c5774410194dcbd361eb21890e7d
Parents: 2803a10
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Dec 1 08:33:02 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 1 08:33:02 2015 -0800

----------------------------------------------------------------------
 .../internal/membership/NetView.java            |  46 ++++-
 .../membership/gms/membership/GMSJoinLeave.java |  25 +--
 .../internal/membership/NetViewJUnitTest.java   | 194 +++++++++++++++++++
 3 files changed, 241 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e4448adc/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetView.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetView.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetView.java
index 8800d9d..a90a45d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetView.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetView.java
@@ -88,6 +88,12 @@ public class NetView implements DataSerializableFixedID {
     Arrays.fill(failureDetectionPorts, -1);
   }
 
+  /**
+   * Create a new view with the contents of the given view and the
+   * specified view ID
+   * @param other
+   * @param viewId
+   */
   public NetView(NetView other, int viewId) {
     this.creator = other.creator;
     this.viewId = viewId;
@@ -129,7 +135,7 @@ public class NetView implements DataSerializableFixedID {
   
   public int getFailureDetectionPort(InternalDistributedMember mbr) {
     int idx = members.indexOf(mbr);
-    if (idx < 0 || failureDetectionPorts == null || idx >= failureDetectionPorts.length) {
+    if (idx < 0 || idx >= failureDetectionPorts.length) {
       return -1;
     }
     return failureDetectionPorts[idx];
@@ -143,15 +149,36 @@ public class NetView implements DataSerializableFixedID {
     ensureFDCapacity(idx);
     failureDetectionPorts[idx] = port;
   }
+  
+  /**
+   * Transfer the failure-detection ports from another view to this one
+   * @param otherView
+   */
+  public void setFailureDetectionPorts(NetView otherView) {
+    int[] ports = otherView.getFailureDetectionPorts();
+    if (ports != null) {
+      int idx = 0;
+      int portsSize = ports.length;
+      for (InternalDistributedMember mbr: otherView.getMembers()) {
+        if (contains(mbr)) {
+          // unit tests create views w/o failure detection ports, so we must check the length
+          // of the array
+          if (idx < portsSize) {
+            setFailureDetectionPort(mbr, ports[idx]);
+          } else {
+            setFailureDetectionPort(mbr, -1);
+          }
+        }
+        idx += 1;
+      }
+    }
+  }
 
   /**
    * ensures that there is a slot at idx to store an int
    */
   private void ensureFDCapacity(int idx) {
-    if (failureDetectionPorts == null) {
-      failureDetectionPorts = new int[idx+10];
-      Arrays.fill(failureDetectionPorts, -1);
-    } else if (idx >= failureDetectionPorts.length) {
+    if (idx >= failureDetectionPorts.length) {
       int[] p = new int[idx+10];
       if (failureDetectionPorts.length > 0) {
         System.arraycopy(failureDetectionPorts, 0, p, 0, failureDetectionPorts.length);
@@ -479,6 +506,15 @@ public class NetView implements DataSerializableFixedID {
         first = false;
       }
     }
+//    sb.append("] fd ports: [");
+//    int[] ports = getFailureDetectionPorts();
+//    int numMembers = size();
+//    for (int i=0; i<numMembers; i++) {
+//      if (i > 0) {
+//        sb.append(' ');
+//      }
+//      sb.append(ports[i]);
+//    }
     sb.append("]");
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e4448adc/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
index 1c7b601..2f8d734 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
@@ -531,7 +531,6 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
       NetView check = new NetView(v, v.getViewId() + 1);
       synchronized (removedMembers) {
         removedMembers.add(mbr);
-        check = new NetView(v, v.getViewId());
         check.addCrashedMembers(removedMembers);
         check.removeAll(removedMembers);
       }
@@ -660,6 +659,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
         mbrs.removeAll(leaving);
         newView = new NetView(this.localAddress, viewNumber, mbrs, leaving,
             removals);
+        newView.setFailureDetectionPorts(currentView);
         newView.setFailureDetectionPort(this.localAddress, services.getHealthMonitor().getFailureDetectionPort());
       }
       if (viewCreator == null || viewCreator.isShutdown()) {
@@ -1890,23 +1890,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
           }
         }
         if (currentView != null) {
-          int[] ports = currentView.getFailureDetectionPorts();
-          if (ports != null) {
-            int idx = 0;
-            int portsSize = ports.length;
-            for (InternalDistributedMember mbr: currentView.getMembers()) {
-              if (newView.contains(mbr)) {
-                // unit tests create views w/o failure detection ports, so we must check the length
-                // of the array
-                if (idx < portsSize) {
-                  newView.setFailureDetectionPort(mbr, ports[idx]);
-                } else {
-                  newView.setFailureDetectionPort(mbr, -1);
-                }
-              }
-              idx += 1;
-            }
-          }
+          newView.setFailureDetectionPorts(currentView);
         }
       }
 
@@ -1995,12 +1979,15 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
             logger.info("adding these new members from a conflicting view to the new view: {}", newMembers);
             for (InternalDistributedMember mbr: newMembers) {
               int port = conflictingView.getFailureDetectionPort(mbr);
-              JoinRequestMessage msg = new JoinRequestMessage(localAddress, mbr, null, port);
               newView.add(mbr);
               newView.setFailureDetectionPort(mbr, port);
               joinReqs.add(mbr);
             }
           }
+          // trump the view ID of the conflicting view so mine will be accepted
+          if (conflictingView.getViewId() >= newView.getViewId()) {
+            newView = new NetView(newView, conflictingView.getViewId()+1);
+          }
         }
 
         if (!unresponsive.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e4448adc/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
new file mode 100755
index 0000000..603c7bf
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
@@ -0,0 +1,194 @@
+package com.gemstone.gemfire.distributed.internal.membership;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.Timer;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.distributed.internal.membership.NetView;
+import com.gemstone.gemfire.internal.SocketCreator;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class NetViewJUnitTest {
+  List<InternalDistributedMember> members;
+  
+  @Before
+  public void initMembers() throws Exception {
+    int numMembers = 10;
+    members = new ArrayList<>(numMembers);
+    for (int i=0; i<numMembers; i++) {
+      members.add(new InternalDistributedMember(SocketCreator.getLocalHost(), 1000+i));
+    }
+    // view creator is a locator
+    members.get(0).setVmKind(DistributionManager.LOCATOR_DM_TYPE);
+    members.get(0).setVmViewId(0);
+    members.get(0).getNetMember().setPreferredForCoordinator(true);
+    
+    // members who joined in view #1
+    for (int i=1; i<(numMembers-1); i++) {
+      members.get(i).setVmViewId(1);
+      members.get(i).setVmKind(DistributionManager.NORMAL_DM_TYPE);
+      members.get(i).getNetMember().setPreferredForCoordinator(false);
+    }
+
+    // member joining in this view
+    members.get(numMembers-1).setVmViewId(2);
+    members.get(numMembers-1).setVmKind(DistributionManager.NORMAL_DM_TYPE);
+  }
+  
+  private void setFailureDetectionPorts(NetView view) {
+    int numMembers = members.size();
+    // use the membership port as the FD port so it's easy to figure out problems
+    for (int i=0; i<numMembers; i++) {
+      view.setFailureDetectionPort(members.get(i), members.get(i).getPort());
+    }
+  }
+  
+  @Test
+  public void testCreateView() throws Exception {
+    int numMembers = members.size();
+    NetView view = new NetView(members.get(0), 2, members, Collections.emptySet(), Collections.emptySet());
+    setFailureDetectionPorts(view);
+    
+    assertTrue(view.getCreator().equals(members.get(0)));
+    assertEquals(2, view.getViewId());
+    assertEquals(members, view.getMembers());
+    assertEquals(0, view.getCrashedMembers().size());
+    assertEquals(members.get(1), view.getLeadMember()); // a locator can't be lead member
+    assertEquals(0, view.getShutdownMembers().size());
+    assertEquals(1, view.getNewMembers().size());
+    assertEquals(members.get(numMembers-1), view.getNewMembers().iterator().next());
+    assertEquals(members.get(0), view.getCoordinator());
+    
+    for (int i=0; i<numMembers; i++) {
+      InternalDistributedMember mbr = members.get(i);
+      assertEquals(mbr.getPort(), view.getFailureDetectionPort(mbr));
+    }
+    
+    assertFalse(view.shouldBeCoordinator(members.get(1)));
+    assertTrue(view.shouldBeCoordinator(members.get(0)));
+    assertEquals(members.get(numMembers-1), view.getCoordinator(Collections.singletonList(members.get(0))));
+    members.get(numMembers-1).getNetMember().setPreferredForCoordinator(false);
+    assertEquals(members.get(1), view.getCoordinator(Collections.singletonList(members.get(0))));
+    
+    members.get(numMembers-1).getNetMember().setPreferredForCoordinator(true);
+    List<InternalDistributedMember> preferred = view.getPreferredCoordinators(Collections.<InternalDistributedMember>singleton(members.get(1)), members.get(0), 2);
+    assertEquals(3, preferred.size());
+    assertEquals(members.get(numMembers-1), preferred.get(0));
+  }
+  
+  @Test
+  public void testRemoveMembers() throws Exception {
+    int numMembers = members.size();
+    NetView view = new NetView(members.get(0), 2, new ArrayList<>(members), Collections.emptySet(),
+        Collections.emptySet());
+    setFailureDetectionPorts(view);
+
+    for (int i=1; i<numMembers; i+=2) {
+      view.remove(members.get(i));
+      assertFalse(view.contains(members.get(i)));
+    }
+    
+    List<InternalDistributedMember> remainingMembers = view.getMembers();
+    int num = remainingMembers.size();
+    for (int i=0; i<num; i++) {
+      InternalDistributedMember mbr = remainingMembers.get(i);
+      assertEquals(mbr.getPort(), view.getFailureDetectionPort(mbr));
+    }
+  }
+  
+  @Test
+  public void testRemoveAll() throws Exception {
+    int numMembers = members.size();
+    NetView view = new NetView(members.get(0), 2, new ArrayList<>(members), Collections.emptySet(),
+        Collections.emptySet());
+    setFailureDetectionPorts(view);
+
+    Collection<InternalDistributedMember> removals = new ArrayList<>(numMembers/2);
+    for (int i=1; i<numMembers; i+=2) {
+      removals.add(members.get(i));
+    }
+    
+    view.removeAll(removals);
+    for (InternalDistributedMember mbr: removals) {
+      assertFalse(view.contains(mbr));
+    }
+    assertEquals(numMembers-removals.size(), view.size());
+
+    List<InternalDistributedMember> remainingMembers = view.getMembers();
+    int num = remainingMembers.size();
+    for (int i=0; i<num; i++) {
+      InternalDistributedMember mbr = remainingMembers.get(i);
+      assertEquals(mbr.getPort(), view.getFailureDetectionPort(mbr));
+    }
+  }
+  
+  @Test
+  public void testCopyView() throws Exception {
+    NetView view = new NetView(members.get(0), 2, new ArrayList<>(members), Collections.emptySet(),
+        Collections.emptySet());
+    setFailureDetectionPorts(view);
+
+    NetView newView = new NetView(view, 3);
+
+    assertTrue(newView.getCreator().equals(members.get(0)));
+    assertEquals(3, newView.getViewId());
+    assertEquals(members, newView.getMembers());
+    assertEquals(0, newView.getCrashedMembers().size());
+    assertEquals(members.get(1), newView.getLeadMember()); // a locator can't be lead member
+    assertEquals(0, newView.getShutdownMembers().size());
+    assertEquals(0, newView.getNewMembers().size());
+    assertTrue(newView.equals(view));
+    assertTrue(view.equals(newView));
+    newView.remove(members.get(1));
+    assertFalse(newView.equals(view));
+  }
+  
+  @Test
+  public void testAddLotsOfMembers() throws Exception {
+    NetView view = new NetView(members.get(0), 2, new ArrayList<>(members), Collections.emptySet(),
+        Collections.emptySet());
+    setFailureDetectionPorts(view);
+    
+    NetView copy = new NetView(view, 2);
+
+    int oldSize = view.size();
+    for (int i=0; i<100; i++) {
+      InternalDistributedMember mbr = new InternalDistributedMember(SocketCreator.getLocalHost(), 2000+i);
+      mbr.setVmKind(DistributionManager.NORMAL_DM_TYPE);
+      mbr.setVmViewId(2);
+      view.add(mbr);
+      view.setFailureDetectionPort(mbr, 2000+i);
+    }
+    
+    assertEquals(oldSize+100, view.size());
+    for (InternalDistributedMember mbr: view.getMembers()) {
+      assertEquals(mbr.getPort(), view.getFailureDetectionPort(mbr));
+    }
+    
+    assertEquals(100, view.getNewMembers(copy).size());
+  }
+  
+}


[29/50] [abbrv] incubator-geode git commit: GEODE-563: Moving gfsh tests from closed

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
new file mode 100644
index 0000000..54aed63
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
@@ -0,0 +1,385 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.SharedConfiguration;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.ClassBuilder;
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * A distributed test suite of test cases for testing the queue commands that are part of Gfsh.
+ *
+ * @author David Hoots
+ * @since 8.0
+ */
+public class QueueCommandsDUnitTest extends CliCommandTestBase {
+  private static final long serialVersionUID = 1L;
+
+  final List<String> filesToBeDeleted = new CopyOnWriteArrayList<String>();
+
+  public QueueCommandsDUnitTest(final String testName) {
+    super(testName);
+  }
+
+  public void testAsyncEventQueue() throws IOException {
+    final String queue1Name = "testAsyncEventQueue1";
+    final String queue2Name = "testAsyncEventQueue2";
+    final String diskStoreName = "testAsyncEventQueueDiskStore";
+
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group0");
+    createDefaultSetup(localProps);
+
+    CommandResult cmdResult = executeCommand(CliStrings.LIST_ASYNC_EVENT_QUEUES);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertTrue(commandResultToString(cmdResult).contains("No Async Event Queues Found"));
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1Name = "VM" + vm1.getPid();
+    final File diskStoreDir = new File(new File(".").getAbsolutePath(), diskStoreName);
+    this.filesToBeDeleted.add(diskStoreDir.getAbsolutePath());
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        diskStoreDir.mkdirs();
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm1Name);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+        getSystem(localProps);
+        getCache();
+      }
+    });
+
+    final VM vm2 = Host.getHost(0).getVM(2);
+    final String vm2Name = "VM" + vm2.getPid();
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm2Name);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(localProps);
+        getCache();
+      }
+    });
+
+    // Deploy a JAR file with an AsyncEventListener/GatewayEventFilter/GatewayEventSubstitutionFilter
+    // that can be instantiated on each server
+    final File jarFile = new File(new File(".").getAbsolutePath(), "QueueCommandsDUnit.jar");
+    QueueCommandsDUnitTest.this.filesToBeDeleted.add(jarFile.getAbsolutePath());
+
+    ClassBuilder classBuilder = new ClassBuilder();
+    byte[] jarBytes = classBuilder.createJarFromClassContent("com/qcdunit/QueueCommandsDUnitTestHelper",
+        "package com.qcdunit;" +
+            "import java.util.List; import java.util.Properties;" +
+            "import com.gemstone.gemfire.internal.cache.xmlcache.Declarable2; import com.gemstone.gemfire.cache.asyncqueue.AsyncEvent;" +
+            "import com.gemstone.gemfire.cache.wan.GatewayEventFilter; import com.gemstone.gemfire.cache.wan.GatewayEventSubstitutionFilter;" +
+            "import com.gemstone.gemfire.cache.asyncqueue.AsyncEventListener; import com.gemstone.gemfire.cache.wan.GatewayQueueEvent;" +
+            "import com.gemstone.gemfire.cache.EntryEvent;" +
+            "public class QueueCommandsDUnitTestHelper implements Declarable2, GatewayEventFilter, GatewayEventSubstitutionFilter, AsyncEventListener {" +
+            "Properties props;" +
+            "public boolean processEvents(List<AsyncEvent> events) { return true; }" +
+            "public void afterAcknowledgement(GatewayQueueEvent event) {}" +
+            "public boolean beforeEnqueue(GatewayQueueEvent event) { return true; }" +
+            "public boolean beforeTransmit(GatewayQueueEvent event) { return true; }" +
+            "public Object getSubstituteValue(EntryEvent event) { return null; }" +
+            "public void close() {}" +
+            "public void init(final Properties props) {this.props = props;}" +
+            "public Properties getConfig() {return this.props;}}");
+    writeJarBytesToFile(jarFile, jarBytes);
+
+    cmdResult = executeCommand("deploy --jar=QueueCommandsDUnit.jar");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_DISK_STORE);
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__NAME, diskStoreName);
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__GROUP, "Group1");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE, diskStoreDir.getAbsolutePath());
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    String stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*Success"));
+
+    commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_ASYNC_EVENT_QUEUE);
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ID, queue1Name);
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__GROUP, "Group1");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCH_SIZE, "514");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT, "true");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISK_STORE, diskStoreName);
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY, "213");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCHTIMEINTERVAL, "946");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PARALLEL, "true");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ENABLEBATCHCONFLATION, "true");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS, "2");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY, "PARTITION");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER,
+        "com.qcdunit.QueueCommandsDUnitTestHelper");
+    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__LISTENER,
+        "com.qcdunit.QueueCommandsDUnitTestHelper");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE, "param1");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE, "param2#value2");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*Success"));
+
+    // Verify that the queue was created on the correct member
+    cmdResult = executeCommand(CliStrings.LIST_ASYNC_EVENT_QUEUES);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult,
+        vm1Name + " .*" + queue1Name + " .*514 .*true .*" + diskStoreName + " .*213 .*" + " .*com.qcdunit.QueueCommandsDUnitTestHelper" + ".*"));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*param2=value2.*"));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*param1=[^\\w].*"));
+    assertFalse(stringContainsLine(stringResult, vm2Name + ".*" + queue1Name + ".*"));
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+        AsyncEventQueue queue = cache.getAsyncEventQueue(queue1Name);
+        assertEquals(queue.getBatchSize(), 514);
+        assertEquals(queue.isPersistent(), true);
+        assertEquals(queue.getDiskStoreName(), diskStoreName);
+        assertEquals(queue.getMaximumQueueMemory(), 213);
+        assertEquals(queue.getBatchTimeInterval(), 946);
+        assertEquals(queue.isParallel(), true);
+        assertEquals(queue.isBatchConflationEnabled(), true);
+        assertEquals(queue.getDispatcherThreads(), 2);
+        assertEquals(queue.getOrderPolicy().toString(), "PARTITION");
+        assertEquals(queue.getGatewayEventFilters().size(), 1);
+        assertEquals(queue.getGatewayEventFilters().get(0).getClass().getName(),
+            "com.qcdunit.QueueCommandsDUnitTestHelper");
+        assertEquals(queue.getGatewayEventSubstitutionFilter().getClass().getName(),
+            "com.qcdunit.QueueCommandsDUnitTestHelper");
+        assertEquals(queue.isDiskSynchronous(), false);
+        assertEquals(queue.getAsyncEventListener().getClass().getName(), "com.qcdunit.QueueCommandsDUnitTestHelper");
+      }
+    });
+
+    commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_ASYNC_EVENT_QUEUE);
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ID, queue2Name);
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER,
+        "com.qcdunit.QueueCommandsDUnitTestHelper");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(5, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Manager.*Success"));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*Success"));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*Success"));
+
+    // Verify that the queue was created on the correct members
+    cmdResult = executeCommand(CliStrings.LIST_ASYNC_EVENT_QUEUES);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(6, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult,
+        "Manager .*" + queue2Name + " .*100 .*false .*null .*100 .*" + " .*com.qcdunit.QueueCommandsDUnitTestHelper"));
+    assertTrue(stringContainsLine(stringResult,
+        vm1Name + " .*" + queue1Name + " .*514 .*true .*" + diskStoreName + " .*213 .*" + " .*com.qcdunit.QueueCommandsDUnitTestHelper" + ".*"));
+    assertTrue(stringContainsLine(stringResult,
+        vm1Name + " .*" + queue2Name + " .*100 .*false .*null .*100 .*" + " .*com.qcdunit.QueueCommandsDUnitTestHelper"));
+    assertTrue(stringContainsLine(stringResult,
+        vm2Name + " .*" + queue2Name + " .*100 .*false .*null .*100 .*" + " .*com.qcdunit.QueueCommandsDUnitTestHelper"));
+  }
+
+  /**
+   * Asserts that creating async event queues correctly updates the shared configuration.
+   */
+  public void testCreateUpdatesSharedConfig() throws IOException {
+    disconnectAllFromDS();
+
+    final String queueName = "testAsyncEventQueueQueue";
+    final String groupName = "testAsyncEventQueueSharedConfigGroup";
+
+    // Start the Locator and wait for shared configuration to be available
+    final int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+
+        final File locatorLogFile = new File("locator-" + locatorPort + ".log");
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, "Locator");
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "fine");
+        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
+              locatorProps);
+
+          DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+            @Override
+            public boolean done() {
+              return locator.isSharedConfigurationRunning();
+            }
+
+            @Override
+            public String description() {
+              return "Waiting for shared configuration to be started";
+            }
+          };
+          DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+        } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        }
+      }
+    });
+
+    // Start the default manager
+    Properties managerProps = new Properties();
+    managerProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    managerProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+    createDefaultSetup(managerProps);
+
+    // Create a cache in VM 1
+    VM vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        getSystem(localProps);
+        assertNotNull(getCache());
+      }
+    });
+
+    // Deploy a JAR file with an AsyncEventListener that can be instantiated on each server
+    final File jarFile = new File(new File(".").getAbsolutePath(), "QueueCommandsDUnit.jar");
+    QueueCommandsDUnitTest.this.filesToBeDeleted.add(jarFile.getAbsolutePath());
+
+    ClassBuilder classBuilder = new ClassBuilder();
+    byte[] jarBytes = classBuilder.createJarFromClassContent("com/qcdunit/QueueCommandsDUnitTestListener",
+        "package com.qcdunit;" +
+            "import java.util.List; import java.util.Properties;" +
+            "import com.gemstone.gemfire.internal.cache.xmlcache.Declarable2; import com.gemstone.gemfire.cache.asyncqueue.AsyncEvent;" +
+            "import com.gemstone.gemfire.cache.asyncqueue.AsyncEventListener;" +
+            "public class QueueCommandsDUnitTestListener implements Declarable2, AsyncEventListener {" +
+            "Properties props;" +
+            "public boolean processEvents(List<AsyncEvent> events) { return true; }" +
+            "public void close() {}" +
+            "public void init(final Properties props) {this.props = props;}" +
+            "public Properties getConfig() {return this.props;}}");
+    writeJarBytesToFile(jarFile, jarBytes);
+
+    CommandResult cmdResult = executeCommand("deploy --jar=QueueCommandsDUnit.jar");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Test creating the queue
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_ASYNC_EVENT_QUEUE);
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ID, queueName);
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__GROUP, groupName);
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER,
+        "com.qcdunit.QueueCommandsDUnitTestListener");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure the queue exists in the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        String xmlFromConfig;
+        try {
+          xmlFromConfig = sharedConfig.getConfiguration(groupName).getCacheXmlContent();
+          assertTrue(xmlFromConfig.contains(queueName));
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service", e);
+        }
+      }
+    });
+
+    //Close cache in the vm1 and restart it to get the shared configuration
+    vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        assertNotNull(cache);
+        cache.close();
+
+        assertTrue(cache.isClosed());
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        localProps.setProperty(DistributionConfig.USE_CLUSTER_CONFIGURATION_NAME, "true");
+        getSystem(localProps);
+        cache = getCache();
+        assertNotNull(cache);
+        AsyncEventQueue aeq = cache.getAsyncEventQueue(queueName);
+
+        assertNotNull(aeq);
+      }
+    });
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    for (String path : this.filesToBeDeleted) {
+      try {
+        final File fileToDelete = new File(path);
+        FileUtil.delete(fileToDelete);
+        if (path.endsWith(".jar")) {
+          executeCommand("undeploy --jar=" + fileToDelete.getName());
+        }
+      } catch (IOException e) {
+        getLogWriter().error("Unable to delete file", e);
+      }
+    }
+    this.filesToBeDeleted.clear();
+    super.tearDown2();
+  }
+
+  private void writeJarBytesToFile(File jarFile, byte[] jarBytes) throws IOException {
+    final OutputStream outStream = new FileOutputStream(jarFile);
+    outStream.write(jarBytes);
+    outStream.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
new file mode 100644
index 0000000..adf5b5f
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
@@ -0,0 +1,338 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.SharedConfiguration;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.ClassBuilder;
+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.HeadlessGfsh;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import com.gemstone.gemfire.management.internal.configuration.SharedConfigurationDUnitTest;
+import com.gemstone.gemfire.management.internal.configuration.domain.Configuration;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+import org.apache.commons.io.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Properties;
+import java.util.Set;
+
+/***
+ * DUnit test to test export and import of shared configuration.
+ *
+ * @author bansods
+ */
+public class SharedConfigurationCommandsDUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+  private static final int TIMEOUT = 10000;
+  private static final int INTERVAL = 500;
+
+  public SharedConfigurationCommandsDUnitTest(String name) {
+    super(name);
+  }
+
+  File newDeployableJarFile = new File("DeployCommandsDUnit1.jar");
+  private transient ClassBuilder classBuilder = new ClassBuilder();
+
+  @SuppressWarnings("unchecked")
+  public void testExportImportSharedConfiguration() {
+    disconnectAllFromDS();
+
+    final String region1Name = "r1";
+    final String region2Name = "r2";
+    final String groupName = "testRegionSharedConfigGroup";
+    final String sharedConfigZipFileName = "sharedConfig.zip";
+    final String deployedJarName = "DeployCommandsDUnit1.jar";
+    final String logLevel = "info";
+    final String startArchiveFileName = "stats.gfs";
+    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(3);
+
+    // TODO Sourabh - the code below is similar to CliCommandTestBase.createDefaultSetup(..); we may want to consider
+    // refactoring this and combine the duplicate code blocks using either the Template Method and/or Strategy design
+    // patterns.  We can talk about this.
+    // Start the Locator and wait for shared configuration to be available
+    final int locator1Port = ports[0];
+    final String locator1Name = "locator1-" + locator1Port;
+    VM locatorAndMgr = Host.getHost(0).getVM(3);
+    Object[] result = (Object[]) locatorAndMgr.invoke(new SerializableCallable() {
+      @Override
+      public Object call() {
+        int httpPort;
+        int jmxPort;
+        String jmxHost;
+
+        try {
+          jmxHost = InetAddress.getLocalHost().getHostName();
+        } catch (UnknownHostException ignore) {
+          jmxHost = "localhost";
+        }
+
+        final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+
+        jmxPort = ports[0];
+        httpPort = ports[1];
+
+        final File locatorLogFile = new File("locator-" + locator1Port + ".log");
+
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, locator1Name);
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "config");
+        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_NAME, "true");
+        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_START_NAME, "true");
+        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_BIND_ADDRESS_NAME, String.valueOf(jmxHost));
+        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
+        locatorProps.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(httpPort));
+
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locator1Port, locatorLogFile,
+              null, locatorProps);
+          DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+            @Override
+            public boolean done() {
+              return locator.isSharedConfigurationRunning();
+            }
+
+            @Override
+            public String description() {
+              return "Waiting for shared configuration to be started";
+            }
+          };
+          DistributedTestCase.waitForCriterion(wc, TIMEOUT, INTERVAL, true);
+        } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        }
+
+        final Object[] result = new Object[4];
+        result[0] = jmxHost;
+        result[1] = jmxPort;
+        result[2] = httpPort;
+        result[3] = CliUtil.getAllNormalMembers(CacheFactory.getAnyInstance());
+
+        return result;
+      }
+    });
+
+    HeadlessGfsh gfsh = getDefaultShell();
+    String jmxHost = (String) result[0];
+    int jmxPort = (Integer) result[1];
+    int httpPort = (Integer) result[2];
+    Set<DistributedMember> normalMembers1 = (Set<DistributedMember>) result[3];
+
+    shellConnect(jmxHost, jmxPort, httpPort, gfsh);
+    // Create a cache in VM 1
+    VM dataMember = Host.getHost(0).getVM(1);
+    normalMembers1 = (Set<DistributedMember>) dataMember.invoke(new SerializableCallable() {
+      @Override
+      public Object call() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locator1Port);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        localProps.setProperty(DistributionConfig.NAME_NAME, "DataMember");
+        getSystem(localProps);
+        Cache cache = getCache();
+        assertNotNull(cache);
+        return CliUtil.getAllNormalMembers(cache);
+      }
+    });
+    // Create a JAR file
+    try {
+      this.classBuilder.writeJarFromName("DeployCommandsDUnitA", this.newDeployableJarFile);
+    } catch (IOException e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+    }
+
+    // Deploy the JAR
+    CommandResult cmdResult = executeCommand("deploy --jar=" + deployedJarName);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    //Create the region1 on the group
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_REGION);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGION, region1Name);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, "REPLICATE");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__STATISTICSENABLED, "true");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__GROUP, groupName);
+
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_REGION);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGION, region2Name);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, "PARTITION");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__STATISTICSENABLED, "true");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    //Alter runtime configuration 
+    commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_RUNTIME_CONFIG);
+    commandStringBuilder.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL, logLevel);
+    commandStringBuilder.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT, "50");
+    commandStringBuilder.addOption(CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT, "32");
+    commandStringBuilder.addOption(CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT, "49");
+    commandStringBuilder.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE, "120");
+    commandStringBuilder.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE, startArchiveFileName);
+    commandStringBuilder.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLING__ENABLED, "true");
+    commandStringBuilder.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, "10");
+    cmdResult = executeCommand(commandStringBuilder.getCommandString());
+    String resultString = commandResultToString(cmdResult);
+
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultString);
+    assertEquals(true, cmdResult.getStatus().equals(Status.OK));
+
+    commandStringBuilder = new CommandStringBuilder(CliStrings.STATUS_SHARED_CONFIG);
+    cmdResult = executeCommand(commandStringBuilder.getCommandString());
+    resultString = commandResultToString(cmdResult);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultString);
+    assertEquals(Status.OK, cmdResult.getStatus());
+
+    commandStringBuilder = new CommandStringBuilder(CliStrings.EXPORT_SHARED_CONFIG);
+    commandStringBuilder.addOption(CliStrings.EXPORT_SHARED_CONFIG__FILE, sharedConfigZipFileName);
+    cmdResult = executeCommand(commandStringBuilder.getCommandString());
+    resultString = commandResultToString(cmdResult);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultString);
+    assertEquals(Status.OK, cmdResult.getStatus());
+
+    //Import into a running system should fail
+    commandStringBuilder = new CommandStringBuilder(CliStrings.IMPORT_SHARED_CONFIG);
+    commandStringBuilder.addOption(CliStrings.IMPORT_SHARED_CONFIG__ZIP, sharedConfigZipFileName);
+    cmdResult = executeCommand(commandStringBuilder.getCommandString());
+    assertEquals(Status.ERROR, cmdResult.getStatus());
+
+    //Stop the data members and remove the shared configuration in the locator.
+    dataMember.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        Cache cache = getCache();
+        cache.close();
+        assertTrue(cache.isClosed());
+        disconnectFromDS();
+        return null;
+      }
+    });
+
+    //Clear shared configuration in this locator to test the import shared configuration
+    locatorAndMgr.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        InternalLocator locator = InternalLocator.getLocator();
+        SharedConfiguration sc = locator.getSharedConfiguration();
+        assertNotNull(sc);
+        sc.clearSharedConfiguration();
+        return null;
+      }
+    });
+
+    //Now execute import shared configuration 
+    //Now import the shared configuration and it should succeed.
+    commandStringBuilder = new CommandStringBuilder(CliStrings.IMPORT_SHARED_CONFIG);
+    commandStringBuilder.addOption(CliStrings.IMPORT_SHARED_CONFIG__ZIP, sharedConfigZipFileName);
+    cmdResult = executeCommand(commandStringBuilder.getCommandString());
+    assertEquals(Status.OK, cmdResult.getStatus());
+
+    //Start a new locator , test if it has all the imported shared configuration artifacts
+    VM newLocator = Host.getHost(0).getVM(2);
+    final int locator2Port = ports[1];
+    final String locator2Name = "Locator2-" + locator2Port;
+
+    newLocator.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        final File locatorLogFile = new File("locator-" + locator2Port + ".log");
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, locator2Name);
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "fine");
+        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        locatorProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locator1Port);
+
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locator2Port, locatorLogFile,
+              null, locatorProps);
+
+          DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+            @Override
+            public boolean done() {
+              return locator.isSharedConfigurationRunning();
+            }
+
+            @Override
+            public String description() {
+              return "Waiting for shared configuration to be started";
+            }
+          };
+          DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+
+          SharedConfiguration sc = locator.getSharedConfiguration();
+          assertNotNull(sc);
+          Configuration groupConfig = sc.getConfiguration(groupName);
+          assertNotNull(groupConfig);
+          assertTrue(groupConfig.getCacheXmlContent().contains(region1Name));
+
+          Configuration clusterConfig = sc.getConfiguration(SharedConfiguration.CLUSTER_CONFIG);
+          assertNotNull(clusterConfig);
+          assertTrue(clusterConfig.getCacheXmlContent().contains(region2Name));
+          assertTrue(clusterConfig.getJarNames().contains(deployedJarName));
+          assertTrue(
+              clusterConfig.getGemfireProperties().getProperty(DistributionConfig.LOG_LEVEL_NAME).equals(logLevel));
+          assertTrue(
+              clusterConfig.getGemfireProperties().getProperty(DistributionConfig.STATISTIC_ARCHIVE_FILE_NAME).equals(
+                  startArchiveFileName));
+        } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service", e);
+        }
+      }
+    });
+
+    //Clean up
+    File sharedConfigZipFile = new File(sharedConfigZipFileName);
+    FileUtils.deleteQuietly(sharedConfigZipFile);
+    FileUtils.deleteQuietly(newDeployableJarFile);
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+    for (int i = 0; i < 4; i++) {
+      Host.getHost(0).getVM(i).invoke(SharedConfigurationDUnitTest.locatorCleanup);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
new file mode 100644
index 0000000..0fc6be4
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
@@ -0,0 +1,365 @@
+/*
+ * 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.AbstractLauncher.Status;
+import com.gemstone.gemfire.distributed.LocatorLauncher;
+import com.gemstone.gemfire.distributed.LocatorLauncher.LocatorState;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.internal.util.IOUtils;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.domain.DataCommandRequest;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+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 org.junit.Before;
+
+import java.io.File;
+import java.util.concurrent.TimeUnit;
+
+public class ShellCommandsDUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public ShellCommandsDUnitTest(String name) {
+    super(name);
+  }
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    getDefaultShell();
+  }
+
+  protected CommandResult connectToLocator(final int locatorPort) {
+    return executeCommand(new CommandStringBuilder(CliStrings.CONNECT).addOption(CliStrings.CONNECT__LOCATOR,
+        "localhost[" + locatorPort + "]").toString());
+  }
+
+  public void testConnectToLocatorBecomesManager() {
+    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+
+    final int jmxManagerPort = ports[0];
+    final int locatorPort = ports[1];
+
+    System.setProperty("gemfire.jmx-manager-port", String.valueOf(jmxManagerPort));
+    System.setProperty("gemfire.jmx-manager-http-port", "0");
+
+    assertEquals(String.valueOf(jmxManagerPort), System.getProperty("gemfire.jmx-manager-port"));
+    assertEquals("0", System.getProperty("gemfire.jmx-manager-http-port"));
+
+    final String pathname = (getClass().getSimpleName() + "_" + getTestName());
+    final File workingDirectory = new File(pathname);
+
+    workingDirectory.mkdir();
+
+    assertTrue(workingDirectory.isDirectory());
+
+    final LocatorLauncher locatorLauncher = new LocatorLauncher.Builder().setBindAddress(null).setForce(
+        true).setMemberName(pathname).setPort(locatorPort).setWorkingDirectory(
+        IOUtils.tryGetCanonicalPathElseGetAbsolutePath(workingDirectory)).build();
+
+    assertNotNull(locatorLauncher);
+    assertEquals(locatorPort, locatorLauncher.getPort().intValue());
+
+    try {
+      // fix for bug 46729
+      locatorLauncher.start();
+
+      final LocatorState locatorState = locatorLauncher.waitOnStatusResponse(60, 10, TimeUnit.SECONDS);
+
+      assertNotNull(locatorState);
+      assertEquals(Status.ONLINE, locatorState.getStatus());
+
+      final Result result = connectToLocator(locatorPort);
+
+      assertNotNull(result);
+      assertEquals(Result.Status.OK, result.getStatus());
+    } finally {
+      assertEquals(Status.STOPPED, locatorLauncher.stop().getStatus());
+      assertEquals(Status.NOT_RESPONDING, locatorLauncher.status().getStatus());
+    }
+  }
+
+  public void testEchoWithVariableAtEnd() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testEcho command gfshInstance is null");
+    }
+    getLogWriter().info("Gsh " + gfshInstance);
+
+    gfshInstance.setEnvProperty("TESTSYS", "SYS_VALUE");
+    printAllEnvs(gfshInstance);
+
+    String command = "echo --string=\"Hello World! This is ${TESTSYS}\"";
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      String stringResult = commandResultToString(cmdResult);
+      assertEquals("Hello World! This is SYS_VALUE", StringUtils.trim(stringResult));
+    } else {
+      fail("testEchoWithVariableAtEnd failed");
+    }
+  }
+
+  public void testEchoWithNoVariable() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testEcho command gfshInstance is null");
+    }
+
+    gfshInstance.setEnvProperty("TESTSYS", "SYS_VALUE");
+    printAllEnvs(gfshInstance);
+
+    String command = "echo --string=\"Hello World! This is Pivotal\"";
+
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      String stringResult = commandResultToString(cmdResult);
+      assertTrue(stringResult.contains("Hello World! This is Pivotal"));
+    } else {
+      fail("testEchoWithNoVariable failed");
+    }
+  }
+
+  public void testEchoWithVariableAtStart() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testEcho command gfshInstance is null");
+    }
+
+    gfshInstance.setEnvProperty("TESTSYS", "SYS_VALUE");
+    printAllEnvs(gfshInstance);
+
+    String command = "echo --string=\"${TESTSYS} Hello World! This is Pivotal\"";
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      String stringResult = commandResultToString(cmdResult);
+      assertTrue(stringResult.contains("SYS_VALUE Hello World! This is Pivotal"));
+    } else {
+      fail("testEchoWithVariableAtStart failed");
+    }
+  }
+
+  public void testEchoWithMultipleVariables() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testEcho command gfshInstance is null");
+    }
+
+    gfshInstance.setEnvProperty("TESTSYS", "SYS_VALUE");
+    printAllEnvs(gfshInstance);
+
+    String command = "echo --string=\"${TESTSYS} Hello World! This is Pivotal ${TESTSYS}\"";
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      String stringResult = commandResultToString(cmdResult);
+      assertTrue(stringResult.contains("SYS_VALUE Hello World! This is Pivotal SYS_VALUE"));
+    } else {
+      fail("testEchoWithMultipleVariables failed");
+    }
+  }
+
+  public void testEchoAllPropertyVariables() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testEcho command gfshInstance is null");
+    }
+
+    String command = "echo --string=\"$*\"";
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testEchoAllPropertyVariables failed");
+    }
+  }
+
+  public void testEchoForSingleVariable() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testEcho command gfshInstance is null");
+    }
+
+    gfshInstance.setEnvProperty("TESTSYS", "SYS_VALUE");
+    printAllEnvs(gfshInstance);
+
+    String command = "echo --string=${TESTSYS}";
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      String stringResult = commandResultToString(cmdResult);
+      assertTrue(stringResult.contains("SYS_VALUE"));
+    } else {
+      fail("testEchoForSingleVariable failed");
+    }
+  }
+
+  public void testEchoForSingleVariable2() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testEcho command gfshInstance is null");
+    }
+
+    gfshInstance.setEnvProperty("TESTSYS", "SYS_VALUE");
+    printAllEnvs(gfshInstance);
+
+    String command = "echo --string=\"${TESTSYS} ${TESTSYS}\"";
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      String stringResult = commandResultToString(cmdResult);
+      assertTrue(stringResult.contains("SYS_VALUE"));
+    } else {
+      fail("testEchoForSingleVariable2 failed");
+    }
+  }
+
+  public void testDebug() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testDebug command gfshInstance is null");
+    }
+
+    gfshInstance.setDebug(false);
+    String command = "debug --state=ON";
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testDebug failed");
+    }
+    assertEquals(gfshInstance.getDebug(), true);
+
+  }
+
+  public void testHistory() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testHistory command gfshInstance is null");
+    }
+
+    gfshInstance.setDebug(false);
+    String command = "history";
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testHistory failed");
+    }
+  }
+
+  public void testHistoryWithFileName() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testHistory command gfshInstance is null");
+    }
+
+    String historyFileName = gfshInstance.getGfshConfig().getHistoryFileName();
+    File historyFile = new File(historyFileName);
+    String fileName = historyFile.getParent();
+    fileName = fileName + File.separator + getClass().getSimpleName() + "_" + getName() + "-exported.history";
+
+    String command = "history --file=" + fileName;
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testHistory failed");
+    }
+  }
+
+  public void testClearHistory() {
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testClearHistory command gfshInstance is null");
+    }
+
+    gfshInstance.setDebug(false);
+    String command = "history --clear";
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      getLogWriter().info("testClearHistory cmdResult=" + commandResultToString(cmdResult));
+      String resultString = commandResultToString(cmdResult);
+      getLogWriter().info("testClearHistory resultString=" + resultString);
+      assertTrue(resultString.contains(CliStrings.HISTORY__MSG__CLEARED_HISTORY));
+      assertTrue(gfshInstance.getGfshHistory().getHistoryList().size() <= 1);
+    } else {
+      fail("testClearHistory failed");
+    }
+  }
+
+  private static void printCommandOutput(CommandResult cmdResult) {
+    assertNotNull(cmdResult);
+    getLogWriter().info("Command Output : ");
+    StringBuilder sb = new StringBuilder();
+    cmdResult.resetToFirstLine();
+    while (cmdResult.hasNextLine()) {
+      sb.append(cmdResult.nextLine()).append(DataCommandRequest.NEW_LINE);
+    }
+    getLogWriter().info(sb.toString());
+    getLogWriter().info("");
+  }
+
+  private void printAllEnvs(Gfsh gfsh) {
+    getLogWriter().info("printAllEnvs : " + StringUtils.objectToString(gfsh.getEnv(), false, 0));
+    /*
+    getLogWriter().info("Gfsh printAllEnvs : " + HydraUtil.ObjectToString(getDefaultShell().getEnv()));    
+    getLogWriter().info("gfsh " + gfsh + " default shell " + getDefaultShell());*/
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowDeadlockDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowDeadlockDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowDeadlockDUnitTest.java
new file mode 100644
index 0000000..2d67129
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowDeadlockDUnitTest.java
@@ -0,0 +1,271 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.cache.execute.ResultCollector;
+import com.gemstone.gemfire.cache30.CacheTestCase;
+import com.gemstone.gemfire.distributed.DistributedLockService;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.deadlock.GemFireDeadlockDetector;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+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.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.remote.CommandProcessor;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * This DUnit tests uses same code as GemFireDeadlockDetectorDUnitTest and uses the command processor for executing the
+ * "show deadlock" command
+ */
+public class ShowDeadlockDUnitTest extends CacheTestCase {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 1L;
+  private static final Set<Thread> stuckThreads = Collections.synchronizedSet(new HashSet<Thread>());
+  private static final Map<String, String> EMPTY_ENV = Collections.emptyMap();
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    // This test does not require an actual Gfsh connection to work, however when run as part of a suite, prior tests
+    // may mess up the environment causing this test to fail. Setting this prevents false failures.
+    CliUtil.isGfshVM = false;
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    invokeInEveryVM(new SerializableRunnable() {
+      private static final long serialVersionUID = 1L;
+
+      public void run() {
+        for (Thread thread : stuckThreads) {
+          thread.interrupt();
+        }
+      }
+    });
+    CliUtil.isGfshVM = true;
+  }
+
+  public ShowDeadlockDUnitTest(String name) {
+    super(name);
+  }
+
+  public void testNoDeadlock() throws ClassNotFoundException, IOException {
+    Host host = Host.getHost(0);
+    VM vm0 = host.getVM(0);
+    VM vm1 = host.getVM(1);
+
+    //Make sure a deadlock from a previous test is cleared.
+    disconnectAllFromDS();
+
+    createCache(vm0);
+    createCache(vm1);
+    createCache(new Properties());
+
+    String fileName = "dependency.txt";
+    GemFireDeadlockDetector detect = new GemFireDeadlockDetector();
+    assertEquals(null, detect.find().findCycle());
+
+    CommandProcessor commandProcessor = new CommandProcessor();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.SHOW_DEADLOCK);
+    csb.addOption(CliStrings.SHOW_DEADLOCK__DEPENDENCIES__FILE, fileName);
+    Result result = commandProcessor.createCommandStatement(csb.toString(), EMPTY_ENV).process();
+
+    String deadLockOutputFromCommand = getResultAsString(result);
+
+    getLogWriter().info("output = " + deadLockOutputFromCommand);
+    assertEquals(true, result.hasIncomingFiles());
+    assertEquals(true, result.getStatus().equals(Status.OK));
+    assertEquals(true, deadLockOutputFromCommand.startsWith(CliStrings.SHOW_DEADLOCK__NO__DEADLOCK));
+    result.saveIncomingFiles(null);
+    File file = new File(fileName);
+    assertTrue(file.exists());
+    file.delete();
+
+    disconnectAllFromDS();
+  }
+
+  private static final Lock lock = new ReentrantLock();
+
+
+  public void testDistributedDeadlockWithFunction() throws InterruptedException, ClassNotFoundException, IOException {
+    Host host = Host.getHost(0);
+    VM vm0 = host.getVM(0);
+    VM vm1 = host.getVM(1);
+    String filename = "gfeDependency.txt";
+    InternalDistributedMember member1 = createCache(vm0);
+    final InternalDistributedMember member2 = createCache(vm1);
+    createCache(new Properties());
+    //Have two threads lock locks on different members in different orders.
+    //This thread locks the lock member1 first, then member2.
+    lockTheLocks(vm0, member2);
+    //This thread locks the lock member2 first, then member1.
+    lockTheLocks(vm1, member1);
+
+    Thread.sleep(5000);
+    CommandProcessor commandProcessor = new CommandProcessor();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.SHOW_DEADLOCK);
+    csb.addOption(CliStrings.SHOW_DEADLOCK__DEPENDENCIES__FILE, filename);
+    Result result = commandProcessor.createCommandStatement(csb.toString(), EMPTY_ENV).process();
+
+    String deadLockOutputFromCommand = getResultAsString(result);
+    getLogWriter().info("Deadlock = " + deadLockOutputFromCommand);
+    result.saveIncomingFiles(null);
+    assertEquals(true, deadLockOutputFromCommand.startsWith(CliStrings.SHOW_DEADLOCK__DEADLOCK__DETECTED));
+    assertEquals(true, result.getStatus().equals(Status.OK));
+    File file = new File(filename);
+    assertTrue(file.exists());
+    file.delete();
+
+  }
+
+
+  private void createCache(Properties props) {
+    getSystem(props);
+    final Cache cache = getCache();
+  }
+
+  private Properties createProperties(Host host, int locatorPort) {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+//    props.setProperty(DistributionConfig.LOCATORS_NAME, getServerHostName(host) + "[" + locatorPort + "]");
+    props.setProperty(DistributionConfig.LOG_LEVEL_NAME, "info");
+    props.setProperty(DistributionConfig.STATISTIC_SAMPLING_ENABLED_NAME, "true");
+    props.setProperty(DistributionConfig.ENABLE_TIME_STATISTICS_NAME, "true");
+    props.put(DistributionConfig.ENABLE_NETWORK_PARTITION_DETECTION_NAME, "true");
+    return props;
+  }
+
+  private void lockTheLocks(VM vm0, final InternalDistributedMember member) {
+    vm0.invokeAsync(new SerializableRunnable() {
+
+      private static final long serialVersionUID = 1L;
+
+      public void run() {
+        lock.lock();
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          fail("interrupted", e);
+        }
+        ResultCollector collector = FunctionService.onMember(system, member).execute(new TestFunction());
+        //wait the function to lock the lock on member.
+        collector.getResult();
+        lock.unlock();
+      }
+    });
+  }
+
+  private void lockTheDLocks(VM vm, final String first, final String second) {
+    vm.invokeAsync(new SerializableRunnable() {
+
+      private static final long serialVersionUID = 1L;
+
+      public void run() {
+        getCache();
+        DistributedLockService dls = DistributedLockService.create("deadlock_test", getSystem());
+        dls.lock(first, 10 * 1000, -1);
+
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+        dls.lock(second, 10 * 1000, -1);
+      }
+    });
+  }
+
+  private InternalDistributedMember createCache(VM vm) {
+    return (InternalDistributedMember) vm.invoke(new SerializableCallable() {
+      /**
+       *
+       */
+      private static final long serialVersionUID = 1L;
+
+      public Object call() {
+        getCache();
+        return getSystem().getDistributedMember();
+      }
+    });
+  }
+
+  private String getResultAsString(Result result) {
+    StringBuilder sb = new StringBuilder();
+    while (result.hasNextLine()) {
+      sb.append(result.nextLine());
+    }
+
+    return sb.toString();
+  }
+
+  private static class TestFunction implements Function {
+
+    private static final long serialVersionUID = 1L;
+    private static final int LOCK_WAIT_TIME = 1000;
+
+    public boolean hasResult() {
+      return true;
+    }
+
+    public void execute(FunctionContext context) {
+      try {
+        stuckThreads.add(Thread.currentThread());
+        lock.tryLock(LOCK_WAIT_TIME, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        //ingore
+      }
+      context.getResultSender().lastResult(null);
+    }
+
+    public String getId() {
+      return getClass().getCanonicalName();
+    }
+
+    public boolean optimizeForWrite() {
+      return false;
+    }
+
+    public boolean isHA() {
+      return false;
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java
new file mode 100644
index 0000000..a69c35a
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java
@@ -0,0 +1,347 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.management.CacheServerMXBean;
+import com.gemstone.gemfire.management.DistributedRegionMXBean;
+import com.gemstone.gemfire.management.DistributedSystemMXBean;
+import com.gemstone.gemfire.management.ManagementService;
+import com.gemstone.gemfire.management.MemberMXBean;
+import com.gemstone.gemfire.management.RegionMXBean;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.cli.Result.Status;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.remote.CommandProcessor;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import javax.management.ObjectName;
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Properties;
+
+/****
+ * @author bansods
+ */
+public class ShowMetricsDUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public ShowMetricsDUnitTest(String name) {
+    super(name);
+    // TODO Auto-generated constructor stub
+  }
+
+  private void createLocalSetUp() {
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Controller");
+    getSystem(localProps);
+    Cache cache = getCache();
+    RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+    Region region1 = dataRegionFactory.create("REGION1");
+    Region region2 = dataRegionFactory.create("REGION2");
+  }
+
+  /*
+   * tests the default version of "show metrics"
+   */
+  public void testShowMetricsDefault() {
+    createDefaultSetup(null);
+    createLocalSetUp();
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1Name = "VM" + vm1.getPid();
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm1Name);
+        getSystem(localProps);
+
+        Cache cache = getCache();
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+        Region region = dataRegionFactory.create("REGION1");
+      }
+    });
+
+    SerializableCallable showMetricCmd = new SerializableCallable() {
+
+      @Override
+      public Object call() throws Exception {
+        WaitCriterion wc = createMBeanWaitCriterion(1, "", null, 0);
+        waitForCriterion(wc, 5000, 500, true);
+        CommandProcessor commandProcessor = new CommandProcessor();
+        Result result = commandProcessor.createCommandStatement("show metrics", Collections.EMPTY_MAP).process();
+        String resultStr = commandResultToString((CommandResult) result);
+        getLogWriter().info(resultStr);
+        assertEquals(resultStr, true, result.getStatus().equals(Status.OK));
+        return resultStr;
+      }
+    };
+
+    //Invoke the command in the Manager VM
+    final VM managerVm = Host.getHost(0).getVM(0);
+    Object managerResultObj = managerVm.invoke(showMetricCmd);
+
+    String managerResult = (String) managerResultObj;
+
+    getLogWriter().info("#SB Manager");
+    getLogWriter().info(managerResult);
+  }
+
+  public void systemSetUp() {
+    createDefaultSetup(null);
+    createLocalSetUp();
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1Name = "VM" + vm1.getPid();
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm1Name);
+        getSystem(localProps);
+
+        Cache cache = getCache();
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+        Region region = dataRegionFactory.create("REGION1");
+      }
+    });
+  }
+
+  public void testShowMetricsRegion() throws InterruptedException {
+    systemSetUp();
+    final String regionName = "REGION1";
+    SerializableCallable showMetricCmd = new SerializableCallable() {
+
+      @Override
+      public Object call() throws Exception {
+        WaitCriterion wc = createMBeanWaitCriterion(2, regionName, null, 0);
+        waitForCriterion(wc, 5000, 500, true);
+        CommandProcessor commandProcessor = new CommandProcessor();
+        Result result = commandProcessor.createCommandStatement("show metrics --region=REGION1",
+            Collections.EMPTY_MAP).process();
+        String resultAsString = commandResultToString((CommandResult) result);
+        assertEquals(resultAsString, true, result.getStatus().equals(Status.OK));
+        return resultAsString;
+      }
+    };
+
+    //Invoke the command in the Manager VM
+    final VM managerVm = Host.getHost(0).getVM(0);
+    Object managerResultObj = managerVm.invoke(showMetricCmd);
+
+    String managerResult = (String) managerResultObj;
+
+    getLogWriter().info("#SB Manager");
+    getLogWriter().info(managerResult);
+  }
+
+  /***
+   * Creates WaitCriterion based on creation of different types of MBeans
+   *
+   * @param beanType
+   * @param regionName
+   * @param memberName
+   * @return
+   */
+  private WaitCriterion createMBeanWaitCriterion(final int beanType, final String regionName,
+      final DistributedMember distributedMember, final int cacheServerPort) {
+
+    WaitCriterion waitCriterion = new WaitCriterion() {
+
+      @Override
+      public boolean done() {
+        boolean done = false;
+        Cache cache = getCache();
+        ManagementService mgmtService = ManagementService.getManagementService(cache);
+        if (beanType == 1) {
+          DistributedSystemMXBean dsMxBean = mgmtService.getDistributedSystemMXBean();
+          if (dsMxBean != null) done = true;
+        } else if (beanType == 2) {
+          DistributedRegionMXBean dsRegionMxBean = mgmtService.getDistributedRegionMXBean("/" + regionName);
+          if (dsRegionMxBean != null) done = true;
+        } else if (beanType == 3) {
+          ObjectName memberMBeanName = mgmtService.getMemberMBeanName(distributedMember);
+          MemberMXBean memberMxBean = mgmtService.getMBeanInstance(memberMBeanName, MemberMXBean.class);
+
+          if (memberMxBean != null) done = true;
+        } else if (beanType == 4) {
+          ObjectName regionMBeanName = mgmtService.getRegionMBeanName(distributedMember, "/" + regionName);
+          RegionMXBean regionMxBean = mgmtService.getMBeanInstance(regionMBeanName, RegionMXBean.class);
+
+          if (regionMxBean != null) done = true;
+        } else if (beanType == 5) {
+          ObjectName csMxBeanName = mgmtService.getCacheServerMBeanName(cacheServerPort, distributedMember);
+          CacheServerMXBean csMxBean = mgmtService.getMBeanInstance(csMxBeanName, CacheServerMXBean.class);
+
+          if (csMxBean != null) {
+            done = true;
+          }
+        }
+
+        return done;
+      }
+
+      @Override
+      public String description() {
+        return "Waiting for the mbean to be created";
+      }
+    };
+
+    return waitCriterion;
+  }
+
+  public void testShowMetricsMember() throws ClassNotFoundException, IOException, InterruptedException {
+    systemSetUp();
+    Cache cache = getCache();
+    final DistributedMember distributedMember = cache.getDistributedSystem().getDistributedMember();
+    final String exportFileName = "memberMetricReport.csv";
+
+    int ports[] = AvailablePortHelper.getRandomAvailableTCPPorts(1);
+    CacheServer cs = getCache().addCacheServer();
+    cs.setPort(ports[0]);
+    cs.start();
+    final int cacheServerPort = cs.getPort();
+
+    SerializableCallable showMetricCmd = new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+
+        WaitCriterion wc = createMBeanWaitCriterion(3, "", distributedMember, 0);
+        waitForCriterion(wc, 5000, 500, true);
+        wc = createMBeanWaitCriterion(5, "", distributedMember, cacheServerPort);
+        waitForCriterion(wc, 10000, 500, true);
+
+        final String command = CliStrings.SHOW_METRICS + " --" + CliStrings.SHOW_METRICS__MEMBER + "=" + distributedMember.getId() + " --" + CliStrings.SHOW_METRICS__CACHESERVER__PORT + "=" + cacheServerPort + " --" + CliStrings.SHOW_METRICS__FILE + "=" + exportFileName;
+
+        CommandProcessor commandProcessor = new CommandProcessor();
+        Result result = commandProcessor.createCommandStatement(command, Collections.EMPTY_MAP).process();
+        String resultAsString = commandResultToString((CommandResult) result);
+        assertEquals(resultAsString, true, result.getStatus().equals(Status.OK));
+        assertTrue(result.hasIncomingFiles());
+        result.saveIncomingFiles(null);
+        File file = new File(exportFileName);
+        file.deleteOnExit();
+        assertTrue(file.exists());
+        file.delete();
+        return resultAsString;
+
+      }
+    };
+
+    //Invoke the command in the Manager VM
+    final VM managerVm = Host.getHost(0).getVM(0);
+    Object managerResultObj = managerVm.invoke(showMetricCmd);
+
+    String managerResult = (String) managerResultObj;
+
+    getLogWriter().info("#SB Manager");
+    getLogWriter().info(managerResult);
+    cs.stop();
+  }
+
+  public void testShowMetricsRegionFromMember() throws ClassNotFoundException, IOException, InterruptedException {
+    systemSetUp();
+    Cache cache = getCache();
+    final DistributedMember distributedMember = cache.getDistributedSystem().getDistributedMember();
+    final String exportFileName = "regionOnAMemberReport.csv";
+    final String regionName = "REGION1";
+
+    SerializableCallable showMetricCmd = new SerializableCallable() {
+
+      @Override
+      public Object call() throws Exception {
+
+        WaitCriterion wc = createMBeanWaitCriterion(4, regionName, distributedMember, 0);
+        waitForCriterion(wc, 5000, 500, true);
+        CommandProcessor commandProcessor = new CommandProcessor();
+        Result result = commandProcessor.createCommandStatement(
+            "show metrics --region=" + regionName + " --member=" + distributedMember.getName() + " --file=" + exportFileName,
+            Collections.EMPTY_MAP).process();
+        String resultAsString = commandResultToString((CommandResult) result);
+        assertEquals(resultAsString, true, result.getStatus().equals(Status.OK));
+        assertTrue(result.hasIncomingFiles());
+        result.saveIncomingFiles(null);
+        File file = new File(exportFileName);
+        file.deleteOnExit();
+        assertTrue(file.exists());
+        file.delete();
+        return resultAsString;
+      }
+    };
+
+    //Invoke the command in the Manager VM
+    final VM managerVm = Host.getHost(0).getVM(0);
+    Object managerResultObj = managerVm.invoke(showMetricCmd);
+
+    String managerResult = (String) managerResultObj;
+
+    getLogWriter().info("#SB Manager");
+    getLogWriter().info(managerResult);
+  }
+
+  public void testShowMetricsRegionFromMemberWithCategories() throws ClassNotFoundException, IOException, InterruptedException {
+    systemSetUp();
+    Cache cache = getCache();
+    final DistributedMember distributedMember = cache.getDistributedSystem().getDistributedMember();
+    final String exportFileName = "regionOnAMemberReport.csv";
+    final String regionName = "REGION1";
+
+    SerializableCallable showMetricCmd = new SerializableCallable() {
+
+      @Override
+      public Object call() throws Exception {
+
+        WaitCriterion wc = createMBeanWaitCriterion(4, regionName, distributedMember, 0);
+        waitForCriterion(wc, 5000, 500, true);
+        CommandProcessor commandProcessor = new CommandProcessor();
+        Result result = commandProcessor.createCommandStatement(
+            "show metrics --region=" + regionName + " --member=" + distributedMember.getName() + " --file=" + exportFileName + " --categories=region,eviction",
+            Collections.EMPTY_MAP).process();
+        String resultAsString = commandResultToString((CommandResult) result);
+        assertEquals(resultAsString, true, result.getStatus().equals(Status.OK));
+        assertTrue(result.hasIncomingFiles());
+        result.saveIncomingFiles(null);
+        File file = new File(exportFileName);
+        file.deleteOnExit();
+        assertTrue(file.exists());
+        file.delete();
+        return resultAsString;
+      }
+    };
+
+    //Invoke the command in the Manager VM
+    final VM managerVm = Host.getHost(0).getVM(0);
+    Object managerResultObj = managerVm.invoke(showMetricCmd);
+
+    String managerResult = (String) managerResultObj;
+
+    getLogWriter().info("#SB Manager");
+    getLogWriter().info(managerResult);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java
new file mode 100644
index 0000000..d1dc87f
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.management.cli.Result.Status;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+
+/***
+ * DUnit test for 'show stack-trace' command
+ *
+ * @author bansods
+ */
+public class ShowStackTraceDUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public ShowStackTraceDUnitTest(String name) {
+    super(name);
+  }
+
+  private void createCache(Properties props) {
+    getSystem(props);
+    getCache();
+  }
+
+  private Properties createProperties(Host host, String name, String groups) {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOG_LEVEL_NAME, "info");
+    props.setProperty(DistributionConfig.STATISTIC_SAMPLING_ENABLED_NAME, "true");
+    props.setProperty(DistributionConfig.ENABLE_TIME_STATISTICS_NAME, "true");
+    props.setProperty(DistributionConfig.NAME_NAME, name);
+    props.setProperty(DistributionConfig.GROUPS_NAME, groups);
+    return props;
+  }
+
+  /***
+   * Sets up a system of 3 peers
+   */
+  private void setupSystem() {
+    disconnectAllFromDS();
+    final Host host = Host.getHost(0);
+    final VM[] servers = {host.getVM(0), host.getVM(1)};
+
+    final Properties propsManager = createProperties(host, "Manager", "G1");
+    final Properties propsServer2 = createProperties(host, "Server", "G2");
+
+    createDefaultSetup(propsManager);
+
+    servers[1].invoke(new SerializableRunnable("Create cache for server1") {
+      public void run() {
+        createCache(propsServer2);
+      }
+    });
+  }
+
+  /***
+   * Tests the default behavior of the show stack-trace command
+   *
+   * @throws ClassNotFoundException
+   * @throws IOException
+   */
+  public void testExportStacktrace() throws ClassNotFoundException, IOException {
+    setupSystem();
+
+    File allStacktracesFile = new File("allStackTraces.txt");
+    allStacktracesFile.createNewFile();
+    allStacktracesFile.deleteOnExit();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.EXPORT_STACKTRACE);
+    csb.addOption(CliStrings.EXPORT_STACKTRACE__FILE, allStacktracesFile.getCanonicalPath());
+    String commandString = csb.toString();
+    getLogWriter().info("CommandString : " + commandString);
+    CommandResult commandResult = executeCommand(commandString);
+    getLogWriter().info("Output : \n" + commandResultToString(commandResult));
+    assertTrue(commandResult.getStatus().equals(Status.OK));
+
+    File mgrStacktraceFile = new File("managerStacktrace.txt");
+    mgrStacktraceFile.createNewFile();
+    mgrStacktraceFile.deleteOnExit();
+    csb = new CommandStringBuilder(CliStrings.EXPORT_STACKTRACE);
+    csb.addOption(CliStrings.EXPORT_STACKTRACE__FILE, mgrStacktraceFile.getCanonicalPath());
+    csb.addOption(CliStrings.EXPORT_STACKTRACE__MEMBER, "Manager");
+    commandString = csb.toString();
+    getLogWriter().info("CommandString : " + commandString);
+    commandResult = executeCommand(commandString);
+    getLogWriter().info("Output : \n" + commandResultToString(commandResult));
+    assertTrue(commandResult.getStatus().equals(Status.OK));
+
+    File serverStacktraceFile = new File("serverStacktrace.txt");
+    serverStacktraceFile.createNewFile();
+    serverStacktraceFile.deleteOnExit();
+    csb = new CommandStringBuilder(CliStrings.EXPORT_STACKTRACE);
+    csb.addOption(CliStrings.EXPORT_STACKTRACE__FILE, serverStacktraceFile.getCanonicalPath());
+    csb.addOption(CliStrings.EXPORT_STACKTRACE__MEMBER, "Server");
+    commandString = csb.toString();
+    getLogWriter().info("CommandString : " + commandString);
+    commandResult = executeCommand(commandString);
+    getLogWriter().info("Output : \n" + commandResultToString(commandResult));
+    assertTrue(commandResult.getStatus().equals(Status.OK));
+
+    File groupStacktraceFile = new File("groupstacktrace.txt");
+    groupStacktraceFile.createNewFile();
+    groupStacktraceFile.deleteOnExit();
+    csb = new CommandStringBuilder(CliStrings.EXPORT_STACKTRACE);
+    csb.addOption(CliStrings.EXPORT_STACKTRACE__FILE, groupStacktraceFile.getCanonicalPath());
+    csb.addOption(CliStrings.EXPORT_STACKTRACE__GROUP, "G2");
+    commandString = csb.toString();
+    getLogWriter().info("CommandString : " + commandString);
+    commandResult = executeCommand(commandString);
+    getLogWriter().info("Output : \n" + commandResultToString(commandResult));
+    assertTrue(commandResult.getStatus().equals(Status.OK));
+
+    File wrongStackTraceFile = new File("wrongStackTrace.txt");
+    wrongStackTraceFile.createNewFile();
+    wrongStackTraceFile.deleteOnExit();
+    csb = new CommandStringBuilder(CliStrings.EXPORT_STACKTRACE);
+    csb.addOption(CliStrings.EXPORT_STACKTRACE__FILE, wrongStackTraceFile.getCanonicalPath());
+    csb.addOption(CliStrings.EXPORT_STACKTRACE__MEMBER, "WrongMember");
+    commandString = csb.toString();
+    getLogWriter().info("CommandString : " + commandString);
+    commandResult = executeCommand(commandString);
+    getLogWriter().info("Output : \n" + commandResultToString(commandResult));
+    assertFalse(commandResult.getStatus().equals(Status.OK));
+  }
+}


[46/50] [abbrv] incubator-geode git commit: Merge branch 'feature/GEODE-53' into develop - This merge will update Apache Geode website - Removing unused images

Posted by kl...@apache.org.
Merge branch 'feature/GEODE-53' into develop
- This merge will update Apache Geode website
- Removing unused images


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

Branch: refs/heads/feature/GEODE-217
Commit: cd75b1f196573524cac16f8565a2f42937ef16c7
Parents: 68dfcab c32a5b2
Author: William Markito <wm...@pivotal.io>
Authored: Tue Dec 8 16:37:37 2015 -0800
Committer: William Markito <wm...@pivotal.io>
Committed: Tue Dec 8 16:58:16 2015 -0800

----------------------------------------------------------------------
 gemfire-site/build.gradle                       |   37 -
 .../content/bootstrap/bootstrap.min.css         |    9 +
 gemfire-site/content/community/index.html       |  629 +++++++
 .../content/css/bootflat-extensions.css         |  356 ++++
 gemfire-site/content/css/bootflat-square.css    |   69 +
 gemfire-site/content/css/bootflat.css           | 1559 ++++++++++++++++++
 gemfire-site/content/css/font-awesome.min.css   |  405 +++++
 gemfire-site/content/css/geode-site.css         | 1554 +++++++++++++++++
 gemfire-site/content/css/usergrid-site.css      | 1554 +++++++++++++++++
 gemfire-site/content/favicon.ico                |  Bin 0 -> 20805 bytes
 gemfire-site/content/font/FontAwesome.otf       |  Bin 0 -> 61896 bytes
 .../content/font/fontawesome-webfont-eot.eot    |  Bin 0 -> 37405 bytes
 .../content/font/fontawesome-webfont-svg.svg    |  399 +++++
 .../content/font/fontawesome-webfont-ttf.ttf    |  Bin 0 -> 79076 bytes
 .../content/font/fontawesome-webfont-woff.woff  |  Bin 0 -> 43572 bytes
 gemfire-site/content/img/apache_geode_logo.png  |  Bin 0 -> 23616 bytes
 .../content/img/apache_geode_logo_white.png     |  Bin 0 -> 22695 bytes
 .../img/apache_geode_logo_white_small.png       |  Bin 0 -> 52948 bytes
 gemfire-site/content/img/check_flat/default.png |  Bin 0 -> 25851 bytes
 gemfire-site/content/img/egg-logo.png           |  Bin 0 -> 9938 bytes
 gemfire-site/content/img/github.png             |  Bin 0 -> 8936 bytes
 gemfire-site/content/index.html                 |  295 ++++
 gemfire-site/content/js/bootstrap.min.js        |    8 +
 gemfire-site/content/js/head.js                 |  708 ++++++++
 gemfire-site/content/js/html5shiv.js            |    8 +
 gemfire-site/content/js/jquery-1.10.1.min.js    |    6 +
 gemfire-site/content/js/jquery.icheck.js        |  397 +++++
 gemfire-site/content/js/respond.min.js          |    6 +
 gemfire-site/content/js/usergrid-site.js        |   50 +
 gemfire-site/content/releases/index.html        |  239 +++
 gemfire-site/content/static/github-btn.html     |    2 +
 gemfire-site/src/jbake.zip                      |  Bin 207030 -> 0 bytes
 gemfire-site/src/jbake/assets/favicon.ico       |  Bin 1150 -> 0 bytes
 .../src/jbake/assets/images/bg-billboard.png    |  Bin 25538 -> 0 bytes
 .../jbake/assets/images/bg-crystals-home.png    |  Bin 41684 -> 0 bytes
 .../assets/images/bg-crystals-secondary.png     |  Bin 26046 -> 0 bytes
 .../src/jbake/assets/images/egg-logo1.png       |  Bin 8626 -> 0 bytes
 .../jbake/assets/images/events/apachecon.png    |  Bin 4528 -> 0 bytes
 .../src/jbake/assets/images/events/oscon.png    |  Bin 26024 -> 0 bytes
 .../src/jbake/assets/images/geode-banner.png    |  Bin 7916 -> 0 bytes
 .../assets/images/logo-apache-geode-white.png   |  Bin 2336 -> 0 bytes
 .../jbake/assets/images/logo-apache-geode.png   |  Bin 3200 -> 0 bytes
 .../jbake/assets/images/logo-geode-white.png    |  Bin 1620 -> 0 bytes
 .../src/jbake/assets/images/logo-geode.png      |  Bin 3345 -> 0 bytes
 .../src/jbake/assets/javascripts/master.js      |  121 --
 .../src/jbake/assets/javascripts/scale.fix.js   |   20 -
 .../jbake/assets/stylesheets/pygment_trac.css   |   60 -
 .../src/jbake/assets/stylesheets/styles.css     |  319 ----
 gemfire-site/src/jbake/content/404.md           |    9 -
 gemfire-site/src/jbake/content/README.md        |   36 -
 gemfire-site/src/jbake/content/about/index.md   |   31 -
 .../src/jbake/content/community/index.md        |   82 -
 .../src/jbake/content/contribute/index.md       |   47 -
 gemfire-site/src/jbake/content/docs/index.md    |   23 -
 .../src/jbake/content/download/index.md         |   13 -
 .../src/jbake/content/getting-started/index.md  |   88 -
 gemfire-site/src/jbake/content/index.md         |   76 -
 gemfire-site/src/jbake/jbake.properties         |    6 -
 gemfire-site/src/jbake/templates/page.groovy    |   80 -
 gemfire-site/website/.gitignore                 |    1 +
 gemfire-site/website/README.md                  |   54 +
 gemfire-site/website/Rules                      |   52 +
 gemfire-site/website/build.sh                   |    1 +
 .../website/content/bootstrap/bootstrap.min.css |    9 +
 .../website/content/community/index.html        |  286 ++++
 .../website/content/css/bootflat-extensions.css |  356 ++++
 .../website/content/css/bootflat-square.css     |   69 +
 gemfire-site/website/content/css/bootflat.css   | 1559 ++++++++++++++++++
 .../website/content/css/font-awesome.min.css    |  405 +++++
 gemfire-site/website/content/css/geode-site.css | 1554 +++++++++++++++++
 gemfire-site/website/content/favicon.ico        |  Bin 0 -> 20805 bytes
 .../website/content/font/FontAwesome.otf        |  Bin 0 -> 61896 bytes
 .../content/font/fontawesome-webfont-eot.eot    |  Bin 0 -> 37405 bytes
 .../content/font/fontawesome-webfont-svg.svg    |  399 +++++
 .../content/font/fontawesome-webfont-ttf.ttf    |  Bin 0 -> 79076 bytes
 .../content/font/fontawesome-webfont-woff.woff  |  Bin 0 -> 43572 bytes
 .../website/content/img/apache_geode_logo.png   |  Bin 0 -> 23616 bytes
 .../content/img/apache_geode_logo_white.png     |  Bin 0 -> 22695 bytes
 .../img/apache_geode_logo_white_small.png       |  Bin 0 -> 52948 bytes
 .../website/content/img/check_flat/default.png  |  Bin 0 -> 25851 bytes
 gemfire-site/website/content/img/egg-logo.png   |  Bin 0 -> 9938 bytes
 gemfire-site/website/content/img/github.png     |  Bin 0 -> 8936 bytes
 gemfire-site/website/content/img/intellij.png   |  Bin 0 -> 9199 bytes
 gemfire-site/website/content/img/yourkit.jpeg   |  Bin 0 -> 7763 bytes
 gemfire-site/website/content/index.html         |  124 ++
 .../website/content/js/bootstrap.min.js         |    8 +
 gemfire-site/website/content/js/head.js         |  708 ++++++++
 gemfire-site/website/content/js/html5shiv.js    |    8 +
 .../website/content/js/jquery-1.10.1.min.js     |    6 +
 .../website/content/js/jquery.icheck.js         |  397 +++++
 gemfire-site/website/content/js/respond.min.js  |    6 +
 .../website/content/js/usergrid-site.js         |   50 +
 .../website/content/releases/index.html         |   65 +
 gemfire-site/website/layouts/community.html     |    1 +
 gemfire-site/website/layouts/default.html       |   44 +
 gemfire-site/website/layouts/docs.html          |    1 +
 gemfire-site/website/layouts/footer.html        |   96 ++
 gemfire-site/website/layouts/header.html        |  231 +++
 gemfire-site/website/lib/default.rb             |   43 +
 gemfire-site/website/lib/helpers_.rb            |    0
 gemfire-site/website/lib/pandoc.template        |    4 +
 gemfire-site/website/nanoc.yaml                 |   77 +
 gemfire-site/website/run.sh                     |    1 +
 gemfire-site/website/utilities/map-markers.rb   |   58 +
 gemfire-site/website/utilities/markers.txt      |  440 +++++
 .../website/utilities/snapshot-apigee.rb        |   71 +
 gemfire-site/website/utilities/usergrid.csv     |  290 ++++
 107 files changed, 15727 insertions(+), 1048 deletions(-)
----------------------------------------------------------------------



[03/50] [abbrv] incubator-geode git commit: Adding code-coverage testing for GMSMembershipManager and removing unused code.

Posted by kl...@apache.org.
Adding code-coverage testing for GMSMembershipManager and removing unused code.


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

Branch: refs/heads/feature/GEODE-217
Commit: 38dd3ed892be2acd1609c0b72ee5f4d4cfe85175
Parents: 253549d
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Nov 20 13:33:21 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 1 08:47:51 2015 -0800

----------------------------------------------------------------------
 .../internal/DistributionChannel.java           |   2 +-
 .../internal/DistributionManager.java           |  29 +-
 .../internal/HighPriorityAckedMessage.java      |   4 +-
 .../distributed/internal/StartupMessage.java    |  19 -
 .../distributed/internal/StartupOperation.java  |   1 -
 .../internal/direct/DirectChannel.java          |  16 +-
 .../internal/direct/DirectChannelListener.java  |  22 ++
 .../DistributedMembershipListener.java          |  13 +-
 .../internal/membership/MemberFactory.java      |   8 -
 .../internal/membership/MemberServices.java     |   6 -
 .../internal/membership/MembershipManager.java  |  13 -
 .../internal/membership/gms/GMSMember.java      |  22 --
 .../membership/gms/GMSMemberFactory.java        |  12 +-
 .../internal/membership/gms/GMSUtil.java        |  19 +
 .../membership/gms/fd/GMSHealthMonitor.java     |   2 +-
 .../gms/messenger/JGroupsMessenger.java         |   9 +-
 .../gms/mgr/GMSMembershipManager.java           | 342 +++--------------
 .../internal/logging/log4j/LogMarker.java       |   1 -
 .../gemfire/internal/tcp/Connection.java        |   7 +
 .../gemfire/internal/tcp/TCPConduit.java        |   6 +-
 .../membership/MembershipJUnitTest.java         |  29 +-
 .../locator/GMSLocatorRecoveryJUnitTest.java    |   4 +-
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  | 370 +++++++++++++++++++
 23 files changed, 520 insertions(+), 436 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionChannel.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionChannel.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionChannel.java
index daccc9c..a31c92f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionChannel.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionChannel.java
@@ -132,7 +132,7 @@ public class DistributionChannel  {
   public long getId() {
     MembershipManager mgr = this.membershipManager;
     if (mgr == null) {
-      throw new DistributedSystemDisconnectedException(LocalizedStrings.DistributionChannel_I_NO_LONGER_HAVE_A_MEMBERSHIP_ID.toLocalizedString(), membershipManager.getShutdownCause());
+      throw new DistributedSystemDisconnectedException(LocalizedStrings.DistributionChannel_I_NO_LONGER_HAVE_A_MEMBERSHIP_ID.toLocalizedString());
     }
     InternalDistributedMember moi = mgr.getLocalMember();
     if (moi == null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
index 837194c..5d3bdce 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
@@ -593,7 +593,7 @@ public class DistributionManager
             }
           }
         }
-        dm.addNewMember(id, null); // add ourselves
+        dm.addNewMember(id); // add ourselves
         dm.selectElder(); // ShutdownException could be thrown here
       }
 
@@ -1440,7 +1440,7 @@ public class DistributionManager
       // Add them all to our view
       Iterator<InternalDistributedMember> it = v.getMembers().iterator();
       while (it.hasNext()) {
-        addNewMember(it.next(), null);
+        addNewMember(it.next());
       }
       
       // Figure out who the elder is...
@@ -1610,15 +1610,6 @@ public class DistributionManager
   }
 
   /**
-   * Returns a remote reference to the channel used for point-to-point
-   * communications, or null if the normal channel is being used for
-   * this.
-   */
-  protected Stub getDirectChannel() {
-    return membershipManager.getDirectChannel();
-  }
-
-  /**
    * Returns an unmodifiable set containing the identities of all of
    * the known (non-admin-only) distribution managers.
    */
@@ -1820,16 +1811,16 @@ public class DistributionManager
     }
   }
 
-  public void addNewMember(InternalDistributedMember member, Stub stub) {
+  public void addNewMember(InternalDistributedMember member) {
     // This is the place to cleanup the zombieMembers
     int vmType = member.getVmKind();
     switch (vmType) {
       case ADMIN_ONLY_DM_TYPE:
-        handleConsoleStartup(member, stub);
+        handleConsoleStartup(member);
         break;
       case LOCATOR_DM_TYPE:
       case NORMAL_DM_TYPE:
-        handleManagerStartup(member, stub);
+        handleManagerStartup(member);
         break;        
       default:
         throw new InternalGemFireError(LocalizedStrings.DistributionManager_UNKNOWN_MEMBER_TYPE_0.toLocalizedString(Integer.valueOf(vmType)));
@@ -2795,7 +2786,7 @@ public class DistributionManager
         if (unresponsiveElder) {
           logger.warn(LocalizedMessage.create(
               LocalizedStrings.DistributionManager_FORCING_AN_ELDER_JOIN_EVENT_SINCE_A_STARTUP_RESPONSE_WAS_NOT_RECEIVED_FROM_ELDER__0_, e));
-          handleManagerStartup(e, null/*stub already registered*/);
+          handleManagerStartup(e);
         }
       } // an elder exists
     } // someone didn't reply
@@ -3104,7 +3095,7 @@ public class DistributionManager
    *        The id of the distribution manager starting up
    *
    */
-  private void handleManagerStartup(InternalDistributedMember theId, Stub directChannel) {
+  private void handleManagerStartup(InternalDistributedMember theId) {
     HashMap<InternalDistributedMember,InternalDistributedMember> tmp = null;
     synchronized (this.membersLock) {
       // Note test is under membersLock
@@ -3150,7 +3141,7 @@ public class DistributionManager
    * the distributed cache.
    *
    */
-  private void handleConsoleStartup(InternalDistributedMember theId, Serializable directChannel) {
+  private void handleConsoleStartup(InternalDistributedMember theId) {
     // if we have an all listener then notify it NOW.
     HashSet tmp = null;
     synchronized (this.membersLock) {
@@ -4400,12 +4391,12 @@ public class DistributionManager
       handleIncomingDMsg(message);
     }
 
-    public void newMemberConnected(InternalDistributedMember member, Stub stub) {
+    public void newMemberConnected(InternalDistributedMember member) {
       // Do not elect the elder here as surprise members invoke this callback
       // without holding the view lock.  That can cause a race condition and
       // subsequent deadlock (#45566).  Elder selection is now done when a view
       // is installed.
-      dm.addNewMember(member, stub);
+      dm.addNewMember(member);
     }
 
     public void memberDeparted(InternalDistributedMember theId, boolean crashed, String reason) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/HighPriorityAckedMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/HighPriorityAckedMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/HighPriorityAckedMessage.java
index cf0a856..66bd9bb 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/HighPriorityAckedMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/HighPriorityAckedMessage.java
@@ -69,7 +69,9 @@ public final class HighPriorityAckedMessage extends HighPriorityDistributionMess
     if (ds != null) {
       this.originDm = (DistributionManager)ds.getDistributionManager();
     }
-    this.id = this.originDm.getDistributionManagerId();
+    if (this.originDm != null) {
+      this.id = this.originDm.getDistributionManagerId();
+    }
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
index 612d3fa..96f8b60 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
@@ -46,8 +46,6 @@ import com.gemstone.gemfire.internal.tcp.Stub;
 public final class StartupMessage extends HighPriorityDistributionMessage implements AdminMessageType {
   private static final Logger logger = LogService.getLogger();
 
-  /** A stub for the direct channel for this manager */
-  private Stub directChannel;
   private String version = GemFireVersion.getGemFireVersion(); // added for bug 29005
   private int replyProcessorId;
   private boolean isMcastEnabled;
@@ -102,13 +100,6 @@ public final class StartupMessage extends HighPriorityDistributionMessage implem
   ///////////////////////  Instance Methods  ///////////////////////
   
   /**
-   * Sets the id of the distribution manager that is starting up
-   */
-  void setDirectChannel(Stub directChannel) {
-    this.directChannel = directChannel;
-  }
-
-  /**
    * Sets the reply processor for this message
    */
   void setReplyProcessorId(int proc) {
@@ -319,10 +310,6 @@ public final class StartupMessage extends HighPriorityDistributionMessage implem
   @Override
   public void toData(DataOutput out) throws IOException {
     super.toData(out);
-    out.writeBoolean(this.directChannel != null);
-    if (this.directChannel != null) {
-      InternalDataSerializer.invokeToData(this.directChannel, out);
-    }
 
     boolean pre9_0_0_0 = InternalDataSerializer.
         getVersionForDataStream(out).compareTo(Version.GFE_90) < 0;
@@ -391,12 +378,6 @@ public final class StartupMessage extends HighPriorityDistributionMessage implem
   public void fromData(DataInput in)
     throws IOException, ClassNotFoundException {
     super.fromData(in);
-    boolean hasDirectChannel = in.readBoolean();
-    if (hasDirectChannel) {
-      this.directChannel = Stub.createFromData(in);
-    } else {
-      this.directChannel = null;
-    }
 
     boolean pre9_0_0_0 = InternalDataSerializer.
         getVersionForDataStream(in).compareTo(Version.GFE_90) < 0;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupOperation.java
index 6d0ccb7..721f95b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupOperation.java
@@ -65,7 +65,6 @@ public class StartupOperation {
     msg.setDistributedSystemId(dm.getConfig().getDistributedSystemId());
     msg.setRedundancyZone(redundancyZone);
     msg.setEnforceUniqueZone(enforceUniqueZone);
-    msg.setDirectChannel(dm.getDirectChannel());
     msg.setMcastEnabled(transport.isMcastEnabled());
     msg.setMcastPort(dm.getSystem().getOriginalConfig().getMcastPort());
     msg.setMcastHostAddress(dm.getSystem().getOriginalConfig().getMcastAddress());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
index fb7572a..f84813e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
@@ -91,7 +91,7 @@ public class DirectChannel {
     private volatile boolean disconnectCompleted = true;
 
     /** this is the DistributionManager, most of the time */
-    private final DistributedMembershipListener receiver;
+    private final DirectChannelListener receiver;
 
     private final InetAddress address;
     
@@ -142,17 +142,17 @@ public class DirectChannel {
 
     /**
      * @param mgr
-     * @param dm
+     * @param listener
      * @param dc
      * @param unused
      * @throws ConnectionException
      */
-    public DirectChannel(MembershipManager mgr, DistributedMembershipListener dm,
-        DistributionConfig dc, Properties unused) 
+    public DirectChannel(MembershipManager mgr, DirectChannelListener listener,
+        DistributionConfig dc) 
         throws ConnectionException {
-      this.receiver = dm;
+      this.receiver = listener;
 
-      this.address = initAddress(dm, dc);
+      this.address = initAddress(dc);
       boolean isBindAddress = dc.getBindAddress() != null;
       try {
         int port = Integer.getInteger("tcpServerPort", 0).intValue();
@@ -889,7 +889,7 @@ public class DirectChannel {
   }
 
   /** returns the receiver to which this DirectChannel is delivering messages */
-  protected DistributedMembershipListener getReceiver() {
+  protected DirectChannelListener getReceiver() {
     return receiver;
   }
 
@@ -909,7 +909,7 @@ public class DirectChannel {
     return this.conduit;
   }
 
-  private InetAddress initAddress(DistributedMembershipListener dm, DistributionConfig dc) {
+  private InetAddress initAddress(DistributionConfig dc) {
 
     String bindAddress = dc.getBindAddress();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannelListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannelListener.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannelListener.java
new file mode 100755
index 0000000..28f481b
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannelListener.java
@@ -0,0 +1,22 @@
+package com.gemstone.gemfire.distributed.internal.direct;
+
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.DistributionMessage;
+
+public interface DirectChannelListener {
+
+  /**
+   * Event indicating a message has been delivered that we need to process.
+   * 
+   * @param o the message that should be processed.
+   */
+  public void messageReceived(DistributionMessage o);
+  
+
+  /**
+   * Return the distribution manager for this receiver
+   * @return the distribution manager
+   */
+  public DistributionManager getDM();
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/DistributedMembershipListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/DistributedMembershipListener.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/DistributedMembershipListener.java
index 5436b47..acfd6ba 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/DistributedMembershipListener.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/DistributedMembershipListener.java
@@ -21,8 +21,9 @@ import java.util.Set;
 
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
+import com.gemstone.gemfire.distributed.internal.direct.DirectChannelListener;
 
-public interface DistributedMembershipListener {
+public interface DistributedMembershipListener extends DirectChannelListener {
 
   /** this method is invoked when the processing of a new view is completed */
   public void viewInstalled(NetView view);
@@ -33,10 +34,8 @@ public interface DistributedMembershipListener {
   /**
    * Event indicating that a new member has joined the system.
    * @param m the new member
-   * @param stub the stub, if any, representing communication to this member
    */
-  public void newMemberConnected(InternalDistributedMember m, 
-      com.gemstone.gemfire.internal.tcp.Stub stub);
+  public void newMemberConnected(InternalDistributedMember m);
 
   /**
    * Event indicating that a member has left the system
@@ -83,10 +82,4 @@ public interface DistributedMembershipListener {
    */
   public String toString();
   
-  /**
-   * Return the distribution manager for this receiver
-   * @return the distribution manager
-   */
-  public DistributionManager getDM();
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberFactory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberFactory.java
index c8a23e8..250a9a3 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberFactory.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberFactory.java
@@ -38,14 +38,6 @@ public class MemberFactory {
   private static final MemberServices services = new GMSMemberFactory();
 
   /**
-   * Return a blank NetMember (used by externalization)
-   * @return the new NetMember
-   */
-  static public NetMember newNetMember() {
-    return services.newNetMember();
-  }
-  
-  /**
    * Return a new NetMember, possibly for a different host
    * 
    * @param i the name of the host for the specified NetMember, the current host (hopefully)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberServices.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberServices.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberServices.java
index 8fb302e..3fb6ef2 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberServices.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberServices.java
@@ -35,12 +35,6 @@ import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 public interface MemberServices {
 
   /**
-   * Return a blank NetMember (used by externalization)
-   * @return the new NetMember
-   */
-  public abstract NetMember newNetMember();
-  
-  /**
    * Return a new NetMember, possibly for a different host
    * 
    * @param i the name of the host for the specified NetMember, the current host (hopefully)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
index f72e6e2..54b82a7 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
@@ -121,13 +121,6 @@ public interface MembershipManager {
    */
   public void startEventProcessing();
   
-  /**
-   * Return the underlying proxy object, if any, associated with this
-   * local side of this connection.
-   * 
-   * @return the Stub
-   */
-  public Stub getDirectChannel();
   
   /**
    * @param destinations list of members to send the message to.  A list of
@@ -345,10 +338,4 @@ public interface MembershipManager {
    */
   public void releaseQuorumChecker(QuorumChecker checker);
   
-  /**
-   * sets the log writer for authentication logging
-   * @param writer
-   */
-  public void setSecurityLogWriter(InternalLogWriter writer);
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
index e7c9315..f4784ed 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
@@ -93,28 +93,6 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
   }
   
   /**
-   * This is the only constructor to refer to a CacheMember other
-   * than the current host.
-   */
-  public GMSMember(GMSMember m) {
-    udpPort=m.udpPort;
-    preferredForCoordinator=m.preferredForCoordinator;
-    splitBrainEnabled=m.splitBrainEnabled;
-    memberWeight=m.memberWeight;
-    inetAddr=m.inetAddr;
-    processId=m.processId;
-    vmKind=m.vmKind;
-    vmViewId=m.vmViewId;
-    directPort=m.directPort;
-    name=m.name;
-    durableClientAttributes=m.durableClientAttributes;
-    groups=m.groups;
-    versionOrdinal=m.versionOrdinal;
-    uuidLSBs=m.uuidLSBs;
-    uuidMSBs=m.uuidMSBs;
-  }
-
-  /**
    * Create a CacheMember referring to the current host (as defined by
    * the given string).
    * 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberFactory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberFactory.java
index 36311a2..2063d7c 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberFactory.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberFactory.java
@@ -95,17 +95,6 @@ public class GMSMemberFactory implements MemberServices {
     return newNetMember(inetAddr, p);
   }
   
-  /**
-   * Return a new Member
-   * 
-   * Used by externalization only.
-   * 
-   * @return blank member for use with externalization
-   */
-  public NetMember newNetMember() {
-    return new GMSMember();
-  }
-
   public MembershipManager newMembershipManager(DistributedMembershipListener listener,
           DistributionConfig config,
           RemoteTransportConfig transport, DMStats stats) throws DistributionException
@@ -140,4 +129,5 @@ public class GMSMemberFactory implements MemberServices {
     
     return new GMSLocator(bindAddress, stateFile, locatorString, usePreferredCoordinators, networkPartitionDetectionEnabled, stats);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSUtil.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSUtil.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSUtil.java
index df8847c..6478c70 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSUtil.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSUtil.java
@@ -136,5 +136,24 @@ public class GMSUtil {
   }
   
   
+  /** compareTo for InetAddresses */
+  public static int compareAddresses(InetAddress one, InetAddress two) {
+    byte[] oneBytes = one.getAddress();
+    byte[] twoBytes = two.getAddress();
+
+    if (oneBytes != twoBytes) {
+      for (int i = 0; i < oneBytes.length; i++) {
+        if (i >= twoBytes.length)
+          return -1; // same as far as they go, but shorter...
+        if (oneBytes[i] < twoBytes[i])
+          return -1;
+        if (oneBytes[i] > twoBytes[i])
+          return 1;
+      }
+      if (oneBytes.length > twoBytes.length)
+        return 1; // same as far as they go, but longer...
+    }
+    return 0;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
index bd60236..fcda1a0 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
@@ -419,7 +419,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
    * @return
    */
   private boolean doCheckMember(InternalDistributedMember member) {
-    if (playingDead) {
+    if (playingDead || beingSick) {
       // a member playingDead should not be sending messages to other
       // members, so we avoid sending heartbeat requests or suspect
       // messages by returning true.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
index 91c32ae..3a00f62 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
@@ -605,7 +605,7 @@ public class JGroupsMessenger implements Messenger {
         }
       }
       if (problem != null) {
-        if (services.getManager().getShutdownCause() != null) {
+        if (services.getShutdownCause() != null) {
           Throwable cause = services.getShutdownCause();
           // If ForcedDisconnectException occurred then report it as actual
           // problem.
@@ -901,13 +901,6 @@ public class JGroupsMessenger implements Messenger {
   }
   
   /**
-   * returns the JChannel for test verification
-   */
-  public JChannel getJGroupsChannel() {
-    return this.myChannel;
-  }
-  
-  /**
    * for unit testing we need to replace UDP with a fake UDP protocol 
    */
   public void setJGroupsStackConfigForTesting(String config) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index e7c937d..4e108be 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -65,12 +65,14 @@ import com.gemstone.gemfire.distributed.internal.SizeableRunnable;
 import com.gemstone.gemfire.distributed.internal.StartupMessage;
 import com.gemstone.gemfire.distributed.internal.ThrottlingMemLinkedQueueWithDMStats;
 import com.gemstone.gemfire.distributed.internal.direct.DirectChannel;
+import com.gemstone.gemfire.distributed.internal.direct.DirectChannelListener;
 import com.gemstone.gemfire.distributed.internal.membership.DistributedMembershipListener;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipTestHook;
 import com.gemstone.gemfire.distributed.internal.membership.NetView;
 import com.gemstone.gemfire.distributed.internal.membership.QuorumChecker;
+import com.gemstone.gemfire.distributed.internal.membership.gms.GMSUtil;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
 import com.gemstone.gemfire.distributed.internal.membership.gms.SuspectMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.fd.GMSHealthMonitor;
@@ -85,7 +87,6 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheServerCreation;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlGenerator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.log4j.AlertAppender;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
@@ -134,18 +135,16 @@ public class GMSMembershipManager implements MembershipManager, Manager
   }
   
   static class StartupEvent  {
-    static final int DEPARTURE = 1;
-    static final int CONNECT = 2;
-    static final int VIEW = 3;
-    static final int MESSAGE = 4;
+    static final int SURPRISE_CONNECT = 1;
+    static final int VIEW = 2;
+    static final int MESSAGE = 3;
     
     /**
      * indicates whether the event is a departure, a surprise connect
      * (i.e., before the view message arrived), a view, or a regular
      * message
      * 
-     * @see #DEPARTURE
-     * @see #CONNECT
+     * @see #SURPRISE_CONNECT
      * @see #VIEW
      * @see #MESSAGE
      */
@@ -164,20 +163,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       StringBuffer sb = new StringBuffer();
       sb.append("kind=");
       switch (kind) {
-      case DEPARTURE:
-        sb.append("departure; member = <")
-          .append(member)
-          .append(">; crashed = ")
-          .append(crashed)
-          .append("; reason = ");
-        if (reason != null && (reason.indexOf("NoSuchMemberException") >= 0)) {
-          sb.append(LocalizedStrings.GroupMembershipService_TCPIP_CONNECTIONS_CLOSED.toLocalizedString());
-        }
-        else {
-          sb.append(reason);
-        }
-        break;
-      case CONNECT:
+      case SURPRISE_CONNECT:
         sb.append("connect; member = <" + member + ">; stub = " + stub);
         break;
       case VIEW:
@@ -193,42 +179,23 @@ public class GMSMembershipManager implements MembershipManager, Manager
       }
       return sb.toString();
     }
-    /**
-     * Create a departure event
-     * @param dm the member that left
-     * @param crashed true if this member crashed
-     * @param reason reason string, esp. if crashed
-     */
-    StartupEvent(InternalDistributedMember dm, boolean crashed, String reason) {
-      this.kind = DEPARTURE;
-      this.member = dm;
-      this.crashed = crashed;
-      this.reason = reason;
-    }
-    /**
-     * Indicate if this is a departure
-     * @return true if this is a departure event
-     */
-    boolean isDepartureEvent() {
-      return this.kind == DEPARTURE;
-    }
 
     /**
-     * Create a connect event
+     * Create a surprise connect event
      * @param member the member connecting
      * @param id the stub
      */
     StartupEvent(final InternalDistributedMember member, final Stub id) {
-      this.kind = CONNECT;
+      this.kind = SURPRISE_CONNECT;
       this.member = member;
       this.stub = id;
     }
     /**
-     * Indicate if this is a connect event
+     * Indicate if this is a surprise connect event
      * @return true if this is a connect event
      */
-    boolean isConnect() {
-      return this.kind == CONNECT;
+    boolean isSurpriseConnect() {
+      return this.kind == SURPRISE_CONNECT;
     }
 
     /**
@@ -398,11 +365,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
    */
   private final long suspectMemberTimeout = 180000;
   
-  /** sleep period, in millis, that the user of this manager should slumber after creating
-      the manager.  This is advice from the JChannel itself when it detects a concurrent
-      startup race condition that requires a settling period. */
-  private long channelPause = 0;  
-
   /**
    * Length of time, in seconds, that a member is retained in the zombie set
    * 
@@ -451,10 +413,10 @@ public class GMSMembershipManager implements MembershipManager, Manager
    * @author jpenney
    * 
    */
-  class MyDCReceiver implements DistributedMembershipListener
+  class MyDCReceiver implements DirectChannelListener
   {
 
-    DistributedMembershipListener upCall;
+    DirectChannelListener upCall;
     
     /**
      * Don't provide events until the caller has told us we are ready.
@@ -465,60 +427,17 @@ public class GMSMembershipManager implements MembershipManager, Manager
      * client; we don't need to put this check before every call...
      *
      */
-   MyDCReceiver(DistributedMembershipListener up) {
+   MyDCReceiver(DirectChannelListener up) {
       upCall = up;
     }
 
-    public void messageReceived(DistributionMessage msg)
-    {
+    public void messageReceived(DistributionMessage msg) {
       // bug 36851 - notify failure detection that we've had contact from a member
       services.getHealthMonitor().contactedBy(msg.getSender());
       handleOrDeferMessage(msg);
     }
 
-    public void newMemberConnected(final InternalDistributedMember member, final Stub id)
-    {
-      handleOrDeferSurpriseConnect(member, id);
-    }
-
-    public void memberDeparted(InternalDistributedMember id, boolean crashed, String reason)
-    {
-      try {
-        handleOrDeferRemove(id, crashed, reason);
-      }
-      catch (DistributedSystemDisconnectedException ignore) {
-        // ignore
-      }
-      catch (RuntimeException e) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.GroupMembershipService_MEMBERSHIP_ERROR_HANDLING_MEMBER_DEPARTURE__0), e);
-      }
-    }
-    
-    public void quorumLost(Set<InternalDistributedMember> failures, List<InternalDistributedMember> remaining) {
-    }
-
-    public void memberSuspect(InternalDistributedMember suspect, InternalDistributedMember whoSuspected) {
-      // the direct channel isn't currently a source of suspect events, though
-      // it does request initiation of suspicion through the membership
-      // manager
-    }
-
-    public boolean isShutdownMsgSent()
-    {
-      return upCall.isShutdownMsgSent();
-    }
-
-    public void membershipFailure(String reason, Throwable t)
-    {
-      upCall.membershipFailure(reason, t);
-    }
-    
-    public void viewInstalled(NetView view) {
-      upCall.viewInstalled(view);
-    }
-
-    public DistributionManager getDM()
-    {
+    public DistributionManager getDM() {
      return upCall.getDM();
     }
 
@@ -547,9 +466,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
       msg.append("Membership: Processing view ");
       msg.append(newView);
       msg.append("} on " + address.toString());
-      if (logger.isDebugEnabled()) {
-        logger.debug(LogMarker.DM_VIEWS, msg);
-      }
       if (!newView.contains(address)) {
         logger.info(LocalizedMessage.create(
             LocalizedStrings.GroupMembershipService_THE_MEMBER_WITH_ID_0_IS_NO_LONGER_IN_MY_OWN_VIEW_1,
@@ -584,10 +500,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
       
       if (newViewId < latestViewId) {
         // ignore this view since it is old news
-        if (newViewId < latestViewId && logger.isDebugEnabled(LogMarker.DISTRIBUTION_VIEWS)) {
-          logger.debug(LogMarker.DISTRIBUTION_VIEWS, "Membership: Ignoring view (with id {}) since it is older than the last view (with id {}); ignoredView={}",
-              newViewId, latestViewId, newView);
-        }
         return;
       }
 
@@ -641,8 +553,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
         }
 
         if (shutdownInProgress()) {
-          logger.info(LogMarker.DM_VIEWS, LocalizedMessage.create(
-              LocalizedStrings.GroupMembershipService_MEMBERSHIP_SHUNNING_MEMBER__0__DURING_OUR_SHUTDOWN, m));
           addShunnedMember(m);
           continue; // no additions processed after shutdown begins
         } else {
@@ -655,7 +565,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
         logger.info(LocalizedMessage.create(LocalizedStrings.GroupMembershipService_MEMBERSHIP_PROCESSING_ADDITION__0_, m));
 
         try {
-          listener.newMemberConnected(m, getStubForMember(m));
+          listener.newMemberConnected(m);
         }
         catch (VirtualMachineError err) {
           SystemFailure.initiateFailure(err);
@@ -690,8 +600,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
         }
 
         try {
-          logger.info(LogMarker.DM_VIEWS, LocalizedMessage.create(
-              LocalizedStrings.GroupMembershipService_MEMBERSHIP_PROCESSING_DEPARTING_MEMBER__0_, m));
           removeWithViewLock(m,
               newView.getCrashedMembers().contains(m) || suspectedMembers.containsKey(m)
               , "departed membership view");
@@ -741,9 +649,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
         if (birthtime.longValue() < oldestAllowed) {
           InternalDistributedMember m = (InternalDistributedMember)entry.getKey();
           it.remove();
-          if (logger.isTraceEnabled(LogMarker.DISTRIBUTION_VIEWS)) {
-            logger.trace(LogMarker.DISTRIBUTION_VIEWS, "Membership: expiring suspect member <{}>", m);
-          }
         }
       }
       try {
@@ -755,8 +660,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     } finally {
       latestViewLock.writeLock().unlock();
     }
-    logger.info(LogMarker.DM_VIEWS, LocalizedMessage.create(
-        LocalizedStrings.GroupMembershipService_MEMBERSHIP_FINISHED_VIEW_PROCESSING_VIEWID___0, Long.valueOf(newViewId)));
   }
 
   /**
@@ -869,7 +772,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
 
     int dcPort = 0;
     if (!tcpDisabled) {
-      directChannel = new DirectChannel(this, dcReceiver, config, null);
+      directChannel = new DirectChannel(this, dcReceiver, config);
       dcPort = directChannel.getPort();
     }
 
@@ -971,23 +874,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
   }
   
   /**
-   * Remove a member, or queue a startup operation to do so
-   * @param dm the member to shun
-   * @param crashed true if crashed
-   * @param reason the reason, esp. if crashed
-   */
-  protected void handleOrDeferRemove(InternalDistributedMember dm,
-      boolean crashed, String reason) {
-    synchronized(startupLock) {
-      if (!processingEvents) {
-        startupMessages.add(new StartupEvent(dm, crashed, reason));
-        return;
-      }
-    }
-    removeMember(dm, crashed, reason);
-  }
-  
-  /**
    * Remove a member.  {@link #latestViewLock} must be held
    * before this method is called.  If member is not already shunned,
    * the uplevel event handler is invoked.
@@ -1007,10 +893,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
       return; // Explicit deletion, no upcall.
     }
     
-    if (logger.isTraceEnabled(LogMarker.DISTRIBUTION_VIEWS)) {
-      logger.trace(LogMarker.DISTRIBUTION_VIEWS, "Membership: dispatching uplevel departure event for < {} >", dm);
-    }
-    
     try {
       listener.memberDeparted(dm, crashed, reason);
     }
@@ -1020,48 +902,12 @@ public class GMSMembershipManager implements MembershipManager, Manager
   }
   
   /**
-   * Automatic removal of a member (for internal
-   * use only).  Write-locks {@link #latestViewLock} and then deletes
-   * the member.
-   * 
-   * @param dm
-   * @param crashed
-   * @param reason
-   */
-  protected void removeMember(InternalDistributedMember dm,
-      boolean crashed, String reason)
-  {
-    if (logger.isTraceEnabled(LogMarker.DISTRIBUTION_VIEWS)) {
-      StringBuffer sb = new StringBuffer(200);
-      sb.append("Membership: removing <")
-         .append(dm)
-         .append(">; crashed = ")
-         .append(crashed)
-         .append("; reason = ");
-      if (reason != null && (reason.indexOf("NoSuchMemberException") >= 0)) {
-        sb.append("tcp/ip connections closed");
-      }
-      else {
-        sb.append(reason);
-      }
-      logger.trace(LogMarker.DISTRIBUTION_VIEWS, sb);
-    }
-    latestViewLock.writeLock().lock();
-    try {
-      removeWithViewLock(dm, crashed, reason);
-    } finally {
-      latestViewLock.writeLock().unlock();
-    }
-  }
-  
- 
-  /**
    * Process a surprise connect event, or place it on the startup queue.
    * @param member the member
    * @param stub its stub
    */
-  protected void handleOrDeferSurpriseConnect(InternalDistributedMember member,
-      Stub stub) {
+  protected void handleOrDeferSurpriseConnect(InternalDistributedMember member) {
+    Stub stub = new Stub(member.getInetAddress(), member.getDirectChannelPort(), member.getVmViewId());
     synchronized (startupLock) {
       if (!processingEvents) {
         startupMessages.add(new StartupEvent(member, stub));
@@ -1113,7 +959,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
         return true;
       }
       if (member.getVmViewId() < 0) {
-        logger.warn("attempt to add a surprise member that has not yet joined the distributed system: " + member, new Exception("stack trace"));
+        logger.warn("adding a surprise member that has not yet joined the distributed system: " + member, new Exception("stack trace"));
       }
       if (latestView.getViewId() > member.getVmViewId()) {
         // tell the process that it should shut down distribution.
@@ -1133,20 +979,11 @@ public class GMSMembershipManager implements MembershipManager, Manager
         return false;
       }
 
-      if (logger.isTraceEnabled(LogMarker.DISTRIBUTION_VIEWS)) {
-        logger.trace(LogMarker.DISTRIBUTION_VIEWS, "Membership: Received message from surprise member: <{}>.  My view number is {} it is {}", 
-            member, latestView.getViewId(), member.getVmViewId());
-      }
-
       // Adding him to this set ensures we won't remove him if a new
       // view comes in and he's still not visible.
       surpriseMembers.put(member, Long.valueOf(System.currentTimeMillis()));
 
       if (shutdownInProgress()) {
-        if (logger.isTraceEnabled(LogMarker.DISTRIBUTION_VIEWS)) {
-          logger.trace(LogMarker.DISTRIBUTION_VIEWS, "Membership: new member during shutdown ignored: <{}>", member); 
-        }
-
         // Force disconnect, esp. the TCPConduit
         String msg = LocalizedStrings.GroupMembershipService_THIS_DISTRIBUTED_SYSTEM_IS_SHUTTING_DOWN.toLocalizedString();
         if (directChannel != null) {
@@ -1166,10 +1003,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
       } else {
 
         // Now that we're sure the member is new, add them.
-        if (logger.isTraceEnabled(LogMarker.DM_VIEWS)) {
-          logger.trace(LogMarker.DM_VIEWS, "Membership: Processing surprise addition <{}>", member);
-        }
-
         // make sure the surprise-member cleanup task is running
         if (this.cleanupTimer == null) {
           startCleanupTimer();
@@ -1206,7 +1039,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       logger.warn(LocalizedMessage.create(
           LocalizedStrings.GroupMembershipService_MEMBERSHIP_IGNORING_SURPRISE_CONNECT_FROM_SHUNNED_MEMBER_0, member));
     } else {
-      listener.newMemberConnected(member, s);
+      listener.newMemberConnected(member);
     }
     return !warn;
   }
@@ -1364,10 +1197,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
       // to avoid blocking a reader thread
       NetView newView = viewArg;
       long newId = viewArg.getViewId();
-      if (logger.isTraceEnabled(LogMarker.DM_VIEWS)) {
-        logger.trace(LogMarker.DM_VIEWS, "Membership: queuing new view for processing, id = {}, view = {}", 
-            newId, newView);
-      }
       LocalViewMessage v = new LocalViewMessage(address, newId, newView,
           GMSMembershipManager.this);
 
@@ -1445,10 +1274,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     else if (o.isGmsView()) { // view event
       processView(o.gmsView.getViewId(), o.gmsView);
     }
-    else if (o.isDepartureEvent()) { // departure
-      removeMember(o.member, o.crashed, o.reason);
-    }
-    else if (o.isConnect()) { // connect
+    else if (o.isSurpriseConnect()) { // connect
       processSurpriseConnect(o.member, o.stub);
     }
     
@@ -1552,6 +1378,12 @@ public class GMSMembershipManager implements MembershipManager, Manager
     }
   }
 
+  /**
+   * for testing we need to validate the startup event list
+   */
+  public List<StartupEvent> getStartupEvents() {
+    return this.startupMessages;
+  }
 
   public ReadWriteLock getViewLock() {
     return this.latestViewLock;
@@ -1639,19 +1471,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     return services;
   }
 
-  public void postConnect()
-  {
-    if (channelPause > 0) {
-      logger.info(LocalizedMessage.create(
-          LocalizedStrings.GroupMembershipService_MEMBERSHIP_PAUSING_TO_ALLOW_OTHER_CONCURRENT_PROCESSES_TO_JOIN_THE_DISTRIBUTED_SYSTEM));
-      try {
-        Thread.sleep(channelPause);
-      }
-      catch (InterruptedException ie) {
-        Thread.currentThread().interrupt();
-      }
-      channelPause = 0;
-    }
+  public void postConnect() {
   }
   
   /**
@@ -1936,7 +1756,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
    * all received it)
    * @throws NotSerializableException if the message is not serializable
    */
-  private Set<InternalDistributedMember> directChannelSend(InternalDistributedMember[] destinations,
+  protected Set<InternalDistributedMember> directChannelSend(InternalDistributedMember[] destinations,
       DistributionMessage content,
       DMStats theStats)
       throws NotSerializableException
@@ -2122,7 +1942,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     
     if (msg instanceof AdminMessageType
         && this.shutdownInProgress) {
-      // no alerts while shutting down - this can cause threads to hang
+      // no admin messages while shutting down - this can cause threads to hang
       return new HashSet(Arrays.asList(msg.getRecipients()));
     }
 
@@ -2243,7 +2063,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       if (result == null) {
         // it may have not been added to the stub->idm map yet, so check the current view
         for (InternalDistributedMember idm: latestView.getMembers()) {
-          if (idm.getInetAddress().equals(s.getInetAddress())
+          if (GMSUtil.compareAddresses(idm.getInetAddress(), s.getInetAddress()) == 0
               && idm.getDirectChannelPort() == s.getPort()) {
             addChannel(idm, s);
             return idm;
@@ -2298,8 +2118,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
    */
   protected void destroyMember(final InternalDistributedMember member,
       boolean crashed, final String reason) {
-    if (logger.isTraceEnabled(LogMarker.DISTRIBUTION_VIEWS))
-      logger.trace(LogMarker.DISTRIBUTION_VIEWS, "Membership: destroying < {} >", member);
     
     // Clean up the maps
     Stub theChannel = (Stub)memberToStubMap.remove(member);
@@ -2362,18 +2180,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     }
   }
   
-  public Stub getDirectChannel()
-  {
-    Stub result;
-    latestViewLock.readLock().lock();
-    try {
-      result = (Stub)memberToStubMap.get(address);
-    } finally {
-      latestViewLock.readLock().unlock();
-    }
-    return result;
-  }
-
   /**
    * Indicate whether the given member is in the zombie list (dead or dying)
    * @param m the member in question
@@ -2398,9 +2204,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
         return true;
       
       // Oh, it _is_ stale.  Remove it while we're here.
-      if (logger.isTraceEnabled(LogMarker.DISTRIBUTION_VIEWS)) {
-        logger.debug("Membership: no longer shunning <  {} >", m);
-      }
       endShun(m);
       return false;
     } finally {
@@ -2459,14 +2262,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     return this.surpriseMemberTimeout;
   }
   
-  /**
-   * returns the shunned member shunset interval, in milliseconds
-   */
-  public int getShunnedMemberTimeout() {
-    return SHUNNED_SUNSET * 1000;
-  }
-  
-
   private boolean endShun(DistributedMember m) {
     boolean wasShunned = (shunnedMembers.remove(m) != null);
     shunnedAndWarnedMembers.remove(m);
@@ -2490,8 +2285,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     // Update the shunned set.
     if (!isShunned(m)) {
       shunnedMembers.put(m, Long.valueOf(System.currentTimeMillis()));
-      if (logger.isTraceEnabled(LogMarker.DISTRIBUTION_VIEWS))
-        logger.trace(LogMarker.DISTRIBUTION_VIEWS, "Membership: added shunned member < {} >", m);
     }
 
     // Remove really really old shunned members.
@@ -2535,13 +2328,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
   }
   
   /**
-   * Set thread-local data for hydra
-   */
-  public void setThreadLocalData(Object data) {
-    Map dataMap = (Map)data;
-  }
-  
-  /**
    * for testing verification purposes, this return the port for the
    * direct channel, or zero if there is no direct
    * channel
@@ -2550,6 +2336,14 @@ public class GMSMembershipManager implements MembershipManager, Manager
     return directChannel == null? 0 : directChannel.getPort();
   }
   
+  /**
+   * for mock testing this allows insertion of a DirectChannel mock
+   */
+  protected void setDirectChannel(DirectChannel dc) {
+    this.directChannel = dc;
+    this.tcpDisabled = false;
+  }
+  
   /* non-thread-owned serial channels and high priority channels are not
    * included
    */
@@ -2664,6 +2458,8 @@ public class GMSMembershipManager implements MembershipManager, Manager
   }
   
 
+  // TODO remove this overly complex method and replace its use with
+  // waitForViewChange using the remote member's view ID
   public boolean waitForMembershipCheck(InternalDistributedMember remoteId) {
     boolean foundRemoteId = false;
     CountDownLatch currentLatch = null;
@@ -2689,8 +2485,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     }
 
     if (!foundRemoteId) {
-      // ARB: wait for hardcoded 1000 ms for latch to open.
-      // if-stmt precondition: currentLatch is non-null
       try {
         if (currentLatch.await(membershipCheckTimeout, TimeUnit.MILLISECONDS)) {
           foundRemoteId = true;
@@ -2783,9 +2577,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
   public synchronized void beSick() {
     if (!beingSick) {
       beingSick = true;
-      if (logger.isInfoEnabled()) {
-        logger.info("GroupMembershipService.beSick invoked for {} - simulating sickness", this.address);
-      }
+      logger.info("GroupMembershipService.beSick invoked for {} - simulating sickness", this.address);
       services.getJoinLeave().beSick();
       services.getHealthMonitor().beSick();
       if (directChannel != null) {
@@ -2800,9 +2592,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
   public synchronized void playDead() {
     if (!playingDead) {
       playingDead = true;
-      if (logger.isDebugEnabled()) {
-        logger.debug("GroupMembershipService.playDead invoked for {}", this.address);
-      }
+      logger.info("GroupMembershipService.playDead invoked for {}", this.address);
       services.getJoinLeave().playDead();
       services.getHealthMonitor().playDead();
       services.getMessenger().playDead();
@@ -2816,9 +2606,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     if (beingSick || playingDead) {
       beingSick = false;
       playingDead = false;
-      if (logger.isDebugEnabled()) {
-        logger.debug("GroupMembershipService.beHealthy invoked for {} - recovering health now", this.address);
-      }
+      logger.info("GroupMembershipService.beHealthy invoked for {} - recovering health now", this.address);
       if (directChannel != null) {
         directChannel.beHealthy();
       }
@@ -2955,11 +2743,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     services.getJoinLeave().disableDisconnectOnQuorumLossForTesting();
   }
 
-  @Override
-  public void setSecurityLogWriter(InternalLogWriter writer) {
-    Services.setSecurityLogWriter(writer);
-  }
-
 
   /**
    * Class <code>BoundedLinkedHashMap</code> is a bounded
@@ -2980,29 +2763,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     /**
      * Constructor.
      *
-     * @param initialCapacity The initial capacity.
-     * @param loadFactor The load factor
-     * @param maximumNumberOfEntries The maximum number of allowed entries
-     */
-    public BoundedLinkedHashMap(int initialCapacity, float loadFactor, int maximumNumberOfEntries) {
-      super(initialCapacity, loadFactor);
-      this._maximumNumberOfEntries = maximumNumberOfEntries;
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param initialCapacity The initial capacity.
-     * @param maximumNumberOfEntries The maximum number of allowed entries
-     */
-    public BoundedLinkedHashMap(int initialCapacity, int maximumNumberOfEntries) {
-      super(initialCapacity);
-      this._maximumNumberOfEntries = maximumNumberOfEntries;
-    }
-
-    /**
-     * Constructor.
-     *
      * @param maximumNumberOfEntries The maximum number of allowed entries
      */
     public BoundedLinkedHashMap(int maximumNumberOfEntries) {
@@ -3010,14 +2770,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
       this._maximumNumberOfEntries = maximumNumberOfEntries;
     }
 
-    /**
-     * Returns the maximum number of entries.
-     * @return the maximum number of entries
-     */
-    public int getMaximumNumberOfEntries(){
-      return this._maximumNumberOfEntries;
-    }
-
     @Override
     protected boolean removeEldestEntry(Map.Entry entry) {
       return size() > this._maximumNumberOfEntries;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/LogMarker.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/LogMarker.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/LogMarker.java
index 04b4223..3b95ca6 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/LogMarker.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/LogMarker.java
@@ -73,7 +73,6 @@ public interface LogMarker {
   public static final Marker DISTRIBUTION_BRIDGE_SERVER = MarkerManager.getMarker("DISTRIBUTION_BRIDGE_SERVER").addParents(DISTRIBUTION, BRIDGE_SERVER);
   public static final Marker DISTRIBUTION_VIEWS = MarkerManager.getMarker("DISTRIBUTION_VIEWS").addParents(DISTRIBUTION);
   public static final Marker DM = MarkerManager.getMarker("DM").addParents(DISTRIBUTION);
-  public static final Marker DM_VIEWS = MarkerManager.getMarker("DM_VIEWS").addParents(DM, DISTRIBUTION_VIEWS);
   public static final Marker DM_BRIDGE_SERVER = MarkerManager.getMarker("DM_BRIDGE").addParents(BRIDGE_SERVER, DM);
   public static final Marker DA = MarkerManager.getMarker("DA").addParents(DISTRIBUTION);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
index c592133..30962e7 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
@@ -771,6 +771,13 @@ public class Connection implements Runnable {
             String peerName;
             if (this.remoteAddr != null) {
               peerName = this.remoteAddr.toString();
+              // late in the life of jdk 1.7 we started seeing connections accepted
+              // when accept() was not even being called.  This started causing timeouts
+              // to occur in the handshake threads instead of causing failures in
+              // connection-formation.  So, we need to initiate suspect processing here
+              owner.getDM().getMembershipManager().suspectMember(this.remoteAddr,
+                  LocalizedStrings.Connection_CONNECTION_HANDSHAKE_WITH_0_TIMED_OUT_AFTER_WAITING_1_MILLISECONDS.toLocalizedString(
+                      new Object[] {peerName, Integer.valueOf(HANDSHAKE_TIMEOUT_MS)}));
             }
             else {
               peerName = "socket " + this.socket.getRemoteSocketAddress().toString()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
index f4fab74..12a03fd 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
@@ -645,9 +645,9 @@ public class TCPConduit implements Runnable {
           continue;
         }
         if (inhibitNewConnections) {
-          if (logger.isTraceEnabled(LogMarker.QA)) {
-            logger.trace(LogMarker.QA, "Test hook: inhibiting acceptance of connection {}", othersock);
-          }
+//          if (logger.isTraceEnabled(LogMarker.QA)) {
+            logger.info("Test hook: inhibiting acceptance of connection {}", othersock);
+//          }
           othersock.close();
           while (inhibitNewConnections && !stopped) {
             this.stopper.checkCancelInProgress(null);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
index fd5aaa7..91889df 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
@@ -19,6 +19,10 @@ package com.gemstone.gemfire.distributed.internal.membership;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+
 import java.io.File;
 import java.net.InetAddress;
 import java.util.ArrayList;
@@ -43,6 +47,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.membership.gms.GMSUtil;
 import com.gemstone.gemfire.distributed.internal.membership.gms.ServiceConfig;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.JoinLeave;
@@ -54,13 +59,9 @@ import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 @Category(IntegrationTest.class)
-public class MembershipJUnitTest extends TestCase {
+public class MembershipJUnitTest {
   static Level baseLogLevel;
   
-  public MembershipJUnitTest(String name) {
-    super(name);
-  }
-
   @BeforeClass
   public static void setupClass() {
 //    baseLogLevel = LogService.getBaseLogLevel();
@@ -68,7 +69,7 @@ public class MembershipJUnitTest extends TestCase {
   }
   
   @AfterClass
-  protected void tearDown() throws Exception {
+  public static void tearDown() throws Exception {
 //    LogService.setBaseLogLevel(baseLogLevel);
   }
   
@@ -263,11 +264,25 @@ public class MembershipJUnitTest extends TestCase {
     GMSJoinLeave joinLeave = new GMSJoinLeave();
     try {
       joinLeave.init(services);
-      fail("expected a GemFireConfigException to be thrown because no locators are configured");
+      throw new Error("expected a GemFireConfigException to be thrown because no locators are configured");
     } catch (GemFireConfigException e) {
       // expected
     }
   }
   
+  /**
+   * test the GMSUtil.formatBytes() method
+   */
+  @Test
+  public void testFormatBytes() throws Exception {
+    byte[] bytes = new byte[200];
+    for (int i=0; i<bytes.length; i++) {
+      bytes[i] = (byte)(i%255);
+    }
+    String str = GMSUtil.formatBytes(bytes, 0, bytes.length);
+    System.out.println(str);
+    assertEquals(600+4, str.length());
+  }
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
index 2d042fc..7badce6 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
@@ -152,8 +152,8 @@ public class GMSLocatorRecoveryJUnitTest {
       nonDefault.put(DistributionConfig.MCAST_PORT_NAME, "0");
       nonDefault.put(DistributionConfig.LOG_FILE_NAME, "");
       nonDefault.put(DistributionConfig.LOG_LEVEL_NAME, "fine");
-      nonDefault.put(DistributionConfig.LOCATORS_NAME, localHost.getHostAddress()+'['+port+']');
-      nonDefault.put(DistributionConfig.BIND_ADDRESS_NAME, localHost.getHostAddress());
+      nonDefault.put(DistributionConfig.LOCATORS_NAME, localHost.getHostName()+'['+port+']');
+      nonDefault.put(DistributionConfig.BIND_ADDRESS_NAME, localHost.getHostName());
       DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
       RemoteTransportConfig transport = new RemoteTransportConfig(config,
           DistributionManager.NORMAL_DM_TYPE);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/38dd3ed8/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
new file mode 100755
index 0000000..2b59ca5
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
@@ -0,0 +1,370 @@
+/*
+ * 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.distributed.internal.membership.gms.mgr;
+
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.Timer;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.internal.AdminMessageType;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.DistributionMessage;
+import com.gemstone.gemfire.distributed.internal.HighPriorityAckedMessage;
+import com.gemstone.gemfire.distributed.internal.HighPriorityDistributionMessage;
+import com.gemstone.gemfire.distributed.internal.direct.DirectChannel;
+import com.gemstone.gemfire.distributed.internal.membership.DistributedMembershipListener;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.distributed.internal.membership.NetView;
+import com.gemstone.gemfire.distributed.internal.membership.gms.ServiceConfig;
+import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
+import com.gemstone.gemfire.distributed.internal.membership.gms.Services.Stopper;
+import com.gemstone.gemfire.distributed.internal.membership.gms.SuspectMember;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Authenticator;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.HealthMonitor;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.JoinLeave;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Messenger;
+import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
+import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
+import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager.StartupEvent;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.admin.remote.AdminRequest;
+import com.gemstone.gemfire.internal.admin.remote.AdminResponse;
+import com.gemstone.gemfire.internal.admin.remote.AlertListenerMessage;
+import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
+import com.gemstone.gemfire.internal.tcp.ConnectExceptions;
+import com.gemstone.gemfire.internal.tcp.Stub;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class GMSMembershipManagerJUnitTest {
+  private Services services;
+  private ServiceConfig mockConfig;
+  private DistributionConfig distConfig;
+  private Authenticator authenticator;
+  private HealthMonitor healthMonitor;
+  private InternalDistributedMember myMemberId;
+  private InternalDistributedMember[] mockMembers;
+  private Messenger messenger;
+  private JoinLeave joinLeave;
+  private Stopper stopper;
+  DistributedMembershipListener listener;
+  private GMSMembershipManager manager;
+  private List<InternalDistributedMember> members;
+  private Set<InternalDistributedMember> emptyMembersSet = new HashSet<>();
+  private DirectChannel dc;
+
+  @Before
+  public void initMocks() throws Exception {
+    Properties nonDefault = new Properties();
+    nonDefault.put(DistributionConfig.DISABLE_TCP_NAME, "true");
+    nonDefault.put(DistributionConfig.MCAST_PORT_NAME, "0");
+    nonDefault.put(DistributionConfig.MCAST_TTL_NAME, "0");
+    nonDefault.put(DistributionConfig.LOG_FILE_NAME, "");
+    nonDefault.put(DistributionConfig.LOG_LEVEL_NAME, "fine");
+    nonDefault.put(DistributionConfig.MEMBER_TIMEOUT_NAME, "2000");
+    nonDefault.put(DistributionConfig.LOCATORS_NAME, "localhost[10344]");
+    distConfig = new DistributionConfigImpl(nonDefault);
+    RemoteTransportConfig tconfig = new RemoteTransportConfig(distConfig,
+        DistributionManager.NORMAL_DM_TYPE);
+    
+
+    mockConfig = mock(ServiceConfig.class);
+    when(mockConfig.getDistributionConfig()).thenReturn(distConfig);
+    when(mockConfig.getTransport()).thenReturn(tconfig);
+    
+    authenticator = mock(Authenticator.class);
+    myMemberId = new InternalDistributedMember("localhost", 8887);
+    
+    messenger = mock(Messenger.class);
+    when(messenger.getMemberID()).thenReturn(myMemberId);
+
+    stopper = mock(Stopper.class);
+    when(stopper.isCancelInProgress()).thenReturn(false);
+    
+    healthMonitor = mock(HealthMonitor.class);
+    when(healthMonitor.getFailureDetectionPort()).thenReturn(Integer.valueOf(-1));
+    
+    joinLeave = mock(JoinLeave.class);
+    
+    services = mock(Services.class);
+    when(services.getAuthenticator()).thenReturn(authenticator);
+    when(services.getConfig()).thenReturn(mockConfig);
+    when(services.getMessenger()).thenReturn(messenger);
+    when(services.getCancelCriterion()).thenReturn(stopper);
+    when(services.getHealthMonitor()).thenReturn(healthMonitor);
+    when(services.getJoinLeave()).thenReturn(joinLeave);
+    
+    Timer t = new Timer(true);
+    when(services.getTimer()).thenReturn(t);
+    
+    mockMembers = new InternalDistributedMember[5];
+    for (int i = 0; i < mockMembers.length; i++) {
+      mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
+    }
+    members = new ArrayList<>(Arrays.asList(mockMembers));
+
+    listener = mock(DistributedMembershipListener.class);
+    
+    manager = new GMSMembershipManager(listener);
+    manager.init(services);
+    when(services.getManager()).thenReturn(manager);
+  }
+  
+  @After
+  public void tearDown() throws Exception {
+    if (manager != null) {
+      manager.stop();
+      manager.stopped();
+    }
+  }
+  
+  @Test
+  public void testSendMessage() throws Exception {
+    HighPriorityAckedMessage m = new HighPriorityAckedMessage();
+    m.setRecipient(mockMembers[0]);
+    manager.start();
+    manager.started();
+    manager.installView(new NetView(myMemberId, 1, members, emptyMembersSet, emptyMembersSet));
+    Set<InternalDistributedMember> failures = manager.send(m);
+    verify(messenger).send(m);
+    if (failures != null) {
+      assertEquals(0, failures.size());
+    }
+  }
+  
+  @Test
+  public void testSendAdminMessageFailsDuringShutdown() throws Exception {
+    AlertListenerMessage m = AlertListenerMessage.create(mockMembers[0], 1, 
+       new Date(System.currentTimeMillis()), "thread", "", 1L, "", "");
+    manager.start();
+    manager.started();
+    manager.installView(new NetView(myMemberId, 1, members, emptyMembersSet, emptyMembersSet));
+    manager.setShutdown();
+    Set<InternalDistributedMember> failures = manager.send(m);
+    verify(messenger, never()).send(m);
+    assertEquals(1, failures.size());
+    assertEquals(mockMembers[0], failures.iterator().next());
+  }
+  
+  @Test
+  public void testSendToEmptyListIsRejected() throws Exception {
+    InternalDistributedMember[] emptyList = new InternalDistributedMember[0];
+    HighPriorityAckedMessage m = new HighPriorityAckedMessage();
+    m.setRecipient(mockMembers[0]);
+    manager.start();
+    manager.started();
+    manager.installView(new NetView(myMemberId, 1, members, emptyMembersSet, emptyMembersSet));
+    Set<InternalDistributedMember> failures = manager.send(null, m, null);
+    verify(messenger, never()).send(m);
+    reset(messenger);
+    failures = manager.send(emptyList, m, null);
+    verify(messenger, never()).send(m);
+  }
+  
+  @Test
+  public void testStartupEvents() throws Exception {
+    manager.start();
+    manager.started();
+    manager.isJoining = true;
+
+    List<InternalDistributedMember> viewmembers = Arrays.asList(new InternalDistributedMember[] {mockMembers[0], myMemberId});
+    manager.installView(new NetView(myMemberId, 2, viewmembers, emptyMembersSet, emptyMembersSet));
+
+    // add a surprise member that will be shunned due to it's having
+    // an old view ID
+    InternalDistributedMember surpriseMember = mockMembers[2];
+    surpriseMember.setVmViewId(1);
+    manager.handleOrDeferSurpriseConnect(surpriseMember);
+    assertEquals(1, manager.getStartupEvents().size());
+
+    // add a surprise member that will be accepted
+    InternalDistributedMember surpriseMember2 = mockMembers[3];
+    surpriseMember2.setVmViewId(3);
+    manager.handleOrDeferSurpriseConnect(surpriseMember2);
+    assertEquals(2, manager.getStartupEvents().size());
+
+    // suspect a member
+    InternalDistributedMember suspectMember = mockMembers[1];
+    manager.handleOrDeferSuspect(new SuspectMember(mockMembers[0], suspectMember));
+    // suspect messages aren't queued - they're ignored before joining the system
+    assertEquals(2, manager.getStartupEvents().size());
+    verify(listener, never()).memberSuspect(suspectMember, mockMembers[0]);
+
+    HighPriorityAckedMessage m = new HighPriorityAckedMessage();
+    mockMembers[0].setVmViewId(1);
+    m.setRecipient(mockMembers[0]);
+    m.setSender(mockMembers[1]);
+    manager.handleOrDeferMessage(m);
+    assertEquals(3, manager.getStartupEvents().size());
+
+    // this view officially adds surpriseMember2
+    viewmembers = Arrays.asList(
+        new InternalDistributedMember[] {mockMembers[0], myMemberId, surpriseMember2});
+    manager.handleOrDeferViewEvent(new NetView(myMemberId, 3, viewmembers, emptyMembersSet, emptyMembersSet));
+    assertEquals(4, manager.getStartupEvents().size());
+    
+    // add a surprise member that will be shunned due to it's having
+    // an old view ID
+    InternalDistributedMember surpriseMember3 = mockMembers[4];
+    surpriseMember.setVmViewId(1);
+    manager.handleOrDeferSurpriseConnect(surpriseMember);
+    assertEquals(5, manager.getStartupEvents().size());
+
+    
+    // process a new view after we finish joining but before event processing has started
+    manager.isJoining = false;
+    mockMembers[4].setVmViewId(4);
+    viewmembers = Arrays.asList(new InternalDistributedMember[] {mockMembers[0], myMemberId, surpriseMember2, mockMembers[4]});
+    manager.handleOrDeferViewEvent(new NetView(myMemberId, 4, viewmembers, emptyMembersSet, emptyMembersSet));
+    assertEquals(6, manager.getStartupEvents().size());
+    
+    // exercise the toString methods for code coverage
+    for (StartupEvent ev: manager.getStartupEvents()) {
+      ev.toString();
+    }
+    
+    manager.startEventProcessing();
+
+    // all startup events should have been processed 
+    assertEquals(0, manager.getStartupEvents().size());
+    // the new view should have been installed
+    assertEquals(4, manager.getView().getViewId());
+    // supriseMember2 should have been announced
+    verify(listener).newMemberConnected(surpriseMember2);
+    // supriseMember should have been rejected (old view ID)
+    verify(listener, never()).newMemberConnected(surpriseMember);
+    
+    // for code coverage also install a view after we finish joining but before
+    // event processing has started.  This should notify the distribution manager
+    // with a LocalViewMessage to process the view
+    reset(listener);
+    manager.handleOrDeferViewEvent(new NetView(myMemberId, 5, viewmembers, emptyMembersSet, emptyMembersSet));
+    assertEquals(0, manager.getStartupEvents().size());
+    verify(listener).messageReceived(isA(LocalViewMessage.class));
+
+    // process a suspect now - it will be passed to the listener
+    reset(listener);
+    suspectMember = mockMembers[1];
+    manager.handleOrDeferSuspect(new SuspectMember(mockMembers[0], suspectMember));
+    verify(listener).memberSuspect(suspectMember, mockMembers[0]);
+    
+    InternalDistributedMember mbr = manager.getMemberForStub(new Stub(myMemberId.getInetAddress(), 2033, 20), false);
+    assertTrue(mbr == null);
+    myMemberId.setDirectChannelPort(2033);
+    mbr = manager.getMemberForStub(new Stub(myMemberId.getInetAddress(), 2033, 20), false);
+    assertTrue(mbr != null);
+    assertEquals(mbr, myMemberId);
+  }
+  
+  /**
+   * Some tests require a DirectChannel mock
+   */
+  private void setUpDirectChannelMock() throws Exception {
+    dc = mock(DirectChannel.class);
+    when(dc.send(any(GMSMembershipManager.class), any(mockMembers.getClass()), any(DistributionMessage.class), anyInt(), anyInt()))
+      .thenReturn(100);
+
+    manager.start();
+    manager.started();
+    
+    manager.setDirectChannel(dc);
+
+    NetView view = new NetView(myMemberId, 1, members, emptyMembersSet, emptyMembersSet);
+    manager.installView(view);
+    when(joinLeave.getView()).thenReturn(view);
+    
+    manager.startEventProcessing();
+  }
+
+  @Test
+  public void testDirectChannelSend() throws Exception {
+    setUpDirectChannelMock();
+    HighPriorityAckedMessage m = new HighPriorityAckedMessage();
+    InternalDistributedMember[] recipients = new InternalDistributedMember[] {mockMembers[2], mockMembers[3]};
+    m.setRecipients(Arrays.asList(recipients));
+    Set<InternalDistributedMember> failures = manager.directChannelSend(recipients, m, null);
+    assertTrue(failures == null);
+    verify(dc).send(isA(GMSMembershipManager.class), isA(mockMembers.getClass()), isA(DistributionMessage.class), anyInt(), anyInt());
+  }
+  
+  @Test
+  public void testDirectChannelSendFailureToOneRecipient() throws Exception {
+    setUpDirectChannelMock();
+    HighPriorityAckedMessage m = new HighPriorityAckedMessage();
+    InternalDistributedMember[] recipients = new InternalDistributedMember[] {mockMembers[2], mockMembers[3]};
+    m.setRecipients(Arrays.asList(recipients));
+    Set<InternalDistributedMember> failures = manager.directChannelSend(recipients, m, null);
+
+    ConnectExceptions exception = new ConnectExceptions();
+    exception.addFailure(recipients[0], new Exception("testing"));
+    when(dc.send(any(GMSMembershipManager.class), any(mockMembers.getClass()), any(DistributionMessage.class), anyInt(), anyInt()))
+      .thenThrow(exception);
+    failures = manager.directChannelSend(recipients, m, null);
+    assertTrue(failures != null);
+    assertEquals(1, failures.size());
+    assertEquals(recipients[0], failures.iterator().next()); 
+  }
+  
+  @Test
+  public void testDirectChannelSendFailureToAll() throws Exception {
+    setUpDirectChannelMock();
+    HighPriorityAckedMessage m = new HighPriorityAckedMessage();
+    InternalDistributedMember[] recipients = new InternalDistributedMember[] {mockMembers[2], mockMembers[3]};
+    m.setRecipients(Arrays.asList(recipients));
+    Set<InternalDistributedMember> failures = manager.directChannelSend(recipients, m, null);
+    when(dc.send(any(GMSMembershipManager.class), any(mockMembers.getClass()), any(DistributionMessage.class), anyInt(), anyInt()))
+      .thenReturn(0);
+    when(stopper.cancelInProgress()).thenReturn("stopping for test");
+    try {
+      manager.directChannelSend(recipients, m, null);
+      throw new RuntimeException("expected directChannelSend to throw an exception");
+    } catch (DistributedSystemDisconnectedException e) {
+      // expected
+    }
+  }
+  
+  @Test
+  public void testDirectChannelSendAllRecipients() throws Exception {
+    setUpDirectChannelMock();
+    HighPriorityAckedMessage m = new HighPriorityAckedMessage();
+    m.setRecipient(DistributionMessage.ALL_RECIPIENTS);
+    assertTrue(m.forAll());
+    Set<InternalDistributedMember> failures = manager.directChannelSend(null, m, null);
+    assertTrue(failures == null);
+    verify(dc).send(isA(GMSMembershipManager.class), isA(mockMembers.getClass()), isA(DistributionMessage.class), anyInt(), anyInt());
+  }
+  
+}
+


[47/50] [abbrv] incubator-geode git commit: GEODE-53 - Removing unused images and adding missing incubating references

Posted by kl...@apache.org.
GEODE-53 - Removing unused images and adding missing incubating references


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

Branch: refs/heads/feature/GEODE-217
Commit: 1f193af7016e125059022e76d7cd053339bc309a
Parents: cd75b1f
Author: William Markito <wm...@pivotal.io>
Authored: Tue Dec 8 17:28:18 2015 -0800
Committer: William Markito <wm...@pivotal.io>
Committed: Tue Dec 8 17:28:18 2015 -0800

----------------------------------------------------------------------
 gemfire-site/content/community/index.html         |   2 +-
 gemfire-site/content/index.html                   |   4 ++--
 gemfire-site/website/content/community/index.html |   2 +-
 gemfire-site/website/content/img/intellij.png     | Bin 9199 -> 0 bytes
 gemfire-site/website/content/img/yourkit.jpeg     | Bin 7763 -> 0 bytes
 gemfire-site/website/content/index.html           |   2 +-
 6 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1f193af7/gemfire-site/content/community/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/community/index.html b/gemfire-site/content/community/index.html
index 3c612b4..d1bac78 100644
--- a/gemfire-site/content/community/index.html
+++ b/gemfire-site/content/community/index.html
@@ -244,7 +244,7 @@
     	<div class="row">
         <br/>
 	    	<h2>Join Our Community of Contributors!</h2>
-        <p>The Apache Geode team welcomes contributors who want to support the Geode technology. Our community builds everything from this website, from the Geode code to documentation and best practices information.</p>
+        <p>The Apache Geode (incubating) team welcomes contributors who want to support the Geode technology. Our community builds everything from this website, from the Geode code to documentation and best practices information.</p>
 
         <p>We especially welcome additions and corrections to the documentation, wiki, and website to improve the user experience. Bug reports and fixes and additions to the Apache Geode code are welcome. Helping users learn best practices also earns good karma in our community.</p>
 		</div>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1f193af7/gemfire-site/content/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/index.html b/gemfire-site/content/index.html
index a9d3425..c0d92a1 100644
--- a/gemfire-site/content/index.html
+++ b/gemfire-site/content/index.html
@@ -79,7 +79,7 @@
                   Take advantage of Apache Geode's unique technology that blends advanced techniques for data replication, partitioning and distributed processing.
 
                   <br/><br/>
-                  Apache Geode provides a database-like consistency model, reliable transaction processing and a shared-nothing architecture to maintain very low latency performance with high concurrency processing.<br/></p>
+                  Apache Geode (incubating) provides a database-like consistency model, reliable transaction processing and a shared-nothing architecture to maintain very low latency performance with high concurrency processing.<br/></p>
             </div>
 
             <div class="btn-wrapper">
@@ -166,7 +166,7 @@
         <div class="row">
             <div class="col-md-12 text-left">
                 <h2>About the Project</h2>
-                <p>Apache Geode is a data management platform that provides real-time, consistent access to data-intensive applications throughout widely distributed cloud architectures.</p>
+                <p>Apache Geode (incubating) is a data management platform that provides real-time, consistent access to data-intensive applications throughout widely distributed cloud architectures.</p>
 
                 <p>By pooling memory, CPU, network resources, and (optionally) local disk across multiple processes to manage application objects and behavior, it uses dynamic replication and data partitioning techniques to implement high availability, improved performance, scalability, and fault tolerance. In addition to being a distributed data container, Apache Geode is an in-memory data management system that provides reliable asynchronous event notifications and guaranteed message delivery.</p>
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1f193af7/gemfire-site/website/content/community/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/community/index.html b/gemfire-site/website/content/community/index.html
index ec58600..affc8e7 100644
--- a/gemfire-site/website/content/community/index.html
+++ b/gemfire-site/website/content/community/index.html
@@ -6,7 +6,7 @@
     	<div class="row">
         <br/>
 	    	<h2>Join Our Community of Contributors!</h2>
-        <p>The Apache Geode team welcomes contributors who want to support the Geode technology. Our community builds everything from this website, from the Geode code to documentation and best practices information.</p>
+        <p>The Apache Geode (incubating) team welcomes contributors who want to support the Geode technology. Our community builds everything from this website, from the Geode code to documentation and best practices information.</p>
 
         <p>We especially welcome additions and corrections to the documentation, wiki, and website to improve the user experience. Bug reports and fixes and additions to the Apache Geode code are welcome. Helping users learn best practices also earns good karma in our community.</p>
 		</div>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1f193af7/gemfire-site/website/content/img/intellij.png
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/img/intellij.png b/gemfire-site/website/content/img/intellij.png
deleted file mode 100644
index 2e8c480..0000000
Binary files a/gemfire-site/website/content/img/intellij.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1f193af7/gemfire-site/website/content/img/yourkit.jpeg
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/img/yourkit.jpeg b/gemfire-site/website/content/img/yourkit.jpeg
deleted file mode 100644
index 4a53b77..0000000
Binary files a/gemfire-site/website/content/img/yourkit.jpeg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1f193af7/gemfire-site/website/content/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/index.html b/gemfire-site/website/content/index.html
index 555d024..5e6a474 100644
--- a/gemfire-site/website/content/index.html
+++ b/gemfire-site/website/content/index.html
@@ -100,7 +100,7 @@ title: Performance is key. Consistency is a must.
         <div class="row">
             <div class="col-md-12 text-left">
                 <h2>About the Project</h2>
-                <p>Apache Geode is a data management platform that provides real-time, consistent access to data-intensive applications throughout widely distributed cloud architectures.</p>
+                <p>Apache Geode (incubating) is a data management platform that provides real-time, consistent access to data-intensive applications throughout widely distributed cloud architectures.</p>
 
                 <p>By pooling memory, CPU, network resources, and (optionally) local disk across multiple processes to manage application objects and behavior, it uses dynamic replication and data partitioning techniques to implement high availability, improved performance, scalability, and fault tolerance. In addition to being a distributed data container, Apache Geode is an in-memory data management system that provides reliable asynchronous event notifications and guaranteed message delivery.</p>
 


[48/50] [abbrv] incubator-geode git commit: GEODE-647: Ignoring failing test for now

Posted by kl...@apache.org.
GEODE-647: Ignoring failing test for now


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

Branch: refs/heads/feature/GEODE-217
Commit: 6e32ffe9cad8cf251dda2807d6fd23ecffb0b1cc
Parents: 1f193af
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed Dec 9 12:37:59 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Wed Dec 9 12:37:59 2015 -0800

----------------------------------------------------------------------
 .../gemfire/management/internal/cli/GfshParserJUnitTest.java       | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6e32ffe9/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
index 2cb1148..68fe251 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.junit.After;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.springframework.shell.core.CommandMarker;
@@ -777,6 +778,7 @@ public class GfshParserJUnitTest {
    * @throws SecurityException
    */
   @Test
+  @Ignore("GEODE-647")
   public void testParse() throws Exception {
     // get a CommandManager, add sample commands
     CommandManager commandManager = CommandManager.getInstance(false);


[34/50] [abbrv] incubator-geode git commit: GEODE-563: Moving gfsh tests from closed

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
new file mode 100644
index 0000000..18dfe67
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
@@ -0,0 +1,1148 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+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.asyncqueue.AsyncEvent;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventListener;
+import com.gemstone.gemfire.cache.wan.GatewaySenderFactory;
+import com.gemstone.gemfire.compression.SnappyCompressor;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.SharedConfiguration;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.ClassBuilder;
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.cache.RegionEntryContext;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
+import com.gemstone.gemfire.management.internal.ManagementConstants;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.text.MessageFormat;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBase {
+  private static final long serialVersionUID = 1L;
+
+  final String alterRegionName = "testAlterRegionRegion";
+  final String alterAsyncEventQueueId1 = "testAlterRegionQueue1";
+  final String alterAsyncEventQueueId2 = "testAlterRegionQueue2";
+  final String alterAsyncEventQueueId3 = "testAlterRegionQueue3";
+  final String alterGatewaySenderId1 = "testAlterRegionSender1";
+  final String alterGatewaySenderId2 = "testAlterRegionSender2";
+  final String alterGatewaySenderId3 = "testAlterRegionSender3";
+  final String region46391 = "region46391";
+  VM alterVm1;
+  String alterVm1Name;
+  VM alterVm2;
+  String alterVm2Name;
+
+  final List<String> filesToBeDeleted = new CopyOnWriteArrayList<String>();
+
+  public CreateAlterDestroyRegionCommandsDUnitTest(String name) {
+    super(name);
+  }
+
+  /**
+   * Asserts that the "compressor" option for the "create region" command succeeds for a recognized compressor.
+   */
+  public void testCreateRegionWithGoodCompressor() {
+    createDefaultSetup(null);
+    VM vm = Host.getHost(0).getVM(1);
+
+    // Create a cache in vm 1
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        assertNotNull(getCache());
+      }
+    });
+
+    // Run create region command with compression
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_REGION);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGION, "compressedRegion");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, "REPLICATE");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__COMPRESSOR,
+        RegionEntryContext.DEFAULT_COMPRESSION_PROVIDER);
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure our region exists with compression enabled
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Region region = getCache().getRegion("compressedRegion");
+        assertNotNull(region);
+        assertTrue(SnappyCompressor.getDefaultInstance().equals(region.getAttributes().getCompressor()));
+      }
+    });
+
+    // cleanup
+    commandStringBuilder = new CommandStringBuilder(CliStrings.DESTROY_REGION);
+    commandStringBuilder.addOption(CliStrings.DESTROY_REGION__REGION, "compressedRegion");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+  }
+
+  /**
+   * Asserts that the "compressor" option for the "create region" command fails for an unrecognized compressorc.
+   */
+  public void testCreateRegionWithBadCompressor() {
+    createDefaultSetup(null);
+
+    VM vm = Host.getHost(0).getVM(1);
+
+    // Create a cache in vm 1
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        assertNotNull(getCache());
+      }
+    });
+
+    // Create a region with an unrecognized compressor
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_REGION);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGION, "compressedRegion");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, "REPLICATE");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__COMPRESSOR, "BAD_COMPRESSOR");
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.ERROR, cmdResult.getStatus());
+
+    // Assert that our region was not created
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Region region = getCache().getRegion("compressedRegion");
+        assertNull(region);
+      }
+    });
+  }
+
+  /**
+   * Asserts that a missing "compressor" option for the "create region" command results in a region with no
+   * compression.
+   */
+  public void testCreateRegionWithNoCompressor() {
+    createDefaultSetup(null);
+
+    VM vm = Host.getHost(0).getVM(1);
+
+    // Create a cache in vm 1
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        assertNotNull(getCache());
+      }
+    });
+
+    // Create a region with no compression
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_REGION);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGION, "testRegion");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, "REPLICATE");
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Assert that our newly created region has no compression
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Region region = getCache().getRegion("testRegion");
+        assertNotNull(region);
+        assertNull(region.getAttributes().getCompressor());
+      }
+    });
+
+    // Cleanup
+    commandStringBuilder = new CommandStringBuilder(CliStrings.DESTROY_REGION);
+    commandStringBuilder.addOption(CliStrings.DESTROY_REGION__REGION, "testRegion");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+  }
+
+  public void testDestroyRegion() {
+    createDefaultSetup(null);
+
+    for (int i = 1; i <= 2; i++) {
+      Host.getHost(0).getVM(i).invoke(new SerializableRunnable() {
+        @Override
+        public void run() {
+          final Cache cache = getCache();
+
+          RegionFactory<Object, Object> factory = cache.createRegionFactory(RegionShortcut.PARTITION);
+          factory.create("Customer");
+
+          PartitionAttributesFactory paFactory = new PartitionAttributesFactory();
+          paFactory.setColocatedWith("Customer");
+          factory.setPartitionAttributes(paFactory.create());
+          factory.create("Order");
+        }
+      });
+    }
+
+    // Make sure that the region has been registered with the Manager MXBean
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+          @Override
+          public boolean done() {
+            try {
+              MBeanServer mbeanServer = MBeanJMXAdapter.mbeanServer;
+              String queryExp = MessageFormat.format(ManagementConstants.OBJECTNAME__REGION_MXBEAN,
+                  new Object[]{"/Customer", "*"});
+              ObjectName queryExpON = new ObjectName(queryExp);
+              return !(mbeanServer.queryNames(null, queryExpON).isEmpty());
+            } catch (MalformedObjectNameException mone) {
+              getLogWriter().error(mone);
+              fail(mone.getMessage());
+              return false;
+            }
+          }
+
+          @Override
+          public String description() {
+            return "Waiting for the region to be registed with the MXBean";
+          }
+        };
+
+        DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+      }
+    });
+
+    // Test failure when region not found
+    String command = "destroy region --name=DOESNOTEXIST";
+    getLogWriter().info("testDestroyRegion command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    String strr = commandResultToString(cmdResult);
+    getLogWriter().info("testDestroyRegion strr=" + strr);
+    assertTrue(stringContainsLine(strr, "Could not find.*\"DOESNOTEXIST\".*"));
+    assertEquals(Result.Status.ERROR, cmdResult.getStatus());
+
+    // Test unable to destroy with co-location
+    command = "destroy region --name=/Customer";
+    getLogWriter().info("testDestroyRegion command=" + command);
+    cmdResult = executeCommand(command);
+    strr = commandResultToString(cmdResult);
+    getLogWriter().info("testDestroyRegion strr=" + strr);
+    assertEquals(Result.Status.ERROR, cmdResult.getStatus());
+
+    // Test success
+    command = "destroy region --name=/Order";
+    getLogWriter().info("testDestroyRegion command=" + command);
+    cmdResult = executeCommand(command);
+    strr = commandResultToString(cmdResult);
+    assertTrue(stringContainsLine(strr, ".*Order.*destroyed successfully.*"));
+    getLogWriter().info("testDestroyRegion strr=" + strr);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    command = "destroy region --name=/Customer";
+    getLogWriter().info("testDestroyRegion command=" + command);
+    cmdResult = executeCommand(command);
+    strr = commandResultToString(cmdResult);
+    assertTrue(stringContainsLine(strr, ".*Customer.*destroyed successfully.*"));
+    getLogWriter().info("testDestroyRegion strr=" + strr);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+  }
+
+
+  public void testCreateRegion46391() throws IOException {
+    createDefaultSetup(null);
+    String command = CliStrings.CREATE_REGION + " --" + CliStrings.CREATE_REGION__REGION + "=" + this.region46391 + " --" + CliStrings.CREATE_REGION__REGIONSHORTCUT + "=REPLICATE";
+
+    getLogWriter().info("testCreateRegion46391 create region command=" + command);
+
+    CommandResult cmdResult = executeCommand(command);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    command = CliStrings.PUT + " --" + CliStrings.PUT__KEY + "=k1" + " --" + CliStrings.PUT__VALUE + "=k1" + " --" + CliStrings.PUT__REGIONNAME + "=" + this.region46391;
+
+    getLogWriter().info("testCreateRegion46391 put command=" + command);
+
+    CommandResult cmdResult2 = executeCommand(command);
+    assertEquals(Result.Status.OK, cmdResult2.getStatus());
+
+    getLogWriter().info("testCreateRegion46391  cmdResult2=" + commandResultToString(cmdResult2));
+    String str1 = "Result      : true";
+    String str2 = "Key         : k1";
+    String str3 = "Key Class   : java.lang.String";
+    String str4 = "Value Class : java.lang.String";
+    String str5 = "Old Value   : <NULL>";
+
+    assertTrue(commandResultToString(cmdResult).contains("Region \"/" + this.region46391 + "\" created on"));
+
+    assertTrue(commandResultToString(cmdResult2).contains(str1));
+    assertTrue(commandResultToString(cmdResult2).contains(str2));
+    assertTrue(commandResultToString(cmdResult2).contains(str3));
+    assertTrue(commandResultToString(cmdResult2).contains(str4));
+    assertTrue(commandResultToString(cmdResult2).contains(str5));
+  }
+
+  public void bug51924_testAlterRegion() throws IOException {
+    createDefaultSetup(null);
+
+    CommandResult cmdResult = executeCommand(CliStrings.LIST_REGION);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertTrue(commandResultToString(cmdResult).contains("No Regions Found"));
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        cache.createRegionFactory(RegionShortcut.PARTITION).setStatisticsEnabled(true).create(alterRegionName);
+      }
+    });
+
+    this.alterVm1 = Host.getHost(0).getVM(1);
+    this.alterVm1Name = "VM" + this.alterVm1.getPid();
+    this.alterVm1.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, alterVm1Name);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+        getSystem(localProps);
+        Cache cache = getCache();
+
+        // Setup queues and gateway senders to be used by all tests
+        cache.createRegionFactory(RegionShortcut.PARTITION).setStatisticsEnabled(true).create(alterRegionName);
+        AsyncEventListener listener = new AsyncEventListener() {
+          @Override
+          public void close() {
+            // Nothing to do
+          }
+
+          @Override
+          public boolean processEvents(List<AsyncEvent> events) {
+            return true;
+          }
+        };
+        cache.createAsyncEventQueueFactory().create(alterAsyncEventQueueId1, listener);
+        cache.createAsyncEventQueueFactory().create(alterAsyncEventQueueId2, listener);
+        cache.createAsyncEventQueueFactory().create(alterAsyncEventQueueId3, listener);
+
+        GatewaySenderFactory gatewaySenderFactory = cache.createGatewaySenderFactory();
+        gatewaySenderFactory.setManualStart(true);
+        gatewaySenderFactory.create(alterGatewaySenderId1, 2);
+        gatewaySenderFactory.create(alterGatewaySenderId2, 3);
+        gatewaySenderFactory.create(alterGatewaySenderId3, 4);
+      }
+    });
+
+    this.alterVm2 = Host.getHost(0).getVM(2);
+    this.alterVm2Name = "VM" + this.alterVm2.getPid();
+    this.alterVm2.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, alterVm2Name);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1,Group2");
+        getSystem(localProps);
+        Cache cache = getCache();
+
+        cache.createRegionFactory(RegionShortcut.PARTITION).setStatisticsEnabled(true).create(alterRegionName);
+      }
+    });
+
+    deployJarFilesForRegionAlter();
+    regionAlterGroupTest();
+    regionAlterSetAllTest();
+    regionAlterNoChangeTest();
+    regionAlterSetDefaultsTest();
+    regionAlterManipulatePlugInsTest();
+
+    this.alterVm1.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        getCache().getRegion(alterRegionName).destroyRegion();
+      }
+    });
+  }
+
+  private void regionAlterGroupTest() {
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_REGION);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGION, this.alterRegionName);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GROUP, "Group1");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__EVICTIONMAX, "5764");
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    String stringResult = commandResultToString(cmdResult);
+    assertEquals(4, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm1Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm2Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+
+    this.alterVm1.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
+        assertEquals(5764, attributes.getEvictionAttributes().getMaximum());
+      }
+    });
+
+    this.alterVm2.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
+        assertEquals(5764, attributes.getEvictionAttributes().getMaximum());
+      }
+    });
+
+    commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_REGION);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGION, "/" + this.alterRegionName);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GROUP, "Group2");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__EVICTIONMAX, "6963");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertFalse(
+        stringContainsLine(stringResult, this.alterVm1Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm2Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+
+    this.alterVm1.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
+        assertEquals(5764, attributes.getEvictionAttributes().getMaximum());
+      }
+    });
+
+    this.alterVm2.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
+        assertEquals(6963, attributes.getEvictionAttributes().getMaximum());
+      }
+    });
+  }
+
+  private void regionAlterSetAllTest() {
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_REGION);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGION, "/" + this.alterRegionName);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__EVICTIONMAX, "35464");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CLONINGENABLED, "true");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID, this.alterAsyncEventQueueId1);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIME, "3453");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIMEACTION, "DESTROY");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ENTRYEXPIRATIONTIMETOLIVE, "7563");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ENTRYEXPIRATIONTTLACTION, "DESTROY");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER, "com.cadrdunit.RegionAlterCacheListenerA");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELOADER, "com.cadrdunit.RegionAlterCacheLoader");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHEWRITER, "com.cadrdunit.RegionAlterCacheWriter");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GATEWAYSENDERID, this.alterGatewaySenderId1);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIME, "6234");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIMEACTION, "DESTROY");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGIONEXPIRATIONTTL, "4562");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGIONEXPIRATIONTTLACTION, "DESTROY");
+
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    String stringResult = commandResultToString(cmdResult);
+    assertEquals(5, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(stringContainsLine(stringResult, "Manager.*Region \"/" + this.alterRegionName + "\" altered.*"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm1Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm2Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+
+    this.alterVm1.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
+        assertEquals(35464, attributes.getEvictionAttributes().getMaximum());
+        assertEquals(3453, attributes.getEntryIdleTimeout().getTimeout());
+        assertTrue(attributes.getEntryIdleTimeout().getAction().isDestroy());
+        assertEquals(7563, attributes.getEntryTimeToLive().getTimeout());
+        assertTrue(attributes.getEntryTimeToLive().getAction().isDestroy());
+        assertEquals(6234, attributes.getRegionIdleTimeout().getTimeout());
+        assertTrue(attributes.getRegionIdleTimeout().getAction().isDestroy());
+        assertEquals(4562, attributes.getRegionTimeToLive().getTimeout());
+        assertTrue(attributes.getRegionTimeToLive().getAction().isDestroy());
+        assertEquals(1, attributes.getAsyncEventQueueIds().size());
+        assertTrue(attributes.getAsyncEventQueueIds().contains(alterAsyncEventQueueId1));
+        assertEquals(1, attributes.getGatewaySenderIds().size());
+        assertTrue(attributes.getGatewaySenderIds().contains(alterGatewaySenderId1));
+        assertEquals(1, attributes.getCacheListeners().length);
+        assertEquals("com.cadrdunit.RegionAlterCacheListenerA", attributes.getCacheListeners()[0].getClass().getName());
+        assertEquals("com.cadrdunit.RegionAlterCacheWriter", attributes.getCacheWriter().getClass().getName());
+        assertEquals("com.cadrdunit.RegionAlterCacheLoader", attributes.getCacheLoader().getClass().getName());
+      }
+    });
+  }
+
+  private void regionAlterNoChangeTest() {
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_REGION);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGION, "/" + this.alterRegionName);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GROUP, "Group1");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CLONINGENABLED, "true");
+
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    String stringResult = commandResultToString(cmdResult);
+    assertEquals(4, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm1Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm2Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+
+    this.alterVm2.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
+        assertEquals(35464, attributes.getEvictionAttributes().getMaximum());
+        assertEquals(3453, attributes.getEntryIdleTimeout().getTimeout());
+        assertTrue(attributes.getEntryIdleTimeout().getAction().isDestroy());
+        assertEquals(7563, attributes.getEntryTimeToLive().getTimeout());
+        assertTrue(attributes.getEntryTimeToLive().getAction().isDestroy());
+        assertEquals(6234, attributes.getRegionIdleTimeout().getTimeout());
+        assertTrue(attributes.getRegionIdleTimeout().getAction().isDestroy());
+        assertEquals(4562, attributes.getRegionTimeToLive().getTimeout());
+        assertTrue(attributes.getRegionTimeToLive().getAction().isDestroy());
+        assertEquals(1, attributes.getAsyncEventQueueIds().size());
+        assertTrue(attributes.getAsyncEventQueueIds().contains(alterAsyncEventQueueId1));
+        assertEquals(1, attributes.getGatewaySenderIds().size());
+        assertTrue(attributes.getGatewaySenderIds().contains(alterGatewaySenderId1));
+        assertEquals(1, attributes.getCacheListeners().length);
+        assertEquals("com.cadrdunit.RegionAlterCacheListenerA", attributes.getCacheListeners()[0].getClass().getName());
+        assertEquals("com.cadrdunit.RegionAlterCacheWriter", attributes.getCacheWriter().getClass().getName());
+        assertEquals("com.cadrdunit.RegionAlterCacheLoader", attributes.getCacheLoader().getClass().getName());
+      }
+    });
+  }
+
+  private void regionAlterSetDefaultsTest() {
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_REGION);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGION, "/" + this.alterRegionName);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GROUP, "Group1");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__EVICTIONMAX);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CLONINGENABLED);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIME);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ENTRYEXPIRATIONTTLACTION);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELOADER);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHEWRITER);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GATEWAYSENDERID);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIME);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIMEACTION);
+
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    String stringResult = commandResultToString(cmdResult);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertEquals(4, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm1Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm2Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+
+    this.alterVm1.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
+        assertEquals(0, attributes.getEvictionAttributes().getMaximum());
+        assertEquals(0, attributes.getEntryIdleTimeout().getTimeout());
+        assertTrue(attributes.getEntryIdleTimeout().getAction().isDestroy());
+        assertEquals(7563, attributes.getEntryTimeToLive().getTimeout());
+        assertTrue(attributes.getEntryTimeToLive().getAction().isInvalidate());
+        assertEquals(0, attributes.getRegionIdleTimeout().getTimeout());
+        assertTrue(attributes.getRegionIdleTimeout().getAction().isInvalidate());
+        assertEquals(4562, attributes.getRegionTimeToLive().getTimeout());
+        assertTrue(attributes.getRegionTimeToLive().getAction().isDestroy());
+        assertEquals(0, attributes.getAsyncEventQueueIds().size());
+        assertEquals(0, attributes.getGatewaySenderIds().size());
+        assertEquals(0, attributes.getCacheListeners().length);
+      }
+    });
+  }
+
+  private void regionAlterManipulatePlugInsTest() {
+
+    // Start out by putting 3 entries into each of the plug-in sets
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_REGION);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGION, "/" + this.alterRegionName);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GROUP, "Group1");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID, this.alterAsyncEventQueueId1);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID, this.alterAsyncEventQueueId2);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID, this.alterAsyncEventQueueId3);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GATEWAYSENDERID, this.alterGatewaySenderId1);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GATEWAYSENDERID, this.alterGatewaySenderId2);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GATEWAYSENDERID, this.alterGatewaySenderId3);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER, "com.cadrdunit.RegionAlterCacheListenerA");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER, "com.cadrdunit.RegionAlterCacheListenerB");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER, "com.cadrdunit.RegionAlterCacheListenerC");
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    String stringResult = commandResultToString(cmdResult);
+
+    assertEquals(4, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm1Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm2Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+
+    this.alterVm1.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
+        assertEquals(3, attributes.getAsyncEventQueueIds().size());
+        assertTrue(attributes.getAsyncEventQueueIds().contains(alterAsyncEventQueueId1));
+        assertTrue(attributes.getAsyncEventQueueIds().contains(alterAsyncEventQueueId2));
+        assertTrue(attributes.getAsyncEventQueueIds().contains(alterAsyncEventQueueId3));
+        assertEquals(3, attributes.getGatewaySenderIds().size());
+        assertTrue(attributes.getGatewaySenderIds().contains(alterGatewaySenderId1));
+        assertTrue(attributes.getGatewaySenderIds().contains(alterGatewaySenderId2));
+        assertTrue(attributes.getGatewaySenderIds().contains(alterGatewaySenderId3));
+        assertEquals(3, attributes.getCacheListeners().length);
+        assertEquals("com.cadrdunit.RegionAlterCacheListenerA", attributes.getCacheListeners()[0].getClass().getName());
+        assertEquals("com.cadrdunit.RegionAlterCacheListenerB", attributes.getCacheListeners()[1].getClass().getName());
+        assertEquals("com.cadrdunit.RegionAlterCacheListenerC", attributes.getCacheListeners()[2].getClass().getName());
+      }
+    });
+
+    // Now take 1 entry out of each of the sets
+    commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_REGION);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGION, "/" + this.alterRegionName);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GROUP, "Group1");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID, this.alterAsyncEventQueueId1);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID, this.alterAsyncEventQueueId2);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GATEWAYSENDERID, this.alterGatewaySenderId1);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GATEWAYSENDERID, this.alterGatewaySenderId3);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER, "com.cadrdunit.RegionAlterCacheListenerB");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER, "com.cadrdunit.RegionAlterCacheListenerC");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(4, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm1Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm2Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+
+    this.alterVm2.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
+        assertEquals(2, attributes.getAsyncEventQueueIds().size());
+        Iterator iterator = attributes.getAsyncEventQueueIds().iterator();
+        assertEquals(alterAsyncEventQueueId1, iterator.next());
+        assertEquals(alterAsyncEventQueueId2, iterator.next());
+        assertEquals(2, attributes.getGatewaySenderIds().size());
+        iterator = attributes.getGatewaySenderIds().iterator();
+        assertEquals(alterGatewaySenderId1, iterator.next());
+        assertEquals(alterGatewaySenderId3, iterator.next());
+        assertEquals(2, attributes.getCacheListeners().length);
+        assertEquals("com.cadrdunit.RegionAlterCacheListenerB", attributes.getCacheListeners()[0].getClass().getName());
+        assertEquals("com.cadrdunit.RegionAlterCacheListenerC", attributes.getCacheListeners()[1].getClass().getName());
+      }
+    });
+
+    // Add 1 back to each of the sets
+    commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_REGION);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGION, "/" + this.alterRegionName);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GROUP, "Group1");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID, this.alterAsyncEventQueueId1);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID, this.alterAsyncEventQueueId2);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID, this.alterAsyncEventQueueId3);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GATEWAYSENDERID, this.alterGatewaySenderId1);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GATEWAYSENDERID, this.alterGatewaySenderId3);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GATEWAYSENDERID, this.alterGatewaySenderId2);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER, "com.cadrdunit.RegionAlterCacheListenerB");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER, "com.cadrdunit.RegionAlterCacheListenerC");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER, "com.cadrdunit.RegionAlterCacheListenerA");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(4, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm1Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+    assertTrue(
+        stringContainsLine(stringResult, this.alterVm2Name + ".*Region \"/" + this.alterRegionName + "\" altered.*"));
+
+    this.alterVm1.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        RegionAttributes attributes = getCache().getRegion(alterRegionName).getAttributes();
+        assertEquals(3, attributes.getAsyncEventQueueIds().size());
+        assertTrue(attributes.getAsyncEventQueueIds().contains(alterAsyncEventQueueId1));
+        assertTrue(attributes.getAsyncEventQueueIds().contains(alterAsyncEventQueueId2));
+        assertTrue(attributes.getAsyncEventQueueIds().contains(alterAsyncEventQueueId3));
+        assertEquals(3, attributes.getGatewaySenderIds().size());
+        assertTrue(attributes.getGatewaySenderIds().contains(alterGatewaySenderId1));
+        assertTrue(attributes.getGatewaySenderIds().contains(alterGatewaySenderId3));
+        assertTrue(attributes.getGatewaySenderIds().contains(alterGatewaySenderId2));
+        assertEquals(3, attributes.getCacheListeners().length);
+        assertEquals("com.cadrdunit.RegionAlterCacheListenerB", attributes.getCacheListeners()[0].getClass().getName());
+        assertEquals("com.cadrdunit.RegionAlterCacheListenerC", attributes.getCacheListeners()[1].getClass().getName());
+        assertEquals("com.cadrdunit.RegionAlterCacheListenerA", attributes.getCacheListeners()[2].getClass().getName());
+      }
+    });
+  }
+
+  /**
+   * Asserts that creating, altering and destroying regions correctly updates the shared configuration.
+   */
+  public void testCreateAlterDestroyUpdatesSharedConfig() {
+    disconnectAllFromDS();
+
+    final String regionName = "testRegionSharedConfigRegion";
+    final String groupName = "testRegionSharedConfigGroup";
+
+    // Start the Locator and wait for shared configuration to be available
+    final int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+
+        final File locatorLogFile = new File("locator-" + locatorPort + ".log");
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, "Locator");
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "fine");
+        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
+              locatorProps);
+
+          DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+            @Override
+            public boolean done() {
+              return locator.isSharedConfigurationRunning();
+            }
+
+            @Override
+            public String description() {
+              return "Waiting for shared configuration to be started";
+            }
+          };
+          DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+        } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        }
+      }
+    });
+
+    // Start the default manager
+    Properties managerProps = new Properties();
+    managerProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    managerProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+    createDefaultSetup(managerProps);
+
+    // Create a cache in VM 1
+    VM vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        getSystem(localProps);
+        assertNotNull(getCache());
+      }
+    });
+
+    // Test creating the region
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_REGION);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGION, regionName);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, "REPLICATE");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__STATISTICSENABLED, "true");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__GROUP, groupName);
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure that the region has been registered with the Manager MXBean
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+          @Override
+          public boolean done() {
+            try {
+              MBeanServer mbeanServer = MBeanJMXAdapter.mbeanServer;
+              String queryExp = MessageFormat.format(ManagementConstants.OBJECTNAME__REGION_MXBEAN,
+                  new Object[]{"/" + regionName, "*"});
+              ObjectName queryExpON = new ObjectName(queryExp);
+              return !(mbeanServer.queryNames(null, queryExpON).isEmpty());
+            } catch (MalformedObjectNameException mone) {
+              getLogWriter().error(mone);
+              fail(mone.getMessage());
+              return false;
+            }
+          }
+
+          @Override
+          public String description() {
+            return "Waiting for the region to be registed with the MXBean";
+          }
+        };
+
+        DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+      }
+    });
+
+    // Make sure the region exists in the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        try {
+          assertTrue(sharedConfig.getConfiguration(groupName).getCacheXmlContent().contains(regionName));
+        } catch (Exception e) {
+          fail("Error in cluster configuration service", e);
+        }
+      }
+    });
+
+    //Restart the data vm to make sure the changes are in place
+    vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        assertNotNull(cache);
+        cache.close();
+        assertTrue(cache.isClosed());
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        localProps.setProperty(DistributionConfig.USE_CLUSTER_CONFIGURATION_NAME, "true");
+        getSystem(localProps);
+        cache = getCache();
+        assertNotNull(cache);
+        Region region = cache.getRegion(regionName);
+        assertNotNull(region);
+      }
+    });
+
+
+    // Test altering the region
+    commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_REGION);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGION, regionName);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__GROUP, groupName);
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ENTRYEXPIRATIONTIMETOLIVE, "45635");
+    commandStringBuilder.addOption(CliStrings.ALTER_REGION__ENTRYEXPIRATIONTTLACTION, "DESTROY");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure the region was altered in the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        try {
+          assertTrue(sharedConfig.getConfiguration(groupName).getCacheXmlContent().contains("45635"));
+        } catch (Exception e) {
+          fail("Error in cluster configuration service");
+        }
+      }
+    });
+
+    //Restart the data vm to make sure the changes are in place
+    vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableCallable() {
+      @Override
+      public Object call() {
+        Cache cache = getCache();
+        assertNotNull(cache);
+        cache.close();
+        assertTrue(cache.isClosed());
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        localProps.setProperty(DistributionConfig.USE_CLUSTER_CONFIGURATION_NAME, "true");
+        getSystem(localProps);
+        cache = getCache();
+        assertNotNull(cache);
+        Region region = cache.getRegion(regionName);
+        assertNotNull(region);
+
+        return null;
+      }
+    });
+
+  }
+
+  public void testDestroyRegionWithSharedConfig() {
+
+    disconnectAllFromDS();
+
+    final String regionName = "testRegionSharedConfigRegion";
+    final String groupName = "testRegionSharedConfigGroup";
+
+    // Start the Locator and wait for shared configuration to be available
+    final int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+
+        final File locatorLogFile = new File("locator-" + locatorPort + ".log");
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, "Locator");
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "fine");
+        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
+              locatorProps);
+
+          DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+            @Override
+            public boolean done() {
+              return locator.isSharedConfigurationRunning();
+            }
+
+            @Override
+            public String description() {
+              return "Waiting for shared configuration to be started";
+            }
+          };
+          DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+        } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        }
+      }
+    });
+
+    // Start the default manager
+    Properties managerProps = new Properties();
+    managerProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    managerProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+    createDefaultSetup(managerProps);
+
+    // Create a cache in VM 1
+    VM vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        getSystem(localProps);
+        assertNotNull(getCache());
+      }
+    });
+
+    // Test creating the region
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_REGION);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGION, regionName);
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, "REPLICATE");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__STATISTICSENABLED, "true");
+    commandStringBuilder.addOption(CliStrings.CREATE_REGION__GROUP, groupName);
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure that the region has been registered with the Manager MXBean
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+          @Override
+          public boolean done() {
+            try {
+              MBeanServer mbeanServer = MBeanJMXAdapter.mbeanServer;
+              String queryExp = MessageFormat.format(ManagementConstants.OBJECTNAME__REGION_MXBEAN,
+                  new Object[]{"/" + regionName, "*"});
+              ObjectName queryExpON = new ObjectName(queryExp);
+              return !(mbeanServer.queryNames(null, queryExpON).isEmpty());
+            } catch (MalformedObjectNameException mone) {
+              getLogWriter().error(mone);
+              fail(mone.getMessage());
+              return false;
+            }
+          }
+
+          @Override
+          public String description() {
+            return "Waiting for the region to be registed with the MXBean";
+          }
+        };
+
+        DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+      }
+    });
+
+    // Make sure the region exists in the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        try {
+          assertTrue(sharedConfig.getConfiguration(groupName).getCacheXmlContent().contains(regionName));
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service");
+        }
+      }
+    });
+
+    // Test destroying the region
+    commandStringBuilder = new CommandStringBuilder(CliStrings.DESTROY_REGION);
+    commandStringBuilder.addOption(CliStrings.DESTROY_REGION__REGION, regionName);
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    getLogWriter().info("#SB" + commandResultToString(cmdResult));
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure the region was removed from the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        try {
+          assertFalse(sharedConfig.getConfiguration(groupName).getCacheXmlContent().contains(regionName));
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service");
+        }
+      }
+    });
+
+
+    //Restart the data vm to make sure the region is not existing any more
+    vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableCallable() {
+      @Override
+      public Object call() {
+        Cache cache = getCache();
+        assertNotNull(cache);
+        cache.close();
+        assertTrue(cache.isClosed());
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        localProps.setProperty(DistributionConfig.USE_CLUSTER_CONFIGURATION_NAME, "true");
+        getSystem(localProps);
+        cache = getCache();
+        assertNotNull(cache);
+        Region region = cache.getRegion(regionName);
+        assertNull(region);
+
+        return null;
+      }
+    });
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    for (String path : this.filesToBeDeleted) {
+      try {
+        final File fileToDelete = new File(path);
+        FileUtil.delete(fileToDelete);
+        if (path.endsWith(".jar")) {
+          executeCommand("undeploy --jar=" + fileToDelete.getName());
+        }
+      } catch (IOException e) {
+        getLogWriter().error("Unable to delete file", e);
+      }
+    }
+    this.filesToBeDeleted.clear();
+    super.tearDown2();
+  }
+
+  /**
+   * Deploys JAR files which contain classes to be instantiated by the "alter region" test.
+   */
+  private void deployJarFilesForRegionAlter() throws IOException {
+    ClassBuilder classBuilder = new ClassBuilder();
+    final File jarFile1 = new File(new File(".").getAbsolutePath(), "testAlterRegion1.jar");
+    this.filesToBeDeleted.add(jarFile1.getAbsolutePath());
+    final File jarFile2 = new File(new File(".").getAbsolutePath(), "testAlterRegion2.jar");
+    this.filesToBeDeleted.add(jarFile2.getAbsolutePath());
+    final File jarFile3 = new File(new File(".").getAbsolutePath(), "testAlterRegion3.jar");
+    this.filesToBeDeleted.add(jarFile3.getAbsolutePath());
+    final File jarFile4 = new File(new File(".").getAbsolutePath(), "testAlterRegion4.jar");
+    this.filesToBeDeleted.add(jarFile4.getAbsolutePath());
+    final File jarFile5 = new File(new File(".").getAbsolutePath(), "testAlterRegion5.jar");
+    this.filesToBeDeleted.add(jarFile5.getAbsolutePath());
+
+    byte[] jarBytes = classBuilder.createJarFromClassContent("com/cadrdunit/RegionAlterCacheListenerA",
+        "package com.cadrdunit;" + "import com.gemstone.gemfire.cache.util.CacheListenerAdapter;" + "public class RegionAlterCacheListenerA extends CacheListenerAdapter {}");
+    writeJarBytesToFile(jarFile1, jarBytes);
+    CommandResult cmdResult = executeCommand("deploy --jar=testAlterRegion1.jar");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    jarBytes = classBuilder.createJarFromClassContent("com/cadrdunit/RegionAlterCacheListenerB",
+        "package com.cadrdunit;" + "import com.gemstone.gemfire.cache.util.CacheListenerAdapter;" + "public class RegionAlterCacheListenerB extends CacheListenerAdapter {}");
+    writeJarBytesToFile(jarFile2, jarBytes);
+    cmdResult = executeCommand("deploy --jar=testAlterRegion2.jar");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    jarBytes = classBuilder.createJarFromClassContent("com/cadrdunit/RegionAlterCacheListenerC",
+        "package com.cadrdunit;" + "import com.gemstone.gemfire.cache.util.CacheListenerAdapter;" + "public class RegionAlterCacheListenerC extends CacheListenerAdapter {}");
+    writeJarBytesToFile(jarFile3, jarBytes);
+    cmdResult = executeCommand("deploy --jar=testAlterRegion3.jar");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    jarBytes = classBuilder.createJarFromClassContent("com/cadrdunit/RegionAlterCacheLoader",
+        "package com.cadrdunit;" + "import com.gemstone.gemfire.cache.CacheLoader;" + "import com.gemstone.gemfire.cache.CacheLoaderException;" + "import com.gemstone.gemfire.cache.LoaderHelper;" + "public class RegionAlterCacheLoader implements CacheLoader {" + "public void close() {}" + "public Object load(LoaderHelper helper) throws CacheLoaderException {return null;}}");
+    writeJarBytesToFile(jarFile4, jarBytes);
+    cmdResult = executeCommand("deploy --jar=testAlterRegion4.jar");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    jarBytes = classBuilder.createJarFromClassContent("com/cadrdunit/RegionAlterCacheWriter",
+        "package com.cadrdunit;" + "import com.gemstone.gemfire.cache.util.CacheWriterAdapter;" + "public class RegionAlterCacheWriter extends CacheWriterAdapter {}");
+    writeJarBytesToFile(jarFile5, jarBytes);
+    cmdResult = executeCommand("deploy --jar=testAlterRegion5.jar");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+  }
+
+  private void writeJarBytesToFile(File jarFile, byte[] jarBytes) throws IOException {
+    final OutputStream outStream = new FileOutputStream(jarFile);
+    outStream.write(jarBytes);
+    outStream.flush();
+    outStream.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
new file mode 100644
index 0000000..eeb9896
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
@@ -0,0 +1,480 @@
+/*
+ * 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.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.SharedConfiguration;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.ClassBuilder;
+import com.gemstone.gemfire.internal.JarDeployer;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.remote.CommandExecutionContext;
+import com.gemstone.gemfire.management.internal.cli.remote.CommandProcessor;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.util.Properties;
+import java.util.regex.Pattern;
+
+/**
+ * Unit tests for the DeployCommands class
+ *
+ * @author David Hoots
+ * @since 7.0
+ */
+public class DeployCommandsDUnitTest extends CliCommandTestBase {
+  private static final long serialVersionUID = 1L;
+
+  File newDeployableJarFile = new File("DeployCommandsDUnit1.jar");
+
+  transient private ClassBuilder classBuilder = new ClassBuilder();
+  transient private CommandProcessor commandProcessor;
+
+  public DeployCommandsDUnitTest(String name) {
+    super(name);
+  }
+
+  @SuppressWarnings("serial")
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+
+    this.commandProcessor = new CommandProcessor();
+    assertFalse(this.commandProcessor.isStopped());
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        deleteSavedJarFiles();
+      }
+    });
+    deleteSavedJarFiles();
+  }
+
+  @SuppressWarnings("serial")
+  @Override
+  public void tearDown2() throws Exception {
+    Host.getHost(0).getVM(1).invoke(new SerializableRunnable() {
+      public void run() {
+        DistributionManager.isDedicatedAdminVM = false;
+      }
+    });
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        deleteSavedJarFiles();
+      }
+    });
+    deleteSavedJarFiles();
+
+    super.tearDown2();
+  }
+
+  @SuppressWarnings("serial")
+  public void testDeploy() throws IOException {
+    final Properties props = new Properties();
+    final Host host = Host.getHost(0);
+    final VM vm = host.getVM(0);
+    final String vmName = "VM" + vm.getPid();
+
+    // Create the cache in this VM
+    props.setProperty(DistributionConfig.NAME_NAME, "Controller");
+    props.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+    getSystem(props);
+    getCache();
+
+    // Create the cache in the other VM
+    vm.invoke(new SerializableRunnable() {
+      public void run() {
+        props.setProperty(DistributionConfig.NAME_NAME, vmName);
+        props.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(props);
+        getCache();
+      }
+    });
+
+    DeployCommands deployCommands = new DeployCommands();
+
+    // Single JAR all members
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit1.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitA")});
+    Result result = deployCommands.deploy(null, "DeployCommandsDUnit1.jar", null);
+
+    assertEquals(true, result.hasNextLine());
+
+    String resultString = result.nextLine();
+    assertEquals(false, resultString.contains("ERROR"));
+    assertEquals(1, countMatchesInString(resultString, "Controller"));
+    assertEquals(1, countMatchesInString(resultString, vmName));
+    assertEquals(4, countMatchesInString(resultString, "DeployCommandsDUnit1.jar"));
+
+    // Single JAR with group
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit2.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitB")});
+    result = deployCommands.deploy(new String[]{"Group2"}, "DeployCommandsDUnit2.jar", null);
+
+    assertEquals(true, result.hasNextLine());
+
+    resultString = result.nextLine();
+    assertEquals(false, resultString.contains("ERROR"));
+    assertEquals(false, resultString.contains("Controller"));
+    assertEquals(1, countMatchesInString(resultString, vmName));
+    assertEquals(2, countMatchesInString(resultString, "DeployCommandsDUnit2.jar"));
+
+    // Multiple JARs to all members
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit3.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitC"), "DeployCommandsDUnit4.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitD")});
+    result = deployCommands.deploy(null, null, "AnyDirectory");
+
+    assertEquals(true, result.hasNextLine());
+
+    resultString = result.nextLine();
+    assertEquals(false, resultString.contains("ERROR"));
+    assertEquals(2, countMatchesInString(resultString, "Controller"));
+    assertEquals(2, countMatchesInString(resultString, vmName));
+    assertEquals(4, countMatchesInString(resultString, "DeployCommandsDUnit3.jar"));
+    assertEquals(4, countMatchesInString(resultString, "DeployCommandsDUnit4.jar"));
+
+    // Multiple JARs to a group
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit5.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitE"), "DeployCommandsDUnit6.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitF")});
+    result = deployCommands.deploy(new String[]{"Group1"}, null, "AnyDirectory");
+
+    assertEquals(true, result.hasNextLine());
+
+    resultString = result.nextLine();
+    assertEquals(false, resultString.contains("ERROR"));
+    assertEquals(2, countMatchesInString(resultString, "Controller"));
+    assertEquals(false, resultString.contains(vmName));
+    assertEquals(2, countMatchesInString(resultString, "DeployCommandsDUnit5.jar"));
+    assertEquals(2, countMatchesInString(resultString, "DeployCommandsDUnit6.jar"));
+  }
+
+  @SuppressWarnings("serial")
+  public void testUndeploy() throws IOException {
+    final Properties props = new Properties();
+    final Host host = Host.getHost(0);
+    final VM vm = host.getVM(0);
+    final String vmName = "VM" + vm.getPid();
+
+    // Create the cache in this VM
+    props.setProperty(DistributionConfig.NAME_NAME, "Controller");
+    props.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+    getSystem(props);
+    getCache();
+
+    // Create the cache in the other VM
+    vm.invoke(new SerializableRunnable() {
+      public void run() {
+        props.setProperty(DistributionConfig.NAME_NAME, vmName);
+        props.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(props);
+        getCache();
+      }
+    });
+
+    DeployCommands deployCommands = new DeployCommands();
+
+    // Deploy a couple of JAR files which can be undeployed
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit1.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitA")});
+    deployCommands.deploy(new String[]{"Group1"}, "DeployCommandsDUnit1.jar", null);
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit2.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitB")});
+    deployCommands.deploy(new String[]{"Group2"}, "DeployCommandsDUnit2.jar", null);
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit3.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitC")});
+    deployCommands.deploy(null, "DeployCommandsDUnit3.jar", null);
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit4.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitD")});
+    deployCommands.deploy(null, "DeployCommandsDUnit4.jar", null);
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit5.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitE")});
+    deployCommands.deploy(null, "DeployCommandsDUnit5.jar", null);
+
+    // Undeploy for 1 group
+    Result result = deployCommands.undeploy(new String[]{"Group1"}, "DeployCommandsDUnit1.jar");
+    assertEquals(true, result.hasNextLine());
+    String resultString = result.nextLine();
+    assertEquals(false, resultString.contains("ERROR"));
+    assertEquals(1, countMatchesInString(resultString, "Controller"));
+    assertEquals(false, resultString.contains(vmName));
+    assertEquals(2, countMatchesInString(resultString, "DeployCommandsDUnit1.jar"));
+
+    // Multiple Undeploy for all members
+    result = deployCommands.undeploy(null, "DeployCommandsDUnit2.jar, DeployCommandsDUnit3.jar");
+    assertEquals(true, result.hasNextLine());
+    resultString = result.nextLine();
+    assertEquals(false, resultString.contains("ERROR"));
+    assertEquals(2, countMatchesInString(resultString, "Controller"));
+    assertEquals(2, countMatchesInString(resultString, vmName));
+    assertEquals(3, countMatchesInString(resultString, "DeployCommandsDUnit2.jar"));
+    assertEquals(4, countMatchesInString(resultString, "DeployCommandsDUnit3.jar"));
+
+    // Undeploy all (no JAR specified)
+    result = deployCommands.undeploy(null, null);
+    assertEquals(true, result.hasNextLine());
+    resultString = result.nextLine();
+    assertEquals(false, resultString.contains("ERROR"));
+    assertEquals(2, countMatchesInString(resultString, "Controller"));
+    assertEquals(2, countMatchesInString(resultString, vmName));
+    assertEquals(4, countMatchesInString(resultString, "DeployCommandsDUnit4.jar"));
+    assertEquals(4, countMatchesInString(resultString, "DeployCommandsDUnit5.jar"));
+  }
+
+  @SuppressWarnings("serial")
+  public void testListDeployed() throws IOException {
+    final Properties props = new Properties();
+    final Host host = Host.getHost(0);
+    final VM vm = host.getVM(0);
+    final String vmName = "VM" + vm.getPid();
+
+    // Create the cache in this VM
+    props.setProperty(DistributionConfig.NAME_NAME, "Controller");
+    props.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+    getSystem(props);
+    getCache();
+
+    // Create the cache in the other VM
+    vm.invoke(new SerializableRunnable() {
+      public void run() {
+        props.setProperty(DistributionConfig.NAME_NAME, vmName);
+        props.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(props);
+        getCache();
+      }
+    });
+
+    DeployCommands deployCommands = new DeployCommands();
+
+    // Deploy a couple of JAR files which can be listed
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit1.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitA")});
+    deployCommands.deploy(new String[]{"Group1"}, "DeployCommandsDUnit1.jar", null);
+    CommandExecutionContext.setBytesFromShell(
+        new byte[][]{"DeployCommandsDUnit2.jar".getBytes(), this.classBuilder.createJarFromName(
+            "DeployCommandsDUnitB")});
+    deployCommands.deploy(new String[]{"Group2"}, "DeployCommandsDUnit2.jar", null);
+
+    // List for all members
+    Result result = deployCommands.listDeployed(null);
+    assertEquals(true, result.hasNextLine());
+    String resultString = result.nextLine();
+    assertEquals(false, resultString.contains("ERROR"));
+    assertEquals(1, countMatchesInString(resultString, "Controller"));
+    assertEquals(1, countMatchesInString(resultString, vmName));
+    assertEquals(2, countMatchesInString(resultString, "DeployCommandsDUnit1.jar"));
+    assertEquals(2, countMatchesInString(resultString, "DeployCommandsDUnit2.jar"));
+
+    // List for members in Group1
+    result = deployCommands.listDeployed("Group1");
+    assertEquals(true, result.hasNextLine());
+    resultString = result.nextLine();
+    assertEquals(false, resultString.contains("ERROR"));
+    assertEquals(1, countMatchesInString(resultString, "Controller"));
+    assertEquals(false, resultString.contains(vmName));
+    assertEquals(2, countMatchesInString(resultString, "DeployCommandsDUnit1.jar"));
+    assertEquals(false, resultString.contains("DeployCommandsDUnit2.jar"));
+
+    // List for members in Group2
+    result = deployCommands.listDeployed("Group2");
+    assertEquals(true, result.hasNextLine());
+    resultString = result.nextLine();
+    assertEquals(false, resultString.contains("ERROR"));
+    assertEquals(false, resultString.contains("Controller"));
+    assertEquals(1, countMatchesInString(resultString, vmName));
+    assertEquals(false, resultString.contains("DeployCommandsDUnit1.jar"));
+    assertEquals(2, countMatchesInString(resultString, "DeployCommandsDUnit2.jar"));
+  }
+
+  /**
+   * Does an end-to-end test using the complete CLI framework while ensuring that the shared configuration is updated.
+   */
+  public void testEndToEnd() throws IOException {
+    disconnectAllFromDS();
+
+    final String groupName = "testDeployEndToEndGroup";
+
+    // Start the Locator and wait for shared configuration to be available
+    final int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+
+        final File locatorLogFile = new File("locator-" + locatorPort + ".log");
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, "Locator");
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "fine");
+        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
+              locatorProps);
+
+          DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+            @Override
+            public boolean done() {
+              return locator.isSharedConfigurationRunning();
+            }
+
+            @Override
+            public String description() {
+              return "Waiting for shared configuration to be started";
+            }
+          };
+          DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+        } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        }
+      }
+    });
+
+    // Start the default manager
+    Properties managerProps = new Properties();
+    managerProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    managerProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+    managerProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+    createDefaultSetup(managerProps);
+
+    // Create a JAR file
+    this.classBuilder.writeJarFromName("DeployCommandsDUnitA", this.newDeployableJarFile);
+
+    // Deploy the JAR
+    CommandResult cmdResult = executeCommand("deploy --jar=DeployCommandsDUnit1.jar");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    String stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Member.*JAR.*JAR Location"));
+    assertTrue(stringContainsLine(stringResult,
+        "Manager.*DeployCommandsDUnit1.jar.*" + JarDeployer.JAR_PREFIX + "DeployCommandsDUnit1.jar#1"));
+
+    // Undeploy the JAR
+    cmdResult = executeCommand("undeploy --jar=DeployCommandsDUnit1.jar");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Member.*JAR.*Un-Deployed From JAR Location"));
+    assertTrue(stringContainsLine(stringResult,
+        "Manager.*DeployCommandsDUnit1.jar.*" + JarDeployer.JAR_PREFIX + "DeployCommandsDUnit1.jar#1"));
+
+    // Deploy the JAR to a group
+    cmdResult = executeCommand("deploy --jar=DeployCommandsDUnit1.jar --group=" + groupName);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Member.*JAR.*JAR Location"));
+    assertTrue(stringContainsLine(stringResult,
+        "Manager.*DeployCommandsDUnit1.jar.*" + JarDeployer.JAR_PREFIX + "DeployCommandsDUnit1.jar#1"));
+
+    // Make sure the deployed jar in the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        try {
+          assertTrue(sharedConfig.getConfiguration(groupName).getJarNames().contains("DeployCommandsDUnit1.jar"));
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service", e);
+        }
+      }
+    });
+
+    // List deployed for group
+    cmdResult = executeCommand("list deployed --group=" + groupName);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Member.*JAR.*JAR Location"));
+    assertTrue(stringContainsLine(stringResult,
+        "Manager.*DeployCommandsDUnit1.jar.*" + JarDeployer.JAR_PREFIX + "DeployCommandsDUnit1.jar#1"));
+
+    // Undeploy for group
+    cmdResult = executeCommand("undeploy --group=" + groupName);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Member.*JAR.*Un-Deployed From JAR Location"));
+    assertTrue(stringContainsLine(stringResult,
+        "Manager.*DeployCommandsDUnit1.jar.*" + JarDeployer.JAR_PREFIX + "DeployCommandsDUnit1.jar#1"));
+
+    // Make sure the deployed jar was removed from the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        try {
+          assertFalse(sharedConfig.getConfiguration(groupName).getJarNames().contains("DeployCommandsDUnit1.jar"));
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service", e);
+        }
+      }
+    });
+
+    // List deployed with nothing deployed
+    cmdResult = executeCommand("list deployed");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertTrue(commandResultToString(cmdResult).contains(CliStrings.LIST_DEPLOYED__NO_JARS_FOUND_MESSAGE));
+  }
+
+  final Pattern pattern = Pattern.compile("^" + JarDeployer.JAR_PREFIX + "DeployCommandsDUnit.*#\\d++$");
+
+  void deleteSavedJarFiles() {
+    this.newDeployableJarFile.delete();
+
+    File dirFile = new File(".");
+    // Find all deployed JAR files
+    File[] oldJarFiles = dirFile.listFiles(new FilenameFilter() {
+      @Override
+      public boolean accept(final File file, final String name) {
+        return DeployCommandsDUnitTest.this.pattern.matcher(name).matches();
+      }
+    });
+
+    // Now delete them
+    if (oldJarFiles != null) {
+      for (File oldJarFile : oldJarFiles) {
+        oldJarFile.delete();
+      }
+    }
+  }
+}


[08/50] [abbrv] incubator-geode git commit: [GEODE-619]: Add GMSMemberJUnitTest Adding unit tests that should improve code coverage of GMSMember to ~90% Fixing array index out of bounds issue when one address is longer than the other

Posted by kl...@apache.org.
[GEODE-619]: Add GMSMemberJUnitTest
Adding unit tests that should improve code coverage of GMSMember to ~90%
Fixing array index out of bounds issue when one address is longer than the other


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

Branch: refs/heads/feature/GEODE-217
Commit: 2258d74cac3e13a162a06079b78eb83141eaa641
Parents: 442faa0
Author: Jason Huynh <hu...@gmail.com>
Authored: Tue Dec 1 14:01:53 2015 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Dec 1 14:06:15 2015 -0800

----------------------------------------------------------------------
 .../internal/membership/gms/GMSMember.java      |  20 +--
 .../membership/gms/GMSMemberJUnitTest.java      | 148 +++++++++++++++++++
 2 files changed, 159 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2258d74c/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
index f4784ed..05b3aee 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
@@ -211,7 +211,7 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
     byte[] hisAddr = his.inetAddr.getAddress();
     if (myAddr != hisAddr) {
       for (int idx=0; idx < myAddr.length; idx++) {
-        if (idx > hisAddr.length) {
+        if (idx >= hisAddr.length) {
           return 1;
         } else if (myAddr[idx] > hisAddr[idx]) {
           return 1;
@@ -219,6 +219,10 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
           return -1;
         }
       }
+      //After checking both addresses we have only gone up to myAddr.length, their address could be longer
+      if (hisAddr.length > myAddr.length) {
+        return -1;
+      }
     }
     if (udpPort < his.udpPort) return -1;
     if (his.udpPort < udpPort) return 1;
@@ -226,16 +230,14 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
 
     // bug #41983, address of kill-9'd member is reused
     // before it can be ejected from membership
-    if (result == 0) {
-      if (this.vmViewId >= 0 && his.vmViewId >= 0) {
-        if (this.vmViewId < his.vmViewId) {
-          result = -1;
-        } else if (his.vmViewId < this.vmViewId) {
-          result = 1;
-        }
+    if (this.vmViewId >= 0 && his.vmViewId >= 0) {
+      if (this.vmViewId < his.vmViewId) {
+        result = -1;
+      } else if (his.vmViewId < this.vmViewId) {
+        result = 1;
       }
     }
-    if (result == 0 && this.uuidMSBs != 0 && his.uuidMSBs != 0) {
+    if (result == 0 && this.uuidMSBs != 0 && his.uuidMSBs != 0) { 
       if (this.uuidMSBs < his.uuidMSBs) {
         result = -1;
       } else if (his.uuidMSBs < this.uuidMSBs) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2258d74c/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
new file mode 100644
index 0000000..0b75d3d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
@@ -0,0 +1,148 @@
+package com.gemstone.gemfire.distributed.internal.membership.gms;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.net.InetAddress;
+
+import org.jgroups.util.UUID;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class GMSMemberJUnitTest {
+
+  @Test
+  public void testEqualsNotSameType() {
+    GMSMember member = new GMSMember();
+    Assert.assertFalse(member.equals("Not a GMSMember"));
+  }
+  
+  @Test
+  public void testEqualsIsSame() {
+    GMSMember member = new GMSMember();
+    Assert.assertTrue(member.equals(member));
+  }
+  
+  @Test
+  public void testCompareToIsSame() {
+    GMSMember member = new GMSMember();
+    UUID uuid = new UUID(0, 0);
+    member.setUUID(uuid);
+    Assert.assertEquals(0, member.compareTo(member));
+  }
+  
+  private GMSMember createGMSMember(byte[] inetAddress, int viewId, long msb, long lsb) {
+    GMSMember member = new GMSMember();
+    InetAddress addr1 = mock(InetAddress.class);
+    when(addr1.getAddress()).thenReturn(inetAddress);
+    member.setInetAddr(addr1);
+    member.setBirthViewId(viewId);
+    member.setUUID(new UUID(msb, lsb));
+    return member;
+  }
+  
+  @Test
+  public void testCompareToInetAddressIsLongerThan() {
+    GMSMember member1 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
+    GMSMember member2 = createGMSMember(new byte[] {1, 1, 1, 1}, 1, 1, 1);
+    Assert.assertEquals(1, member1.compareTo(member2));
+  }
+  
+  @Test
+  public void testCompareToInetAddressIsShorterThan() {
+    GMSMember member1 = createGMSMember(new byte[] {1, 1, 1, 1}, 1, 1, 1);
+    GMSMember member2 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
+    Assert.assertEquals(-1, member1.compareTo(member2));
+  }
+  
+  @Test
+  public void testCompareToInetAddressIsGreater() {
+    GMSMember member1 = createGMSMember(new byte[] {1, 2, 1, 1, 1}, 1, 1, 1);
+    GMSMember member2 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
+    Assert.assertEquals(1, member1.compareTo(member2));
+  }
+  
+  @Test
+  public void testCompareToInetAddressIsLessThan() {
+    GMSMember member1 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
+    GMSMember member2 = createGMSMember(new byte[] {1, 2, 1, 1, 1}, 1, 1, 1);
+    Assert.assertEquals(-1, member1.compareTo(member2));
+  }
+  
+  @Test
+  public void testCompareToMyViewIdLarger() {
+    GMSMember member1 = createGMSMember(new byte[] {1}, 2, 1, 1);
+    GMSMember member2 = createGMSMember(new byte[] {1}, 1, 1, 1);
+    Assert.assertEquals(1, member1.compareTo(member2));
+  }
+  
+  @Test
+  public void testCompareToTheirViewIdLarger() {
+    GMSMember member1 = createGMSMember(new byte[] {1}, 1, 1, 1);
+    GMSMember member2 = createGMSMember(new byte[] {1}, 2, 1, 1);
+    Assert.assertEquals(-1, member1.compareTo(member2));
+  }
+  
+  @Test
+  public void testCompareToMyMSBLarger() {
+    GMSMember member1 = createGMSMember(new byte[] {1}, 1, 2, 1);
+    GMSMember member2 = createGMSMember(new byte[] {1}, 1, 1, 1);
+    Assert.assertEquals(1, member1.compareTo(member2));
+  }
+
+  @Test
+  public void testCompareToTheirMSBLarger() {
+    GMSMember member1 = createGMSMember(new byte[] {1}, 1, 1, 1);
+    GMSMember member2 = createGMSMember(new byte[] {1}, 1, 2, 1);
+    Assert.assertEquals(-1, member1.compareTo(member2));
+  }
+
+  @Test
+  public void testCompareToMyLSBLarger() {
+    GMSMember member1 = createGMSMember(new byte[] {1}, 1, 1, 2);
+    GMSMember member2 = createGMSMember(new byte[] {1}, 1, 1, 1);
+    Assert.assertEquals(1, member1.compareTo(member2));
+  }
+  
+  @Test
+  public void testCompareToTheirLSBLarger() {
+    GMSMember member1 = createGMSMember(new byte[] {1}, 1, 1, 1);
+    GMSMember member2 = createGMSMember(new byte[] {1}, 1, 1, 2);
+    Assert.assertEquals(-1, member1.compareTo(member2));
+  }
+
+  
+  //Makes sure a NPE is not thrown
+  @Test
+  public void testNoNPEWhenSetAttributesWithNull() {
+    GMSMember member = new GMSMember();
+    member.setAttributes(null);
+    MemberAttributes attrs = member.getAttributes(); 
+    MemberAttributes invalid = MemberAttributes.INVALID;
+    Assert.assertEquals(attrs.getVmKind(), invalid.getVmKind());
+    Assert.assertEquals(attrs.getPort(), invalid.getPort());
+    Assert.assertEquals(attrs.getVmViewId(), invalid.getVmViewId());
+    Assert.assertEquals(attrs.getName(), invalid.getName());
+  }
+  
+  @Test
+  public void testGetUUIDReturnsNullWhenUUIDIs0() {
+    GMSMember member = new GMSMember();
+    UUID uuid = new UUID(0, 0);
+    member.setUUID(uuid);
+    Assert.assertNull(member.getUUID());
+  }
+  
+  @Test
+  public void testGetUUID() {
+    GMSMember member = new GMSMember();
+    UUID uuid = new UUID(1, 1);
+    member.setUUID(uuid);
+    Assert.assertNotNull(member.getUUID());
+  }
+}


[10/50] [abbrv] incubator-geode git commit: GEODE-621: failure in AnalyzeSerializablesJUnitTest

Posted by kl...@apache.org.
GEODE-621: failure in AnalyzeSerializablesJUnitTest

fixed the entry for StartupMessage


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

Branch: refs/heads/feature/GEODE-217
Commit: b7030d1cd7bf5461889a8a2a0eb400796021c307
Parents: dce479e
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Dec 1 16:55:36 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 1 16:58:03 2015 -0800

----------------------------------------------------------------------
 .../gemfire/codeAnalysis/sanctionedDataSerializables.txt       | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b7030d1c/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
index 137850f..af0ee36 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
@@ -243,9 +243,9 @@ fromData,27,2a2bb7000c2a2bb9000d0100b500032a2bb8000ec0000fb50002b1
 toData,24,2a2bb700092b2ab40003b9000a02002ab400022bb8000bb1
 
 com/gemstone/gemfire/distributed/internal/StartupMessage,3
-fromDataProblem,38,2ab4003ac7000e2abb006d59b7006eb5003a2ab4003a2bb6006f572ab4003a1270b6006f57b1
-fromData,354,2a2bb700712bb9007201003d1c99000e2a2bb80073b5000ca700082a01b5000c2bb80074b20051b600529c000704a70004033e1d9900082bb80075572a2bb80076b500092a2bb900770100b5000d2a2bb900720100b5000e2a2bb900720100b500122bb900770100360403360515051504a2003e2bb800783a062bb90077010036071906c6000d19060301011507b80079a700183a082ab2007a04bd00235903190853b60027b7007b840501a7ffc12bb900770100360503360615061505a200492bb800783a072bb800783a082bb90077010036091907c600121908c6000d19071908150903b8007ca700183a0a2ab2007d04bd00235903190a53b60027b7007b840601a7ffb62a2bb80075c0007eb500132a2bb900770100b500182a2bb80076b500192a2bb900720100b5001abb006659b700673a0619062bb6007f2a1906b60080b5000a2a1906b60081b5000b2a1906b60082b5000f2a1906b60083b50010b1
-toData,399,2a2bb7004d2b2ab4000cc6000704a7000403b9004e02002ab4000cc6000b2ab4000c2bb8004f2bb80050b20051b600529c000704a70004033d1c99000ebb005359b700542bb800552ab400092bb800562b2ab4000db9005702002b2ab4000eb9004e02002b2ab40012b9004e0200b800584e2b2dbeb90057020003360415042dbea200212d150432b600592bb8005a2b2d150432b6005bb900570200840401a7ffdeb8005c3a042b1904beb90057020003360515051904bea2007f1904150532c1005d9900331904150532c0005db6005eb6005f3a061904150532c0005db60060b6005f3a071904150532c0005db600613608a7002a1904150532c00062b600633a061904150532c00062b600643a071904150532c00062b60065360819062bb8005a19072bb8005a2b1508b900570200840501a7ff7f2ab400132bb800552b2ab40018b9005702002ab400192bb800562b2ab4001ab9004e0200bb006659b700673a0519052ab4000ab6006819052ab4000bb6006919052ab4000fb6006a19052ab40010b6006b19052bb6006cb1
+fromDataProblem,38,2ab40039c7000e2abb006b59b7006cb500392ab400392bb6006d572ab40039126eb6006d57b1
+fromData,325,2a2bb7006f2bb80070b2004eb6004f9c000704a70004033d1c9900082bb80071572a2bb80072b500092a2bb900730100b5000c2a2bb900740100b5000d2a2bb900740100b500112bb9007301003e03360415041da2003e2bb800753a052bb90073010036061905c6000d19050301011506b80076a700183a072ab2007704bd00225903190753b60026b70078840401a7ffc22bb900730100360403360515051504a200492bb800753a062bb800753a072bb90073010036081906c600121907c6000d19061907150803b80079a700183a092ab2007a04bd00225903190953b60026b70078840501a7ffb62a2bb80071c0007bb500122a2bb900730100b500172a2bb80072b500182a2bb900740100b50019bb006459b700653a0519052bb6007c2a1905b6007db5000a2a1905b6007eb5000b2a1905b6007fb5000e2a1905b60080b5000fb1
+toData,366,2a2bb7004c2bb8004db2004eb6004f9c000704a70004033d1c99000ebb005059b700512bb800522ab400092bb800532b2ab4000cb9005402002b2ab4000db9005502002b2ab40011b900550200b800564e2b2dbeb90054020003360415042dbea200212d150432b600572bb800582b2d150432b60059b900540200840401a7ffdeb8005a3a042b1904beb90054020003360515051904bea2007f1904150532c1005b9900331904150532c0005bb6005cb6005d3a061904150532c0005bb6005eb6005d3a071904150532c0005bb6005f3608a7002a1904150532c00060b600613a061904150532c00060b600623a071904150532c00060b60063360819062bb8005819072bb800582b1508b900540200840501a7ff7f2ab400122bb800522b2ab40017b9005402002ab400182bb800532b2ab40019b900550200bb006459b700653a0519052ab4000ab6006619052ab4000bb6006719052ab4000eb6006819052ab4000fb6006919052bb6006ab1
 
 com/gemstone/gemfire/distributed/internal/StartupResponseMessage,3
 fromDataProblem,43,2ab40026c7000e2abb003959b7003ab500262ab400262bb6003b572ab40026123c123db8003eb6003b57b1


[39/50] [abbrv] incubator-geode git commit: GEODE-642: fix race in OffHeapStorageJUnitTest

Posted by kl...@apache.org.
GEODE-642: fix race in OffHeapStorageJUnitTest

The problem was caused by the listener not rechecking the system
property every time it was notified of an OutOfOffHeapMemoryException.
Since the code that disconnects the distributed system when
we run out of off-heap memory is async the test would intermittently pass.


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

Branch: refs/heads/feature/GEODE-217
Commit: 80b59bfa3dcf9b2ac1317c6996591b1fd68494e1
Parents: a6398d9
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Dec 8 10:42:58 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Dec 8 11:36:25 2015 -0800

----------------------------------------------------------------------
 .../com/gemstone/gemfire/internal/offheap/OffHeapStorage.java     | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/80b59bfa/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
index 82cbfeb..3eb839b 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
@@ -395,7 +395,6 @@ public class OffHeapStorage implements OffHeapMemoryStats {
   }
   
   static class DisconnectingOutOfOffHeapMemoryListener implements OutOfOffHeapMemoryListener {
-    private final boolean stayConnectedOnOutOfOffHeapMemory = Boolean.getBoolean(STAY_CONNECTED_ON_OUTOFOFFHEAPMEMORY_PROPERTY);
     private final Object lock = new Object();
     private InternalDistributedSystem ids;
     
@@ -415,7 +414,7 @@ public class OffHeapStorage implements OffHeapMemoryStats {
         if (this.ids == null) {
           return;
         }
-        if (stayConnectedOnOutOfOffHeapMemory) {
+        if (Boolean.getBoolean(STAY_CONNECTED_ON_OUTOFOFFHEAPMEMORY_PROPERTY)) {
           return;
         }
         


[23/50] [abbrv] incubator-geode git commit: new unit tests and code clean-up

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
index fbdcdf5..4b9c01f 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
@@ -16,26 +16,22 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.messenger;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
+
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutput;
-import java.io.DataOutputStream;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
-import junit.framework.Assert;
-
+import org.apache.commons.lang.SerializationException;
 import org.jgroups.Event;
+import org.jgroups.JChannel;
 import org.jgroups.Message;
 import org.jgroups.conf.ClassConfigurator;
 import org.jgroups.protocols.UNICAST3;
@@ -43,9 +39,10 @@ import org.jgroups.util.UUID;
 import org.junit.After;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
 
 import com.gemstone.gemfire.ForcedDisconnectException;
+import com.gemstone.gemfire.GemFireIOException;
+import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
@@ -58,15 +55,21 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.GMSMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.ServiceConfig;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services.Stopper;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.HealthMonitor;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.JoinLeave;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.MessageHandler;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinRequestMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinResponseMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.LeaveRequestMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messenger.JGroupsMessenger.JGroupsReceiver;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
+import com.gemstone.gemfire.internal.cache.DistributedCacheOperation;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -76,6 +79,7 @@ public class JGroupsMessengerJUnitTest {
   private JoinLeave joinLeave;
   private Manager manager;
   private Stopper stopper;
+  private HealthMonitor healthMonitor;
   private InterceptUDP interceptor;
 
 
@@ -83,6 +87,10 @@ public class JGroupsMessengerJUnitTest {
    * Create stub and mock objects
    */
   private void initMocks(boolean enableMcast) throws Exception {
+    if (messenger != null) {
+      messenger.stop();
+      messenger = null;
+    }
     Properties nonDefault = new Properties();
     nonDefault.put(DistributionConfig.DISABLE_TCP_NAME, "true");
     nonDefault.put(DistributionConfig.MCAST_PORT_NAME, enableMcast? ""+AvailablePortHelper.getRandomAvailableUDPPort() : "0");
@@ -100,6 +108,8 @@ public class JGroupsMessengerJUnitTest {
     manager = mock(Manager.class);
     when(manager.isMulticastAllowed()).thenReturn(enableMcast);
     
+    healthMonitor = mock(HealthMonitor.class);
+    
     joinLeave = mock(JoinLeave.class);
     
     ServiceConfig serviceConfig = new ServiceConfig(tconfig, config);
@@ -107,6 +117,7 @@ public class JGroupsMessengerJUnitTest {
     services = mock(Services.class);
     when(services.getConfig()).thenReturn(serviceConfig);
     when(services.getCancelCriterion()).thenReturn(stopper);
+    when(services.getHealthMonitor()).thenReturn(healthMonitor);
     when(services.getManager()).thenReturn(manager);
     when(services.getJoinLeave()).thenReturn(joinLeave);
     when(services.getStatistics()).thenReturn(mock(DMStats.class));
@@ -121,7 +132,7 @@ public class JGroupsMessengerJUnitTest {
         "<"+InterceptUDP.class.getName()+"/>" +
         jgroupsConfig.substring(insertIdx);
     messenger.setJGroupsStackConfigForTesting(jgroupsConfig);
-    System.out.println("jgroups config: " + jgroupsConfig);
+//    System.out.println("jgroups config: " + jgroupsConfig);
     
     messenger.start();
     messenger.started();
@@ -141,13 +152,195 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testMemberWeightIsSerialized() throws Exception {
     HeapDataOutputStream out = new HeapDataOutputStream(500, Version.CURRENT);
-    InternalDistributedMember m = new InternalDistributedMember("localhost", 8888);
-    ((GMSMember)m.getNetMember()).setMemberWeight((byte)40);
-    m.toData(out);
+    InternalDistributedMember mbr = createAddress(8888);
+    ((GMSMember)mbr.getNetMember()).setMemberWeight((byte)40);
+    mbr.toData(out);
     DataInputStream in = new DataInputStream(new ByteArrayInputStream(out.toByteArray()));
-    m = new InternalDistributedMember();
-    m.fromData(in);
-    assertEquals(40, m.getNetMember().getMemberWeight());
+    mbr = new InternalDistributedMember();
+    mbr.fromData(in);
+    assertEquals(40, mbr.getNetMember().getMemberWeight());
+  }
+  
+  @Test
+  public void testSerializationError() throws Exception {
+    for (int i=0; i<2 ; i++) {
+      boolean enableMcast = (i==1);
+      initMocks(enableMcast);
+      InternalDistributedMember mbr = createAddress(8888);
+      DistributedCacheOperation.CacheOperationMessage msg = mock(DistributedCacheOperation.CacheOperationMessage.class);
+      when(msg.getRecipients()).thenReturn(new InternalDistributedMember[] {mbr});
+      when(msg.getMulticast()).thenReturn(enableMcast);
+      if (!enableMcast) {
+        // for non-mcast we send a message with a reply-processor
+        when(msg.getProcessorId()).thenReturn(1234);
+      } else {
+        // for mcast we send a direct-ack message and expect the messenger
+        // to register it
+        stub(msg.isDirectAck()).toReturn(true);
+      }
+      when(msg.getDSFID()).thenReturn((int)DataSerializableFixedID.PUT_ALL_MESSAGE);
+      
+      // for code coverage we need to test with both a SerializationException and
+      // an IOException.  The former is wrapped in a GemfireIOException while the
+      // latter is not
+      doThrow(new SerializationException()).when(msg).toData(any(DataOutput.class));
+      try {
+        messenger.send(msg);
+        fail("expected a failure");
+      } catch (GemFireIOException e) {
+        // success
+      }
+      if (enableMcast) {
+        verify(msg, atLeastOnce()).registerProcessor();
+      }
+      doThrow(new IOException()).when(msg).toData(any(DataOutput.class));
+      try {
+        messenger.send(msg);
+        fail("expected a failure");
+      } catch (GemFireIOException e) {
+        // success
+      }
+    }
+  }
+  
+  @Test
+  public void testJChannelError() throws Exception {
+    for (int i=0; i<2 ; i++) {
+      boolean enableMcast = (i==1);
+      initMocks(enableMcast);
+      JChannel mockChannel = mock(JChannel.class);
+      when(mockChannel.isConnected()).thenReturn(true);
+      doThrow(new RuntimeException()).when(mockChannel).send(any(Message.class));
+      JChannel realChannel = messenger.myChannel;
+      messenger.myChannel = mockChannel;
+      try {
+        InternalDistributedMember mbr = createAddress(8888);
+        DistributedCacheOperation.CacheOperationMessage msg = mock(DistributedCacheOperation.CacheOperationMessage.class);
+        when(msg.getRecipients()).thenReturn(new InternalDistributedMember[] {mbr});
+        when(msg.getMulticast()).thenReturn(enableMcast);
+        when(msg.getProcessorId()).thenReturn(1234);
+        when(msg.getDSFID()).thenReturn((int)DataSerializableFixedID.PUT_ALL_MESSAGE);
+        try {
+          messenger.send(msg);
+          fail("expected a failure");
+        } catch (DistributedSystemDisconnectedException e) {
+          // success
+        }
+        verify(mockChannel).send(isA(Message.class));
+      } finally {
+        messenger.myChannel = realChannel;
+      }
+    }
+  }
+  
+  @Test
+  public void testJChannelErrorDuringDisconnect() throws Exception {
+    for (int i=0; i<4 ; i++) {
+      System.out.println("loop #"+i);
+      boolean enableMcast = (i%2 == 1);
+      initMocks(enableMcast);
+      JChannel mockChannel = mock(JChannel.class);
+      when(mockChannel.isConnected()).thenReturn(true);
+      Exception ex, shutdownCause;
+      if (i < 2) {
+        ex = new RuntimeException("");
+        shutdownCause = new RuntimeException("shutdownCause");
+      } else {
+        shutdownCause = new ForcedDisconnectException("");
+        ex = new RuntimeException("", shutdownCause);
+      }
+      doThrow(ex).when(mockChannel).send(any(Message.class));
+      JChannel realChannel = messenger.myChannel;
+      messenger.myChannel = mockChannel;
+      
+      when(services.getShutdownCause()).thenReturn(shutdownCause);
+      
+      try {
+        InternalDistributedMember mbr = createAddress(8888);
+        DistributedCacheOperation.CacheOperationMessage msg = mock(DistributedCacheOperation.CacheOperationMessage.class);
+        when(msg.getRecipients()).thenReturn(new InternalDistributedMember[] {mbr});
+        when(msg.getMulticast()).thenReturn(enableMcast);
+        when(msg.getProcessorId()).thenReturn(1234);
+        when(msg.getDSFID()).thenReturn((int)DataSerializableFixedID.PUT_ALL_MESSAGE);
+        try {
+          messenger.send(msg);
+          fail("expected a failure");
+        } catch (DistributedSystemDisconnectedException e) {
+          // the ultimate cause should be the shutdownCause returned
+          // by Services.getShutdownCause()
+          Throwable cause = e;
+          while (cause.getCause() != null) {
+            cause = cause.getCause();
+          }
+          assertTrue(cause != e);
+          assertTrue(cause == shutdownCause);
+        }
+        verify(mockChannel).send(isA(Message.class));
+      } finally {
+        messenger.myChannel = realChannel;
+      }
+    }
+  }
+  
+  @Test
+  public void testSendWhenChannelIsClosed() throws Exception {
+    for (int i=0; i<2 ; i++) {
+      initMocks(false);
+      JChannel mockChannel = mock(JChannel.class);
+      when(mockChannel.isConnected()).thenReturn(false);
+      doThrow(new RuntimeException()).when(mockChannel).send(any(Message.class));
+      JChannel realChannel = messenger.myChannel;
+      messenger.myChannel = mockChannel;
+      try {
+        InternalDistributedMember mbr = createAddress(8888);
+        DistributedCacheOperation.CacheOperationMessage msg = mock(DistributedCacheOperation.CacheOperationMessage.class);
+        when(msg.getRecipients()).thenReturn(new InternalDistributedMember[] {mbr});
+        when(msg.getMulticast()).thenReturn(false);
+        when(msg.getProcessorId()).thenReturn(1234);
+        try {
+          messenger.send(msg);
+          fail("expected a failure");
+        } catch (DistributedSystemDisconnectedException e) {
+          // success
+        }
+        verify(mockChannel, never()).send(isA(Message.class));
+      } finally {
+        messenger.myChannel = realChannel;
+      }
+    }
+  }
+
+  @Test
+  public void testSendUnreliably() throws Exception {
+    for (int i=0; i<2 ; i++) {
+      boolean enableMcast = (i==1);
+      initMocks(enableMcast);
+      InternalDistributedMember mbr = createAddress(8888);
+      DistributedCacheOperation.CacheOperationMessage msg = mock(DistributedCacheOperation.CacheOperationMessage.class);
+      when(msg.getRecipients()).thenReturn(new InternalDistributedMember[] {mbr});
+      when(msg.getMulticast()).thenReturn(enableMcast);
+      if (!enableMcast) {
+        // for non-mcast we send a message with a reply-processor
+        when(msg.getProcessorId()).thenReturn(1234);
+      } else {
+        // for mcast we send a direct-ack message and expect the messenger
+        // to register it
+        stub(msg.isDirectAck()).toReturn(true);
+      }
+      when(msg.getDSFID()).thenReturn((int)DataSerializableFixedID.PUT_ALL_MESSAGE);
+      interceptor.collectMessages = true;
+      try {
+        messenger.sendUnreliably(msg);
+      } catch (GemFireIOException e) {
+        fail("expected success");
+      }
+      if (enableMcast) {
+        verify(msg, atLeastOnce()).registerProcessor();
+      }
+      verify(msg).toData(isA(DataOutput.class));
+      assertTrue("expected 1 message but found " + interceptor.collectedMessages, interceptor.collectedMessages.size() == 1);
+      assertTrue(interceptor.collectedMessages.get(0).isFlagSet(Message.Flag.NO_RELIABILITY));
+    }
   }
   
   @Test
@@ -265,7 +458,7 @@ public class JGroupsMessengerJUnitTest {
   public void testSendToMultipleMembers() throws Exception {
     initMocks(false);
     InternalDistributedMember sender = messenger.getMemberID();
-    InternalDistributedMember other = new InternalDistributedMember("localhost", 8888);
+    InternalDistributedMember other = createAddress(8888);
 
     NetView v = new NetView(sender);
     v.add(other);
@@ -285,11 +478,11 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testChannelStillConnectedAfterEmergencyCloseAfterForcedDisconnectWithAutoReconnect() throws Exception {
     initMocks(false);
-    Mockito.doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
-    Mockito.doCallRealMethod().when(services).getShutdownCause();
-    Mockito.doCallRealMethod().when(services).emergencyClose();
-    Mockito.doCallRealMethod().when(services).isShutdownDueToForcedDisconnect();
-    Mockito.doCallRealMethod().when(services).isAutoReconnectEnabled();
+    doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
+    doCallRealMethod().when(services).getShutdownCause();
+    doCallRealMethod().when(services).emergencyClose();
+    doCallRealMethod().when(services).isShutdownDueToForcedDisconnect();
+    doCallRealMethod().when(services).isAutoReconnectEnabled();
     services.setShutdownCause(new ForcedDisconnectException("Test Forced Disconnect"));
     assertTrue(messenger.myChannel.isConnected());
     messenger.emergencyClose();
@@ -299,11 +492,11 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testChannelStillConnectedAfterStopAfterForcedDisconnectWithAutoReconnect() throws Exception {
     initMocks(false);
-    Mockito.doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
-    Mockito.doCallRealMethod().when(services).getShutdownCause();
-    Mockito.doCallRealMethod().when(services).emergencyClose();
-    Mockito.doCallRealMethod().when(services).isShutdownDueToForcedDisconnect();
-    Mockito.doCallRealMethod().when(services).isAutoReconnectEnabled();
+    doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
+    doCallRealMethod().when(services).getShutdownCause();
+    doCallRealMethod().when(services).emergencyClose();
+    doCallRealMethod().when(services).isShutdownDueToForcedDisconnect();
+    doCallRealMethod().when(services).isAutoReconnectEnabled();
     services.setShutdownCause(new ForcedDisconnectException("Test Forced Disconnect"));
     assertTrue(messenger.myChannel.isConnected());
     messenger.stop();
@@ -313,12 +506,12 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testChannelStillConnectedAfteremergencyWhileReconnectingDS() throws Exception {
     initMocks(false);
-    Mockito.doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
-    Mockito.doCallRealMethod().when(services).getShutdownCause();
-    Mockito.doCallRealMethod().when(services).emergencyClose();
-    Mockito.doReturn(false).when(services).isShutdownDueToForcedDisconnect();
-    Mockito.doReturn(false).when(services).isAutoReconnectEnabled();
-    Mockito.doReturn(true).when(manager).isReconnectingDS();
+    doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
+    doCallRealMethod().when(services).getShutdownCause();
+    doCallRealMethod().when(services).emergencyClose();
+    doReturn(false).when(services).isShutdownDueToForcedDisconnect();
+    doReturn(false).when(services).isAutoReconnectEnabled();
+    doReturn(true).when(manager).isReconnectingDS();
     services.setShutdownCause(new ForcedDisconnectException("Test Forced Disconnect"));
     assertTrue(messenger.myChannel.isConnected());
     messenger.emergencyClose();
@@ -329,12 +522,12 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testChannelStillConnectedAfterStopWhileReconnectingDS() throws Exception {
     initMocks(false);
-    Mockito.doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
-    Mockito.doCallRealMethod().when(services).getShutdownCause();
-    Mockito.doCallRealMethod().when(services).emergencyClose();
-    Mockito.doReturn(false).when(services).isShutdownDueToForcedDisconnect();
-    Mockito.doReturn(false).when(services).isAutoReconnectEnabled();
-    Mockito.doReturn(true).when(manager).isReconnectingDS();
+    doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
+    doCallRealMethod().when(services).getShutdownCause();
+    doCallRealMethod().when(services).emergencyClose();
+    doReturn(false).when(services).isShutdownDueToForcedDisconnect();
+    doReturn(false).when(services).isAutoReconnectEnabled();
+    doReturn(true).when(manager).isReconnectingDS();
     services.setShutdownCause(new ForcedDisconnectException("Test Forced Disconnect"));
     assertTrue(messenger.myChannel.isConnected());
     messenger.stop();
@@ -344,12 +537,12 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testChannelClosedOnEmergencyClose() throws Exception {
     initMocks(false);
-    Mockito.doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
-    Mockito.doCallRealMethod().when(services).getShutdownCause();
-    Mockito.doCallRealMethod().when(services).emergencyClose();
-    Mockito.doReturn(false).when(services).isShutdownDueToForcedDisconnect();
-    Mockito.doReturn(false).when(services).isAutoReconnectEnabled();
-    Mockito.doReturn(false).when(manager).isReconnectingDS();
+    doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
+    doCallRealMethod().when(services).getShutdownCause();
+    doCallRealMethod().when(services).emergencyClose();
+    doReturn(false).when(services).isShutdownDueToForcedDisconnect();
+    doReturn(false).when(services).isAutoReconnectEnabled();
+    doReturn(false).when(manager).isReconnectingDS();
     services.setShutdownCause(new ForcedDisconnectException("Test Forced Disconnect"));
     assertTrue(messenger.myChannel.isConnected());
     messenger.emergencyClose();
@@ -359,12 +552,12 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testChannelClosedOnStop() throws Exception {
     initMocks(false);
-    Mockito.doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
-    Mockito.doCallRealMethod().when(services).getShutdownCause();
-    Mockito.doCallRealMethod().when(services).emergencyClose();
-    Mockito.doReturn(false).when(services).isShutdownDueToForcedDisconnect();
-    Mockito.doReturn(false).when(services).isAutoReconnectEnabled();
-    Mockito.doReturn(false).when(manager).isReconnectingDS();
+    doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
+    doCallRealMethod().when(services).getShutdownCause();
+    doCallRealMethod().when(services).emergencyClose();
+    doReturn(false).when(services).isShutdownDueToForcedDisconnect();
+    doReturn(false).when(services).isAutoReconnectEnabled();
+    doReturn(false).when(manager).isReconnectingDS();
     services.setShutdownCause(new ForcedDisconnectException("Test Forced Disconnect"));
     assertTrue(messenger.myChannel.isConnected());
     messenger.stop();
@@ -374,12 +567,12 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testChannelClosedAfterEmergencyCloseForcedDisconnectWithoutAutoReconnect() throws Exception {
     initMocks(false);
-    Mockito.doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
-    Mockito.doCallRealMethod().when(services).getShutdownCause();
-    Mockito.doCallRealMethod().when(services).emergencyClose();
-    Mockito.doReturn(true).when(services).isShutdownDueToForcedDisconnect();
-    Mockito.doReturn(false).when(services).isAutoReconnectEnabled();
-    Mockito.doReturn(false).when(manager).isReconnectingDS();
+    doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
+    doCallRealMethod().when(services).getShutdownCause();
+    doCallRealMethod().when(services).emergencyClose();
+    doReturn(true).when(services).isShutdownDueToForcedDisconnect();
+    doReturn(false).when(services).isAutoReconnectEnabled();
+    doReturn(false).when(manager).isReconnectingDS();
     services.setShutdownCause(new ForcedDisconnectException("Test Forced Disconnect"));
     assertTrue(messenger.myChannel.isConnected());
     messenger.emergencyClose();
@@ -389,12 +582,12 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testChannelStillConnectedStopAfterForcedDisconnectWithoutAutoReconnect() throws Exception {
     initMocks(false);
-    Mockito.doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
-    Mockito.doCallRealMethod().when(services).getShutdownCause();
-    Mockito.doCallRealMethod().when(services).emergencyClose();
-    Mockito.doReturn(true).when(services).isShutdownDueToForcedDisconnect();
-    Mockito.doReturn(false).when(services).isAutoReconnectEnabled();
-    Mockito.doReturn(false).when(manager).isReconnectingDS();
+    doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
+    doCallRealMethod().when(services).getShutdownCause();
+    doCallRealMethod().when(services).emergencyClose();
+    doReturn(true).when(services).isShutdownDueToForcedDisconnect();
+    doReturn(false).when(services).isAutoReconnectEnabled();
+    doReturn(false).when(manager).isReconnectingDS();
     services.setShutdownCause(new ForcedDisconnectException("Test Forced Disconnect"));
     assertTrue(messenger.myChannel.isConnected());
     messenger.stop();
@@ -404,12 +597,12 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testChannelClosedAfterEmergencyCloseNotForcedDisconnectWithAutoReconnect() throws Exception {
     initMocks(false);
-    Mockito.doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
-    Mockito.doCallRealMethod().when(services).getShutdownCause();
-    Mockito.doCallRealMethod().when(services).emergencyClose();
-    Mockito.doReturn(false).when(services).isShutdownDueToForcedDisconnect();
-    Mockito.doReturn(true).when(services).isAutoReconnectEnabled();
-    Mockito.doReturn(false).when(manager).isReconnectingDS();
+    doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
+    doCallRealMethod().when(services).getShutdownCause();
+    doCallRealMethod().when(services).emergencyClose();
+    doReturn(false).when(services).isShutdownDueToForcedDisconnect();
+    doReturn(true).when(services).isAutoReconnectEnabled();
+    doReturn(false).when(manager).isReconnectingDS();
     services.setShutdownCause(new ForcedDisconnectException("Test Forced Disconnect"));
     assertTrue(messenger.myChannel.isConnected());
     messenger.emergencyClose();
@@ -419,18 +612,150 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testChannelStillConnectedStopNotForcedDisconnectWithAutoReconnect() throws Exception {
     initMocks(false);
-    Mockito.doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
-    Mockito.doCallRealMethod().when(services).getShutdownCause();
-    Mockito.doCallRealMethod().when(services).emergencyClose();
-    Mockito.doReturn(false).when(services).isShutdownDueToForcedDisconnect();
-    Mockito.doReturn(true).when(services).isAutoReconnectEnabled();
-    Mockito.doReturn(false).when(manager).isReconnectingDS();
+    doCallRealMethod().when(services).setShutdownCause(any(ForcedDisconnectException.class));
+    doCallRealMethod().when(services).getShutdownCause();
+    doCallRealMethod().when(services).emergencyClose();
+    doReturn(false).when(services).isShutdownDueToForcedDisconnect();
+    doReturn(true).when(services).isAutoReconnectEnabled();
+    doReturn(false).when(manager).isReconnectingDS();
     services.setShutdownCause(new ForcedDisconnectException("Test Forced Disconnect"));
     assertTrue(messenger.myChannel.isConnected());
     messenger.stop();
     assertFalse(messenger.myChannel.isConnected());
   }
   
+  @Test
+  public void testMessageFiltering() throws Exception {
+    initMocks(true);
+    InternalDistributedMember mbr = createAddress(8888);
+    NetView view = new NetView(mbr);
+    
+    // the digest should be set in an outgoing join response
+    JoinResponseMessage joinResponse = new JoinResponseMessage(mbr, view);
+    messenger.filterOutgoingMessage(joinResponse);
+    assertNotNull(joinResponse.getMessengerData());
+    
+    // save the view digest for later
+    byte[] data = joinResponse.getMessengerData();
+    
+    // the digest should be used and the message bytes nulled out in an incoming join response
+    messenger.filterIncomingMessage(joinResponse);
+    assertNull(joinResponse.getMessengerData());
+    
+    // the digest shouldn't be set in an outgoing rejection message
+    joinResponse = new JoinResponseMessage("you can't join my distributed system.  nyah nyah nyah!");
+    messenger.filterOutgoingMessage(joinResponse);
+    assertNull(joinResponse.getMessengerData());
+    
+    // the digest shouldn't be installed from an incoming rejection message
+    joinResponse.setMessengerData(data);
+    messenger.filterIncomingMessage(joinResponse);
+    assertNotNull(joinResponse.getMessengerData());
+  }
+  
+  @Test
+  public void testPingPong() throws Exception {
+    initMocks(false);
+    GMSPingPonger pinger = messenger.getPingPonger();
+    InternalDistributedMember mbr = createAddress(8888);
+    JGAddress addr = new JGAddress(mbr);
+    
+    Message pingMessage = pinger.createPingMessage(null, addr);
+    assertTrue(pinger.isPingMessage(pingMessage.getBuffer()));
+    assertFalse(pinger.isPongMessage(pingMessage.getBuffer()));
+    
+    Message pongMessage = pinger.createPongMessage(null, addr);
+    assertTrue(pinger.isPongMessage(pongMessage.getBuffer()));
+    assertFalse(pinger.isPingMessage(pongMessage.getBuffer()));
+    
+    interceptor.collectMessages = true;
+    pinger.sendPingMessage(messenger.myChannel, null, addr);
+    assertEquals("expected 1 message but found " + interceptor.collectedMessages, interceptor.collectedMessages.size(), 1);
+    pingMessage = interceptor.collectedMessages.get(0);
+    assertTrue(pinger.isPingMessage(pingMessage.getBuffer()));
+    
+    interceptor.collectedMessages.clear();
+    pinger.sendPongMessage(messenger.myChannel, null, addr);
+    assertEquals("expected 1 message but found " + interceptor.collectedMessages, interceptor.collectedMessages.size(), 1);
+    pongMessage = interceptor.collectedMessages.get(0);
+    assertTrue(pinger.isPongMessage(pongMessage.getBuffer()));
+
+    interceptor.collectedMessages.clear();
+    JGroupsReceiver receiver = (JGroupsReceiver)messenger.myChannel.getReceiver();
+    long pongsReceived = messenger.pongsReceived;
+    receiver.receive(pongMessage);
+    assertEquals(pongsReceived+1, messenger.pongsReceived);
+    receiver.receive(pingMessage);
+    assertEquals("expected 1 message but found " + interceptor.collectedMessages, interceptor.collectedMessages.size(), 1);
+    Message m = interceptor.collectedMessages.get(0);
+    assertTrue(pinger.isPongMessage(m.getBuffer()));
+  }
+  
+  @Test
+  public void testJGroupsIOExceptionHandler() throws Exception {
+    initMocks(false);
+    InternalDistributedMember mbr = createAddress(8888);
+    NetView v = new NetView(mbr);
+    v.add(messenger.getMemberID());
+    messenger.installView(v);
+
+    IOException ioe = new IOException("test exception");
+    messenger.handleJGroupsIOException(ioe, new JGAddress(mbr));
+    messenger.handleJGroupsIOException(ioe, new JGAddress(mbr)); // should be ignored
+    verify(healthMonitor).checkIfAvailable(mbr, "Unable to send messages to this member via JGroups", true);
+  }
+  
+  @Test
+  public void testReceiver() throws Exception {
+    initMocks(false);
+    JGroupsReceiver receiver = (JGroupsReceiver)messenger.myChannel.getReceiver();
+    
+    // a zero-length message is ignored
+    Message msg = new Message(new JGAddress(messenger.getMemberID()));
+    Object result = messenger.readJGMessage(msg);
+    assertNull(result);
+    
+    // for code coverage we need to pump this message through the receiver
+    receiver.receive(msg);
+    
+    // for more code coverage we need to actually set a buffer in the message
+    msg.setBuffer(new byte[0]);
+    result = messenger.readJGMessage(msg);
+    assertNull(result);
+    receiver.receive(msg);
+    
+    // now create a view and a real distribution-message
+    InternalDistributedMember myAddress = messenger.getMemberID();
+    InternalDistributedMember other = createAddress(8888);
+    NetView v = new NetView(myAddress);
+    v.add(other);
+    when(joinLeave.getView()).thenReturn(v);
+    messenger.installView(v);
+
+    List<InternalDistributedMember> recipients = v.getMembers();
+    SerialAckedMessage dmsg = new SerialAckedMessage();
+    dmsg.setRecipients(recipients);
+
+    // a message is ignored during manager shutdown
+    msg = messenger.createJGMessage(dmsg, new JGAddress(other), Version.CURRENT_ORDINAL);
+    when(manager.shutdownInProgress()).thenReturn(Boolean.TRUE);
+    receiver.receive(msg);
+    verify(manager, never()).processMessage(isA(DistributionMessage.class));
+  }
+  
+  @Test
+  public void testUseOldJChannel() throws Exception {
+    initMocks(false);
+    JChannel channel = messenger.myChannel;
+    services.getConfig().getTransport().setOldDSMembershipInfo(channel);
+    JGroupsMessenger newMessenger = new JGroupsMessenger();
+    newMessenger.init(services);
+    newMessenger.start();
+    newMessenger.started();
+    newMessenger.stop();
+    assertTrue(newMessenger.myChannel == messenger.myChannel);
+  }
+  
   /**
    * creates an InternalDistributedMember address that can be used
    * with the doctored JGroups channel.  This includes a logical
@@ -439,7 +764,7 @@ public class JGroupsMessengerJUnitTest {
    * @param port the UDP port to use for the new address
    */
   private InternalDistributedMember createAddress(int port) {
-    GMSMember gms = new GMSMember("localhost", 8888);
+    GMSMember gms = new GMSMember("localhost",  port);
     gms.setUUID(UUID.randomUUID());
     gms.setVmKind(DistributionManager.NORMAL_DM_TYPE);
     gms.setVersionOrdinal(Version.CURRENT_ORDINAL);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java b/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java
index 0004246..5dffa47 100644
--- a/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java
+++ b/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java
@@ -31,4 +31,6 @@ public interface RemoteDUnitVMIF extends Remote {
   MethExecutorResult executeMethodOnClass(String name, String methodName,
       Object[] args) throws RemoteException;
 
+  void shutDownVM() throws RemoteException;
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/test/java/dunit/standalone/ChildVM.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/standalone/ChildVM.java b/gemfire-core/src/test/java/dunit/standalone/ChildVM.java
index 67b2710..45a236a 100644
--- a/gemfire-core/src/test/java/dunit/standalone/ChildVM.java
+++ b/gemfire-core/src/test/java/dunit/standalone/ChildVM.java
@@ -34,6 +34,15 @@ import dunit.standalone.DUnitLauncher.MasterRemote;
  */
 public class ChildVM {
   
+  private static boolean stopMainLoop = false;
+  
+  /**
+   * tells the main() loop to exit
+   */
+  public static void stopVM() {
+    stopMainLoop = true;
+  }
+  
   static {
     createHydraLogWriter();
   }
@@ -54,7 +63,7 @@ public class ChildVM {
       Naming.rebind("//localhost:" + namingPort + "/vm" + vmNum, dunitVM);
       holder.signalVMReady();
       //This loop is here so this VM will die even if the master is mean killed.
-      while(true) {
+      while (!stopMainLoop) {
         holder.ping();
         Thread.sleep(1000);
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java b/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
index f3109f3..72c33d6 100644
--- a/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
+++ b/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
@@ -169,6 +169,30 @@ public class DUnitLauncher {
 
     Runtime.getRuntime().addShutdownHook(new Thread() {
       public void run() {
+//        System.out.println("shutting down DUnit JVMs");
+//        for (int i=0; i<NUM_VMS; i++) {
+//          try {
+//            processManager.getStub(i).shutDownVM();
+//          } catch (Exception e) {
+//            System.out.println("exception shutting down vm_"+i+": " + e);
+//          }
+//        }
+//        // TODO - hasLiveVMs always returns true
+//        System.out.print("waiting for JVMs to exit");
+//        long giveUp = System.currentTimeMillis() + 5000;
+//        while (giveUp > System.currentTimeMillis()) {
+//          if (!processManager.hasLiveVMs()) {
+//            return;
+//          }
+//          System.out.print(".");
+//          System.out.flush();
+//          try {
+//            Thread.sleep(1000);
+//          } catch (InterruptedException e) {
+//            break;
+//          }
+//        }
+//        System.out.println("\nkilling any remaining JVMs");
         processManager.killVMs();
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java b/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
index 60ac04d..7fc762f 100644
--- a/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
+++ b/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
@@ -98,12 +98,20 @@ public class ProcessManager {
   public synchronized void killVMs() {
     for(ProcessHolder process : processes.values()) {
       if(process != null) {
-        //TODO - stop it gracefully? Why bother
         process.kill();
       }
     }
   }
   
+  public synchronized boolean hasLiveVMs() {
+    for(ProcessHolder process : processes.values()) {
+      if(process != null && process.isAlive()) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
   public synchronized void bounce(int vmNum) {
     if(!processes.containsKey(vmNum)) {
       throw new IllegalStateException("No such process " + vmNum);
@@ -240,6 +248,10 @@ public class ProcessManager {
     public boolean isKilled() {
       return killed;
     }
+    
+    public boolean isAlive() {
+      return !killed && process.isAlive();
+    }
   }
 
   public RemoteDUnitVMIF getStub(int i) throws AccessException, RemoteException, NotBoundException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/test/java/dunit/standalone/RemoteDUnitVM.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/standalone/RemoteDUnitVM.java b/gemfire-core/src/test/java/dunit/standalone/RemoteDUnitVM.java
index 15acc2e..742dc55 100644
--- a/gemfire-core/src/test/java/dunit/standalone/RemoteDUnitVM.java
+++ b/gemfire-core/src/test/java/dunit/standalone/RemoteDUnitVM.java
@@ -135,11 +135,10 @@ public class RemoteDUnitVM extends UnicastRemoteObject implements RemoteDUnitVMI
     
   }
 
-  public void shutDownVM(boolean disconnect, boolean runShutdownHook)
-      throws RemoteException {
+  public void shutDownVM() throws RemoteException {
+    ChildVM.stopVM();
   }
 
-  public void disconnectVM()
-  throws RemoteException {
+  public void disconnectVM() throws RemoteException {
   }
 }



[42/50] [abbrv] incubator-geode git commit: GEODE-637: Additional tests for AsyncEventQueues

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/476c6cd3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
new file mode 100644
index 0000000..1eafbb0
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
@@ -0,0 +1,1911 @@
+/*=========================================================================
+ * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
+ * This product is protected by U.S. and international copyright
+ * and intellectual property laws. Pivotal products are covered by
+ * one or more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+package com.gemstone.gemfire.internal.cache.wan.asyncqueue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.junit.Ignore;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueueFactory;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
+import com.gemstone.gemfire.cache.wan.GatewaySender.OrderPolicy;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.cache.wan.AsyncEventQueueTestBase;
+
+public class AsyncEventListenerDUnitTest extends AsyncEventQueueTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public AsyncEventListenerDUnitTest(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  /**
+   * Test to verify that AsyncEventQueue can not be created when null listener
+   * is passed.
+   */
+  public void testCreateAsyncEventQueueWithNullListener() {
+    AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(testName);
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    InternalDistributedSystem ds = test.getSystem(props);
+    cache = CacheFactory.create(ds);
+
+    AsyncEventQueueFactory asyncQueueFactory = cache
+        .createAsyncEventQueueFactory();
+    try {
+      asyncQueueFactory.create("testId", null);
+      fail("AsyncQueueFactory should not allow to create AsyncEventQueue with null listener");
+    }
+    catch (IllegalArgumentException e) {
+      // expected
+    }
+
+  }
+
+  public void testSerialAsyncEventQueueAttributes() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 150, true, true, "testDS", true });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventQueueAttributes",
+        new Object[] { "ln", 100, 150, AsyncEventQueueFactoryImpl.DEFAULT_BATCH_TIME_INTERVAL, true, "testDS", true, true });
+  }
+  
+  public void testSerialAsyncEventQueueSize() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm5
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm6
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm7
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    pause(1000);// pause at least for the batchTimeInterval
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        1000 });
+
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventQueueSize", new Object[] { "ln" });
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventQueueSize", new Object[] { "ln" });
+    assertEquals("Size of AsyncEventQueue is incorrect", 1000, vm4size);
+    assertEquals("Size of AsyncEventQueue is incorrect", 1000, vm5size);
+  }
+  
+  /**
+   * Added to reproduce defect #50366: 
+   * NullPointerException with AsyncEventQueue#size() when number of dispatchers is more than 1
+   */
+  public void testConcurrentSerialAsyncEventQueueSize() {
+	Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+		"createFirstLocatorWithDSId", new Object[] { 1 });
+
+	vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+	vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+	vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+	vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 150, true, false, null, false, 2, OrderPolicy.KEY });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 150, true, false, null, false, 2, OrderPolicy.KEY });
+
+	vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+		new Object[] { testName + "_RR", "ln", isOffHeap() });
+	vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+		new Object[] { testName + "_RR", "ln", isOffHeap() });
+	vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+	    new Object[] { testName + "_RR", "ln", isOffHeap() });
+	vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+		new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+	vm4
+	  .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+		new Object[] { "ln" });
+	vm5
+	  .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+		new Object[] { "ln" });
+
+	pause(1000);// pause at least for the batchTimeInterval
+
+	vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+		1000 });
+
+	int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class,
+		"getAsyncEventQueueSize", new Object[] { "ln" });
+	int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class,
+		"getAsyncEventQueueSize", new Object[] { "ln" });
+	assertEquals("Size of AsyncEventQueue is incorrect", 1000, vm4size);
+	assertEquals("Size of AsyncEventQueue is incorrect", 1000, vm5size);
+  }
+  
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated WAN: Serial Region persistence enabled: false Async
+   * channel persistence enabled: false
+   */
+
+  public void testReplicatedSerialAsyncEventQueue() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        1000 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+  }
+  
+  /**
+   * Verify that the events loaded by CacheLoader reach the AsyncEventListener
+   * with correct operation detail (added for defect #50237).
+   */
+  public void testReplicatedSerialAsyncEventQueueWithCacheLoader() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false, "MyAsyncEventListener_CacheLoader" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false, "MyAsyncEventListener_CacheLoader" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false, "MyAsyncEventListener_CacheLoader" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false, "MyAsyncEventListener_CacheLoader" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithCacheLoaderAndAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithCacheLoaderAndAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithCacheLoaderAndAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithCacheLoaderAndAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doGets", new Object[] { testName + "_RR",
+        10 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventForOperationDetail",
+        new Object[] { "ln", 10, true, false });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventForOperationDetail",
+        new Object[] { "ln", 0, true, false });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventForOperationDetail",
+        new Object[] { "ln", 0, true, false });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventForOperationDetail",
+        new Object[] { "ln", 0, true, false });// secondary
+  }
+  
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated 
+   * WAN: Serial 
+   * Region persistence enabled: false 
+   * Async queue persistence enabled: false
+   * 
+   * Error is thrown from AsyncEventListener implementation while processing the batch.
+   * Added to test the fix done for defect #45152.
+   */
+
+  public void testReplicatedSerialAsyncEventQueue_ExceptionScenario() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithCustomListener", new Object[] { "ln",
+        false, 100, 100, false, false, null, false, 1 });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithCustomListener", new Object[] { "ln",
+        false, 100, 100, false, false, null, false, 1 });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithCustomListener", new Object[] { "ln",
+        false, 100, 100, false, false, null, false, 1 });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithCustomListener", new Object[] { "ln",
+        false, 100, 100, false, false, null, false, 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm5
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm6
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm7
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    pause(2000);// pause at least for the batchTimeInterval
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        100 });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateCustomAsyncEventListener",
+        new Object[] { "ln", 100 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateCustomAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateCustomAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateCustomAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+  }
+
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated WAN: Serial Region persistence enabled: false Async
+   * channel persistence enabled: false AsyncEventQueue conflation enabled: true
+   */
+  public void testReplicatedSerialAsyncEventQueueWithConflationEnabled() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm5
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm6
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm7
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    pause(1000);// pause at least for the batchTimeInterval
+
+    final Map keyValues = new HashMap();
+    final Map updateKeyValues = new HashMap();
+    for (int i = 0; i < 1000; i++) {
+      keyValues.put(i, i);
+    }
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_RR", keyValues });
+
+    pause(1000);
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] {
+        "ln", keyValues.size() });
+
+    for (int i = 0; i < 500; i++) {
+      updateKeyValues.put(i, i + "_updated");
+    }
+
+    // Put the update events and check the queue size.
+    // There should be no conflation with the previous create events.
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_RR", updateKeyValues });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] {
+        "ln", keyValues.size() + updateKeyValues.size() });
+
+    // Put the update events again and check the queue size.
+    // There should be conflation with the previous update events.
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_RR", updateKeyValues });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] {
+        "ln", keyValues.size() + updateKeyValues.size() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+  }
+
+  
+
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated WAN: Serial Region persistence enabled: false Async
+   * event queue persistence enabled: false
+   * 
+   * Note: The test doesn't create a locator but uses MCAST port instead.
+   */
+  @Ignore("Disabled until I can sort out the hydra dependencies - see bug 52214")
+  public void DISABLED_testReplicatedSerialAsyncEventQueueWithoutLocator() {
+    int mPort = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCacheWithoutLocator",
+        new Object[] { mPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCacheWithoutLocator",
+        new Object[] { mPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCacheWithoutLocator",
+        new Object[] { mPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCacheWithoutLocator",
+        new Object[] { mPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        1000 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+  }
+
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated WAN: Serial Region persistence enabled: false Async
+   * channel persistence enabled: true
+   * 
+   * No VM is restarted.
+   */
+
+  public void testReplicatedSerialAsyncEventQueueWithPeristenceEnabled() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        1000 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+  }
+
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated WAN: Serial Region persistence enabled: false Async
+   * channel persistence enabled: true
+   * 
+   * There is only one vm in the site and that vm is restarted
+   */
+
+  @Ignore("Disabled for 52351")
+  public void DISABLED_testReplicatedSerialAsyncEventQueueWithPeristenceEnabled_Restart() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    String firstDStore = (String)vm4.invoke(AsyncEventQueueTestBase.class,
+        "createAsyncEventQueueWithDiskStore", new Object[] { "ln", false, 100,
+            100, true, null });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    // pause async channel and then do the puts
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        1000 });
+
+    // ------------------ KILL VM4 AND REBUILD
+    // ------------------------------------------
+    vm4.invoke(AsyncEventQueueTestBase.class, "killSender", new Object[] {});
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithDiskStore",
+        new Object[] { "ln", false, 100, 100, true, firstDStore });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    // -----------------------------------------------------------------------------------
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });// primary sender
+  }
+
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated WAN: Serial Region persistence enabled: false Async
+   * channel persistence enabled: true
+   * 
+   * There are 3 VMs in the site and the VM with primary sender is shut down.
+   */
+  @Ignore("Disabled for 52351")
+  public void DISABLED_testReplicatedSerialAsyncEventQueueWithPeristenceEnabled_Restart2() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithDiskStore",
+        new Object[] { "ln", false, 100, 100, true, null });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithDiskStore",
+        new Object[] { "ln", false, 100, 100, true, null });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithDiskStore",
+        new Object[] { "ln", false, 100, 100, true, null });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm4.invoke(AsyncEventQueueTestBase.class, "addCacheListenerAndCloseCache",
+        new Object[] { testName + "_RR" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm5.invoke(AsyncEventQueueTestBase.class, "doPuts",
+        new Object[] { testName + "_RR", 2000 });
+
+    // -----------------------------------------------------------------------------------
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForSenderToBecomePrimary",
+        new Object[] { AsyncEventQueueImpl
+            .getSenderIdFromAsyncEventQueueId("ln") });
+    
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventListenerMapSize", new Object[] { "ln" });
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventListenerMapSize", new Object[] { "ln" });
+
+    getLogWriter().info("vm4 size is: " + vm4size);
+    getLogWriter().info("vm5 size is: " + vm5size);
+    // verify that there is no event loss
+    assertTrue(
+        "Total number of entries in events map on vm4 and vm5 should be at least 2000",
+        (vm4size + vm5size) >= 2000);
+  }
+  
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated 
+   * WAN: Serial 
+   * Dispatcher threads: more than 1
+   * Order policy: key based ordering
+   */
+  public void testConcurrentSerialAsyncEventQueueWithReplicatedRegion() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, OrderPolicy.KEY });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, OrderPolicy.KEY });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, OrderPolicy.KEY });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, OrderPolicy.KEY });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        1000 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 1000 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 0 });// secondary
+  }
+  
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated 
+   * WAN: Serial 
+   * Region persistence enabled: false 
+   * Async queue persistence enabled: false
+   */
+  public void testConcurrentSerialAsyncEventQueueWithReplicatedRegion_2() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, OrderPolicy.THREAD });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, OrderPolicy.THREAD });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, OrderPolicy.THREAD });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, OrderPolicy.THREAD });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4.invokeAsync(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        500 });
+    vm4.invokeAsync(AsyncEventQueueTestBase.class, "doNextPuts", new Object[] { testName + "_RR",
+      500, 1000 });
+    vm4.invokeAsync(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+      1000, 1500 });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 1000 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 0 });// secondary
+  }
+  
+  /**
+   * Dispatcher threads set to more than 1 but no order policy set.
+   * Added for defect #50514.
+   */
+  public void testConcurrentSerialAsyncEventQueueWithoutOrderPolicy() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, null });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, null });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, null });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false, 3, null });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        1000 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 1000 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] {"ln", 0 });// secondary
+  }
+
+  /**
+   * Test configuration::
+   * 
+   * Region: Partitioned WAN: Serial Region persistence enabled: false Async
+   * channel persistence enabled: false
+   */
+  public void testPartitionedSerialAsyncEventQueue() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_PR",
+        500 });
+    vm5.invoke(AsyncEventQueueTestBase.class, "doPutsFrom", new Object[] {
+        testName + "_PR", 500, 1000 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+  }
+
+  /**
+   * Test configuration::
+   * 
+   * Region: Partitioned WAN: Serial Region persistence enabled: false Async
+   * channel persistence enabled: false AsyncEventQueue conflation enabled: true
+   */
+  public void testPartitionedSerialAsyncEventQueueWithConflationEnabled() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm5
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm6
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm7
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    
+    pause(2000);
+
+    final Map keyValues = new HashMap();
+    final Map updateKeyValues = new HashMap();
+    for (int i = 0; i < 1000; i++) {
+      keyValues.put(i, i);
+    }
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_PR", keyValues });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] {
+        "ln", keyValues.size() });
+
+    for (int i = 0; i < 500; i++) {
+      updateKeyValues.put(i, i + "_updated");
+    }
+
+    // Put the update events and check the queue size.
+    // There should be no conflation with the previous create events.
+    vm5.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_PR", updateKeyValues });
+
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] {
+        "ln", keyValues.size() + updateKeyValues.size() });
+
+    // Put the update events again and check the queue size.
+    // There should be conflation with the previous update events.
+    vm5.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+      testName + "_PR", updateKeyValues });
+
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] {
+      "ln", keyValues.size() + updateKeyValues.size() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+  }
+
+  /**
+   * Test configuration::
+   * 
+   * Region: Partitioned WAN: Serial Region persistence enabled: false Async
+   * channel persistence enabled: true
+   * 
+   * No VM is restarted.
+   */
+  public void testPartitionedSerialAsyncEventQueueWithPeristenceEnabled() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, true, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, true, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, true, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, true, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_PR",
+        500 });
+    vm5.invoke(AsyncEventQueueTestBase.class, "doPutsFrom", new Object[] {
+        testName + "_PR", 500, 1000 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+  }
+
+  /**
+   * Test configuration::
+   * 
+   * Region: Partitioned WAN: Serial Region persistence enabled: false Async
+   * channel persistence enabled: true
+   * 
+   * There is only one vm in the site and that vm is restarted
+   */
+  public void testPartitionedSerialAsyncEventQueueWithPeristenceEnabled_Restart() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    String firstDStore = (String)vm4.invoke(AsyncEventQueueTestBase.class,
+        "createAsyncEventQueueWithDiskStore", new Object[] { "ln", false, 100,
+            100, true, null });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    // pause async channel and then do the puts
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueueAndWaitForDispatcherToPause",
+            new Object[] { "ln" });
+  
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_PR",
+        1000 });
+
+    // ------------------ KILL VM4 AND REBUILD
+    // ------------------------------------------
+    vm4.invoke(AsyncEventQueueTestBase.class, "killSender", new Object[] {});
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithDiskStore",
+        new Object[] { "ln", false, 100, 100, true, firstDStore });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    // -----------------------------------------------------------------------------------
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });// primary sender
+  }
+
+  public void testParallelAsyncEventQueueWithReplicatedRegion() {
+    try {
+      Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+          "createFirstLocatorWithDSId", new Object[] { 1 });
+
+      vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+      vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+      vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+      vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+      vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] {
+          "ln", true, 100, 100, true, false, null, false });
+      vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] {
+          "ln", true, 100, 100, true, false, null, false });
+      vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] {
+          "ln", true, 100, 100, true, false, null, false });
+      vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] {
+          "ln", true, 100, 100, true, false, null, false });
+
+      vm4.invoke(AsyncEventQueueTestBase.class,
+          "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+              testName + "_RR", "ln", isOffHeap() });
+      fail("Expected GatewaySenderConfigException where parallel async event queue can not be used with replicated region");
+    }
+    catch (Exception e) {
+      if (!e.getCause().getMessage()
+          .contains("can not be used with replicated region")) {
+        fail("Expected GatewaySenderConfigException where parallel async event queue can not be used with replicated region");
+      }
+    }
+  }
+  
+  public void testParallelAsyncEventQueue() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_PR",
+        256 });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm6size = (Integer)vm6.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm7size = (Integer)vm7.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    
+    assertEquals(vm4size + vm5size + vm6size + vm7size, 256);
+  }
+  
+  /**
+   * Verify that the events reaching the AsyncEventListener have correct operation detail.
+   * (added for defect #50237).
+   */
+  public void testParallelAsyncEventQueueWithCacheLoader() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+	  "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+    	true, 100, 100, false, false, null, false, "MyAsyncEventListener_CacheLoader" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+    	true, 100, 100, false, false, null, false, "MyAsyncEventListener_CacheLoader" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+    	true, 100, 100, false, false, null, false, "MyAsyncEventListener_CacheLoader" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+    	true, 100, 100, false, false, null, false, "MyAsyncEventListener_CacheLoader" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithCacheLoaderAndAsyncQueue",
+    	new Object[] { testName + "_PR", "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithCacheLoaderAndAsyncQueue",
+    	new Object[] { testName + "_PR", "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithCacheLoaderAndAsyncQueue",
+    	new Object[] { testName + "_PR", "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithCacheLoaderAndAsyncQueue",
+    	new Object[] { testName + "_PR", "ln" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPutAll", new Object[] { testName + "_PR",
+    	100, 10 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventForOperationDetail",
+    	new Object[] { "ln", 250, false, true });
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventForOperationDetail",
+    	new Object[] { "ln", 250, false, true });
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventForOperationDetail",
+    	new Object[] { "ln", 250, false, true });
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventForOperationDetail",
+    	new Object[] { "ln", 250, false, true });
+  }
+  
+  public void testParallelAsyncEventQueueSize() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm5
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm6
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm7
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    pause(1000);// pause at least for the batchTimeInterval
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_PR",
+        1000 });
+
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventQueueSize", new Object[] { "ln" });
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventQueueSize", new Object[] { "ln" });
+    
+    assertEquals("Size of AsyncEventQueue is incorrect", 1000, vm4size);
+    assertEquals("Size of AsyncEventQueue is incorrect", 1000, vm5size);
+  }
+  
+  /**
+   * Added to reproduce defect #50366: 
+   * NullPointerException with AsyncEventQueue#size() when number of dispatchers is more than 1
+   */
+  public void testConcurrentParallelAsyncEventQueueSize() {
+	Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+	  "createFirstLocatorWithDSId", new Object[] { 1 });
+
+	vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+	vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+	vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+	vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+	vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+	  true, 100, 100, false, false, null, false, 2, OrderPolicy.KEY });
+	vm5.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+	  true, 100, 100, false, false, null, false, 2, OrderPolicy.KEY });
+	vm6.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+	  true, 100, 100, false, false, null, false, 2, OrderPolicy.KEY });
+	vm7.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+	  true, 100, 100, false, false, null, false, 2, OrderPolicy.KEY });
+
+	vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+	  new Object[] { testName + "_PR", "ln", isOffHeap() });
+	vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+	  new Object[] { testName + "_PR", "ln", isOffHeap() });
+	vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+	  new Object[] { testName + "_PR", "ln", isOffHeap() });
+	vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+	  new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+	vm4
+	  .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+		new Object[] { "ln" });
+	vm5
+	  .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+		new Object[] { "ln" });
+	vm6
+	  .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+		new Object[] { "ln" });
+	vm7
+	  .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+		new Object[] { "ln" });
+	pause(1000);// pause at least for the batchTimeInterval
+
+	vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_PR",
+	  1000 });
+
+	int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class,
+	  "getAsyncEventQueueSize", new Object[] { "ln" });
+	int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class,
+	  "getAsyncEventQueueSize", new Object[] { "ln" });
+	    
+	assertEquals("Size of AsyncEventQueue is incorrect", 1000, vm4size);
+	assertEquals("Size of AsyncEventQueue is incorrect", 1000, vm5size);
+  }
+  
+  public void testParallelAsyncEventQueueWithConflationEnabled() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, true, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, true, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, true, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, true, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm5
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm6
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm7
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+
+    pause(2000);// pause for the batchTimeInterval to ensure that all the
+    // senders are paused
+
+    final Map keyValues = new HashMap();
+    final Map updateKeyValues = new HashMap();
+    for (int i = 0; i < 1000; i++) {
+      keyValues.put(i, i);
+    }
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_PR", keyValues });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] {
+        "ln", keyValues.size() });
+
+    for (int i = 0; i < 500; i++) {
+      updateKeyValues.put(i, i + "_updated");
+    }
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_PR", updateKeyValues });
+
+ 
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncEventQueueSize", new Object[] {
+        "ln", keyValues.size() + updateKeyValues.size() }); // no conflation of creates
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_PR", updateKeyValues });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncEventQueueSize", new Object[] {
+        "ln", keyValues.size() + updateKeyValues.size() }); // conflation of updates
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm6size = (Integer)vm6.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm7size = (Integer)vm7.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    
+    assertEquals(vm4size + vm5size + vm6size + vm7size, keyValues.size());
+  }
+
+  /**
+   * Added to reproduce defect #47213
+   */
+  public void testParallelAsyncEventQueueWithConflationEnabled_bug47213() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, true, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, true, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, true, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, true, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPRWithRedundantCopyWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPRWithRedundantCopyWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPRWithRedundantCopyWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPRWithRedundantCopyWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm5
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm6
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm7
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+
+    pause(2000);// pause for the batchTimeInterval to ensure that all the
+    // senders are paused
+
+    final Map keyValues = new HashMap();
+    final Map updateKeyValues = new HashMap();
+    for (int i = 0; i < 1000; i++) {
+      keyValues.put(i, i);
+    }
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_PR", keyValues });
+
+    pause(2000);
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] {
+        "ln", keyValues.size() });
+
+    for (int i = 0; i < 500; i++) {
+      updateKeyValues.put(i, i + "_updated");
+    }
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_PR", updateKeyValues });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] {
+        testName + "_PR", updateKeyValues });
+
+    // pause to ensure that events have been conflated.
+    pause(2000);
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] {
+        "ln", keyValues.size() + updateKeyValues.size() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm6size = (Integer)vm6.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm7size = (Integer)vm7.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    
+    assertEquals(vm4size + vm5size + vm6size + vm7size, keyValues.size());
+    
+  }
+
+  public void testParallelAsyncEventQueueWithOneAccessor() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm3.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm3.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+
+    vm3.invoke(AsyncEventQueueTestBase.class,
+        "createPartitionedRegionAccessorWithAsyncEventQueue", new Object[] {
+            testName + "_PR", "ln" });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm3.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_PR",
+        256 });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    
+    vm3.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });
+    
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm6size = (Integer)vm6.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm7size = (Integer)vm7.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    
+    assertEquals(vm4size + vm5size + vm6size + vm7size, 256);
+
+  }
+
+  public void testParallelAsyncEventQueueWithPersistence() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, true, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, true, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, true, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, true, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_PR",
+        256 });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm6size = (Integer)vm6.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    int vm7size = (Integer)vm7.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+        new Object[] { "ln"});
+    
+    assertEquals(vm4size + vm5size + vm6size + vm7size, 256);
+  }
+  
+  /**
+   * Below test is disabled intentionally Replicated region with Parallel Async
+   * Event queue is not supported. Test is added for the same
+   * testParallelAsyncEventQueueWithReplicatedRegion
+   * 
+   * We are gone support this configuration in upcoming releases
+   */
+  
+  public void DISABLED_DUETO_BUG51491_testReplicatedParallelAsyncEventQueue() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        true, 100, 100, false, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        1000 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventListenerMapSize", new Object[] { "ln" });
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventListenerMapSize", new Object[] { "ln" });
+    int vm6size = (Integer)vm6.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventListenerMapSize", new Object[] { "ln" });
+    int vm7size = (Integer)vm7.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventListenerMapSize", new Object[] { "ln" });
+
+    assertEquals(vm4size + vm5size + vm6size + vm7size, 1000);
+  }
+  
+/**
+ * Test case to test possibleDuplicates. vm4 & vm5 are hosting the PR. vm5 is
+ * killed so the buckets hosted by it are shifted to vm4.
+ */
+  @Ignore("Disabled for 52349")
+  public void DISABLED_testParallelAsyncEventQueueHA_Scenario1() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+      "createFirstLocatorWithDSId", new Object[] { 1 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    getLogWriter().info("Created the cache");
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithListener2",
+        new Object[] { "ln", true, 100, 5, false, null });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueueWithListener2",
+        new Object[] { "ln", true, 100, 5, false, null });
+
+    getLogWriter().info("Created the AsyncEventQueue");
+
+    vm4.invoke(AsyncEventQueueTestBase.class,
+        "createPRWithRedundantCopyWithAsyncEventQueue", new Object[] {
+            testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class,
+        "createPRWithRedundantCopyWithAsyncEventQueue", new Object[] {
+            testName + "_PR", "ln", isOffHeap() });
+
+    getLogWriter().info("Created PR with AsyncEventQueue");
+
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    vm5
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    pause(1000);// pause for the batchTimeInterval to make sure the AsyncQueue
+                // is paused
+
+    getLogWriter().info("Paused the AsyncEventQueue");
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts",
+        new Object[] { testName + "_PR", 80 });
+
+    getLogWriter().info("Done puts");
+
+    Set<Integer> primaryBucketsVm5 = (Set<Integer>)vm5.invoke(
+        AsyncEventQueueTestBase.class, "getAllPrimaryBucketsOnTheNode",
+        new Object[] { testName + "_PR" });
+
+    getLogWriter().info("Primary buckets on vm5: " + primaryBucketsVm5);
+    // ---------------------------- Kill vm5 --------------------------
+    vm5.invoke(AsyncEventQueueTestBase.class, "killSender", new Object[] {});
+
+    pause(1000);// give some time for rebalancing to happen
+    vm4.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue",
+        new Object[] { "ln" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { 

<TRUNCATED>


[05/50] [abbrv] incubator-geode git commit: GEODE-611: Clean up distribution directives

Posted by kl...@apache.org.
GEODE-611: Clean up distribution directives

Removes the exclude for the findbugs annotation jar which is
no longer a dependency.


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

Branch: refs/heads/feature/GEODE-217
Commit: bff59d1548d71dae61897b72a01ece02a3f3a2b1
Parents: 4d6df30
Author: Anthony Baker <ab...@pivotal.io>
Authored: Tue Dec 1 07:50:40 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Tue Dec 1 11:38:16 2015 -0800

----------------------------------------------------------------------
 gemfire-assembly/build.gradle | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bff59d15/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index e097b65..8de6d4e 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -173,8 +173,6 @@ distributions {
       }
       
       into ('lib') {
-        exclude 'annotation*.jar'
-
         from project(":gemfire-common").configurations.runtime
         from project(":gemfire-common").configurations.archives.allArtifacts.files
 


[45/50] [abbrv] incubator-geode git commit: GEODE-53 - Minor fixes to text and adding links to Apache Incubator

Posted by kl...@apache.org.
GEODE-53 - Minor fixes to text and adding links to Apache Incubator


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

Branch: refs/heads/feature/GEODE-217
Commit: c32a5b27070ff727aa3838430a63df045715fc2e
Parents: d16e78d
Author: William Markito <wm...@pivotal.io>
Authored: Tue Dec 8 16:34:23 2015 -0800
Committer: William Markito <wm...@pivotal.io>
Committed: Tue Dec 8 16:34:23 2015 -0800

----------------------------------------------------------------------
 gemfire-site/content/community/index.html         | 4 ++--
 gemfire-site/content/index.html                   | 2 +-
 gemfire-site/content/releases/index.html          | 2 +-
 gemfire-site/website/content/community/index.html | 2 +-
 gemfire-site/website/content/index.html           | 2 +-
 gemfire-site/website/layouts/footer.html          | 2 +-
 6 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c32a5b27/gemfire-site/content/community/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/community/index.html b/gemfire-site/content/community/index.html
index b1d0783..3c612b4 100644
--- a/gemfire-site/content/community/index.html
+++ b/gemfire-site/content/community/index.html
@@ -246,7 +246,7 @@
 	    	<h2>Join Our Community of Contributors!</h2>
         <p>The Apache Geode team welcomes contributors who want to support the Geode technology. Our community builds everything from this website, from the Geode code to documentation and best practices information.</p>
 
-        <p>We especially welcome additions and corrections to the documentation, wiki, and website to improve the user experience. Bug reports and fixes and additions to the Apache Geode code are welcome. Helping users learn best practices also earns karma in our community.</p>
+        <p>We especially welcome additions and corrections to the documentation, wiki, and website to improve the user experience. Bug reports and fixes and additions to the Apache Geode code are welcome. Helping users learn best practices also earns good karma in our community.</p>
 		</div>
 	</div>
 </section>
@@ -592,7 +592,7 @@
         <div class="row">
           <center>
             <div id="copyright">
-                <img src="/img/egg-logo.png" /><br/><br/>
+                <a href="http://incubator.apache.org" target="_blank"><img src="/img/egg-logo.png" /></a><br/><br/>
                 <p class="credits">
                 Apache Geode is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.
                 </p>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c32a5b27/gemfire-site/content/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/index.html b/gemfire-site/content/index.html
index a450eea..a9d3425 100644
--- a/gemfire-site/content/index.html
+++ b/gemfire-site/content/index.html
@@ -258,7 +258,7 @@ Today Apache Geode is used by over 600 enterprise customers for high-scale busin
         <div class="row">
           <center>
             <div id="copyright">
-                <img src="/img/egg-logo.png" /><br/><br/>
+                <a href="http://incubator.apache.org" target="_blank"><img src="/img/egg-logo.png" /></a><br/><br/>
                 <p class="credits">
                 Apache Geode is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.
                 </p>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c32a5b27/gemfire-site/content/releases/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/releases/index.html b/gemfire-site/content/releases/index.html
index 25bd47c..fb986ff 100644
--- a/gemfire-site/content/releases/index.html
+++ b/gemfire-site/content/releases/index.html
@@ -202,7 +202,7 @@
         <div class="row">
           <center>
             <div id="copyright">
-                <img src="/img/egg-logo.png" /><br/><br/>
+                <a href="http://incubator.apache.org" target="_blank"><img src="/img/egg-logo.png" /></a><br/><br/>
                 <p class="credits">
                 Apache Geode is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.
                 </p>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c32a5b27/gemfire-site/website/content/community/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/community/index.html b/gemfire-site/website/content/community/index.html
index 7abb0a9..ec58600 100644
--- a/gemfire-site/website/content/community/index.html
+++ b/gemfire-site/website/content/community/index.html
@@ -8,7 +8,7 @@
 	    	<h2>Join Our Community of Contributors!</h2>
         <p>The Apache Geode team welcomes contributors who want to support the Geode technology. Our community builds everything from this website, from the Geode code to documentation and best practices information.</p>
 
-        <p>We especially welcome additions and corrections to the documentation, wiki, and website to improve the user experience. Bug reports and fixes and additions to the Apache Geode code are welcome. Helping users learn best practices also earns karma in our community.</p>
+        <p>We especially welcome additions and corrections to the documentation, wiki, and website to improve the user experience. Bug reports and fixes and additions to the Apache Geode code are welcome. Helping users learn best practices also earns good karma in our community.</p>
 		</div>
 	</div>
 </section>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c32a5b27/gemfire-site/website/content/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/index.html b/gemfire-site/website/content/index.html
index 17058e7..555d024 100644
--- a/gemfire-site/website/content/index.html
+++ b/gemfire-site/website/content/index.html
@@ -13,7 +13,7 @@ title: Performance is key. Consistency is a must.
                   Take advantage of Apache Geode's unique technology that blends advanced techniques for data replication, partitioning and distributed processing.
 
                   <br/><br/>
-                  Apache Geode provides a database-like consistency model, reliable transaction processing and a shared-nothing architecture to maintain very low latency performance with high concurrency processing.<br/></p>
+                  Apache Geode (incubating) provides a database-like consistency model, reliable transaction processing and a shared-nothing architecture to maintain very low latency performance with high concurrency processing.<br/></p>
             </div>
 
             <div class="btn-wrapper">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c32a5b27/gemfire-site/website/layouts/footer.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/layouts/footer.html b/gemfire-site/website/layouts/footer.html
index 158ae46..c024af6 100644
--- a/gemfire-site/website/layouts/footer.html
+++ b/gemfire-site/website/layouts/footer.html
@@ -64,7 +64,7 @@
         <div class="row">
           <center>
             <div id="copyright">
-                <img src="/img/egg-logo.png" /><br/><br/>
+                <a href="http://incubator.apache.org" target="_blank"><img src="/img/egg-logo.png" /></a><br/><br/>
                 <p class="credits">
                 Apache Geode is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.
                 </p>


[18/50] [abbrv] incubator-geode git commit: GEODE-626: add unit tests for MemoryChunk

Posted by kl...@apache.org.
GEODE-626: add unit tests for MemoryChunk

Added testGetSize and testCopyBytes.


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

Branch: refs/heads/feature/GEODE-217
Commit: 9283282c64076c5f8e599f3d47895f3c48e205e8
Parents: 3259c02
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Fri Dec 4 16:33:50 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Fri Dec 4 16:47:24 2015 -0800

----------------------------------------------------------------------
 .../offheap/MemoryChunkJUnitTestBase.java       | 47 ++++++++++++++++++++
 1 file changed, 47 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9283282c/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkJUnitTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkJUnitTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkJUnitTestBase.java
index 2e4eabb..c8c0b2b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkJUnitTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkJUnitTestBase.java
@@ -50,6 +50,53 @@ public abstract class MemoryChunkJUnitTestBase {
   }
   
   @Test
+  public void testGetSize() {
+    MemoryChunk mc = createChunk(5);
+    try {
+      assertEquals(5, mc.getSize());
+    } finally {
+      mc.release();
+    }
+    mc = createChunk(0);
+    try {
+      assertEquals(0, mc.getSize());
+    } finally {
+      mc.release();
+    }
+    mc = createChunk(1024);
+    try {
+      assertEquals(1024, mc.getSize());
+    } finally {
+      mc.release();
+    }
+  }
+  
+  @Test
+  public void testCopyBytes() {
+    int CHUNK_SIZE = 1024;
+    MemoryChunk mc = createChunk(CHUNK_SIZE*2);
+    try {
+      for (int i=0; i<CHUNK_SIZE; i++) {
+        mc.writeByte(i, (byte)(i%128));
+      }
+      for (int i=0; i<CHUNK_SIZE; i++) {
+        assertEquals(i%128, mc.readByte(i));
+      }
+      mc.copyBytes(0, CHUNK_SIZE, CHUNK_SIZE);
+      for (int i=0; i<CHUNK_SIZE; i++) {
+        assertEquals(i%128, mc.readByte(CHUNK_SIZE+i));
+      }
+      mc.copyBytes(0, 1, CHUNK_SIZE);
+      for (int i=0; i<CHUNK_SIZE; i++) {
+        assertEquals(i%128, mc.readByte(1+i));
+      }
+    } finally {
+      mc.release();
+    }
+  }
+ 
+  
+  @Test
   public void testByteArrayReadWrite() {
     byte[] writeBytes = new byte[256];
     int v = Byte.MIN_VALUE;


[19/50] [abbrv] incubator-geode git commit: fixing merge error from 38dd3ed that caused this test to start failing again

Posted by kl...@apache.org.
fixing merge error from 38dd3ed that caused this test to start failing again


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

Branch: refs/heads/feature/GEODE-217
Commit: dec83b4ebcc75a9425101985f34d2b9afe650f0d
Parents: 9283282
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Dec 7 08:43:12 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Dec 7 08:43:12 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/locator/GMSLocatorRecoveryJUnitTest.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dec83b4e/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
index 7badce6..2d042fc 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
@@ -152,8 +152,8 @@ public class GMSLocatorRecoveryJUnitTest {
       nonDefault.put(DistributionConfig.MCAST_PORT_NAME, "0");
       nonDefault.put(DistributionConfig.LOG_FILE_NAME, "");
       nonDefault.put(DistributionConfig.LOG_LEVEL_NAME, "fine");
-      nonDefault.put(DistributionConfig.LOCATORS_NAME, localHost.getHostName()+'['+port+']');
-      nonDefault.put(DistributionConfig.BIND_ADDRESS_NAME, localHost.getHostName());
+      nonDefault.put(DistributionConfig.LOCATORS_NAME, localHost.getHostAddress()+'['+port+']');
+      nonDefault.put(DistributionConfig.BIND_ADDRESS_NAME, localHost.getHostAddress());
       DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
       RemoteTransportConfig transport = new RemoteTransportConfig(config,
           DistributionManager.NORMAL_DM_TYPE);


[30/50] [abbrv] incubator-geode git commit: GEODE-563: Moving gfsh tests from closed

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListIndexCommandDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListIndexCommandDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListIndexCommandDUnitTest.java
new file mode 100644
index 0000000..22a38d2
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListIndexCommandDUnitTest.java
@@ -0,0 +1,672 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.query.Index;
+import com.gemstone.gemfire.cache.query.IndexStatistics;
+import com.gemstone.gemfire.cache.query.IndexType;
+import com.gemstone.gemfire.cache.query.SelectResults;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.lang.MutableIdentifiable;
+import com.gemstone.gemfire.internal.lang.ObjectUtils;
+import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.domain.IndexDetails;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * The ListIndexCommandDUnitTest class is distributed test suite of test cases for testing the index-based GemFire shell
+ * (Gfsh) commands. </p>
+ *
+ * @author John Blum
+ * @see com.gemstone.gemfire.management.internal.cli.commands.CliCommandTestBase
+ * @see com.gemstone.gemfire.management.internal.cli.commands.IndexCommands
+ * @since 7.0
+ */
+@SuppressWarnings("unused")
+public class ListIndexCommandDUnitTest extends CliCommandTestBase {
+
+  protected static final int DEFAULT_REGION_INITIAL_CAPACITY = 10000;
+
+  private final AtomicLong idGenerator = new AtomicLong(0l);
+
+  protected static String toString(final Result result) {
+    assert result != null : "The Result object from the command execution cannot be null!";
+
+    final StringBuilder buffer = new StringBuilder(System.getProperty("line.separator"));
+
+    while (result.hasNextLine()) {
+      buffer.append(result.nextLine());
+      buffer.append(System.getProperty("line.separator"));
+    }
+
+    return buffer.toString();
+  }
+
+
+  public ListIndexCommandDUnitTest(final String testName) {
+    super(testName);
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    createDefaultSetup(null);
+    setupGemFire();
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+  }
+
+  protected Index createIndex(final String name, final String indexedExpression, final String fromClause) {
+    return createIndex(name, IndexType.FUNCTIONAL, indexedExpression, fromClause);
+  }
+
+  protected Index createIndex(final String name, final IndexType type, final String indexedExpression,
+      final String fromClause) {
+    return new IndexAdapter(name, type, indexedExpression, fromClause);
+  }
+
+  protected Peer createPeer(final VM vm, final Properties distributedSystemProperties,
+      final RegionDefinition... regions) {
+    final Peer peer = new Peer(vm, distributedSystemProperties);
+    peer.add(regions);
+    return peer;
+  }
+
+  protected RegionDefinition createRegionDefinition(final String regionName, final Class<?> keyConstraint,
+      final Class<?> valueConstraint, final Index... indexes) {
+    final RegionDefinition regionDefinition = new RegionDefinition(regionName, keyConstraint, valueConstraint);
+    regionDefinition.add(indexes);
+    return regionDefinition;
+  }
+
+  protected void setupGemFire() throws Exception {
+    final Host host = Host.getHost(0);
+
+    final VM vm1 = host.getVM(1);
+    final VM vm2 = host.getVM(2);
+
+    final Peer peer1 = createPeer(vm1, createDistributedSystemProperties("consumerServer"),
+        createRegionDefinition("consumers", Long.class, Consumer.class,
+            createIndex("cidIdx", IndexType.PRIMARY_KEY, "id", "/consumers"),
+            createIndex("cnameIdx", "name", "/consumers")));
+
+    final Peer peer2 = createPeer(vm2, createDistributedSystemProperties("producerServer"),
+        createRegionDefinition("producers", Long.class, Producer.class, createIndex("pidIdx", "id", "/producers")));
+
+    createRegionWithIndexes(peer1);
+    createRegionWithIndexes(peer2);
+
+    loadConsumerData(peer1, 10000);
+    loadProducerData(peer2, 10000);
+  }
+
+  protected Properties createDistributedSystemProperties(final String gemfireName) {
+    final Properties distributedSystemProperties = new Properties();
+
+    distributedSystemProperties.setProperty(DistributionConfig.LOG_LEVEL_NAME, getDUnitLogLevel());
+    distributedSystemProperties.setProperty(DistributionConfig.NAME_NAME, gemfireName);
+
+    return distributedSystemProperties;
+  }
+
+  protected void createRegionWithIndexes(final Peer peer) {
+    peer.run(new SerializableRunnable(
+        String.format("Creating Regions with Indexes on GemFire peer (%1$s).", peer.getName())) {
+      public void run() {
+        // create the GemFire distributed system with custom configuration properties...
+        getSystem(peer.getConfiguration());
+
+        final Cache cache = getCache();
+        final RegionFactory regionFactory = cache.createRegionFactory();
+
+        for (RegionDefinition regionDefinition : peer) {
+          regionFactory.setDataPolicy(DataPolicy.REPLICATE);
+          regionFactory.setIndexMaintenanceSynchronous(true);
+          regionFactory.setInitialCapacity(DEFAULT_REGION_INITIAL_CAPACITY);
+          regionFactory.setKeyConstraint(regionDefinition.getKeyConstraint());
+          regionFactory.setScope(Scope.DISTRIBUTED_NO_ACK);
+          regionFactory.setStatisticsEnabled(true);
+          regionFactory.setValueConstraint(regionDefinition.getValueConstraint());
+
+          final Region region = regionFactory.create(regionDefinition.getRegionName());
+          String indexName = null;
+
+          try {
+            for (Index index : regionDefinition) {
+              indexName = index.getName();
+              if (IndexType.PRIMARY_KEY.equals(index.getType())) {
+                cache.getQueryService().createKeyIndex(indexName, index.getIndexedExpression(), region.getFullPath());
+              } else {
+                cache.getQueryService().createIndex(indexName, index.getIndexedExpression(), region.getFullPath());
+              }
+            }
+          } catch (Exception e) {
+            getLogWriter().error(
+                String.format("Error occurred creating Index (%1$s) on Region (%2$s) - (%3$s)", indexName,
+                    region.getFullPath(), e.getMessage()));
+          }
+        }
+      }
+    });
+  }
+
+  protected void loadConsumerData(final Peer peer, final int operationsTotal) {
+    peer.run(new SerializableRunnable("Load /consumers Region with data") {
+      public void run() {
+        final Cache cache = getCache();
+        final Region<Long, Consumer> consumerRegion = cache.getRegion("/consumers");
+
+        final Random random = new Random(System.currentTimeMillis());
+        int count = 0;
+
+        final List<Proxy> proxies = new ArrayList<Proxy>();
+
+        Consumer consumer;
+        Proxy proxy;
+
+        while (count++ < operationsTotal) {
+          switch (CrudOperation.values()[random.nextInt(CrudOperation.values().length)]) {
+            case RETRIEVE:
+              if (!proxies.isEmpty()) {
+                proxy = proxies.get(random.nextInt(proxies.size()));
+                consumer = query(consumerRegion, "id = " + proxy.getId() + "l"); // works
+                //consumer = query(consumerRegion, "Id = " + proxy.getId()); // works
+                //consumer = query(consumerRegion, "id = " + proxy.getId()); // does not work
+                proxy.setUnitsSnapshot(consumer.getUnits());
+                break;
+              }
+            case UPDATE:
+              if (!proxies.isEmpty()) {
+                proxy = proxies.get(random.nextInt(proxies.size()));
+                consumer = query(consumerRegion, "Name = " + proxy.getName());
+                consumer.consume();
+                break;
+              }
+            case CREATE:
+            default:
+              consumer = new Consumer(idGenerator.incrementAndGet());
+              proxies.add(new Proxy(consumer));
+              consumerRegion.put(consumer.getId(), consumer);
+              assertTrue(consumerRegion.containsKey(consumer.getId()));
+              assertTrue(consumerRegion.containsValueForKey(consumer.getId()));
+              assertSame(consumer, consumerRegion.get(consumer.getId()));
+          }
+        }
+      }
+    });
+  }
+
+  protected void loadProducerData(final Peer peer, final int operationsTotal) {
+    peer.run(new SerializableRunnable("Load /producers Region with data") {
+      public void run() {
+        final Cache cache = getCache();
+        final Region<Long, Producer> producerRegion = cache.getRegion("/producers");
+
+        final Random random = new Random(System.currentTimeMillis());
+        int count = 0;
+
+        final List<Proxy> proxies = new ArrayList<Proxy>();
+
+        Producer producer;
+        Proxy proxy;
+
+        while (count++ < operationsTotal) {
+          switch (CrudOperation.values()[random.nextInt(CrudOperation.values().length)]) {
+            case RETRIEVE:
+              if (!proxies.isEmpty()) {
+                proxy = proxies.get(random.nextInt(proxies.size()));
+                producer = query(producerRegion, "Id = " + proxy.getId());
+                proxy.setUnitsSnapshot(producer.getUnits());
+                break;
+              }
+            case UPDATE:
+              if (!proxies.isEmpty()) {
+                proxy = proxies.get(random.nextInt(proxies.size()));
+                producer = query(producerRegion, "Id = " + proxy.getId());
+                producer.produce();
+                break;
+              }
+            case CREATE:
+            default:
+              producer = new Producer(idGenerator.incrementAndGet());
+              proxies.add(new Proxy(producer));
+              producerRegion.put(producer.getId(), producer);
+              assertTrue(producerRegion.containsKey(producer.getId()));
+              assertTrue(producerRegion.containsValueForKey(producer.getId()));
+              assertSame(producer, producerRegion.get(producer.getId()));
+          }
+        }
+      }
+    });
+  }
+
+  @SuppressWarnings("unchecked")
+  protected <T extends Comparable<T>, B extends AbstractBean<T>> B query(final Cache cache, final String queryString) {
+    try {
+      getLogWriter().info(String.format("Running Query (%1$s) in GemFire...", queryString));
+
+      final SelectResults<B> results = (SelectResults<B>) cache.getQueryService().newQuery(queryString).execute();
+
+      getLogWriter().info(
+          String.format("Running Query (%1$s) in GemFire returned (%2$d) result(s).", queryString, results.size()));
+
+      return (results.iterator().hasNext() ? results.iterator().next() : null);
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("An error occurred running Query (%1$s)!", queryString), e);
+    }
+  }
+
+  protected <T extends Comparable<T>, B extends AbstractBean<T>> B query(final Region<T, B> region,
+      final String queryPredicate) {
+    try {
+      getLogWriter().info(
+          String.format("Running Query (%1$s) on Region (%2$s)...", queryPredicate, region.getFullPath()));
+
+      final SelectResults<B> results = region.query(queryPredicate);
+
+      getLogWriter().info(
+          String.format("Running Query (%1$s) on Region (%2$s) returned (%3$d) result(s).", queryPredicate,
+              region.getFullPath(), results.size()));
+
+      return (results.iterator().hasNext() ? results.iterator().next() : null);
+    } catch (Exception e) {
+      throw new RuntimeException(
+          String.format("An error occurred running Query (%1$s) on Region (%2$s)!", queryPredicate,
+              region.getFullPath()), e);
+    }
+  }
+
+  public void testListIndex() throws Exception {
+    final Result result = executeCommand(CliStrings.LIST_INDEX + " --" + CliStrings.LIST_INDEX__STATS);
+
+    assertNotNull(result);
+    getLogWriter().info(toString(result));
+    assertEquals(Result.Status.OK, result.getStatus());
+  }
+
+  protected static class Peer implements Iterable<RegionDefinition>, Serializable {
+
+    private final Properties distributedSystemProperties;
+
+    private final Set<RegionDefinition> regions = new HashSet<RegionDefinition>();
+
+    private final VM vm;
+
+    public Peer(final VM vm, final Properties distributedSystemProperties) {
+      assert distributedSystemProperties != null : "The GemFire Distributed System configuration properties cannot be null!";
+      this.distributedSystemProperties = distributedSystemProperties;
+      this.vm = vm;
+    }
+
+    public Properties getConfiguration() {
+      return this.distributedSystemProperties;
+    }
+
+    public String getName() {
+      return getConfiguration().getProperty(DistributionConfig.NAME_NAME);
+    }
+
+    public VM getVm() {
+      return vm;
+    }
+
+    public boolean add(final RegionDefinition... regionDefinitions) {
+      return (regionDefinitions != null && regions.addAll(Arrays.asList(regionDefinitions)));
+    }
+
+    public Iterator<RegionDefinition> iterator() {
+      return Collections.unmodifiableSet(regions).iterator();
+    }
+
+    public boolean remove(final RegionDefinition... regionDefinitions) {
+      return (regionDefinitions != null && regions.removeAll(Arrays.asList(regionDefinitions)));
+    }
+
+    public void run(final Runnable runnable) {
+      if (getVm() == null) {
+        runnable.run();
+      } else {
+        getVm().invoke(runnable);
+      }
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder buffer = new StringBuilder(getClass().getSimpleName());
+      buffer.append(" {configuration = ").append(getConfiguration());
+      buffer.append(", name = ").append(getName());
+      buffer.append(", pid = ").append(getVm().getPid());
+      buffer.append("}");
+      return buffer.toString();
+    }
+  }
+
+  protected static class IndexAdapter implements Index, Serializable {
+
+    private final IndexDetails.IndexType type;
+
+    private final String fromClause;
+    private final String indexedExpression;
+    private final String name;
+
+    protected IndexAdapter(final String name, final String indexedExpression, final String fromClause) {
+      this(name, IndexType.FUNCTIONAL, indexedExpression, fromClause);
+    }
+
+    protected IndexAdapter(final String name, final IndexType type, final String indexedExpression,
+        final String fromClause) {
+      assert name != null : "The name of the Index cannot be null!";
+      assert indexedExpression != null : String.format("The expression to index for Index (%1$s) cannot be null!",
+          name);
+      assert fromClause != null : String.format("The from clause for Index (%1$s) cannot be null!", name);
+
+      this.type = ObjectUtils.defaultIfNull(IndexDetails.IndexType.valueOf(type), IndexDetails.IndexType.FUNCTIONAL);
+      this.name = name;
+      this.indexedExpression = indexedExpression;
+      this.fromClause = fromClause;
+    }
+
+    public String getName() {
+      return this.name;
+    }
+
+    public String getFromClause() {
+      return this.fromClause;
+    }
+
+    public String getCanonicalizedFromClause() {
+      return this.fromClause;
+    }
+
+    public String getIndexedExpression() {
+      return this.indexedExpression;
+    }
+
+    public String getCanonicalizedIndexedExpression() {
+      return this.indexedExpression;
+    }
+
+    public String getProjectionAttributes() {
+      throw new UnsupportedOperationException("Not Implemented!");
+    }
+
+    public String getCanonicalizedProjectionAttributes() {
+      throw new UnsupportedOperationException("Not Implemented!");
+    }
+
+    public Region<?, ?> getRegion() {
+      throw new UnsupportedOperationException("Not Implemented!");
+    }
+
+    public IndexStatistics getStatistics() {
+      throw new UnsupportedOperationException("Not Implemented!");
+    }
+
+    public IndexType getType() {
+      return type.getType();
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder buffer = new StringBuilder(getClass().getSimpleName());
+      buffer.append(" {indexName = ").append(getName());
+      buffer.append(", indexType = ").append(getType());
+      buffer.append(", indexedExpression = ").append(getIndexedExpression());
+      buffer.append(", fromClause = ").append(getFromClause());
+      buffer.append("}");
+      return buffer.toString();
+    }
+  }
+
+  protected static class RegionDefinition implements Iterable<Index>, Serializable {
+
+    private final Class<?> keyConstraint;
+    private final Class<?> valueConstraint;
+
+    private final Set<Index> indexes = new HashSet<Index>();
+
+    private final String regionName;
+
+    @SuppressWarnings("unchecked")
+    protected RegionDefinition(final String regionName, final Class<?> keyConstraint, final Class<?> valueConstraint) {
+      assert !StringUtils.isBlank(regionName) : "The name of the Region must be specified!";
+      this.regionName = regionName;
+      this.keyConstraint = ObjectUtils.defaultIfNull(keyConstraint, Object.class);
+      this.valueConstraint = ObjectUtils.defaultIfNull(valueConstraint, Object.class);
+    }
+
+    public String getRegionName() {
+      return regionName;
+    }
+
+    public Class<?> getKeyConstraint() {
+      return keyConstraint;
+    }
+
+    public Class<?> getValueConstraint() {
+      return valueConstraint;
+    }
+
+    public boolean add(final Index... indexes) {
+      return (indexes != null && this.indexes.addAll(Arrays.asList(indexes)));
+    }
+
+    public Iterator<Index> iterator() {
+      return Collections.unmodifiableSet(indexes).iterator();
+    }
+
+    public boolean remove(final Index... indexes) {
+      return (indexes != null && this.indexes.removeAll(Arrays.asList(indexes)));
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+      if (obj == this) {
+        return true;
+      }
+
+      if (!(obj instanceof RegionDefinition)) {
+        return false;
+      }
+
+      final RegionDefinition that = (RegionDefinition) obj;
+
+      return ObjectUtils.equals(getRegionName(), that.getRegionName());
+    }
+
+    @Override
+    public int hashCode() {
+      int hashValue = 17;
+      hashValue = 37 * hashValue + ObjectUtils.hashCode(getRegionName());
+      return hashValue;
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder buffer = new StringBuilder(getClass().getSimpleName());
+      buffer.append(" {regionName = ").append(getRegionName());
+      buffer.append(", keyConstraint = ").append(getKeyConstraint());
+      buffer.append(", valueConstraint = ").append(getValueConstraint());
+      buffer.append("}");
+      return buffer.toString();
+    }
+  }
+
+  protected static abstract class AbstractBean<T extends Comparable<T>> implements MutableIdentifiable<T>, Serializable {
+
+    private T id;
+    private String name;
+
+    public AbstractBean() {
+    }
+
+    public AbstractBean(final T id) {
+      this.id = id;
+    }
+
+    public T getId() {
+      return id;
+    }
+
+    public void setId(final T id) {
+      this.id = id;
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    public void setName(final String name) {
+      this.name = name;
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+      if (obj == this) {
+        return true;
+      }
+
+      if (!(getClass().isInstance(obj))) {
+        return false;
+      }
+
+      final AbstractBean bean = (AbstractBean) obj;
+
+      return ObjectUtils.equals(getId(), bean.getId());
+    }
+
+    @Override
+    public int hashCode() {
+      int hashValue = 17;
+      hashValue = 37 * hashValue + ObjectUtils.hashCode(getId());
+      return hashValue;
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder buffer = new StringBuilder(getClass().getSimpleName());
+      buffer.append(" {id = ").append(getId());
+      buffer.append(", name = ").append(getName());
+      buffer.append("}");
+      return buffer.toString();
+    }
+  }
+
+  public static class Consumer extends AbstractBean<Long> {
+
+    private volatile int units;
+
+    public Consumer() {
+    }
+
+    public Consumer(final Long id) {
+      super(id);
+    }
+
+    public int getUnits() {
+      return units;
+    }
+
+    public int consume() {
+      return ++units;
+    }
+  }
+
+  public static class Producer extends AbstractBean<Long> {
+
+    private volatile int units;
+
+    public Producer() {
+    }
+
+    public Producer(final Long id) {
+      super(id);
+    }
+
+    public int getUnits() {
+      return units;
+    }
+
+    public int produce() {
+      return ++units;
+    }
+  }
+
+  public static class Proxy extends AbstractBean<Long> {
+
+    private final AbstractBean<Long> bean;
+    private int unitsSnapshot;
+
+    public Proxy(final AbstractBean<Long> bean) {
+      assert bean != null : "The bean to proxy cannot be null!";
+      this.bean = bean;
+    }
+
+    public AbstractBean<Long> getBean() {
+      return bean;
+    }
+
+    @Override
+    public Long getId() {
+      return getBean().getId();
+    }
+
+    @Override
+    public String getName() {
+      return getBean().getName();
+    }
+
+    public int getUnitsSnapshot() {
+      return unitsSnapshot;
+    }
+
+    public void setUnitsSnapshot(final int unitsSnapshot) {
+      this.unitsSnapshot = unitsSnapshot;
+    }
+  }
+
+  protected static enum CrudOperation {
+    CREATE,
+    RETRIEVE,
+    UPDATE,
+    DELETE
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
new file mode 100644
index 0000000..6623403
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
@@ -0,0 +1,286 @@
+/*
+ * 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.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.FixedPartitionAttributes;
+import com.gemstone.gemfire.cache.PartitionAttributes;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache30.CacheTestCase;
+import com.gemstone.gemfire.distributed.DistributedMember;
+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.GemFireCacheImpl;
+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.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.remote.CommandProcessor;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+/****
+ * @author bansods since 7.0
+ */
+
+public class MemberCommandsDUnitTest extends CacheTestCase {
+  private static final long serialVersionUID = 1L;
+  private static final Map<String, String> EMPTY_ENV = Collections.emptyMap();
+  private static final String REGION1 = "region1";
+  private static final String REGION2 = "region2";
+  private static final String REGION3 = "region3";
+  private static final String SUBREGION1A = "subregion1A";
+  private static final String SUBREGION1B = "subregion1B";
+  private static final String SUBREGION1C = "subregion1C";
+  private static final String PR1 = "PartitionedRegion1";
+  private static final String PR2 = "ParitionedRegion2";
+
+  public MemberCommandsDUnitTest(String name) {
+    super(name);
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    // This test does not require an actual Gfsh connection to work, however when run as part of a suite, prior tests
+    // may mess up the environment causing this test to fail. Setting this prevents false failures.
+    CliUtil.isGfshVM = false;
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+    CliUtil.isGfshVM = true;
+  }
+
+  private Properties createProperties(String name, String groups) {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOG_LEVEL_NAME, "info");
+    props.setProperty(DistributionConfig.STATISTIC_SAMPLING_ENABLED_NAME, "true");
+    props.setProperty(DistributionConfig.ENABLE_TIME_STATISTICS_NAME, "true");
+    props.setProperty(DistributionConfig.NAME_NAME, name);
+    props.setProperty(DistributionConfig.GROUPS_NAME, groups);
+    return props;
+  }
+
+  private void createRegionsWithSubRegions() {
+    final Cache cache = getCache();
+
+    RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+    dataRegionFactory.setConcurrencyLevel(3);
+    Region<String, Integer> region1 = dataRegionFactory.create(REGION1);
+    region1.createSubregion(SUBREGION1C, region1.getAttributes());
+    Region<String, Integer> subregion2 = region1.createSubregion(SUBREGION1A, region1.getAttributes());
+
+    subregion2.createSubregion(SUBREGION1B, subregion2.getAttributes());
+    dataRegionFactory.create(REGION2);
+    dataRegionFactory.create(REGION3);
+  }
+
+  private void createPartitionedRegion1() {
+    final Cache cache = getCache();
+    // Create the data region
+    RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+    dataRegionFactory.create(PR1);
+  }
+
+  private void createPartitionedRegion(String regionName) {
+    final Cache cache = getCache();
+    // Create the data region
+    RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+    dataRegionFactory.setConcurrencyLevel(4);
+    EvictionAttributes ea = EvictionAttributes.createLIFOEntryAttributes(100, EvictionAction.LOCAL_DESTROY);
+    dataRegionFactory.setEvictionAttributes(ea);
+    dataRegionFactory.setEnableAsyncConflation(true);
+
+    FixedPartitionAttributes fpa = FixedPartitionAttributes.createFixedPartition("Par1", true);
+    PartitionAttributes pa = new PartitionAttributesFactory().setLocalMaxMemory(100).setRecoveryDelay(
+        2).setTotalMaxMemory(200).setRedundantCopies(1).addFixedPartitionAttributes(fpa).create();
+    dataRegionFactory.setPartitionAttributes(pa);
+
+    dataRegionFactory.create(regionName);
+  }
+
+
+  private void createLocalRegion() {
+    final Cache cache = getCache();
+    // Create the data region
+    RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.LOCAL);
+    dataRegionFactory.create("LocalRegion");
+  }
+
+  private void setupSystem() throws IOException {
+    disconnectAllFromDS();
+    final Host host = Host.getHost(0);
+    final VM[] servers = {host.getVM(0), host.getVM(1)};
+
+    final Properties propsMe = createProperties("me", "G1");
+    final Properties propsServer1 = createProperties("Server1", "G1");
+    final Properties propsServer2 = createProperties("Server2", "G2");
+
+
+    getSystem(propsMe);
+    final Cache cache = getCache();
+    RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE_PROXY);
+    dataRegionFactory.setConcurrencyLevel(5);
+    Region<String, Integer> region1 = dataRegionFactory.create(REGION1);
+
+
+    servers[1].invoke(new SerializableRunnable("Create cache for server1") {
+      public void run() {
+        getSystem(propsServer2);
+        createRegionsWithSubRegions();
+        createLocalRegion();
+        createPartitionedRegion("ParReg1");
+      }
+    });
+    servers[0].invoke(new SerializableRunnable("Create cache for server0") {
+      public void run() {
+        getSystem(propsServer1);
+        createRegionsWithSubRegions();
+        createLocalRegion();
+      }
+    });
+  }
+
+  private Properties createProperties(Host host, int locatorPort) {
+    Properties props = new Properties();
+
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, getServerHostName(host) + "[" + locatorPort + "]");
+    props.setProperty(DistributionConfig.LOG_LEVEL_NAME, "info");
+    props.setProperty(DistributionConfig.STATISTIC_SAMPLING_ENABLED_NAME, "true");
+    props.setProperty(DistributionConfig.ENABLE_TIME_STATISTICS_NAME, "true");
+    props.put(DistributionConfig.ENABLE_NETWORK_PARTITION_DETECTION_NAME, "true");
+
+    return props;
+  }
+
+  /**
+   * Creates the cache.
+   */
+  private void createCache(Properties props) {
+    getSystem(props);
+    final Cache cache = getCache();
+  }
+
+  /***
+   * Tests the execution of "list member" command which should list out all the members in the DS
+   *
+   * @throws IOException
+   * @throws ClassNotFoundException
+   */
+  public void testListMemberAll() throws IOException, ClassNotFoundException {
+    setupSystem();
+    CommandProcessor commandProcessor = new CommandProcessor();
+    Result result = commandProcessor.createCommandStatement(CliStrings.LIST_MEMBER, EMPTY_ENV).process();
+    getLogWriter().info("#SB" + getResultAsString(result));
+    assertEquals(true, result.getStatus().equals(Status.OK));
+  }
+
+  /****
+   * Tests the execution of "list member" command, when no cache is created
+   *
+   * @throws IOException
+   * @throws ClassNotFoundException
+   */
+  public void testListMemberWithNoCache() throws IOException, ClassNotFoundException {
+    final Host host = Host.getHost(0);
+    final VM[] servers = {host.getVM(0), host.getVM(1)};
+    final int openPorts[] = AvailablePortHelper.getRandomAvailableTCPPorts(1);
+    final File logFile = new File(getUniqueName() + "-locator" + openPorts[0] + ".log");
+
+    Locator locator = Locator.startLocator(openPorts[0], logFile);
+    try {
+
+      final Properties props = createProperties(host, openPorts[0]);
+      CommandProcessor commandProcessor = new CommandProcessor();
+      Result result = commandProcessor.createCommandStatement(CliStrings.LIST_MEMBER, EMPTY_ENV).process();
+
+      getLogWriter().info("#SB" + getResultAsString(result));
+      assertEquals(true, result.getStatus().equals(Status.ERROR));
+    } finally {
+      locator.stop(); // fix for bug 46562
+    }
+  }
+
+  /***
+   * Tests list member --group=G1
+   *
+   * @throws IOException
+   * @throws ClassNotFoundException
+   */
+  public void testListMemberWithGroups() throws IOException, ClassNotFoundException {
+    setupSystem();
+    CommandProcessor commandProcessor = new CommandProcessor();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.LIST_MEMBER);
+    csb.addOption(CliStrings.LIST_MEMBER__GROUP, "G1");
+    Result result = commandProcessor.createCommandStatement(csb.toString(), EMPTY_ENV).process();
+    getLogWriter().info("#SB" + getResultAsString(result));
+    assertEquals(true, result.getStatus().equals(Status.OK));
+  }
+
+  /***
+   * Tests the "describe member" command for all the members in the DS
+   *
+   * @throws IOException
+   * @throws ClassNotFoundException
+   */
+  public void testDescribeMember() throws IOException, ClassNotFoundException {
+    setupSystem();
+    CommandProcessor commandProcessor = new CommandProcessor();
+    GemFireCacheImpl cache = (GemFireCacheImpl) CacheFactory.getAnyInstance();
+    Set<DistributedMember> members = cache.getDistributedSystem().getAllOtherMembers();
+
+    Iterator<DistributedMember> iters = members.iterator();
+
+    while (iters.hasNext()) {
+      DistributedMember member = iters.next();
+      Result result = commandProcessor.createCommandStatement("describe member --name=" + member.getId(),
+          EMPTY_ENV).process();
+      assertEquals(true, result.getStatus().equals(Status.OK));
+      getLogWriter().info("#SB" + getResultAsString(result));
+      //assertEquals(true, result.getStatus().equals(Status.OK));
+    }
+  }
+
+  private String getResultAsString(Result result) {
+    StringBuilder sb = new StringBuilder();
+    while (result.hasNextLine()) {
+      sb.append(result.nextLine());
+    }
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java
new file mode 100644
index 0000000..ca3f94d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java
@@ -0,0 +1,492 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.lang.ThreadUtils;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.cli.Result.Status;
+import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData;
+import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData.SectionResultData;
+import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
+import com.gemstone.gemfire.management.internal.cli.result.ResultData;
+import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+import org.junit.Ignore;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Dunit class for testing gemfire function commands : GC, Shutdown
+ *
+ * @author apande
+ */
+public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+  private static String cachedLogLevel;
+
+  public MiscellaneousCommandsDUnitTest(String name) {
+    super(name);
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    invokeInEveryVM(new SerializableRunnable("reset log level") {
+      public void run() {
+        if (cachedLogLevel != null) {
+          System.setProperty("gemfire.log-level", cachedLogLevel);
+          cachedLogLevel = null;
+        }
+      }
+    });
+  }
+
+  public void testGCForGroup() {
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+    createDefaultSetup(localProps);
+    String command = "gc --group=Group1";
+    CommandResult cmdResult = executeCommand(command);
+    cmdResult.resetToFirstLine();
+    if (cmdResult != null) {
+      String cmdResultStr = commandResultToString(cmdResult);
+      getLogWriter().info("testGCForGroup cmdResultStr=" + cmdResultStr + "; cmdResult=" + cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      if (cmdResult.getType().equals(ResultData.TYPE_TABULAR)) {
+        TabularResultData table = (TabularResultData) cmdResult.getResultData();
+        List<String> memberNames = table.retrieveAllValues(CliStrings.GC__MSG__MEMBER_NAME);
+        assertEquals(true, memberNames.size() == 1 ? true : false);
+      } else {
+        fail("testGCForGroup failed as CommandResult should be table type");
+      }
+    } else {
+      fail("testGCForGroup failed as did not get CommandResult");
+    }
+  }
+
+  public static String getMemberId() {
+    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    return cache.getDistributedSystem().getDistributedMember().getId();
+  }
+
+  public void testGCForMemberID() {
+    createDefaultSetup(null);
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1MemberId = (String) vm1.invoke(MiscellaneousCommandsDUnitTest.class, "getMemberId");
+    String command = "gc --member=" + vm1MemberId;
+    CommandResult cmdResult = executeCommand(command);
+    cmdResult.resetToFirstLine();
+    if (cmdResult != null) {
+      String cmdResultStr = commandResultToString(cmdResult);
+      getLogWriter().info("testGCForMemberID cmdResultStr=" + cmdResultStr);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      if (cmdResult.getType().equals(ResultData.TYPE_TABULAR)) {
+        TabularResultData table = (TabularResultData) cmdResult.getResultData();
+        List<String> memberNames = table.retrieveAllValues(CliStrings.GC__MSG__MEMBER_NAME);
+        assertEquals(true, memberNames.size() == 1 ? true : false);
+      } else {
+        fail("testGCForGroup failed as CommandResult should be table type");
+      }
+    } else {
+      fail("testGCForCluster failed as did not get CommandResult");
+    }
+  }
+
+  public void testShowLogDefault() throws IOException {
+    Properties props = new Properties();
+    try {
+      props.setProperty("log-file", "testShowLogDefault.log");
+      createDefaultSetup(props);
+      final VM vm1 = Host.getHost(0).getVM(0);
+      final String vm1MemberId = (String) vm1.invoke(MiscellaneousCommandsDUnitTest.class, "getMemberId");
+      String command = "show log --member=" + vm1MemberId;
+      CommandResult cmdResult = executeCommand(command);
+      if (cmdResult != null) {
+        String log = commandResultToString(cmdResult);
+        assertNotNull(log);
+        getLogWriter().info("Show Log is" + log);
+        assertEquals(Result.Status.OK, cmdResult.getStatus());
+      } else {
+        fail("testShowLog failed as did not get CommandResult");
+      }
+    } finally {
+      disconnectAllFromDS();
+    }
+  }
+
+  public void testShowLogNumLines() {
+    Properties props = new Properties();
+    props.setProperty("log-file", "testShowLogNumLines.log");
+    try {
+      createDefaultSetup(props);
+      final VM vm1 = Host.getHost(0).getVM(0);
+      final String vm1MemberId = (String) vm1.invoke(MiscellaneousCommandsDUnitTest.class, "getMemberId");
+      String command = "show log --member=" + vm1MemberId + " --lines=50";
+      CommandResult cmdResult = executeCommand(command);
+      if (cmdResult != null) {
+        String log = commandResultToString(cmdResult);
+        assertNotNull(log);
+        getLogWriter().info("Show Log is" + log);
+        assertEquals(Result.Status.OK, cmdResult.getStatus());
+      } else {
+        fail("testShowLog failed as did not get CommandResult");
+      }
+    } finally {
+      disconnectAllFromDS();
+    }
+  }
+
+  public void testGCForEntireCluster() {
+    setupForGC();
+    String command = "gc";
+    CommandResult cmdResult = executeCommand(command);
+    cmdResult.resetToFirstLine();
+    if (cmdResult != null) {
+      String cmdResultStr = commandResultToString(cmdResult);
+      getLogWriter().info("testGCForEntireCluster cmdResultStr=" + cmdResultStr + "; cmdResult=" + cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      if (cmdResult.getType().equals(ResultData.TYPE_TABULAR)) {
+        TabularResultData table = (TabularResultData) cmdResult.getResultData();
+        List<String> memberNames = table.retrieveAllValues(CliStrings.GC__MSG__MEMBER_NAME);
+        assertEquals(3, memberNames.size());
+      } else {
+        fail("testGCForGroup failed as CommandResult should be table type");
+      }
+    } else {
+      fail("testGCForGroup failed as did not get CommandResult");
+    }
+  }
+
+  void setupForGC() {
+    disconnectAllFromDS();
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+
+    createDefaultSetup(null);
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create("testRegion");
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        dataRegionFactory.create("testRegion");
+      }
+    });
+  }
+
+  public void testShutDownWithoutTimeout() {
+
+    addExpectedException("EntryDestroyedException");
+
+    setupForShutDown();
+    ThreadUtils.sleep(2500);
+
+    String command = "shutdown";
+    CommandResult cmdResult = executeCommand(command);
+
+    if (cmdResult != null) {
+      String cmdResultStr = commandResultToString(cmdResult);
+      getLogWriter().info("testShutDownWithoutTimeout cmdResultStr=" + cmdResultStr);
+    }
+
+    verifyShutDown();
+
+    final HeadlessGfsh defaultShell = getDefaultShell();
+
+    // Need for the Gfsh HTTP enablement during shutdown to properly assess the
+    // state of the connection.
+    waitForCriterion(new WaitCriterion() {
+      public boolean done() {
+        return !defaultShell.isConnectedAndReady();
+      }
+
+      public String description() {
+        return "Waits for the shell to disconnect!";
+      }
+    }, 1000, 250, true);
+
+    assertFalse(defaultShell.isConnectedAndReady());
+  }
+
+  @Ignore("Disabled for 52350")
+  public void DISABLED_testShutDownWithTimeout() {
+    setupForShutDown();
+    ThreadUtils.sleep(2500);
+
+    addExpectedException("EntryDestroyedException");
+
+    String command = "shutdown --time-out=15";
+    CommandResult cmdResult = executeCommand(command);
+
+    if (cmdResult != null) {
+      String cmdResultStr = commandResultToString(cmdResult);
+      getLogWriter().info("testShutDownWithTIMEOUT cmdResultStr=" + cmdResultStr);
+    }
+
+    verifyShutDown();
+
+    final HeadlessGfsh defaultShell = getDefaultShell();
+
+    // Need for the Gfsh HTTP enablement during shutdown to properly assess the state of the connection.
+    waitForCriterion(new WaitCriterion() {
+      public boolean done() {
+        return !defaultShell.isConnectedAndReady();
+      }
+
+      public String description() {
+        return "Waits for the shell to disconnect!";
+      }
+    }, 1000, 250, false);
+
+    assertFalse(defaultShell.isConnectedAndReady());
+  }
+
+  public void testShutDownForTIMEOUT() {
+    setupForShutDown();
+    ThreadUtils.sleep(2500);
+    final VM vm0 = Host.getHost(0).getVM(0);
+    vm0.invoke(new SerializableRunnable() {
+      public void run() {
+        System.setProperty("ThrowTimeoutException", "true");
+      }
+    });
+
+
+    String command = "shutdown --time-out=15";
+    CommandResult cmdResult = executeCommand(command);
+
+    if (cmdResult != null) {
+      String cmdResultStr = commandResultToString(cmdResult);
+      getLogWriter().info("testShutDownForTIMEOUT cmdResultStr = " + cmdResultStr);
+      CommandResult result = (CommandResult) ResultBuilder.createInfoResult(CliStrings.SHUTDOWN_TIMEDOUT);
+      String expectedResult = commandResultToString(result);
+      assertEquals(expectedResult, cmdResultStr);
+    }
+    vm0.invoke(new SerializableRunnable() {
+      public void run() {
+        System.clearProperty("ThrowTimeoutException");
+      }
+    });
+  }
+
+  void setupForChangeLogLelvel() {
+    final VM vm0 = Host.getHost(0).getVM(0);
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+    createDefaultSetup(null);
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create("testRegion");
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+  void setupForShutDown() {
+    final VM vm0 = Host.getHost(0).getVM(0);
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+    System.setProperty(CliStrings.IGNORE_INTERCEPTORS, "true");
+    createDefaultSetup(null);
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create("testRegion");
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+  void verifyShutDown() {
+    final VM vm0 = Host.getHost(0).getVM(0);
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+    @SuppressWarnings("serial") final SerializableCallable connectedChecker = new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        boolean cacheExists = true;
+        try {
+          Cache cacheInstance = CacheFactory.getAnyInstance();
+          cacheExists = cacheInstance.getDistributedSystem().isConnected();
+        } catch (CacheClosedException e) {
+          cacheExists = false;
+        }
+        return cacheExists;
+      }
+    };
+
+    WaitCriterion waitCriterion = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return Boolean.FALSE.equals(vm0.invoke(connectedChecker)) && Boolean.FALSE.equals(vm1.invoke(connectedChecker));
+      }
+
+      @Override
+      public String description() {
+        return "Wait for gfsh to get disconnected from Manager.";
+      }
+    };
+    waitForCriterion(waitCriterion, 5000, 200, true);
+
+    assertTrue(Boolean.FALSE.equals(vm1.invoke(connectedChecker)));
+    assertTrue(Boolean.FALSE.equals(vm0.invoke(connectedChecker)));
+  }
+
+  public void testChangeLogLevelForMembers() {
+    final VM vm0 = Host.getHost(0).getVM(0);
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+    setupForChangeLogLelvel();
+
+    String serverName1 = (String) vm0.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        cachedLogLevel = System.getProperty("gemfire.log-level");
+        return GemFireCacheImpl.getInstance().getDistributedSystem().getDistributedMember().getId();
+      }
+    });
+
+    String serverName2 = (String) vm1.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        cachedLogLevel = System.getProperty("gemfire.log-level");
+        return GemFireCacheImpl.getInstance().getDistributedSystem().getDistributedMember().getId();
+      }
+    });
+
+    String commandString = CliStrings.CHANGE_LOGLEVEL + " --" + CliStrings.CHANGE_LOGLEVEL__LOGLEVEL + "=finer" + " --" + CliStrings.CHANGE_LOGLEVEL__MEMBER + "=" + serverName1 + "," + serverName2;
+
+    CommandResult commandResult = executeCommand(commandString);
+    getLogWriter().info("testChangeLogLevel commandResult=" + commandResult);
+    assertTrue(Status.OK.equals(commandResult.getStatus()));
+    CompositeResultData resultData = (CompositeResultData) commandResult.getResultData();
+    SectionResultData section = resultData.retrieveSection("section");
+    assertNotNull(section);
+    TabularResultData tableRsultData = section.retrieveTable("ChangeLogLevel");
+    assertNotNull(tableRsultData);
+
+    List<String> columns = tableRsultData.retrieveAllValues(CliStrings.CHANGE_LOGLEVEL__COLUMN_MEMBER);
+    List<String> status = tableRsultData.retrieveAllValues(CliStrings.CHANGE_LOGLEVEL__COLUMN_STATUS);
+
+    assertEquals(columns.size(), 2);
+    assertEquals(status.size(), 2);
+
+    assertTrue(columns.contains(serverName1));
+    assertTrue(columns.contains(serverName2));
+    assertTrue(status.contains("true"));
+  }
+
+  public void testChangeLogLevelForGrps() {
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group0");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+    final String grp1 = "Group1";
+    final String grp2 = "Group2";
+
+    createDefaultSetup(localProps);
+
+    String vm1id = (String) vm1.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, grp1);
+        getSystem(localProps);
+        Cache cache = getCache();
+        return cache.getDistributedSystem().getDistributedMember().getId();
+      }
+    });
+
+    String vm2id = (String) vm2.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, grp2);
+        getSystem(localProps);
+        Cache cache = getCache();
+        return cache.getDistributedSystem().getDistributedMember().getId();
+      }
+    });
+
+    String commandString = CliStrings.CHANGE_LOGLEVEL + " --" + CliStrings.CHANGE_LOGLEVEL__LOGLEVEL + "=finer" + " --" + CliStrings.CHANGE_LOGLEVEL__GROUPS + "=" + grp1 + "," + grp2;
+
+    CommandResult commandResult = executeCommand(commandString);
+    getLogWriter().info("testChangeLogLevelForGrps commandResult=" + commandResult);
+
+    assertTrue(Status.OK.equals(commandResult.getStatus()));
+
+    CompositeResultData resultData = (CompositeResultData) commandResult.getResultData();
+    SectionResultData section = resultData.retrieveSection("section");
+    assertNotNull(section);
+    TabularResultData tableRsultData = section.retrieveTable("ChangeLogLevel");
+    assertNotNull(tableRsultData);
+
+    List<String> columns = tableRsultData.retrieveAllValues(CliStrings.CHANGE_LOGLEVEL__COLUMN_MEMBER);
+    List<String> status = tableRsultData.retrieveAllValues(CliStrings.CHANGE_LOGLEVEL__COLUMN_STATUS);
+
+    assertEquals(columns.size(), 2);
+    assertEquals(status.size(), 2);
+
+    assertTrue(columns.contains(vm1id));
+    assertTrue(columns.contains(vm2id));
+    assertTrue(status.contains("true"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java
new file mode 100644
index 0000000..6afa7ee
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java
@@ -0,0 +1,139 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.logging.LogWriterImpl;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+/**
+ * Dunit class for testing gemfire function commands : export logs
+ *
+ * @author apande
+ */
+
+public class MiscellaneousCommandsExportLogsPart1DUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public MiscellaneousCommandsExportLogsPart1DUnitTest(String name) {
+    super(name);
+  }
+
+  public static String getMemberId() {
+    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    return cache.getDistributedSystem().getDistributedMember().getId();
+  }
+
+  void setupForExportLogs() {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    createDefaultSetup(null);
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create("testRegion");
+        for (int i = 0; i < 5; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+  String getCurrentTimeString() {
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy_MM_dd_HH_mm_ss_SSS_z");
+    Date startDate = new Date(System.currentTimeMillis());
+    String formattedStartDate = sf.format(startDate);
+    return ("_" + formattedStartDate);
+  }
+
+  public void testExportLogs() throws IOException {
+    Date startDate = new Date(System.currentTimeMillis() - 2 * 60 * 1000);
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy/MM/dd");
+    String start = sf.format(startDate);
+
+    Date enddate = new Date(System.currentTimeMillis() + 2 * 60 * 60 * 1000);
+    String end = sf.format(enddate);
+    String dir = getCurrentTimeString();
+
+    setupForExportLogs();
+    String logLevel = LogWriterImpl.levelToString(LogWriterImpl.INFO_LEVEL);
+
+    MiscellaneousCommands misc = new MiscellaneousCommands();
+    getCache();
+
+    Result cmdResult = misc.exportLogsPreprocessing("./testExportLogs" + dir, null, null, logLevel, false, false, start,
+        end, 1);
+
+    getLogWriter().info("testExportLogs command result =" + cmdResult);
+
+    if (cmdResult != null) {
+      String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
+      getLogWriter().info("testExportLogs cmdStringRsult=" + cmdStringRsult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testExportLogs failed as did not get CommandResult");
+    }
+    FileUtil.delete(new File("./testExportLogs" + dir));
+  }
+
+  public void testExportLogsForMerge() throws IOException {
+    setupForExportLogs();
+    Date startDate = new Date(System.currentTimeMillis() - 2 * 60 * 1000);
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy/MM/dd");
+    String start = sf.format(startDate);
+
+    Date enddate = new Date(System.currentTimeMillis() + 2 * 60 * 60 * 1000);
+    String end = sf.format(enddate);
+    String dir = getCurrentTimeString();
+
+    String logLevel = LogWriterImpl.levelToString(LogWriterImpl.INFO_LEVEL);
+
+    MiscellaneousCommands misc = new MiscellaneousCommands();
+    getCache();
+
+    Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForMerge" + dir, null, null, logLevel, false, true,
+        start, end, 1);
+    getLogWriter().info("testExportLogsForMerge command=" + cmdResult);
+
+    if (cmdResult != null) {
+      String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
+      getLogWriter().info("testExportLogsForMerge cmdStringRsult=" + cmdStringRsult);
+
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testExportLogsForMerge failed as did not get CommandResult");
+    }
+    FileUtil.delete(new File("./testExportLogsForMerge" + dir));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java
new file mode 100644
index 0000000..6a1d86c
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java
@@ -0,0 +1,148 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.logging.LogWriterImpl;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+/**
+ * Dunit class for testing gemfire function commands : export logs
+ *
+ * @author apande
+ */
+
+public class MiscellaneousCommandsExportLogsPart2DUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public MiscellaneousCommandsExportLogsPart2DUnitTest(String name) {
+    super(name);
+  }
+
+  public static String getMemberId() {
+    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    return cache.getDistributedSystem().getDistributedMember().getId();
+  }
+
+  void setupForExportLogs() {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    createDefaultSetup(null);
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create("testRegion");
+        for (int i = 0; i < 5; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+  String getCurrentTimeString() {
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy_MM_dd_HH_mm_ss_SSS_z");
+    Date startDate = new Date(System.currentTimeMillis());
+    String formattedStartDate = sf.format(startDate);
+    return ("_" + formattedStartDate);
+  }
+
+  public void testExportLogsForLogLevel() throws IOException {
+    setupForExportLogs();
+
+    Date startDate = new Date(System.currentTimeMillis() - 60 * 1000);
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy/MM/dd");
+    String start = sf.format(startDate);
+
+    Date enddate = new Date(System.currentTimeMillis() + 60 * 1000);
+    String end = sf.format(enddate);
+    String dir = getCurrentTimeString();
+
+    String logLevel = LogWriterImpl.levelToString(LogWriterImpl.CONFIG_LEVEL);
+
+    MiscellaneousCommands misc = new MiscellaneousCommands();
+    getCache();
+
+    Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForLogLevel" + dir, null, null, logLevel, false,
+        false, start, end, 1);
+
+    getLogWriter().info("testExportLogsForLogLevel command=" + cmdResult);
+
+    if (cmdResult != null) {
+      String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
+      getLogWriter().info("testExportLogsForLogLevel cmdStringRsult=" + cmdStringRsult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testExportLogsForLogLevel failed as did not get CommandResult");
+    }
+    FileUtil.delete(new File("testExportLogsForLogLevel" + dir));
+  }
+
+
+  public void testExportLogsForLogLevelWithUPTOLOGLEVEL() throws IOException {
+    setupForExportLogs();
+
+    Date startDate = new Date(System.currentTimeMillis() - 2 * 60 * 1000);
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy/MM/dd");
+    String start = sf.format(startDate);
+
+    Date enddate = new Date(System.currentTimeMillis() + 2 * 60 * 60 * 1000);
+    String end = sf.format(enddate);
+    String dir = getCurrentTimeString();
+
+    String logLevel = LogWriterImpl.levelToString(LogWriterImpl.SEVERE_LEVEL);
+
+    MiscellaneousCommands misc = new MiscellaneousCommands();
+    getCache();
+
+    Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForLogLevelWithUPTOLOGLEVEL" + dir, null, null,
+        logLevel, true, false, start, end, 1);
+
+    getLogWriter().info("testExportLogsForLogLevelWithUPTOLOGLEVEL command=" + cmdResult);
+
+    if (cmdResult != null) {
+      String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
+      getLogWriter().info("testExportLogsForLogLevelWithUPTOLOGLEVEL cmdStringRsult=" + cmdStringRsult);
+
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testExportLogsForLogLevelWithUPTOLOGLEVEL failed as did not get CommandResult");
+    }
+    FileUtil.delete(new File("testExportLogsForLogLevelWithUPTOLOGLEVEL" + dir));
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java
new file mode 100644
index 0000000..1c2933d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java
@@ -0,0 +1,150 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.logging.LogWriterImpl;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Properties;
+
+/**
+ * Dunit class for testing gemfire function commands : export logs
+ *
+ * @author apande
+ */
+
+public class MiscellaneousCommandsExportLogsPart3DUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public MiscellaneousCommandsExportLogsPart3DUnitTest(String name) {
+    super(name);
+  }
+
+  public static String getMemberId() {
+    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    return cache.getDistributedSystem().getDistributedMember().getId();
+  }
+
+  void setupForExportLogs() {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    createDefaultSetup(null);
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create("testRegion");
+        for (int i = 0; i < 5; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+  String getCurrentTimeString() {
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy_MM_dd_HH_mm_ss_SSS_z");
+    Date startDate = new Date(System.currentTimeMillis());
+    String formattedStartDate = sf.format(startDate);
+    return ("_" + formattedStartDate);
+  }
+
+  public void testExportLogsForGroup() throws IOException {
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+    createDefaultSetup(localProps);
+    String dir = getCurrentTimeString();
+
+    Date startDate = new Date(System.currentTimeMillis() - 2 * 60 * 1000);
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy/MM/dd");
+    String start = sf.format(startDate);
+
+    Date enddate = new Date(System.currentTimeMillis() + 2 * 60 * 60 * 1000);
+    String end = sf.format(enddate);
+
+    String logLevel = LogWriterImpl.levelToString(LogWriterImpl.INFO_LEVEL);
+
+    MiscellaneousCommands misc = new MiscellaneousCommands();
+    getCache();
+    String[] groups = new String[1];
+    groups[0] = "Group1";
+
+    Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForGroup" + dir, groups, null, logLevel, false,
+        false, start, end, 1);
+
+    getLogWriter().info("testExportLogsForGroup command result =" + cmdResult);
+    if (cmdResult != null) {
+      String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
+      getLogWriter().info("testExportLogsForGroup cmdStringRsult=" + cmdStringRsult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testExportLogsForGroup failed as did not get CommandResult");
+    }
+    FileUtil.delete(new File("testExportLogsForGroup" + dir));
+  }
+
+  public void testExportLogsForMember() throws IOException {
+    createDefaultSetup(null);
+
+    Date startDate = new Date(System.currentTimeMillis() - 2 * 60 * 1000);
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy/MM/dd");
+    String start = sf.format(startDate);
+
+    Date enddate = new Date(System.currentTimeMillis() + 2 * 60 * 60 * 1000);
+    String end = sf.format(enddate);
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1MemberId = (String) vm1.invoke(MiscellaneousCommandsDUnitTest.class, "getMemberId");
+    String dir = getCurrentTimeString();
+
+    String logLevel = LogWriterImpl.levelToString(LogWriterImpl.INFO_LEVEL);
+
+    MiscellaneousCommands misc = new MiscellaneousCommands();
+    getCache();
+
+    Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForMember" + dir, null, vm1MemberId, logLevel,
+        false, false, start, end, 1);
+
+    getLogWriter().info("testExportLogsForMember command result =" + cmdResult);
+
+    if (cmdResult != null) {
+      String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
+      getLogWriter().info("testExportLogsForMember cmdStringRsult=" + cmdStringRsult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testExportLogsForMember failed as did not get CommandResult");
+    }
+    FileUtil.delete(new File("testExportLogsForMember" + dir));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.java
new file mode 100644
index 0000000..da12c6e
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.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.management.internal.cli.commands;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.logging.LogWriterImpl;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+/**
+ * Dunit class for testing gemfire function commands : export logs
+ *
+ * @author apande
+ */
+public class MiscellaneousCommandsExportLogsPart4DUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public MiscellaneousCommandsExportLogsPart4DUnitTest(String name) {
+    super(name);
+  }
+
+  public static String getMemberId() {
+    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    return cache.getDistributedSystem().getDistributedMember().getId();
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+  }
+
+  void setupForExportLogs() {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    createDefaultSetup(null);
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create("testRegion");
+        for (int i = 0; i < 5; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+  String getCurrentTimeString() {
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy_MM_dd_HH_mm_ss_SSS_z");
+    Date startDate = new Date(System.currentTimeMillis());
+    String formattedStartDate = sf.format(startDate);
+    return ("_" + formattedStartDate);
+  }
+
+  public void testExportLogsForTimeRange1() throws IOException {
+    setupForExportLogs();
+    Date startDate = new Date(System.currentTimeMillis() - 1 * 60 * 1000);
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy/MM/dd");
+    String start = sf.format(startDate);
+
+    Date enddate = new Date(System.currentTimeMillis() + 1 * 60 * 60 * 1000);
+    String end = sf.format(enddate);
+    String dir = getCurrentTimeString();
+
+    String logLevel = LogWriterImpl.levelToString(LogWriterImpl.INFO_LEVEL);
+
+    MiscellaneousCommands misc = new MiscellaneousCommands();
+    getCache();
+
+    Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForTimeRange1" + dir, null, null, logLevel, false,
+        false, start, end, 1);
+
+    getLogWriter().info("testExportLogsForTimeRange1 command result =" + cmdResult);
+
+    if (cmdResult != null) {
+      String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
+      getLogWriter().info("testExportLogsForTimeRange1 cmdStringRsult=" + cmdStringRsult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testExportLogsForTimeRange1 failed as did not get CommandResult");
+    }
+    FileUtil.delete(new File("testExportLogsForTimeRange1" + dir));
+  }
+
+  public void testExportLogsForTimeRangeForOnlyStartTime() throws IOException {
+    setupForExportLogs();
+    Date date = new Date();
+    date.setTime(System.currentTimeMillis() - 30 * 1000);
+    SimpleDateFormat sf = new SimpleDateFormat("yyyy/MM/dd/HH:mm");
+    String s = sf.format(date);
+    String dir = getCurrentTimeString();
+
+    String logLevel = LogWriterImpl.levelToString(LogWriterImpl.INFO_LEVEL);
+
+    MiscellaneousCommands misc = new MiscellaneousCommands();
+    getCache();
+
+    Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForTimeRangeForOnlyStartTime" + dir, null, null,
+        logLevel, false, false, s, null, 1);
+
+    getLogWriter().info("testExportLogsForTimeRangeForOnlyStartTime command result =" + cmdResult);
+
+    if (cmdResult != null) {
+      String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
+      getLogWriter().info("testExportLogsForTimeRangeForOnlyStartTime cmdStringRsult=" + cmdStringRsult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testExportLogsForTimeRangeForOnlyStartTime failed as did not get CommandResult");
+    }
+    FileUtil.delete(new File("testExportLogsForTimeRangeForOnlyStartTime" + dir));
+  }
+}
\ No newline at end of file


[35/50] [abbrv] incubator-geode git commit: GEODE-563: Moving gfsh tests from closed

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java b/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
new file mode 100644
index 0000000..383012e
--- /dev/null
+++ b/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
@@ -0,0 +1,434 @@
+/*=========================================================================
+ * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
+ * This product is protected by U.S. and international copyright
+ * and intellectual property laws. Pivotal products are covered by
+ * one or more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+package com.gemstone.gemfire.management.internal.configuration;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.wan.GatewaySender.OrderPolicy;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.ClassBuilder;
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.JarDeployer;
+import com.gemstone.gemfire.internal.admin.remote.ShutdownAllRequest;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.management.cli.Result.Status;
+import com.gemstone.gemfire.management.internal.cli.CliUtil;
+import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
+import com.gemstone.gemfire.management.internal.cli.commands.CliCommandTestBase;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.VM;
+import org.apache.commons.io.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+
+public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
+  private static final int TIMEOUT = 10000;
+  private static final int INTERVAL = 500;
+  private static final String REGION1 = "R1";
+  private static final String REGION2 = "R2";
+  private static final String INDEX1 = "ID1";
+  private transient ClassBuilder classBuilder = new ClassBuilder();
+  public static Set<String> serverNames = new HashSet<String>();
+  public static Set<String> jarFileNames = new HashSet<String>();
+
+  public SharedConfigurationEndToEndDUnitTest(String name) {
+    super(name);
+    // TODO Auto-generated constructor stub
+  }
+
+  private static final long serialVersionUID = -2276690105585944041L;
+
+  public Set<String> startServers(HeadlessGfsh gfsh, String locatorString, int numServers, String serverNamePrefix, int startNum) throws ClassNotFoundException, IOException {
+    Set<String> serverNames = new HashSet<String>();
+
+    final int[] serverPorts = AvailablePortHelper.getRandomAvailableTCPPorts(numServers);
+    for (int i=0; i<numServers; i++) {
+      int port = serverPorts[i];
+      String serverName = serverNamePrefix+ Integer.toString(i+startNum) + "-" + port;
+      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.START_SERVER);
+      csb.addOption(CliStrings.START_SERVER__NAME, serverName);
+      csb.addOption(CliStrings.START_SERVER__LOCATORS, locatorString);
+      csb.addOption(CliStrings.START_SERVER__SERVER_PORT, Integer.toString(port));
+      CommandResult cmdResult = executeCommand(gfsh, csb.getCommandString());
+      assertEquals(Status.OK, cmdResult.getStatus());
+    }
+    return serverNames;
+  }
+
+  public void testStartServerAndExecuteCommands() throws InterruptedException, ClassNotFoundException, IOException, ExecutionException {
+    addExpectedException("EntryDestroyedException");
+    Object[] result = setup();
+    final int locatorPort = (Integer) result[0];
+    final String jmxHost = (String) result[1];
+    final int jmxPort = (Integer) result[2];
+    final int httpPort = (Integer) result[3];
+    final String locatorString = "localHost[" + locatorPort + "]";
+
+    final HeadlessGfsh gfsh = new HeadlessGfsh("gfsh2", 300);
+    assertNotNull(gfsh);
+    shellConnect(jmxHost, jmxPort, httpPort, gfsh);
+
+    serverNames.addAll(startServers(gfsh, locatorString, 2, "Server", 1));
+    doCreateCommands();
+    serverNames.addAll(startServers(gfsh, locatorString, 1, "NewMember", 4));
+    verifyRegionCreateOnAllMembers(REGION1);
+    verifyRegionCreateOnAllMembers(REGION2);
+    verifyIndexCreationOnAllMembers(INDEX1);
+    verifyAsyncEventQueueCreation();
+   
+
+
+    //shutdown everything
+    getLogWriter().info("Shutting down all the members");
+    shutdownAll();
+    deleteSavedJarFiles();
+  }
+
+
+  private void doCreateCommands() {
+    createRegion(REGION1, RegionShortcut.REPLICATE, null);
+    createRegion(REGION2, RegionShortcut.PARTITION, null);
+    createIndex(INDEX1 , "AAPL", REGION1, null);
+    createAndDeployJar("Deploy1.jar");
+    createAsyncEventQueue("q1");
+    final String autoCompact = "true";
+    final String allowForceCompaction = "true";
+    final String compactionThreshold = "50";
+    final String duCritical = "90";
+    final String duWarning = "85";
+    final String maxOplogSize = "1000";
+    final String queueSize = "300";
+    final String timeInterval = "10";
+    final String writeBufferSize="100";
+    final String diskStoreName = "ds1";
+    final String diskDirs = "ds1";
+    
+    createDiskStore(diskStoreName, diskDirs, autoCompact, allowForceCompaction, compactionThreshold, duCritical, duWarning, maxOplogSize, queueSize, timeInterval, writeBufferSize);
+  }
+
+
+  protected void executeAndVerifyCommand(String commandString) {
+    CommandResult cmdResult = executeCommand(commandString);
+    getLogWriter().info("Command Result : \n" + commandResultToString(cmdResult));
+    assertEquals(Status.OK, cmdResult.getStatus());
+    assertFalse(cmdResult.failedToPersist());
+  }
+
+  private void createRegion(String regionName, RegionShortcut regionShortCut, String group) {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_REGION);
+    csb.addOption(CliStrings.CREATE_REGION__REGION, regionName);
+    csb.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, regionShortCut.name());
+    executeAndVerifyCommand(csb.getCommandString());
+  }
+
+  private void destroyRegion(String regionName) {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESTROY_REGION);
+    csb.addOption(CliStrings.DESTROY_REGION__REGION, regionName);
+    executeAndVerifyCommand(csb.getCommandString());
+  }
+
+  private void stopServer(String serverName) {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.STOP_SERVER);
+    csb.addOption(CliStrings.STOP_SERVER__MEMBER, serverName);
+    executeAndVerifyCommand(csb.getCommandString());
+  }
+
+  public void createAsyncEventQueue(String queueName) {
+    String queueCommandsJarName = "testEndToEndSC-QueueCommands.jar";
+    final File jarFile = new File(queueCommandsJarName);
+
+    try {
+      ClassBuilder classBuilder = new ClassBuilder();
+      byte[] jarBytes = classBuilder.createJarFromClassContent("com/qcdunit/QueueCommandsDUnitTestListener",
+          "package com.qcdunit;" +
+              "import java.util.List; import java.util.Properties;" +
+              "import com.gemstone.gemfire.internal.cache.xmlcache.Declarable2; import com.gemstone.gemfire.cache.asyncqueue.AsyncEvent;" +
+              "import com.gemstone.gemfire.cache.asyncqueue.AsyncEventListener;" +
+              "public class QueueCommandsDUnitTestListener implements Declarable2, AsyncEventListener {" +
+              "Properties props;" +
+              "public boolean processEvents(List<AsyncEvent> events) { return true; }" +
+              "public void close() {}" +
+              "public void init(final Properties props) {this.props = props;}" +
+          "public Properties getConfig() {return this.props;}}");
+
+      FileUtils.writeByteArrayToFile(jarFile, jarBytes);
+      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DEPLOY);
+      csb.addOption(CliStrings.DEPLOY__JAR, queueCommandsJarName);
+      executeAndVerifyCommand(csb.getCommandString());
+
+      csb = new CommandStringBuilder(CliStrings.CREATE_ASYNC_EVENT_QUEUE);
+      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ID, queueName);
+      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER, "com.qcdunit.QueueCommandsDUnitTestListener");
+      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCH_SIZE, "100");
+      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCHTIMEINTERVAL, "200");
+      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__MAXIMUM_QUEUE_MEMORY, "1000");
+      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY, OrderPolicy.KEY.toString());
+      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT, "true");
+      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PARALLEL, "true");
+      
+      executeAndVerifyCommand(csb.getCommandString());
+
+    } catch (IOException e) {
+      e.printStackTrace();
+    } finally {
+      FileUtils.deleteQuietly(jarFile);
+    }
+  }
+  private void createDiskStore(String diskStoreName, 
+      String diskDirs, 
+      String autoCompact, 
+      String allowForceCompaction, 
+      String compactionThreshold, 
+      String duCritical, 
+      String duWarning,
+      String maxOplogSize,
+      String queueSize,
+      String timeInterval,
+      String writeBufferSize) {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_DISK_STORE);
+    csb.addOption(CliStrings.CREATE_DISK_STORE__NAME, diskStoreName);
+    csb.addOption(CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE, diskDirs);
+    csb.addOptionWithValueCheck(CliStrings.CREATE_DISK_STORE__AUTO_COMPACT, autoCompact);
+    csb.addOptionWithValueCheck(CliStrings.CREATE_DISK_STORE__ALLOW_FORCE_COMPACTION, allowForceCompaction);
+    csb.addOptionWithValueCheck(CliStrings.CREATE_DISK_STORE__COMPACTION_THRESHOLD, compactionThreshold);
+    csb.addOptionWithValueCheck(CliStrings.CREATE_DISK_STORE__DISK_USAGE_CRITICAL_PCT, duCritical);
+    csb.addOptionWithValueCheck(CliStrings.CREATE_DISK_STORE__DISK_USAGE_WARNING_PCT, duWarning);
+    csb.addOptionWithValueCheck(CliStrings.CREATE_DISK_STORE__MAX_OPLOG_SIZE, maxOplogSize);
+    csb.addOptionWithValueCheck(CliStrings.CREATE_DISK_STORE__QUEUE_SIZE, queueSize);
+    csb.addOptionWithValueCheck(CliStrings.CREATE_DISK_STORE__TIME_INTERVAL, timeInterval);
+    csb.addOptionWithValueCheck(CliStrings.CREATE_DISK_STORE__WRITE_BUFFER_SIZE, writeBufferSize);
+    executeAndVerifyCommand(csb.getCommandString());
+  }
+  
+  private void destroyDiskStore(String diskStoreName, String group) {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESTROY_DISK_STORE);
+    csb.addOption(CliStrings.DESTROY_DISK_STORE__NAME, diskStoreName);
+    csb.addOptionWithValueCheck(CliStrings.DESTROY_DISK_STORE__GROUP, group);
+    executeAndVerifyCommand(csb.toString());
+  }
+  public void createIndex(String indexName, String expression, String regionName, String group) {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, expression);
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, regionName);
+    executeAndVerifyCommand(csb.getCommandString());
+  }
+
+  public void destoyIndex(String indexName, String regionName, String group) {
+    if (StringUtils.isBlank(indexName) && StringUtils.isBlank(regionName) && StringUtils.isBlank(group)) {
+      return;
+    }
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    if (!StringUtils.isBlank(indexName)) {
+      csb.addOption(CliStrings.DESTROY_INDEX__NAME, indexName);
+    }
+
+    if (!StringUtils.isBlank(regionName)) {
+      csb.addOption(CliStrings.DESTROY_INDEX__REGION, regionName);
+    }
+
+    if (!StringUtils.isBlank(group)) {
+      csb.addOption(CliStrings.DESTROY_INDEX__GROUP, group);
+    }
+    executeAndVerifyCommand(csb.getCommandString());
+  }
+
+  public void createAndDeployJar(String jarName) {
+    File newDeployableJarFile = new File(jarName);
+    try {
+      this.classBuilder.writeJarFromName("ShareConfigClass", newDeployableJarFile);
+      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DEPLOY);
+      csb.addOption(CliStrings.DEPLOY__JAR, jarName);
+      executeAndVerifyCommand(csb.getCommandString());
+      jarFileNames.add(jarName);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  public void deleteSavedJarFiles() {
+    try {
+      FileUtil.deleteMatching(new File("."), "^" + JarDeployer.JAR_PREFIX + "Deploy1.*#\\d++$");
+      FileUtil.delete(new File("Deploy1.jar"));
+    } catch (IOException ioe) {
+      ioe.printStackTrace();
+    }
+  }
+
+  public Object[] setup() {
+    disconnectAllFromDS();
+    final int [] ports = AvailablePortHelper.getRandomAvailableTCPPorts(3);
+    final int locator1Port = ports[0];
+    final String locator1Name = "locator1-" + locator1Port;
+    VM locatorAndMgr = Host.getHost(0).getVM(3);
+
+    Object[] result = (Object[]) locatorAndMgr.invoke(new SerializableCallable() {
+      @Override
+      public Object call() {
+        int httpPort;
+        int jmxPort;
+        String jmxHost;
+
+        try {
+          jmxHost = InetAddress.getLocalHost().getHostName();
+        }
+        catch (UnknownHostException ignore) {
+          jmxHost = "localhost";
+        }
+
+        final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+
+        jmxPort = ports[0];
+        httpPort = ports[1];
+
+        final File locatorLogFile = new File("locator-" + locator1Port + ".log");
+
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, locator1Name);
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "config");
+        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_NAME, "true");
+        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_START_NAME, "true");
+        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_BIND_ADDRESS_NAME, String.valueOf(jmxHost));
+        locatorProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
+        locatorProps.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(httpPort));
+
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locator1Port, locatorLogFile, null,
+              locatorProps);
+          DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+            @Override
+            public boolean done() {
+              return locator.isSharedConfigurationRunning();
+            }
+
+            @Override
+            public String description() {
+              return "Waiting for shared configuration to be started";
+            }
+          };
+          DistributedTestCase.waitForCriterion(wc, TIMEOUT, INTERVAL, true);
+        } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        }
+
+        final Object[] result = new Object[4];
+        result[0] = locator1Port;
+        result[1] = jmxHost;
+        result[2] = jmxPort;
+        result[3] = httpPort;
+        return result;
+      }
+    });
+
+    HeadlessGfsh gfsh = getDefaultShell();
+    String jmxHost = (String)result[1];
+    int jmxPort = (Integer)result[2];
+    int httpPort = (Integer)result[3];
+
+    shellConnect(jmxHost, jmxPort, httpPort, gfsh);
+    // Create a cache in VM 1
+    VM dataMember = Host.getHost(0).getVM(1);
+    dataMember.invoke(new SerializableCallable() {
+      @Override
+      public Object call() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locator1Port);
+        localProps.setProperty(DistributionConfig.NAME_NAME, "DataMember");
+        getSystem(localProps);
+        Cache cache = getCache();
+        assertNotNull(cache);
+        return CliUtil.getAllNormalMembers(cache);
+      }
+    });
+    return result;
+  }
+
+  private void shutdownAll() throws IOException {
+    VM locatorAndMgr = Host.getHost(0).getVM(3);
+    locatorAndMgr.invoke(new SerializableCallable() {
+      /**
+       * 
+       */
+      private static final long serialVersionUID = 1L;
+
+      @Override
+      public Object call() throws Exception {
+        GemFireCacheImpl cache = (GemFireCacheImpl)CacheFactory.getAnyInstance();
+        ShutdownAllRequest.send(cache.getDistributedSystem().getDistributionManager(), -1);
+        return null;
+      }
+    });
+
+    locatorAndMgr.invoke(SharedConfigurationDUnitTest.locatorCleanup);
+    //Clean up the directories
+    if (!serverNames.isEmpty()) {
+      for (String serverName : serverNames) {
+        final File serverDir = new File(serverName);
+        FileUtils.cleanDirectory(serverDir);
+        FileUtils.deleteDirectory(serverDir);
+      }
+    }
+    serverNames.clear();
+    serverNames = null;
+  }
+
+  private void verifyRegionCreateOnAllMembers(String regionName) {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESCRIBE_REGION);
+    csb.addOption(CliStrings.DESCRIBE_REGION__NAME, regionName);
+    CommandResult cmdResult = executeCommand(csb.getCommandString());
+    String resultAsString = commandResultToString(cmdResult);
+
+    for (String serverName : serverNames) {
+      assertTrue(resultAsString.contains(serverName));
+    }
+  }     
+
+  private void verifyIndexCreationOnAllMembers(String indexName) {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
+    CommandResult cmdResult = executeCommand(csb.getCommandString());
+    String resultAsString = commandResultToString(cmdResult);
+
+    for (String serverName : serverNames) {
+      assertTrue(resultAsString.contains(serverName));
+    }
+  }
+  
+  private void verifyAsyncEventQueueCreation() {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.LIST_ASYNC_EVENT_QUEUES);
+    CommandResult cmdResult = executeCommand(csb.toString());
+    String resultAsString = commandResultToString(cmdResult);
+    
+    for (String serverName : serverNames) {
+      assertTrue(resultAsString.contains(serverName));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfsh.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfsh.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfsh.java
new file mode 100644
index 0000000..9ca9809
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfsh.java
@@ -0,0 +1,376 @@
+/*
+ * 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;
+
+import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
+import com.gemstone.gemfire.management.internal.cli.shell.GfshConfig;
+import com.gemstone.gemfire.management.internal.cli.shell.jline.GfshUnsupportedTerminal;
+import edu.umd.cs.findbugs.annotations.SuppressWarnings;
+import jline.ConsoleReader;
+import org.springframework.shell.core.ExitShellRequest;
+import org.springframework.shell.event.ShellStatus.Status;
+
+import java.io.BufferedWriter;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.io.Writer;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.logging.Level;
+
+
+/**
+ * This is headless shell which can be used to submit random commands and get command-result It is used for commands
+ * testing but can be used as for anything like programmatically sending commands to operate on GemFire Distributed
+ * systems. TODO : Merge HeadlessGfsh and HeadlessGfshShell TODO : Provide constructor for optionally specifying
+ * GfshConfig to provide logDirectory and logLevel
+ *
+ * @author tushark
+ */
+@SuppressWarnings("rawtypes")
+public class HeadlessGfsh implements ResultHandler {
+
+  public static final String ERROR_RESULT = "_$_ERROR_RESULT";
+
+  private HeadlessGfshShell shell = null;
+  private LinkedBlockingQueue queue = new LinkedBlockingQueue<>();
+  private long timeout = 20;
+  public String outputString = null;
+
+  public HeadlessGfsh(String name, int timeout) throws ClassNotFoundException, IOException {
+    this(name, timeout, null);
+  }
+
+  public HeadlessGfsh(String name, int timeout, Properties envProps) throws ClassNotFoundException, IOException {
+    this.timeout = timeout;
+    System.setProperty("jline.terminal", GfshUnsupportedTerminal.class.getName());
+    this.shell = new HeadlessGfshShell(name, this);
+    this.shell.setEnvProperty(Gfsh.ENV_APP_RESULT_VIEWER, "non-basic");
+
+    if (envProps != null) {
+      for (String key : envProps.stringPropertyNames()) {
+        this.shell.setEnvProperty(key, envProps.getProperty(key));
+      }
+    }
+
+    // This allows us to avoid race conditions during startup - in particular a NPE on the ConsoleReader which is
+    // created in a separate thread during start()
+    CountDownLatch shellStarted = new CountDownLatch(1);
+    this.shell.addShellStatusListener((oldStatus, newStatus) -> {
+      if (newStatus.getStatus() == Status.STARTED) {
+        shellStarted.countDown();
+      }
+    });
+
+    this.shell.start();
+    this.setThreadLocalInstance();
+
+    try {
+      shellStarted.await();
+    } catch (InterruptedException e) {
+      e.printStackTrace(System.out);
+    }
+  }
+
+  public void setThreadLocalInstance() {
+    shell.setThreadLocalInstance();
+  }
+
+  //TODO : Have non-blocking method also where we move executeCommand call to separate thread-pool
+  public boolean executeCommand(String command) {
+    boolean status = false;
+    try {
+      outputString = null;
+      status = shell.executeCommand(command);
+    } catch (Exception e) {
+      outputString = e.getMessage();
+    }
+    return status;
+  }
+
+  int getCommandExecutionStatus() {
+    return shell.getCommandExecutionStatus();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void handleExecutionResult(Object result, String sysout) {
+    queue.add(result);
+    outputString = sysout;
+  }
+
+  public Object getResult() throws InterruptedException {
+    //Dont wait for when some command calls gfsh.stop();
+    if (shell.stopCalledThroughAPI) return null;
+    try {
+      Object result = queue.poll(timeout, TimeUnit.SECONDS);
+      queue.clear();
+      return result;
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+  public void clear() {
+    queue.clear();
+    outputString = null;
+  }
+
+  public void clearEvents() {
+    queue.clear();
+    outputString = null;
+  }
+
+  public void terminate() {
+    shell.terminate();
+  }
+
+  public boolean isConnectedAndReady() {
+    return shell.isConnectedAndReady();
+  }
+
+  public String getErrorString() {
+    return shell.errorString;
+  }
+
+  public boolean hasError() {
+    return shell.hasError();
+  }
+
+  public String getError() {
+    return shell.errorString;
+  }
+
+  public static class HeadlessGfshShell extends Gfsh {
+
+    private ResultHandler handler = null;
+    private final Lock lock = new ReentrantLock();
+    private final Condition endOfShell = lock.newCondition();
+    private ByteArrayOutputStream output = null;
+    private String errorString = null;
+    private boolean hasError = false;
+    boolean stopCalledThroughAPI = false;
+
+    protected HeadlessGfshShell(String testName, ResultHandler handler) throws ClassNotFoundException, IOException {
+      super(false, new String[]{}, new HeadlessGfshConfig(testName));
+      this.handler = handler;
+    }
+
+    public void setThreadLocalInstance() {
+      gfshThreadLocal.set(this);
+    }
+
+    protected void handleExecutionResult(Object result) {
+      if (!result.equals(ERROR_RESULT)) {
+        super.handleExecutionResult(result);
+        handler.handleExecutionResult(result, output.toString());
+        output.reset();
+      } else {
+        //signal waiting queue with error condition with empty output
+        output.reset();
+        handler.handleExecutionResult(result, output.toString());
+      }
+    }
+
+    int getCommandExecutionStatus() {
+      return getLastExecutionStatus();
+    }
+
+    public void terminate() {
+      closeShell();
+      stopPromptLoop();
+      stop();
+    }
+
+    public void stop() {
+      stopCalledThroughAPI = true;
+    }
+
+    private void stopPromptLoop() {
+      lock.lock();
+      try {
+        endOfShell.signalAll();
+      } finally {
+        lock.unlock();
+      }
+    }
+
+    public String getErrorString() {
+      return errorString;
+    }
+
+    public boolean hasError() {
+      return hasError;
+    }
+
+    /**
+     * We override this method just to fool runner thread in reading from nothing. It waits for Condition endOfShell
+     * which is signalled when terminate is called. This achieves clean shutdown of runner thread.
+     */
+    @Override
+    public void promptLoop() {
+      lock.lock();
+      try {
+        while (true) {
+          try {
+            endOfShell.await();
+          } catch (InterruptedException e) {
+            //e.printStackTrace();
+          }
+          this.exitShellRequest = ExitShellRequest.NORMAL_EXIT;
+          setShellStatus(Status.SHUTTING_DOWN);
+          break;
+        }
+      } finally {
+        lock.unlock();
+      }
+    }
+
+    private static void setGfshOutErr(PrintStream outToUse) {
+      Gfsh.gfshout = outToUse;
+      Gfsh.gfsherr = outToUse;
+    }
+
+    /**
+     * This prints out error messages when Exceptions occur in shell. Capture it and set error flag=true and send
+     * ERROR_RESULT on the queue to signal thread waiting for CommandResult
+     */
+    @Override
+    public void logWarning(String message, Throwable t) {
+      super.logWarning(message, t);
+      errorString = message;
+      hasError = true;
+      //signal waiting queue with error condition
+      handleExecutionResult(ERROR_RESULT);
+    }
+
+    /**
+     * This prints out error messages when Exceptions occur in shell. Capture it and set error flag=true and send
+     * ERROR_RESULT on the queue to signal thread waiting for CommandResult
+     */
+    @Override
+    public void logSevere(String message, Throwable t) {
+      super.logSevere(message, t);
+      errorString = message;
+      hasError = true;
+      //signal waiting queue with error condition
+      handleExecutionResult(ERROR_RESULT);
+    }
+
+    /**
+     * Setup console-reader to capture Shell output
+     */
+    @Override
+    protected ConsoleReader createConsoleReader() {
+      try {
+        output = new ByteArrayOutputStream(1024 * 10);
+        PrintStream sysout = new PrintStream(output);
+        Writer wrappedOut = new BufferedWriter(new OutputStreamWriter(sysout));
+        setGfshOutErr(sysout);
+        return new ConsoleReader(new FileInputStream(FileDescriptor.in), new PrintWriter(wrappedOut));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+
+  /**
+   * HeadlessGfshConfig for tests. Taken from TestableGfsh
+   */
+  static class HeadlessGfshConfig extends GfshConfig {
+    {
+      // set vm as a gfsh vm
+      CliUtil.isGfshVM = true;
+    }
+
+    private File parentDir;
+    private String fileNamePrefix;
+    private String name;
+    private String generatedHistoryFileName = null;
+
+    public HeadlessGfshConfig(String name) {
+      this.name = name;
+
+      if (isDUnitTest(this.name)) {
+        fileNamePrefix = this.name;
+      } else {
+        fileNamePrefix = "non-hydra-client";
+      }
+
+      parentDir = new File("gfsh_files");
+      parentDir.mkdirs();
+    }
+
+    private static boolean isDUnitTest(String name) {
+      boolean isDUnitTest = false;
+      if (name != null) {
+        String[] split = name.split("_");
+        if (split.length != 0 && split[0].endsWith("DUnitTest")) {
+          isDUnitTest = true;
+        }
+      }
+      return isDUnitTest;
+    }
+
+    @Override
+    public String getLogFilePath() {
+      return new File(parentDir, getFileNamePrefix() + "-gfsh.log").getAbsolutePath();
+    }
+
+    private String getFileNamePrefix() {
+      String timeStamp = new java.sql.Time(System.currentTimeMillis()).toString();
+      timeStamp = timeStamp.replace(':', '_');
+      return fileNamePrefix + "-" + timeStamp;
+    }
+
+    @Override
+    public String getHistoryFileName() {
+      if (generatedHistoryFileName == null) {
+        String fileName = new File(parentDir, (getFileNamePrefix() + "-gfsh.history")).getAbsolutePath();
+        generatedHistoryFileName = fileName;
+        return fileName;
+      } else {
+        return generatedHistoryFileName;
+      }
+    }
+
+    @Override
+    public boolean isTestConfig() {
+      return true;
+    }
+
+    @Override
+    public Level getLogLevel() {
+      // Keep log level fine for tests
+      return Level.FINE;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfshJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfshJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfshJUnitTest.java
new file mode 100644
index 0000000..0807898
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfshJUnitTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.util.Properties;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * TODO : Add more tests for error-catch, different type of results etc
+ *
+ * @author tushark
+ */
+@Category(UnitTest.class)
+public class HeadlessGfshJUnitTest {
+
+  @SuppressWarnings({"unused", "deprecation", "unused"})
+  @Test
+  public void testHeadlessGfshTest() throws ClassNotFoundException, IOException, InterruptedException {
+    GemFireCacheImpl cache = null;
+    DistributedSystem ds = null;
+    Properties pr = new Properties();
+    pr.put("name", "testHeadlessGfshTest");
+    pr.put(DistributionConfig.JMX_MANAGER_NAME, "true");
+    pr.put(DistributionConfig.JMX_MANAGER_START_NAME, "true");
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    pr.put(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(port));
+    pr.put(DistributionConfig.HTTP_SERVICE_PORT_NAME, "0");
+    pr.put(DistributionConfig.MCAST_PORT_NAME, "0");
+
+    ds = DistributedSystem.connect(pr);
+    cache = (GemFireCacheImpl) CacheFactory.create(ds);
+    ObjectName name = MBeanJMXAdapter.getDistributedSystemName();
+
+    HeadlessGfsh gfsh = new HeadlessGfsh("Test", 25);
+    for (int i = 0; i < 5; i++) {
+      gfsh.executeCommand("connect --jmx-manager=localhost[" + port + "]");
+      Object result = gfsh.getResult();
+      assertTrue(gfsh.isConnectedAndReady());
+      assertNotNull(result);
+      gfsh.clear();
+      gfsh.executeCommand("list members");
+      result = gfsh.getResult();
+      assertNotNull(result);
+      gfsh.executeCommand("disconnect");
+      gfsh.getResult();
+    }
+
+    long l1 = System.currentTimeMillis();
+    gfsh.executeCommand("exit");
+    long l2 = System.currentTimeMillis();
+    gfsh.getResult();
+    long l3 = System.currentTimeMillis();
+    System.out.println("L3-l2=" + (l3 - l2) + " Total time= " + (l3 - l1) / 1000);
+    gfsh.terminate();
+    cache.close();
+    ds.disconnect();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/ResultHandler.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/ResultHandler.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/ResultHandler.java
new file mode 100644
index 0000000..2b90b60
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/ResultHandler.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.
+ */
+package com.gemstone.gemfire.management.internal.cli;
+
+public interface ResultHandler {
+  
+  void handleExecutionResult(Object result, String sysout);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/TableBuilderJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/TableBuilderJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/TableBuilderJUnitTest.java
new file mode 100644
index 0000000..e5f1d86
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/TableBuilderJUnitTest.java
@@ -0,0 +1,183 @@
+/*
+ * 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;
+
+import com.gemstone.gemfire.management.internal.cli.result.TableBuilder;
+import com.gemstone.gemfire.management.internal.cli.result.TableBuilder.Row;
+import com.gemstone.gemfire.management.internal.cli.result.TableBuilder.RowGroup;
+import com.gemstone.gemfire.management.internal.cli.result.TableBuilder.Table;
+import com.gemstone.gemfire.management.internal.cli.result.TableBuilderHelper;
+import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+import java.util.Properties;
+import java.util.Random;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * TODO: fails when running integrationTest from gradle command-line or in Eclipse on Windows 7
+ * <p>
+ * com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest > testBasicScrapping FAILED
+ * java.lang.AssertionError: Expected length < 100 is 101 at org.junit.Assert.fail(Assert.java:88) at
+ * com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest.doTableBuilderTestUnit(TableBuilderJUnitTest.java:115)
+ * at com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest.testBasicScrapping(TableBuilderJUnitTest.java:134)
+ * <p>
+ * com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest > testManyColumns FAILED java.lang.AssertionError:
+ * Expected length < 100 is 101 at org.junit.Assert.fail(Assert.java:88) at com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest.doTableBuilderTestUnit(TableBuilderJUnitTest.java:115)
+ * at com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest.testManyColumns(TableBuilderJUnitTest.java:155)
+ *
+ * @author tushark
+ */
+@Category(IntegrationTest.class)
+public class TableBuilderJUnitTest {
+
+  @Rule
+  public TestName testName = new TestName();
+
+  private final Table createTable(int rows, int cols, int width, String separator) {
+    Table resultTable = TableBuilder.newTable();
+    resultTable.setTabularResult(true);
+    resultTable.setColumnSeparator(separator);
+
+    resultTable.newBlankRow();
+    resultTable.newRow().newLeftCol("Displaying all fields for member: ");
+    resultTable.newBlankRow();
+    RowGroup rowGroup = resultTable.newRowGroup();
+    Row row = rowGroup.newRow();
+    for (int colIndex = 0; colIndex < cols; colIndex++) {
+      row.newCenterCol("Field" + colIndex);
+    }
+
+    rowGroup.newRowSeparator('-', false);
+
+    int counter = rows;
+    for (int i = 0; i < counter; i++) {
+      row = rowGroup.newRow();
+      for (int k = 0; k < cols; k++) {
+        row.newLeftCol(getString(i, width / cols));
+      }
+    }
+    resultTable.newBlankRow();
+
+    return resultTable;
+  }
+
+  private Object getString(int i, int width) {
+    StringBuilder sb = new StringBuilder();
+    Random random = new Random();
+    int k = 0;
+    double d = random.nextDouble();
+    // .09 probability
+    if (d <= 0.9) {
+      k = random.nextInt(width);
+    } else {
+      k = width / 2 + random.nextInt(width);
+    }
+    random.nextInt(10);
+    for (int j = 0; j < k; j++) {
+      sb.append(i);
+      if (sb.length() > k) break;
+    }
+    return sb.toString();
+  }
+
+  private HeadlessGfsh createShell(Properties props) throws ClassNotFoundException, IOException {
+    String shellId = getClass().getSimpleName() + "_" + testName;
+    HeadlessGfsh shell = new HeadlessGfsh(shellId, 30, props);
+    return shell;
+  }
+
+  private void doTableBuilderTestUnit(int rows, int cols, String sep, boolean shouldTrim,
+      boolean expectTooManyColEx) throws ClassNotFoundException, IOException {
+    int width = Gfsh.getCurrentInstance().getTerminalWidth();
+    Table table = createTable(rows, cols, width, sep);
+    String st = table.buildTable();
+    System.out.println(st);
+
+    String[] array = st.split("\n");
+
+    int line = 0;
+    for (String s : array) {
+      System.out.println("For line " + line++ + " length is " + s.length() + " isWider = " + (s.length() > width));
+
+      if (shouldTrim) {
+        if (s.length() > width) {
+          fail("Expected length < " + width + " is " + s.length());
+        }
+      } else {
+        if (s.length() > 50 && s.length() <= width) {
+          fail("Expected length <= " + width + " is " + s.length());
+        }
+      }
+
+    }
+  }
+
+  /**
+   * Test Variations tablewide separator true false
+   */
+  @Test
+  public void testBasicScraping() throws ClassNotFoundException, IOException {
+    Properties props = new Properties();
+    props.setProperty(Gfsh.ENV_APP_RESULT_VIEWER, Gfsh.DEFAULT_APP_RESULT_VIEWER);
+    createShell(props);
+    assertTrue(TableBuilderHelper.shouldTrimColumns());
+    doTableBuilderTestUnit(15, 4, "|", true, false);
+  }
+
+
+  @Test
+  public void testSeparatorWithMultipleChars() throws ClassNotFoundException, IOException {
+    Properties props = new Properties();
+    props.setProperty(Gfsh.ENV_APP_RESULT_VIEWER, Gfsh.DEFAULT_APP_RESULT_VIEWER);
+    createShell(props);
+    assertTrue(TableBuilderHelper.shouldTrimColumns());
+    doTableBuilderTestUnit(15, 4, " | ", true, false);
+  }
+
+  /**
+   * multiple columns upto 8 : done
+   */
+  @Test
+  @Ignore("Bug 52051")
+  public void testManyColumns() throws ClassNotFoundException, IOException {
+    createShell(null);
+    assertTrue(TableBuilderHelper.shouldTrimColumns());
+    doTableBuilderTestUnit(15, 6, "|", true, true);
+  }
+
+  /**
+   * set gfsh env property result_viewer to basic disable for external reader
+   */
+  //
+  @Test
+  public void testDisableColumnAdjustment() throws ClassNotFoundException, IOException {
+    createShell(null);
+    assertFalse(TableBuilderHelper.shouldTrimColumns());
+    doTableBuilderTestUnit(15, 12, "|", false, false);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
new file mode 100644
index 0000000..a0fb8f8
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
@@ -0,0 +1,560 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache30.CacheTestCase;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.management.ManagementService;
+import com.gemstone.gemfire.management.internal.cli.CommandManager;
+import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+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 dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import util.TestException;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Base class for all the CLI/gfsh command dunit tests.
+ *
+ * @author Tushar Khairnar
+ * @author Abhishek Chaudhari
+ * @author David Hoots
+ * @author John Blum
+ */
+public class CliCommandTestBase extends CacheTestCase {
+
+  private static final long serialVersionUID = 1L;
+
+  protected static final String USE_HTTP_SYSTEM_PROPERTY = "useHTTP";
+
+  private ManagementService managementService;
+
+  private transient HeadlessGfsh shell;
+
+  private boolean useHttpOnConnect = Boolean.getBoolean("useHTTP");
+
+  private int httpPort;
+  private int jmxPort;
+
+  private String jmxHost;
+
+  public CliCommandTestBase(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    destroyDefaultSetup();
+    super.tearDown2();
+  }
+
+  /**
+   * Create all of the components necessary for the default setup. The provided properties will be used when creating
+   * the default cache. This will create GFSH in the controller VM (VM[4]) (no cache) and the manager in VM[0] (with
+   * cache). When adding regions, functions, keys, whatever to your cache for tests, you'll need to use
+   * Host.getHost(0).getVM(0).invoke(new SerializableRunnable() { public void run() { ... } } in order to have this
+   * setup run in the same VM as the manager.
+   * <p>
+   *
+   * @param props the Properties used when creating the cache for this default setup.
+   * @return the default testable GemFire shell.
+   */
+  @SuppressWarnings("serial")
+  protected final HeadlessGfsh createDefaultSetup(final Properties props) {
+    Object[] result = (Object[]) Host.getHost(0).getVM(0).invoke(new SerializableCallable() {
+      public Object call() {
+        final Object[] result = new Object[3];
+        final Properties localProps = (props != null ? props : new Properties());
+
+        try {
+          jmxHost = InetAddress.getLocalHost().getHostName();
+        } catch (UnknownHostException ignore) {
+          jmxHost = "localhost";
+        }
+
+        if (!localProps.containsKey(DistributionConfig.NAME_NAME)) {
+          localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
+        }
+
+        final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+
+        jmxPort = ports[0];
+        httpPort = ports[1];
+
+        localProps.setProperty(DistributionConfig.JMX_MANAGER_NAME, "true");
+        localProps.setProperty(DistributionConfig.JMX_MANAGER_START_NAME, "true");
+        localProps.setProperty(DistributionConfig.JMX_MANAGER_BIND_ADDRESS_NAME, String.valueOf(jmxHost));
+        localProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
+        localProps.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(httpPort));
+
+        getSystem(localProps);
+        verifyManagementServiceStarted(getCache());
+
+        result[0] = jmxHost;
+        result[1] = jmxPort;
+        result[2] = httpPort;
+
+        return result;
+      }
+    });
+
+    this.jmxHost = (String) result[0];
+    this.jmxPort = (Integer) result[1];
+    this.httpPort = (Integer) result[2];
+
+    return defaultShellConnect();
+  }
+
+  protected boolean useHTTPByTest() {
+    return false;
+  }
+
+  /**
+   * Destroy all of the components created for the default setup.
+   */
+  @SuppressWarnings("serial")
+  protected final void destroyDefaultSetup() {
+    if (this.shell != null) {
+      executeCommand(shell, "exit");
+      this.shell.terminate();
+      this.shell = null;
+    }
+
+    disconnectAllFromDS();
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        verifyManagementServiceStopped();
+      }
+    });
+  }
+
+  /**
+   * Start the default management service using the provided Cache.
+   *
+   * @param cache Cache to use when creating the management service
+   */
+  private void verifyManagementServiceStarted(Cache cache) {
+    assert (cache != null);
+
+    this.managementService = ManagementService.getExistingManagementService(cache);
+    assertNotNull(this.managementService);
+    assertTrue(this.managementService.isManager());
+    assertTrue(checkIfCommandsAreLoadedOrNot());
+  }
+
+  public static boolean checkIfCommandsAreLoadedOrNot() {
+    CommandManager manager;
+    try {
+      manager = CommandManager.getInstance();
+      Map<String, CommandTarget> commands = manager.getCommands();
+      Set set = commands.keySet();
+      if (commands.size() < 1) {
+        return false;
+      }
+      return true;
+    } catch (ClassNotFoundException | IOException e) {
+      throw new RuntimeException("Could not load commands", e);
+    }
+  }
+
+  /**
+   * Stop the default management service.
+   */
+  private void verifyManagementServiceStopped() {
+    if (this.managementService != null) {
+      assertFalse(this.managementService.isManager());
+      this.managementService = null;
+    }
+  }
+
+  /**
+   * Connect the default shell to the default JMX server.
+   *
+   * @return The default shell.
+   */
+  private HeadlessGfsh defaultShellConnect() {
+    HeadlessGfsh shell = getDefaultShell();
+    shellConnect(this.jmxHost, this.jmxPort, this.httpPort, shell);
+    return shell;
+  }
+
+  /**
+   * 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) {
+    assert (host != null);
+    assert (shell != null);
+
+    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.CONNECT);
+    String endpoint;
+
+    if (useHttpOnConnect) {
+      endpoint = "http://" + host + ":" + httpPort + "/gemfire/v1";
+      command.addOption(CliStrings.CONNECT__USE_HTTP, Boolean.TRUE.toString());
+      command.addOption(CliStrings.CONNECT__URL, endpoint);
+    } else {
+      endpoint = host + "[" + jmxPort + "]";
+      command.addOption(CliStrings.CONNECT__JMX_MANAGER, endpoint);
+    }
+
+    CommandResult result = executeCommand(shell, command.toString());
+
+    if (!shell.isConnectedAndReady()) {
+      throw new TestException(
+          "Connect command failed to connect to manager " + endpoint + " result=" + commandResultToString(result));
+    }
+
+    info("Successfully connected to managing node using " + (useHttpOnConnect ? "HTTP" : "JMX"));
+    assertEquals(true, shell.isConnectedAndReady());
+  }
+
+  /**
+   * Get the default shell (will create one if it doesn't already exist).
+   *
+   * @return The default shell
+   */
+  protected synchronized final HeadlessGfsh getDefaultShell() {
+    if (this.shell == null) {
+      this.shell = createShell();
+    }
+
+    return this.shell;
+  }
+
+  /**
+   * Create a HeadlessGfsh object.
+   *
+   * @return The created shell.
+   */
+  protected HeadlessGfsh createShell() {
+    try {
+      Gfsh.SUPPORT_MUTLIPLESHELL = true;
+      String shellId = getClass().getSimpleName() + "_" + getName();
+      HeadlessGfsh shell = new HeadlessGfsh(shellId, 30);
+      //Added to avoid trimming of the columns
+      info("Started testable shell: " + shell);
+      return shell;
+    } catch (ClassNotFoundException e) {
+      throw new TestException(getStackTrace(e));
+    } catch (IOException e) {
+      throw new TestException(getStackTrace(e));
+    }
+  }
+
+  /**
+   * Execute a command using the default shell and clear the shell events before returning.
+   *
+   * @param command Command to execute
+   * @return The result of the command execution
+   */
+  protected CommandResult executeCommand(String command) {
+    assert (command != null);
+
+    return executeCommand(getDefaultShell(), command);
+  }
+
+  /**
+   * Execute a command in the provided shell and clear the shell events before returning.
+   *
+   * @param shell   Shell in which to execute the command.
+   * @param command Command to execute
+   * @return The result of the command execution
+   */
+  protected CommandResult executeCommand(HeadlessGfsh shell, String command) {
+    assert (shell != null);
+    assert (command != null);
+
+    CommandResult commandResult = executeCommandWithoutClear(shell, command);
+    shell.clearEvents();
+    return commandResult;
+  }
+
+  /**
+   * Execute a command using the default shell. Useful for getting additional information from the shell after the
+   * command has been executed (using getDefaultShell().???). Caller is responsible for calling
+   * getDefaultShell().clearEvents() when done.
+   *
+   * @param command Command to execute
+   * @return The result of the command execution
+   */
+  @SuppressWarnings("unused")
+  protected CommandResult executeCommandWithoutClear(String command) {
+    assert (command != null);
+
+    return executeCommandWithoutClear(getDefaultShell(), command);
+  }
+
+  /**
+   * Execute a command in the provided shell. Useful for getting additional information from the shell after the command
+   * has been executed (using getDefaultShell().???). Caller is responsible for calling getDefaultShell().clearEvents()
+   * when done.
+   *
+   * @param shell   Shell in which to execute the command.
+   * @param command Command to execute
+   * @return The result of the command execution
+   */
+  protected CommandResult executeCommandWithoutClear(HeadlessGfsh shell, String command) {
+    assert (shell != null);
+    assert (command != null);
+
+    try {
+      info("Executing command " + command + " with command Mgr " + CommandManager.getInstance());
+    } catch (ClassNotFoundException cnfex) {
+      throw new TestException(getStackTrace(cnfex));
+    } catch (IOException ioex) {
+      throw new TestException(getStackTrace(ioex));
+    }
+
+    shell.executeCommand(command);
+    if (shell.hasError()) {
+      error("executeCommand completed with error : " + shell.getError());
+    }
+
+    CommandResult result = null;
+    try {
+      result = (CommandResult) shell.getResult();
+    } catch (InterruptedException ex) {
+      error("shell received InterruptedException");
+    }
+
+    if (result != null) {
+      result.resetToFirstLine();
+    }
+
+    return result;
+  }
+
+  /**
+   * Utility method for viewing the results of a command.
+   *
+   * @param commandResult Results to dump
+   * @param printStream   Stream to dump the results to
+   */
+  protected void printResult(final CommandResult commandResult, PrintStream printStream) {
+    assert (commandResult != null);
+    assert (printStream != null);
+
+    commandResult.resetToFirstLine();
+    printStream.print(commandResultToString(commandResult));
+  }
+
+  protected String commandResultToString(final CommandResult commandResult) {
+    assertNotNull(commandResult);
+
+    commandResult.resetToFirstLine();
+
+    StringBuilder buffer = new StringBuilder(commandResult.getHeader());
+
+    while (commandResult.hasNextLine()) {
+      buffer.append(commandResult.nextLine());
+    }
+
+    buffer.append(commandResult.getFooter());
+
+    return buffer.toString();
+  }
+
+  /**
+   * Utility method for finding the CommandResult object in the Map of CommandOutput objects.
+   *
+   * @param commandOutput CommandOutput Map to search
+   * @return The CommandResult object or null if not found.
+   */
+  protected CommandResult extractCommandResult(Map<String, Object> commandOutput) {
+    assert (commandOutput != null);
+
+    for (Object resultObject : commandOutput.values()) {
+      if (resultObject instanceof CommandResult) {
+        CommandResult result = (CommandResult) resultObject;
+        result.resetToFirstLine();
+        return result;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Utility method to determine how many times a string occurs in another string. Note that when looking for matches
+   * substrings of other matches will be counted as a match. For example, looking for "AA" in the string "AAAA" will
+   * result in a return value of 3.
+   *
+   * @param stringToSearch String to search
+   * @param stringToCount  String to look for and count
+   * @return The number of matches.
+   */
+  protected int countMatchesInString(final String stringToSearch, final String stringToCount) {
+    assert (stringToSearch != null);
+    assert (stringToCount != null);
+
+    int length = stringToSearch.length();
+    int count = 0;
+    for (int i = 0; i < length; i++) {
+      if (stringToSearch.substring(i).startsWith(stringToCount)) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  /**
+   * Determines if a string contains a trimmed line that matches the pattern. So, any single line whose leading and
+   * trailing spaces have been removed which contains a string that exactly matches the given pattern will be considered
+   * a match.
+   *
+   * @param stringToSearch String to search
+   * @param stringPattern  Pattern to search for
+   * @return True if a match is found, false otherwise
+   */
+  protected boolean stringContainsLine(final String stringToSearch, final String stringPattern) {
+    assert (stringToSearch != null);
+    assert (stringPattern != null);
+
+    Pattern pattern = Pattern.compile("^\\s*" + stringPattern + "\\s*$", Pattern.MULTILINE);
+    Matcher matcher = pattern.matcher(stringToSearch);
+    return matcher.find();
+  }
+
+  /**
+   * Counts the number of distinct lines in a String.
+   *
+   * @param stringToSearch  String to search for lines.
+   * @param countBlankLines Whether to count blank lines (true to count)
+   * @return The number of lines found.
+   */
+  protected int countLinesInString(final String stringToSearch, final boolean countBlankLines) {
+    assert (stringToSearch != null);
+
+    int length = stringToSearch.length();
+    int count = 0;
+    char character = 0;
+    boolean foundNonSpaceChar = false;
+
+    for (int i = 0; i < length; i++) {
+      character = stringToSearch.charAt(i);
+      if (character == '\r' && (i + 1) < length && stringToSearch.charAt(i + 1) == '\n') {
+        i++;
+      }
+      if (character == '\n' || character == '\r') {
+        if (countBlankLines) {
+          count++;
+        } else {
+          if (foundNonSpaceChar) {
+            count++;
+          }
+        }
+        foundNonSpaceChar = false;
+      } else if (character != ' ' && character != '\t') {
+        foundNonSpaceChar = true;
+      }
+    }
+
+    // Even if the last line isn't terminated, it still counts as a line
+    if (character != '\n' && character != '\r') {
+      count++;
+    }
+
+    return count;
+  }
+
+  /**
+   * Get a specific line from the string (using \n or \r as a line separator).
+   *
+   * @param stringToSearch String to get the line from
+   * @param lineNumber     Line number to get
+   * @return The line
+   */
+  protected String getLineFromString(final String stringToSearch, final int lineNumber) {
+    assert (stringToSearch != null);
+    assert (lineNumber > 0);
+
+    int length = stringToSearch.length();
+    int count = 0;
+    int startIndex = 0;
+    char character;
+    int endIndex = length;
+
+    for (int i = 0; i < length; i++) {
+      character = stringToSearch.charAt(i);
+      if (character == '\r' && (i + 1) < length && stringToSearch.charAt(i + 1) == '\n') {
+        i++;
+      }
+      if (character == '\n' || character == '\r') {
+        if (lineNumber == 1) {
+          endIndex = i;
+          break;
+        }
+        if (++count == lineNumber - 1) {
+          startIndex = i + 1;
+        } else if (count >= lineNumber) {
+          endIndex = i;
+          break;
+        }
+      }
+    }
+
+    return stringToSearch.substring(startIndex, endIndex);
+  }
+
+  protected static String getStackTrace(Throwable aThrowable) {
+    StringWriter sw = new StringWriter();
+    aThrowable.printStackTrace(new PrintWriter(sw, true));
+    return sw.toString();
+  }
+
+  protected void info(String string) {
+    getLogWriter().info(string);
+  }
+
+  protected void debug(String string) {
+    getLogWriter().fine(string);
+  }
+
+  protected void error(String string) {
+    getLogWriter().error(string);
+  }
+
+  protected void error(String string, Throwable e) {
+    getLogWriter().error(string, e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java
new file mode 100644
index 0000000..81536db
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java
@@ -0,0 +1,497 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.SharedConfiguration;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlGenerator;
+import com.gemstone.gemfire.internal.logging.LogWriterImpl;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.cli.Result.Status;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.remote.CommandProcessor;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+import org.apache.commons.io.FileUtils;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Dunit class for testing GemFire config commands : export config
+ *
+ * @author David Hoots
+ * @author Sourabh Bansod
+ * @since 7.0
+ */
+public class ConfigCommandsDUnitTest extends CliCommandTestBase {
+  private static final long serialVersionUID = 1L;
+
+  File managerConfigFile = new File("Manager-cache.xml");
+  File managerPropsFile = new File("Manager-gf.properties");
+  File vm1ConfigFile = new File("VM1-cache.xml");
+  File vm1PropsFile = new File("VM1-gf.properties");
+  File vm2ConfigFile = new File("VM2-cache.xml");
+  File vm2PropsFile = new File("VM2-gf.properties");
+  File shellConfigFile = new File("Shell-cache.xml");
+  File shellPropsFile = new File("Shell-gf.properties");
+  File subDir = new File("ConfigCommandsDUnitTestSubDir");
+  File subManagerConfigFile = new File(subDir, managerConfigFile.getName());
+
+  public ConfigCommandsDUnitTest(String name) {
+    super(name);
+  }
+
+  public void tearDown2() throws Exception {
+    deleteTestFiles();
+    invokeInEveryVM(new SerializableRunnable() {
+
+      @Override
+      public void run() {
+        try {
+          deleteTestFiles();
+        } catch (IOException e) {
+          fail("error", e);
+        }
+      }
+    });
+    super.tearDown2();
+  }
+
+  public void testDescribeConfig() throws ClassNotFoundException, IOException {
+    createDefaultSetup(null);
+    final String controllerName = "Member2";
+
+    /***
+     * Create properties for the controller VM
+     */
+    final Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    localProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "info");
+    localProps.setProperty(DistributionConfig.STATISTIC_SAMPLING_ENABLED_NAME, "true");
+    localProps.setProperty(DistributionConfig.ENABLE_TIME_STATISTICS_NAME, "true");
+    localProps.setProperty(DistributionConfig.NAME_NAME, controllerName);
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "G1");
+    getSystem(localProps);
+    Cache cache = getCache();
+    int ports[] = AvailablePortHelper.getRandomAvailableTCPPorts(1);
+    CacheServer cs = getCache().addCacheServer();
+    cs.setPort(ports[0]);
+    cs.setMaxThreads(10);
+    cs.setMaxConnections(9);
+    cs.start();
+
+    RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
+    List<String> jvmArgs = runtimeBean.getInputArguments();
+
+    getLogWriter().info("#SB Actual JVM Args : ");
+
+    for (String jvmArg : jvmArgs) {
+      getLogWriter().info("#SB JVM " + jvmArg);
+    }
+
+    InternalDistributedSystem system = (InternalDistributedSystem) cache.getDistributedSystem();
+    DistributionConfig config = system.getConfig();
+    config.setArchiveFileSizeLimit(1000);
+
+    String command = CliStrings.DESCRIBE_CONFIG + " --member=" + controllerName;
+    CommandProcessor cmdProcessor = new CommandProcessor();
+    cmdProcessor.createCommandStatement(command, Collections.EMPTY_MAP).process();
+
+    CommandResult cmdResult = executeCommand(command);
+
+    String resultStr = commandResultToString(cmdResult);
+    getLogWriter().info("#SB Hiding the defaults\n" + resultStr);
+
+    assertEquals(true, cmdResult.getStatus().equals(Status.OK));
+    assertEquals(true, resultStr.contains("G1"));
+    assertEquals(true, resultStr.contains(controllerName));
+    assertEquals(true, resultStr.contains("archive-file-size-limit"));
+    assertEquals(true, !resultStr.contains("copy-on-read"));
+
+    cmdResult = executeCommand(command + " --" + CliStrings.DESCRIBE_CONFIG__HIDE__DEFAULTS + "=false");
+    resultStr = commandResultToString(cmdResult);
+    getLogWriter().info("#SB No hiding of defaults\n" + resultStr);
+
+    assertEquals(true, cmdResult.getStatus().equals(Status.OK));
+    assertEquals(true, resultStr.contains("is-server"));
+    assertEquals(true, resultStr.contains(controllerName));
+    assertEquals(true, resultStr.contains("copy-on-read"));
+
+    cs.stop();
+  }
+
+  @SuppressWarnings("serial")
+  public void testExportConfig() throws IOException {
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+    createDefaultSetup(localProps);
+
+    // Create a cache in another VM (VM1)
+    Host.getHost(0).getVM(1).invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, "VM1");
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(localProps);
+        getCache();
+      }
+    });
+
+    // Create a cache in a 3rd VM (VM2)
+    Host.getHost(0).getVM(2).invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, "VM2");
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(localProps);
+        getCache();
+      }
+    });
+
+    // Create a cache in the local VM
+    localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Shell");
+    getSystem(localProps);
+    Cache cache = getCache();
+
+    // Test export config for all members
+    deleteTestFiles();
+    CommandResult cmdResult = executeCommand("export config");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    assertTrue(this.managerConfigFile.exists());
+    assertTrue(this.managerPropsFile.exists());
+    assertTrue(this.vm1ConfigFile.exists());
+    assertTrue(this.vm1PropsFile.exists());
+    assertTrue(this.vm2ConfigFile.exists());
+    assertTrue(this.vm2PropsFile.exists());
+    assertTrue(this.shellConfigFile.exists());
+    assertTrue(this.shellPropsFile.exists());
+
+    // Test exporting member
+    deleteTestFiles();
+    cmdResult = executeCommand("export config --member=Manager");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    assertTrue(this.managerConfigFile.exists());
+    assertFalse(this.vm1ConfigFile.exists());
+    assertFalse(this.vm2ConfigFile.exists());
+    assertFalse(this.shellConfigFile.exists());
+
+    // Test exporting group
+    deleteTestFiles();
+    cmdResult = executeCommand("export config --group=Group2");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    assertFalse(this.managerConfigFile.exists());
+    assertTrue(this.vm1ConfigFile.exists());
+    assertTrue(this.vm2ConfigFile.exists());
+    assertFalse(this.shellConfigFile.exists());
+
+    // Test export to directory
+    deleteTestFiles();
+    cmdResult = executeCommand("export config --dir=" + subDir.getAbsolutePath());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    assertFalse(this.managerConfigFile.exists());
+    assertTrue(this.subManagerConfigFile.exists());
+
+    // Test the contents of the file
+    StringWriter stringWriter = new StringWriter();
+    PrintWriter printWriter = new PrintWriter(stringWriter);
+    CacheXmlGenerator.generate(cache, printWriter, false, false, false);
+    String configToMatch = stringWriter.toString();
+
+    deleteTestFiles();
+    cmdResult = executeCommand("export config --member=Shell");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    char[] fileContents = new char[configToMatch.length()];
+    try {
+      FileReader reader = new FileReader(shellConfigFile);
+      reader.read(fileContents);
+    } catch (Exception ex) {
+      fail("Unable to read file contents for comparison", ex);
+    }
+
+    assertEquals(configToMatch, new String(fileContents));
+  }
+
+  public void testAlterRuntimeConfig() throws ClassNotFoundException, IOException {
+    final String controller = "controller";
+    createDefaultSetup(null);
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, controller);
+    localProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "error");
+    getSystem(localProps);
+    final GemFireCacheImpl cache = (GemFireCacheImpl) getCache();
+    final DistributionConfig config = cache.getSystem().getConfig();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.ALTER_RUNTIME_CONFIG);
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__MEMBER, controller);
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL, "info");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT, "50");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT, "32");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT, "49");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE, "2000");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE, "stat.gfs");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLING__ENABLED, "true");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, "10");
+    CommandResult cmdResult = executeCommand(csb.getCommandString());
+    String resultString = commandResultToString(cmdResult);
+    getLogWriter().info("Result\n");
+    getLogWriter().info(resultString);
+    assertEquals(true, cmdResult.getStatus().equals(Status.OK));
+    assertEquals(LogWriterImpl.INFO_LEVEL, config.getLogLevel());
+    assertEquals(50, config.getLogFileSizeLimit());
+    assertEquals(32, config.getArchiveDiskSpaceLimit());
+    assertEquals(2000, config.getStatisticSampleRate());
+    assertEquals("stat.gfs", config.getStatisticArchiveFile().getName());
+    assertEquals(true, config.getStatisticSamplingEnabled());
+    assertEquals(10, config.getLogDiskSpaceLimit());
+
+
+    CommandProcessor commandProcessor = new CommandProcessor();
+    Result result = commandProcessor.createCommandStatement("alter runtime", Collections.EMPTY_MAP).process();
+  }
+
+  public void testAlterRuntimeConfigRandom() {
+    final String member1 = "VM1";
+    final String controller = "controller";
+    createDefaultSetup(null);
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, controller);
+    localProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "error");
+    getSystem(localProps);
+    final GemFireCacheImpl cache = (GemFireCacheImpl) getCache();
+    final DistributionConfig config = cache.getSystem().getConfig();
+
+    Host.getHost(0).getVM(1).invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, member1);
+        getSystem(localProps);
+        Cache cache = getCache();
+      }
+    });
+
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.ALTER_RUNTIME_CONFIG);
+    CommandResult cmdResult = executeCommand(csb.getCommandString());
+    String resultAsString = commandResultToString(cmdResult);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultAsString);
+    assertEquals(true, cmdResult.getStatus().equals(Status.ERROR));
+    assertTrue(resultAsString.contains(CliStrings.ALTER_RUNTIME_CONFIG__RELEVANT__OPTION__MESSAGE));
+
+    csb = new CommandStringBuilder(CliStrings.ALTER_RUNTIME_CONFIG);
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, "2000000000");
+    cmdResult = executeCommand(csb.getCommandString());
+    resultAsString = commandResultToString(cmdResult);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultAsString);
+    assertEquals(true, cmdResult.getStatus().equals(Status.ERROR));
+
+  }
+
+  public void testAlterRuntimeConfigOnAllMembers() {
+    final String member1 = "VM1";
+    final String controller = "controller";
+    createDefaultSetup(null);
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, controller);
+    localProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "error");
+    getSystem(localProps);
+    final GemFireCacheImpl cache = (GemFireCacheImpl) getCache();
+    final DistributionConfig config = cache.getSystem().getConfig();
+
+    Host.getHost(0).getVM(1).invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, member1);
+        getSystem(localProps);
+        Cache cache = getCache();
+      }
+    });
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.ALTER_RUNTIME_CONFIG);
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL, "info");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT, "50");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT, "32");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT, "49");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE, "2000");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE, "stat.gfs");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLING__ENABLED, "true");
+    csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, "10");
+    CommandResult cmdResult = executeCommand(csb.getCommandString());
+    String resultString = commandResultToString(cmdResult);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultString);
+    assertEquals(true, cmdResult.getStatus().equals(Status.OK));
+    assertEquals(LogWriterImpl.INFO_LEVEL, config.getLogLevel());
+    assertEquals(50, config.getLogFileSizeLimit());
+    assertEquals(49, config.getArchiveFileSizeLimit());
+    assertEquals(32, config.getArchiveDiskSpaceLimit());
+    assertEquals(2000, config.getStatisticSampleRate());
+    assertEquals("stat.gfs", config.getStatisticArchiveFile().getName());
+    assertEquals(true, config.getStatisticSamplingEnabled());
+    assertEquals(10, config.getLogDiskSpaceLimit());
+
+    // Validate the changes in the vm1
+    Host.getHost(0).getVM(1).invoke(new SerializableRunnable() {
+      public void run() {
+        GemFireCacheImpl cacheVM1 = (GemFireCacheImpl) getCache();
+        final DistributionConfig configVM1 = cacheVM1.getSystem().getConfig();
+        assertEquals(LogWriterImpl.INFO_LEVEL, configVM1.getLogLevel());
+        assertEquals(50, configVM1.getLogFileSizeLimit());
+        assertEquals(49, configVM1.getArchiveFileSizeLimit());
+        assertEquals(32, configVM1.getArchiveDiskSpaceLimit());
+        assertEquals(2000, configVM1.getStatisticSampleRate());
+        assertEquals("stat.gfs", configVM1.getStatisticArchiveFile().getName());
+        assertEquals(true, configVM1.getStatisticSamplingEnabled());
+        assertEquals(10, configVM1.getLogDiskSpaceLimit());
+      }
+    });
+  }
+
+  /**
+   * Asserts that altering the runtime config correctly updates the shared configuration.
+   * <p>
+   * Disabled: this test frequently fails during unit test runs. See ticket #52204
+   */
+  public void disabledtestAlterUpdatesSharedConfig() {
+    disconnectAllFromDS();
+
+    final String groupName = "testAlterRuntimeConfigSharedConfigGroup";
+
+    // Start the Locator and wait for shared configuration to be available
+    final int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+
+        final File locatorLogFile = new File("locator-" + locatorPort + ".log");
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, "Locator");
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
+              locatorProps);
+
+          DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+            @Override
+            public boolean done() {
+              return locator.isSharedConfigurationRunning();
+            }
+
+            @Override
+            public String description() {
+              return "Waiting for shared configuration to be started";
+            }
+          };
+          DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+        } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        }
+      }
+    });
+
+    // Start the default manager
+    Properties managerProps = new Properties();
+    managerProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    managerProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+    createDefaultSetup(managerProps);
+
+    // Create a cache in VM 1
+    VM vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        //Make sure no previous shared config is screwing up this test.
+        FileUtil.delete(new File("ConfigDiskDir_Locator"));
+        FileUtil.delete(new File("cluster_config"));
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "error");
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        getSystem(localProps);
+
+        assertNotNull(getCache());
+        assertEquals("error", system.getConfig().getAttribute(DistributionConfig.LOG_LEVEL_NAME));
+        return null;
+      }
+    });
+
+    // Test altering the runtime config
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_RUNTIME_CONFIG);
+    commandStringBuilder.addOption(CliStrings.ALTER_RUNTIME_CONFIG__GROUP, groupName);
+    commandStringBuilder.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL, "fine");
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure the shared config was updated
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        Properties gemfireProperties;
+        try {
+          gemfireProperties = sharedConfig.getConfiguration(groupName).getGemfireProperties();
+          assertEquals("fine", gemfireProperties.get(DistributionConfig.LOG_LEVEL_NAME));
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service", e);
+        }
+      }
+    });
+  }
+
+  private final void deleteTestFiles() throws IOException {
+    this.managerConfigFile.delete();
+    this.managerPropsFile.delete();
+    this.vm1ConfigFile.delete();
+    this.vm1PropsFile.delete();
+    this.vm2ConfigFile.delete();
+    this.vm2PropsFile.delete();
+    this.shellConfigFile.delete();
+    this.shellPropsFile.delete();
+
+    FileUtils.deleteDirectory(this.subDir);
+  }
+}


[20/50] [abbrv] incubator-geode git commit: GEODE-617: Change xsd namespace for lucene to geode.apache.org

Posted by kl...@apache.org.
GEODE-617: Change xsd namespace for lucene to geode.apache.org


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

Branch: refs/heads/feature/GEODE-217
Commit: e414a493325d9742772e647965e1bcd1f45650d1
Parents: dec83b4
Author: Dan Smith <up...@apache.org>
Authored: Tue Dec 1 16:42:08 2015 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Mon Dec 7 10:10:52 2015 -0800

----------------------------------------------------------------------
 .../cache/xmlcache/GeodeEntityResolver.java     |  8 +--
 .../lucene/internal/xml/LuceneXmlConstants.java |  2 +-
 .../geode.apache.org/lucene/lucene-1.0.xsd      | 57 +++++++++++++++++++
 .../lucene/lucene-1.0.xsd                       | 58 --------------------
 ...erIntegrationJUnitTest.createIndex.cache.xml |  6 +-
 ...serIntegrationJUnitTest.parseIndex.cache.xml |  6 +-
 6 files changed, 68 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e414a493/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/GeodeEntityResolver.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/GeodeEntityResolver.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/GeodeEntityResolver.java
index 559a1f8..67cda99 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/GeodeEntityResolver.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/GeodeEntityResolver.java
@@ -26,8 +26,8 @@ import org.xml.sax.ext.EntityResolver2;
 
 /**
  * Resolves entities for XSDs or DTDs with SYSTEM IDs rooted at
- * http://www.pivotal.io/xml/ns from the classpath at
- * /META-INF/schemas/schema.pivotal.io/.
+ * http://geode.apache.org/schema from the classpath at
+ * /META-INF/schemas/geode.apache.org/.
  * 
  * Loaded by {@link ServiceLoader} on {@link EntityResolver2} class. See file
  * <code>META-INF/services/org.xml.sax.ext.EntityResolver2</code>
@@ -38,9 +38,9 @@ import org.xml.sax.ext.EntityResolver2;
  */
 public final class GeodeEntityResolver extends DefaultEntityResolver2 {
 
-  private static final String SYSTEM_ID_ROOT = "http://geode.incubator.apache.org/schema";
+  private static final String SYSTEM_ID_ROOT = "http://geode.apache.org/schema";
 
-  private static final String CLASSPATH_ROOT = "/META-INF/schemas/geode.incubator.apache.org/";
+  private static final String CLASSPATH_ROOT = "/META-INF/schemas/geode.apache.org/";
 
   @Override
   public InputSource resolveEntity(final String name, final String publicId, final String baseURI, final String systemId) throws SAXException, IOException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e414a493/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneXmlConstants.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneXmlConstants.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneXmlConstants.java
index 303424e..bc80180 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneXmlConstants.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneXmlConstants.java
@@ -20,7 +20,7 @@
 package com.gemstone.gemfire.cache.lucene.internal.xml;
 
 public class LuceneXmlConstants {
-  public static final String NAMESPACE= "http://geode.incubator.apache.org/schema/lucene";
+  public static final String NAMESPACE= "http://geode.apache.org/schema/lucene";
   public static final String PREFIX = "lucene";
   public static final String SERVICE = "service";
   public static final String NAME = "name";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e414a493/gemfire-lucene/src/main/resources/META-INF/schemas/geode.apache.org/lucene/lucene-1.0.xsd
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/resources/META-INF/schemas/geode.apache.org/lucene/lucene-1.0.xsd b/gemfire-lucene/src/main/resources/META-INF/schemas/geode.apache.org/lucene/lucene-1.0.xsd
new file mode 100644
index 0000000..bfe9f6c
--- /dev/null
+++ b/gemfire-lucene/src/main/resources/META-INF/schemas/geode.apache.org/lucene/lucene-1.0.xsd
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<xsd:schema
+    targetNamespace="http://geode.apache.org/schema/lucene"
+    xmlns:xsd="http://www.w3.org/2001/XMLSchema"
+    elementFormDefault="qualified"
+    attributeFormDefault="unqualified"
+    version="1.0">
+  
+  <xsd:import
+      namespace="http://schema.pivotal.io/gemfire/cache"
+      schemaLocation="http://schema.pivotal.io/gemfire/cache/cache-9.0.xsd"/>
+  
+  <xsd:annotation>
+    <xsd:documentation><![CDATA[
+XML schema for Lucene indexes in Geode.
+
+  <cache
+    xmlns="http://schema.pivotal.io/gemfire/cache"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://schema.pivotal.io/gemfire/cache
+        http://schema.pivotal.io/gemfire/cache/cache-9.0.xsd
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
+    version="9.0">
+    
+    ]]></xsd:documentation>
+  </xsd:annotation>
+  <xsd:element name="index">
+    <xsd:complexType>
+    	<xsd:sequence>
+    	  <xsd:element name="field" maxOccurs="unbounded">
+			<xsd:complexType>
+				<xsd:attribute name="name" type="xsd:string" />
+			</xsd:complexType>
+    	  </xsd:element>
+    	</xsd:sequence>
+    	<xsd:attribute name="name" type="xsd:string"/>
+    </xsd:complexType>
+  </xsd:element>
+</xsd:schema>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e414a493/gemfire-lucene/src/main/resources/META-INF/schemas/geode.incubator.apache.org/lucene/lucene-1.0.xsd
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/resources/META-INF/schemas/geode.incubator.apache.org/lucene/lucene-1.0.xsd b/gemfire-lucene/src/main/resources/META-INF/schemas/geode.incubator.apache.org/lucene/lucene-1.0.xsd
deleted file mode 100644
index b1eae03..0000000
--- a/gemfire-lucene/src/main/resources/META-INF/schemas/geode.incubator.apache.org/lucene/lucene-1.0.xsd
+++ /dev/null
@@ -1,58 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one or more
-contributor license agreements.  See the NOTICE file distributed with
-this work for additional information regarding copyright ownership.
-The ASF licenses this file to You under the Apache License, Version 2.0
-(the "License"); you may not use this file except in compliance with
-the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
--->
-<xsd:schema
-    targetNamespace="http://geode.incubator.apache.org/schema/lucene"
-    xmlns:gpdb="http://geode.incubator.apache.org/schema/lucene"
-    xmlns:xsd="http://www.w3.org/2001/XMLSchema"
-    elementFormDefault="qualified"
-    attributeFormDefault="unqualified"
-    version="1.0">
-  
-  <xsd:import
-      namespace="http://schema.pivotal.io/gemfire/cache"
-      schemaLocation="http://schema.pivotal.io/gemfire/cache/cache-9.0.xsd"/>
-  
-  <xsd:annotation>
-    <xsd:documentation><![CDATA[
-XML schema for Lucene indexes in Geode.
-
-  <cache
-    xmlns="http://schema.pivotal.io/gemfire/cache"
-    xmlns:lucene="http://geode.incubator.apache.org/schema/lucene"
-    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-    xsi:schemaLocation="http://schema.pivotal.io/gemfire/cache
-        http://schema.pivotal.io/gemfire/cache/cache-9.0.xsd
-        http://geode.incubator.apache.org/schema/lucene
-        http://geode.incubator.apache.org/schema/lucene/lucene-1.0.xsd"
-    version="9.0">
-    
-    ]]></xsd:documentation>
-  </xsd:annotation>
-  <xsd:element name="index">
-    <xsd:complexType>
-    	<xsd:sequence>
-    	  <xsd:element name="field" maxOccurs="unbounded">
-			<xsd:complexType>
-				<xsd:attribute name="name" type="xsd:string" />
-			</xsd:complexType>
-    	  </xsd:element>
-    	</xsd:sequence>
-    	<xsd:attribute name="name" type="xsd:string"/>
-    </xsd:complexType>
-  </xsd:element>
-</xsd:schema>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e414a493/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.createIndex.cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.createIndex.cache.xml b/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.createIndex.cache.xml
index 7f804e0..42e4e84 100644
--- a/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.createIndex.cache.xml
+++ b/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.createIndex.cache.xml
@@ -1,12 +1,12 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
-    xmlns:lucene="http://geode.incubator.apache.org/schema/lucene"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
     xsi:schemaLocation="http://schema.pivotal.io/gemfire/cache
         http://schema.pivotal.io/gemfire/cache/cache-9.0.xsd
-        http://geode.incubator.apache.org/schema/lucene
-        http://geode.incubator.apache.org/schema/lucene/lucene-1.0.xsd"
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
     version="9.0">
 
 	<region name="region" refid="PARTITION">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e414a493/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.parseIndex.cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.parseIndex.cache.xml b/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.parseIndex.cache.xml
index 7f804e0..42e4e84 100644
--- a/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.parseIndex.cache.xml
+++ b/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.parseIndex.cache.xml
@@ -1,12 +1,12 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
-    xmlns:lucene="http://geode.incubator.apache.org/schema/lucene"
+    xmlns:lucene="http://geode.apache.org/schema/lucene"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
     xsi:schemaLocation="http://schema.pivotal.io/gemfire/cache
         http://schema.pivotal.io/gemfire/cache/cache-9.0.xsd
-        http://geode.incubator.apache.org/schema/lucene
-        http://geode.incubator.apache.org/schema/lucene/lucene-1.0.xsd"
+        http://geode.apache.org/schema/lucene
+        http://geode.apache.org/schema/lucene/lucene-1.0.xsd"
     version="9.0">
 
 	<region name="region" refid="PARTITION">


[16/50] [abbrv] incubator-geode git commit: Added GMSJoinLeave tests

Posted by kl...@apache.org.
Added GMSJoinLeave 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/e0bf6858
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/e0bf6858
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/e0bf6858

Branch: refs/heads/feature/GEODE-217
Commit: e0bf6858ab3db9c043853cef32cba0cffc1f1b68
Parents: fba6867
Author: Hitesh Khamesra <hi...@yahoo.com>
Authored: Fri Dec 4 09:04:18 2015 -0800
Committer: Hitesh Khamesra <hi...@yahoo.com>
Committed: Fri Dec 4 09:05:41 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/membership/GMSJoinLeave.java |  17 ++-
 .../gms/membership/GMSJoinLeaveJUnitTest.java   | 149 +++++++++++++++++++
 2 files changed, 165 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0bf6858/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
index 2f9c514..2986238 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
@@ -1476,12 +1476,24 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     return result;
   }
 
+  /***
+   * test method
+   * @return ViewReplyProcessor
+   */
+  protected ViewReplyProcessor getPrepareViewReplyProcessor() {
+    return prepareProcessor;
+  }
+  
+  protected boolean testPrepareProcessorWaiting(){
+    return prepareProcessor.isWaiting();
+  }
+  
   class ViewReplyProcessor {
     volatile int viewId = -1;
     final Set<InternalDistributedMember> notRepliedYet = new HashSet<>();
     NetView conflictingView;
     InternalDistributedMember conflictingViewSender;
-    boolean waiting;
+    volatile boolean waiting;
     final boolean isPrepareViewProcessor;
     final Set<InternalDistributedMember> pendingRemovals = new HashSet<>();
 
@@ -1498,6 +1510,9 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
       pendingRemovals.clear();
     }
 
+    boolean isWaiting(){
+      return waiting;
+    }
     synchronized void processPendingRequests(Set<InternalDistributedMember> pendingLeaves, Set<InternalDistributedMember> pendingRemovals) {
       // there's no point in waiting for members who have already
       // requested to leave or who have been declared crashed.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0bf6858/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
index e49e4ae..abc7a2f 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
@@ -41,8 +41,14 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+import org.mockito.internal.verification.Times;
+import org.mockito.internal.verification.api.VerificationData;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
+import org.mockito.verification.Timeout;
+import org.mockito.verification.VerificationMode;
+import org.mockito.verification.VerificationWithTimeout;
 
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
@@ -57,13 +63,17 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manag
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Messenger;
 import com.gemstone.gemfire.distributed.internal.membership.gms.locator.FindCoordinatorResponse;
 import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave.SearchState;
+import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave.ViewReplyProcessor;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.InstallViewMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinRequestMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinResponseMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.LeaveRequestMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.NetworkPartitionMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.RemoveMemberMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.ViewAckMessage;
 import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.admin.remote.AddStatListenerResponse;
+import com.gemstone.gemfire.internal.admin.remote.StatListenerMessage;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
@@ -793,5 +803,144 @@ public class GMSJoinLeaveJUnitTest {
     b.run();
     verify(messenger).sendUnreliably(isA(InstallViewMessage.class));
   }
+  
+  private void installView(int viewId,InternalDistributedMember coordinator, List<InternalDistributedMember> members) throws IOException {
+    Set<InternalDistributedMember> shutdowns = new HashSet<>();
+    Set<InternalDistributedMember> crashes = new HashSet<>();
+    
+    when(services.getMessenger()).thenReturn(messenger);
+    
+    //prepare the view
+    NetView netView = new NetView(coordinator, viewId, members, shutdowns, crashes);
+    InstallViewMessage installViewMessage = new InstallViewMessage(netView, credentials, false);
+    gmsJoinLeave.processMessage(installViewMessage);
+   // verify(messenger).send(any(ViewAckMessage.class));
+  }
+  
+  @Test
+  public void testIgnoreoldView() throws Exception {
+    initMocks(false);
+    installView(3, mockMembers[0], createMemberList(mockMembers[0], mockMembers[1], mockMembers[2], gmsJoinLeaveMemberId, mockMembers[3]));
+    //now try to intall old view..
+    installView(1, mockMembers[0], createMemberList(mockMembers[0], mockMembers[1], mockMembers[2], gmsJoinLeaveMemberId, mockMembers[3]));
+    
+    assertFalse("Expected view id is 3 but found " + gmsJoinLeave.getView().getViewId(), gmsJoinLeave.getView().getViewId() == 1);
+  }
+  
+  @Test
+  public void testClearViewRequests() throws Exception {
+    try {
+    initMocks(false);
+    System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
+    gmsJoinLeave.join();
+    gmsJoinLeave.processMessage(new JoinRequestMessage(mockMembers[0], mockMembers[0], credentials, -1));
+    int viewRequests = gmsJoinLeave.getViewRequests().size();
+    
+    assertTrue( "There should be 1 viewRequest but found " + viewRequests, viewRequests == 1);
+    Thread.sleep(2 * GMSJoinLeave.MEMBER_REQUEST_COLLECTION_INTERVAL);
+    
+    viewRequests = gmsJoinLeave.getViewRequests().size();
+    assertTrue( "There should be 0 viewRequest but found " + viewRequests, viewRequests == 0);
+    }finally {
+      System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
+    }
+  }
+  
+  /***
+   * validating ViewReplyProcessor's memberSuspected, 
+   * processLeaveRequest, processRemoveRequest, processViewResponse method
+   */
+  @Test
+  public void testViewReplyProcessor() throws Exception {
+    try {
+      initMocks(false);
+      System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
+      gmsJoinLeave.join();
+      Set<InternalDistributedMember> recips = new HashSet<>();
+      recips.add(mockMembers[0]);
+      recips.add(mockMembers[1]);
+      recips.add(mockMembers[2]);
+      recips.add(mockMembers[3]);
+      ViewReplyProcessor prepareProcessor = gmsJoinLeave.getPrepareViewReplyProcessor(); 
+      prepareProcessor.initialize( 1, recips);
+      assertTrue("Prepare processor should be waiting ", gmsJoinLeave.testPrepareProcessorWaiting());
+      
+      prepareProcessor.memberSuspected(gmsJoinLeaveMemberId, mockMembers[0]);
+      prepareProcessor.processLeaveRequest(mockMembers[1]);
+      prepareProcessor.processRemoveRequest(mockMembers[2]);
+      prepareProcessor.processViewResponse(1, mockMembers[3], null);
+      
+      assertFalse("Prepare processor should not be waiting ", gmsJoinLeave.testPrepareProcessorWaiting());
+      }finally {
+        System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
+      }
+  }
+  
+  /***
+   * validating ViewReplyProcessor's processPendingRequests method
+   */
+  @Test
+  public void testViewReplyProcessor2() throws Exception {
+    try {
+      initMocks(false);
+      System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
+      gmsJoinLeave.join();
+      Set<InternalDistributedMember> recips = new HashSet<>();
+      recips.add(mockMembers[0]);
+      recips.add(mockMembers[1]);
+      recips.add(mockMembers[2]);
+      recips.add(mockMembers[3]);
+      ViewReplyProcessor prepareProcessor = gmsJoinLeave.getPrepareViewReplyProcessor();
+      prepareProcessor.initialize(1, recips);
+      assertTrue("Prepare processor should be waiting ", gmsJoinLeave.testPrepareProcessorWaiting());
+      Set<InternalDistributedMember> pendingLeaves = new HashSet<>();
+      pendingLeaves.add(mockMembers[0]);
+      Set<InternalDistributedMember> pendingRemovals = new HashSet<>();
+      pendingRemovals.add(mockMembers[1]);
+      
+      prepareProcessor.processPendingRequests(pendingLeaves, pendingRemovals);
+      
+      prepareProcessor.processViewResponse(1, mockMembers[2], null);
+      prepareProcessor.processViewResponse(1, mockMembers[3], null);
+      
+      assertFalse("Prepare processor should not be waiting ", gmsJoinLeave.testPrepareProcessorWaiting());
+      }finally {
+        System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
+      }
+  }
+  
+  @Test
+  public void testJoinResponseMsgWithBecomeCoordinator() throws Exception {
+    initMocks(false);
+    gmsJoinLeaveMemberId.getNetMember().setPreferredForCoordinator(false);
+    JoinRequestMessage reqMsg = new JoinRequestMessage(gmsJoinLeaveMemberId, mockMembers[0], null, 56734);
+    InternalDistributedMember ids = new InternalDistributedMember("localhost", 97898);
+    ids.getNetMember().setPreferredForCoordinator(true);
+    gmsJoinLeave.processMessage(reqMsg);
+    ArgumentCaptor<JoinResponseMessage> ac = new ArgumentCaptor<>();
+    verify(messenger).send(ac.capture());
+    
+    assertTrue("Should have asked for becoming a coordinator", ac.getValue().getBecomeCoordinator());
+  }
+  
+  @Test
+  public void testNetworkPartionMessage() throws Exception {
+    try {
+      initMocks(true);
+      System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
+      gmsJoinLeave.join();
+      installView(1, gmsJoinLeaveMemberId, createMemberList(mockMembers[0], mockMembers[1], mockMembers[2], gmsJoinLeaveMemberId, mockMembers[3]));
+      for(int i = 1; i < 4; i++) {
+        RemoveMemberMessage msg = new RemoveMemberMessage(gmsJoinLeaveMemberId, mockMembers[i], "crashed");
+        msg.setSender(gmsJoinLeaveMemberId);
+        gmsJoinLeave.processMessage(msg);
+      }
+      Timeout to = new Timeout(2 * GMSJoinLeave.MEMBER_REQUEST_COLLECTION_INTERVAL, new Times(1));
+      verify(messenger, to).send( isA(NetworkPartitionMessage.class));
+                 
+    }finally {
+      System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
+    }    
+  }
 }
 


[38/50] [abbrv] incubator-geode git commit: GEODE-638: Add build task to allow for custom set of tests to be run

Posted by kl...@apache.org.
GEODE-638: Add build task to allow for custom set of tests to be run


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

Branch: refs/heads/feature/GEODE-217
Commit: a6398d919685d63c2bc89c1f4f605a5b73f3f257
Parents: eddef32
Author: Jens Deppe <jd...@pivotal.io>
Authored: Mon Dec 7 15:25:15 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Tue Dec 8 09:34:52 2015 -0800

----------------------------------------------------------------------
 build.gradle | 18 +++++++++++++++++-
 1 file changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6398d91/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index b5465b8..0c94573 100755
--- a/build.gradle
+++ b/build.gradle
@@ -386,7 +386,23 @@ subprojects {
     //I'm hoping this might deal with SOME OOMEs I've seen
     forkEvery 30
   }
-  
+
+  // By proving a file with an arbitrary list of test classes, we can select only those
+  // tests to run. Activated using -Dcustom.tests=<file> customTest
+  def customTestList = []
+  def customTestFile = System.getProperty('custom.tests')
+  if (customTestFile != null) {
+    new File(customTestFile).eachLine { customTestList << it }
+  }
+
+  task customTest(type:Test) {
+    include { x ->
+      (x.isDirectory() || customTestList.any { y -> x.getName().contains(y) } ) ? true : false
+    }
+
+    forkEvery 30
+  }
+
   // apply common test configuration
   gradle.taskGraph.whenReady( { graph ->
     tasks.withType(Test).each { test ->


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

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


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

Branch: refs/heads/feature/GEODE-217
Commit: 4c72833c9e490f48e2013755b0ac64f220c9649c
Parents: 3733ae7 812d51c
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Dec 7 17:36:49 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Mon Dec 7 17:36:49 2015 -0800

----------------------------------------------------------------------
 README.md                                       |   11 +-
 build.gradle                                    |  121 +-
 dev-tools/docker/base/Dockerfile                |   54 +
 dev-tools/docker/base/build-base-docker.sh      |   30 +
 dev-tools/docker/compile/Dockerfile             |   21 +
 .../docker/compile/start-compile-docker.sh      |   62 +
 docker/Dockerfile                               |   56 +-
 docker/README.md                                |    6 +-
 docker/build-runtime-docker.sh                  |   32 +
 gemfire-assembly/build.gradle                   |   57 +-
 gemfire-assembly/src/main/dist/bin/gfsh         |    2 +-
 gemfire-assembly/src/main/dist/bin/gfsh.bat     |    2 +-
 .../src/test/java/AgentUtilJUnitTest.java       |  109 -
 .../management/internal/AgentUtilJUnitTest.java |  106 +
 gemfire-common/build.gradle                     |    3 +
 .../gemfire/annotations/Experimental.java       |   56 +
 .../annotations/ExperimentalJUnitTest.java      |  199 +
 .../ClassInExperimentalPackage.java             |   27 +
 .../experimentalpackage/package-info.java       |   27 +
 .../ClassInNonExperimentalPackage.java          |   27 +
 .../nonexperimentalpackage/package-info.java    |   24 +
 gemfire-core/build.gradle                       |  187 +-
 .../internal/ra/GFConnectionFactoryImpl.java    |   21 +-
 .../gemfire/internal/ra/GFConnectionImpl.java   |   21 +-
 .../internal/ra/spi/JCALocalTransaction.java    |   21 +-
 .../internal/ra/spi/JCAManagedConnection.java   |   21 +-
 .../ra/spi/JCAManagedConnectionFactory.java     |   21 +-
 .../ra/spi/JCAManagedConnectionMetaData.java    |   21 +-
 gemfire-core/src/jca/ra.xml                     |    2 +-
 .../com/gemstone/gemfire/CancelCriterion.java   |   21 +-
 .../com/gemstone/gemfire/CancelException.java   |   21 +-
 .../gemstone/gemfire/CanonicalInstantiator.java |   21 +-
 .../com/gemstone/gemfire/CopyException.java     |   21 +-
 .../java/com/gemstone/gemfire/CopyHelper.java   |   21 +-
 .../com/gemstone/gemfire/DataSerializable.java  |   21 +-
 .../com/gemstone/gemfire/DataSerializer.java    |   34 +-
 .../main/java/com/gemstone/gemfire/Delta.java   |   21 +-
 .../gemfire/DeltaSerializationException.java    |   21 +-
 .../gemfire/ForcedDisconnectException.java      |   21 +-
 .../gemstone/gemfire/GemFireCacheException.java |   21 +-
 .../gemfire/GemFireCheckedException.java        |   21 +-
 .../gemfire/GemFireConfigException.java         |   21 +-
 .../com/gemstone/gemfire/GemFireException.java  |   21 +-
 .../gemstone/gemfire/GemFireIOException.java    |   21 +-
 .../gemstone/gemfire/GemFireRethrowable.java    |   21 +-
 .../gemfire/IncompatibleSystemException.java    |   21 +-
 .../java/com/gemstone/gemfire/Instantiator.java |   21 +-
 .../gemstone/gemfire/InternalGemFireError.java  |   21 +-
 .../gemfire/InternalGemFireException.java       |   21 +-
 .../gemstone/gemfire/InvalidDeltaException.java |   21 +-
 .../gemstone/gemfire/InvalidValueException.java |   21 +-
 .../gemfire/InvalidVersionException.java        |   16 +
 .../com/gemstone/gemfire/LicenseException.java  |   21 +-
 .../java/com/gemstone/gemfire/LogWriter.java    |   21 +-
 .../com/gemstone/gemfire/NoSystemException.java |   21 +-
 .../gemfire/OutOfOffHeapMemoryException.java    |   21 +-
 .../gemfire/SerializationException.java         |   21 +-
 .../gemstone/gemfire/StatisticDescriptor.java   |   21 +-
 .../java/com/gemstone/gemfire/Statistics.java   |   21 +-
 .../com/gemstone/gemfire/StatisticsFactory.java |   21 +-
 .../com/gemstone/gemfire/StatisticsType.java    |   21 +-
 .../gemstone/gemfire/StatisticsTypeFactory.java |   21 +-
 .../gemfire/SystemConnectException.java         |   21 +-
 .../com/gemstone/gemfire/SystemFailure.java     |   25 +-
 .../gemfire/SystemIsRunningException.java       |   21 +-
 .../gemfire/ThreadInterruptedException.java     |   18 +-
 .../com/gemstone/gemfire/ToDataException.java   |   21 +-
 .../gemfire/UncreatedSystemException.java       |   21 +-
 .../gemstone/gemfire/UnmodifiableException.java |   21 +-
 .../gemfire/UnstartedSystemException.java       |   21 +-
 .../com/gemstone/gemfire/admin/AdminConfig.java |   21 +-
 .../gemfire/admin/AdminDistributedSystem.java   |   26 +-
 .../admin/AdminDistributedSystemFactory.java    |   21 +-
 .../gemstone/gemfire/admin/AdminException.java  |   21 +-
 .../gemfire/admin/AdminXmlException.java        |   21 +-
 .../java/com/gemstone/gemfire/admin/Alert.java  |   21 +-
 .../com/gemstone/gemfire/admin/AlertLevel.java  |   21 +-
 .../gemstone/gemfire/admin/AlertListener.java   |   21 +-
 .../gemstone/gemfire/admin/BackupStatus.java    |   21 +-
 .../admin/CacheDoesNotExistException.java       |   21 +-
 .../gemfire/admin/CacheHealthConfig.java        |   21 +-
 .../com/gemstone/gemfire/admin/CacheServer.java |   21 +-
 .../gemfire/admin/CacheServerConfig.java        |   21 +-
 .../com/gemstone/gemfire/admin/CacheVm.java     |   21 +-
 .../gemstone/gemfire/admin/CacheVmConfig.java   |   21 +-
 .../gemfire/admin/ConfigurationParameter.java   |   21 +-
 .../gemfire/admin/DistributedSystemConfig.java  |   21 +-
 .../admin/DistributedSystemHealthConfig.java    |   21 +-
 .../gemfire/admin/DistributionLocator.java      |   21 +-
 .../admin/DistributionLocatorConfig.java        |   21 +-
 .../gemstone/gemfire/admin/GemFireHealth.java   |   21 +-
 .../gemfire/admin/GemFireHealthConfig.java      |   21 +-
 .../gemfire/admin/GemFireMemberStatus.java      |   34 +-
 .../gemstone/gemfire/admin/ManagedEntity.java   |   21 +-
 .../gemfire/admin/ManagedEntityConfig.java      |   21 +-
 .../gemfire/admin/MemberHealthConfig.java       |   21 +-
 .../admin/OperationCancelledException.java      |   21 +-
 .../gemfire/admin/RegionNotFoundException.java  |   21 +-
 .../gemfire/admin/RegionSubRegionSnapshot.java  |   21 +-
 .../gemfire/admin/RuntimeAdminException.java    |   21 +-
 .../com/gemstone/gemfire/admin/Statistic.java   |   21 +-
 .../gemfire/admin/StatisticResource.java        |   21 +-
 .../gemstone/gemfire/admin/SystemMember.java    |   21 +-
 .../gemfire/admin/SystemMemberBridgeServer.java |   26 +-
 .../gemfire/admin/SystemMemberCache.java        |   48 +-
 .../gemfire/admin/SystemMemberCacheEvent.java   |   16 +
 .../admin/SystemMemberCacheListener.java        |   21 +-
 .../gemfire/admin/SystemMemberCacheServer.java  |   21 +-
 .../gemfire/admin/SystemMemberRegion.java       |   21 +-
 .../gemfire/admin/SystemMemberRegionEvent.java  |   16 +
 .../gemfire/admin/SystemMemberType.java         |   21 +-
 .../gemfire/admin/SystemMembershipEvent.java    |   21 +-
 .../gemfire/admin/SystemMembershipListener.java |   21 +-
 .../UnmodifiableConfigurationException.java     |   21 +-
 .../admin/internal/AbstractHealthEvaluator.java |   21 +-
 .../internal/AdminDistributedSystemImpl.java    |   45 +-
 .../admin/internal/BackupStatusImpl.java        |   21 +-
 .../admin/internal/CacheHealthConfigImpl.java   |   21 +-
 .../admin/internal/CacheHealthEvaluator.java    |   21 +-
 .../admin/internal/CacheServerConfigImpl.java   |   21 +-
 .../gemfire/admin/internal/CacheServerImpl.java |   21 +-
 .../internal/ConfigurationParameterImpl.java    |   21 +-
 .../ConfigurationParameterListener.java         |   21 +-
 .../DisabledManagedEntityController.java        |   21 +-
 .../internal/DistributedSystemConfigImpl.java   |   22 +-
 .../DistributedSystemHealthConfigImpl.java      |   21 +-
 .../DistributedSystemHealthEvaluator.java       |   21 +-
 .../DistributedSystemHealthMonitor.java         |   23 +-
 .../internal/DistributionLocatorConfigImpl.java |   21 +-
 .../admin/internal/DistributionLocatorImpl.java |   21 +-
 .../EnabledManagedEntityController.java         |   21 +-
 .../admin/internal/FinishBackupRequest.java     |   21 +-
 .../admin/internal/FinishBackupResponse.java    |   21 +-
 .../admin/internal/FlushToDiskRequest.java      |   21 +-
 .../admin/internal/FlushToDiskResponse.java     |   21 +-
 .../admin/internal/GemFireHealthConfigImpl.java |   21 +-
 .../admin/internal/GemFireHealthEvaluator.java  |   21 +-
 .../admin/internal/GemFireHealthImpl.java       |   21 +-
 .../gemfire/admin/internal/InetAddressUtil.java |   23 +-
 .../admin/internal/InternalManagedEntity.java   |   21 +-
 .../gemfire/admin/internal/LogCollator.java     |   21 +-
 .../admin/internal/ManagedEntityConfigImpl.java |   21 +-
 .../admin/internal/ManagedEntityConfigXml.java  |   21 +-
 .../ManagedEntityConfigXmlGenerator.java        |   42 +-
 .../internal/ManagedEntityConfigXmlParser.java  |   21 +-
 .../admin/internal/ManagedEntityController.java |   21 +-
 .../ManagedEntityControllerFactory.java         |   21 +-
 .../admin/internal/ManagedSystemMemberImpl.java |   21 +-
 .../admin/internal/MemberHealthConfigImpl.java  |   21 +-
 .../admin/internal/MemberHealthEvaluator.java   |   21 +-
 .../admin/internal/PrepareBackupRequest.java    |   21 +-
 .../admin/internal/PrepareBackupResponse.java   |   21 +-
 .../gemfire/admin/internal/StatisticImpl.java   |   21 +-
 .../admin/internal/StatisticResourceImpl.java   |   20 +-
 .../internal/SystemMemberBridgeServerImpl.java  |   21 +-
 .../internal/SystemMemberCacheEventImpl.java    |   21 +-
 .../SystemMemberCacheEventProcessor.java        |   21 +-
 .../admin/internal/SystemMemberCacheImpl.java   |   38 +-
 .../admin/internal/SystemMemberImpl.java        |   21 +-
 .../internal/SystemMemberRegionEventImpl.java   |   21 +-
 .../admin/internal/SystemMemberRegionImpl.java  |   21 +-
 .../internal/SystemMembershipEventImpl.java     |   21 +-
 .../gemfire/admin/internal/package.html         |   16 +
 .../com/gemstone/gemfire/admin/jmx/Agent.java   |   21 +-
 .../gemstone/gemfire/admin/jmx/AgentConfig.java |   21 +-
 .../gemfire/admin/jmx/AgentFactory.java         |   21 +-
 .../internal/AdminDistributedSystemJmxImpl.java |   21 +-
 .../admin/jmx/internal/AgentConfigImpl.java     |   21 +-
 .../gemfire/admin/jmx/internal/AgentImpl.java   |   35 +-
 .../admin/jmx/internal/AgentLauncher.java       |   21 +-
 .../admin/jmx/internal/CacheServerJmxImpl.java  |   21 +-
 .../admin/jmx/internal/ConfigAttributeInfo.java |   20 +-
 .../internal/ConfigurationParameterJmxImpl.java |   21 +-
 .../DistributedSystemHealthConfigJmxImpl.java   |   21 +-
 .../internal/DistributionLocatorJmxImpl.java    |   22 +-
 .../admin/jmx/internal/DynamicManagedBean.java  |   20 +-
 .../internal/GemFireHealthConfigJmxImpl.java    |   35 +-
 .../jmx/internal/GemFireHealthJmxImpl.java      |   21 +-
 .../admin/jmx/internal/GenerateMBeanHTML.java   |   21 +-
 .../gemfire/admin/jmx/internal/MBeanUtil.java   |   20 +-
 .../admin/jmx/internal/MX4JModelMBean.java      |   21 +-
 .../jmx/internal/MX4JServerSocketFactory.java   |   22 +-
 .../gemfire/admin/jmx/internal/MailManager.java |   21 +-
 .../admin/jmx/internal/ManagedResource.java     |   21 +-
 .../admin/jmx/internal/ManagedResourceType.java |   21 +-
 .../jmx/internal/MemberInfoWithStatsMBean.java  |   21 +-
 .../admin/jmx/internal/RMIRegistryService.java  |   20 +-
 .../jmx/internal/RMIRegistryServiceMBean.java   |   20 +-
 .../jmx/internal/RefreshNotificationType.java   |   21 +-
 .../jmx/internal/StatAlertNotification.java     |   21 +-
 .../jmx/internal/StatAlertsAggregator.java      |   21 +-
 .../jmx/internal/StatisticAttributeInfo.java    |   20 +-
 .../jmx/internal/StatisticResourceJmxImpl.java  |   21 +-
 .../SystemMemberBridgeServerJmxImpl.java        |   21 +-
 .../jmx/internal/SystemMemberCacheJmxImpl.java  |   34 +-
 .../admin/jmx/internal/SystemMemberJmx.java     |   21 +-
 .../admin/jmx/internal/SystemMemberJmxImpl.java |   21 +-
 .../jmx/internal/SystemMemberRegionJmxImpl.java |   21 +-
 .../gemfire/admin/jmx/internal/package.html     |   16 +
 .../com/gemstone/gemfire/admin/jmx/package.html |   16 +
 .../com/gemstone/gemfire/admin/package.html     |   16 +
 .../gemfire/cache/AttributesFactory.java        |  101 +-
 .../gemfire/cache/AttributesMutator.java        |   21 +-
 .../java/com/gemstone/gemfire/cache/Cache.java  |   50 +-
 .../gemstone/gemfire/cache/CacheCallback.java   |   21 +-
 .../gemfire/cache/CacheClosedException.java     |   21 +-
 .../com/gemstone/gemfire/cache/CacheEvent.java  |   21 +-
 .../gemstone/gemfire/cache/CacheException.java  |   21 +-
 .../gemfire/cache/CacheExistsException.java     |   21 +-
 .../gemstone/gemfire/cache/CacheFactory.java    |   21 +-
 .../gemstone/gemfire/cache/CacheListener.java   |   21 +-
 .../com/gemstone/gemfire/cache/CacheLoader.java |   21 +-
 .../gemfire/cache/CacheLoaderException.java     |   21 +-
 .../gemfire/cache/CacheRuntimeException.java    |   21 +-
 .../gemstone/gemfire/cache/CacheStatistics.java |   21 +-
 .../gemfire/cache/CacheTransactionManager.java  |   21 +-
 .../com/gemstone/gemfire/cache/CacheWriter.java |   21 +-
 .../gemfire/cache/CacheWriterException.java     |   21 +-
 .../gemfire/cache/CacheXmlException.java        |   21 +-
 .../gemstone/gemfire/cache/ClientSession.java   |   21 +-
 .../gemfire/cache/CommitConflictException.java  |   21 +-
 .../cache/CommitDistributionException.java      |   21 +-
 .../cache/CommitIncompleteException.java        |   16 +
 .../gemfire/cache/CustomEvictionAttributes.java |   22 +-
 .../gemstone/gemfire/cache/CustomExpiry.java    |   21 +-
 .../com/gemstone/gemfire/cache/DataPolicy.java  |   40 +-
 .../com/gemstone/gemfire/cache/Declarable.java  |   21 +-
 .../gemfire/cache/DiskAccessException.java      |   21 +-
 .../com/gemstone/gemfire/cache/DiskStore.java   |   21 +-
 .../gemfire/cache/DiskStoreFactory.java         |   36 +-
 .../gemfire/cache/DiskWriteAttributes.java      |   21 +-
 .../cache/DiskWriteAttributesFactory.java       |   21 +-
 .../DuplicatePrimaryPartitionException.java     |   21 +-
 .../gemfire/cache/DynamicRegionFactory.java     |  142 +-
 .../gemfire/cache/DynamicRegionListener.java    |   21 +-
 .../gemfire/cache/EntryDestroyedException.java  |   21 +-
 .../com/gemstone/gemfire/cache/EntryEvent.java  |   21 +-
 .../gemfire/cache/EntryExistsException.java     |   21 +-
 .../gemfire/cache/EntryNotFoundException.java   |   21 +-
 .../gemfire/cache/EntryNotFoundInRegion.java    |   21 +-
 .../gemstone/gemfire/cache/EntryOperation.java  |   23 +-
 .../gemstone/gemfire/cache/EvictionAction.java  |   26 +-
 .../gemfire/cache/EvictionAlgorithm.java        |   26 +-
 .../gemfire/cache/EvictionAttributes.java       |   20 +-
 .../cache/EvictionAttributesMutator.java        |   23 +-
 .../gemfire/cache/EvictionCriteria.java         |   22 +-
 .../gemfire/cache/ExpirationAction.java         |   21 +-
 .../gemfire/cache/ExpirationAttributes.java     |   21 +-
 .../cache/FailedSynchronizationException.java   |   21 +-
 .../gemfire/cache/FixedPartitionAttributes.java |   21 +-
 .../gemfire/cache/FixedPartitionResolver.java   |   20 +-
 .../cache/GatewayConfigurationException.java    |   21 +-
 .../gemfire/cache/GatewayException.java         |   21 +-
 .../gemstone/gemfire/cache/GemFireCache.java    |   46 +-
 .../cache/IncompatibleVersionException.java     |   21 +-
 .../gemstone/gemfire/cache/InterestPolicy.java  |   21 +-
 .../cache/InterestRegistrationEvent.java        |   21 +-
 .../cache/InterestRegistrationListener.java     |   21 +-
 .../gemfire/cache/InterestResultPolicy.java     |   21 +-
 .../gemstone/gemfire/cache/LoaderHelper.java    |   21 +-
 .../com/gemstone/gemfire/cache/LossAction.java  |   21 +-
 .../gemfire/cache/LowMemoryException.java       |   21 +-
 .../gemfire/cache/MembershipAttributes.java     |   21 +-
 .../com/gemstone/gemfire/cache/MirrorType.java  |   21 +-
 .../cache/NoQueueServersAvailableException.java |   21 +-
 ...NoSubscriptionServersAvailableException.java |   21 +-
 .../com/gemstone/gemfire/cache/Operation.java   |   21 +-
 .../cache/OperationAbortedException.java        |   21 +-
 .../gemfire/cache/PartitionAttributes.java      |   21 +-
 .../cache/PartitionAttributesFactory.java       |   20 +-
 .../gemfire/cache/PartitionResolver.java        |   20 +-
 .../PartitionedRegionDistributionException.java |   21 +-
 .../PartitionedRegionStorageException.java      |   21 +-
 .../java/com/gemstone/gemfire/cache/Region.java |   21 +-
 .../gemfire/cache/RegionAccessException.java    |   21 +-
 .../gemfire/cache/RegionAttributes.java         |   21 +-
 .../gemfire/cache/RegionDestroyedException.java |   21 +-
 .../cache/RegionDistributionException.java      |   21 +-
 .../com/gemstone/gemfire/cache/RegionEvent.java |   21 +-
 .../gemfire/cache/RegionExistsException.java    |   21 +-
 .../gemstone/gemfire/cache/RegionFactory.java   |   46 +-
 .../gemfire/cache/RegionMembershipListener.java |   21 +-
 .../cache/RegionReinitializedException.java     |   21 +-
 .../gemfire/cache/RegionRoleException.java      |   21 +-
 .../gemfire/cache/RegionRoleListener.java       |   21 +-
 .../gemstone/gemfire/cache/RegionService.java   |   23 +-
 .../gemstone/gemfire/cache/RegionShortcut.java  |   71 +-
 .../cache/RemoteTransactionException.java       |   21 +-
 .../gemstone/gemfire/cache/RequiredRoles.java   |   21 +-
 .../gemfire/cache/ResourceException.java        |   21 +-
 .../gemfire/cache/ResumptionAction.java         |   21 +-
 .../com/gemstone/gemfire/cache/RoleEvent.java   |   21 +-
 .../gemstone/gemfire/cache/RoleException.java   |   21 +-
 .../java/com/gemstone/gemfire/cache/Scope.java  |   21 +-
 .../gemfire/cache/SerializedCacheValue.java     |   21 +-
 .../cache/StatisticsDisabledException.java      |   21 +-
 .../gemfire/cache/SubscriptionAttributes.java   |   21 +-
 .../SynchronizationCommitConflictException.java |   21 +-
 .../gemfire/cache/TimeoutException.java         |   21 +-
 ...TransactionDataNodeHasDepartedException.java |   21 +-
 .../TransactionDataNotColocatedException.java   |   21 +-
 .../TransactionDataRebalancedException.java     |   21 +-
 .../gemfire/cache/TransactionEvent.java         |   21 +-
 .../gemfire/cache/TransactionException.java     |   21 +-
 .../gemstone/gemfire/cache/TransactionId.java   |   21 +-
 .../cache/TransactionInDoubtException.java      |   21 +-
 .../gemfire/cache/TransactionListener.java      |   21 +-
 .../gemfire/cache/TransactionWriter.java        |   21 +-
 .../cache/TransactionWriterException.java       |   21 +-
 ...upportedOperationInTransactionException.java |   21 +-
 .../cache/UnsupportedVersionException.java      |   21 +-
 .../gemfire/cache/VersionException.java         |   21 +-
 .../gemfire/cache/asyncqueue/AsyncEvent.java    |   21 +-
 .../cache/asyncqueue/AsyncEventListener.java    |   21 +-
 .../cache/asyncqueue/AsyncEventQueue.java       |   21 +-
 .../asyncqueue/AsyncEventQueueFactory.java      |   23 +-
 .../internal/AsyncEventQueueFactoryImpl.java    |   30 +-
 .../internal/AsyncEventQueueImpl.java           |   24 +-
 .../internal/AsyncEventQueueStats.java          |   21 +-
 .../internal/ParallelAsyncEventQueueImpl.java   |   18 +-
 .../internal/SerialAsyncEventQueueImpl.java     |   18 +-
 .../client/AllConnectionsInUseException.java    |   21 +-
 .../gemfire/cache/client/ClientCache.java       |   21 +-
 .../cache/client/ClientCacheFactory.java        |   21 +-
 .../cache/client/ClientNotReadyException.java   |   21 +-
 .../cache/client/ClientRegionFactory.java       |   21 +-
 .../cache/client/ClientRegionShortcut.java      |   23 +-
 .../client/NoAvailableLocatorsException.java    |   21 +-
 .../client/NoAvailableServersException.java     |   21 +-
 .../com/gemstone/gemfire/cache/client/Pool.java |   21 +-
 .../gemfire/cache/client/PoolFactory.java       |   31 +-
 .../gemfire/cache/client/PoolManager.java       |   21 +-
 .../client/ServerConnectivityException.java     |   21 +-
 .../cache/client/ServerOperationException.java  |   21 +-
 .../ServerRefusedConnectionException.java       |   21 +-
 .../client/SubscriptionNotEnabledException.java |   21 +-
 .../cache/client/internal/AbstractOp.java       |   23 +-
 .../cache/client/internal/AddPDXEnumOp.java     |   21 +-
 .../cache/client/internal/AddPDXTypeOp.java     |   21 +-
 .../client/internal/AuthenticateUserOp.java     |   23 +-
 .../internal/AutoConnectionSourceImpl.java      |   21 +-
 .../cache/client/internal/BridgePoolImpl.java   |  479 --
 .../internal/BridgeServerLoadMessage.java       |   99 -
 .../client/internal/CacheServerLoadMessage.java |  108 +
 .../gemfire/cache/client/internal/ClearOp.java  |   21 +-
 .../client/internal/ClientMetadataService.java  |   20 +-
 .../client/internal/ClientPartitionAdvisor.java |   21 +-
 .../internal/ClientRegionFactoryImpl.java       |   21 +-
 .../cache/client/internal/ClientUpdater.java    |   21 +-
 .../client/internal/CloseConnectionOp.java      |   21 +-
 .../gemfire/cache/client/internal/CommitOp.java |   21 +-
 .../cache/client/internal/Connection.java       |   21 +-
 .../client/internal/ConnectionFactory.java      |   21 +-
 .../client/internal/ConnectionFactoryImpl.java  |   52 +-
 .../cache/client/internal/ConnectionImpl.java   |   53 +-
 .../cache/client/internal/ConnectionSource.java |   21 +-
 .../cache/client/internal/ConnectionStats.java  |   21 +-
 .../cache/client/internal/ContainsKeyOp.java    |   21 +-
 .../DataSerializerRecoveryListener.java         |   21 +-
 .../cache/client/internal/DestroyOp.java        |   26 +-
 .../cache/client/internal/DestroyRegionOp.java  |   21 +-
 .../gemfire/cache/client/internal/Endpoint.java |   21 +-
 .../cache/client/internal/EndpointManager.java  |   21 +-
 .../client/internal/EndpointManagerImpl.java    |   31 +-
 .../cache/client/internal/ExecutablePool.java   |   21 +-
 .../client/internal/ExecuteFunctionHelper.java  |   21 +-
 .../client/internal/ExecuteFunctionNoAckOp.java |   21 +-
 .../client/internal/ExecuteFunctionOp.java      |   21 +-
 .../internal/ExecuteRegionFunctionNoAckOp.java  |   21 +-
 .../internal/ExecuteRegionFunctionOp.java       |   21 +-
 .../ExecuteRegionFunctionSingleHopOp.java       |   21 +-
 .../internal/ExplicitConnectionSourceImpl.java  |  106 +-
 .../gemfire/cache/client/internal/GetAllOp.java |   21 +-
 .../client/internal/GetClientPRMetaDataOp.java  |   20 +-
 .../GetClientPartitionAttributesOp.java         |   20 +-
 .../cache/client/internal/GetEntryOp.java       |   21 +-
 .../cache/client/internal/GetEventValueOp.java  |   20 +-
 .../client/internal/GetFunctionAttributeOp.java |   21 +-
 .../gemfire/cache/client/internal/GetOp.java    |   26 +-
 .../cache/client/internal/GetPDXEnumByIdOp.java |   21 +-
 .../cache/client/internal/GetPDXEnumsOp.java    |   21 +-
 .../client/internal/GetPDXIdForEnumOp.java      |   21 +-
 .../client/internal/GetPDXIdForTypeOp.java      |   21 +-
 .../cache/client/internal/GetPDXTypeByIdOp.java |   21 +-
 .../cache/client/internal/GetPDXTypesOp.java    |   21 +-
 .../internal/InstantiatorRecoveryListener.java  |   21 +-
 .../cache/client/internal/InternalPool.java     |   21 +-
 .../cache/client/internal/InvalidateOp.java     |   21 +-
 .../gemfire/cache/client/internal/KeySetOp.java |   21 +-
 .../cache/client/internal/LiveServerPinger.java |   21 +-
 .../internal/LocatorDiscoveryCallback.java      |   21 +-
 .../LocatorDiscoveryCallbackAdapter.java        |   21 +-
 .../cache/client/internal/MakePrimaryOp.java    |   21 +-
 .../gemfire/cache/client/internal/Op.java       |   23 +-
 .../cache/client/internal/OpExecutorImpl.java   |   21 +-
 .../internal/PdxRegistryRecoveryListener.java   |   21 +-
 .../gemfire/cache/client/internal/PingOp.java   |   37 +-
 .../gemfire/cache/client/internal/PoolImpl.java |   35 +-
 .../cache/client/internal/PrimaryAckOp.java     |   21 +-
 .../cache/client/internal/ProxyCache.java       |   21 +-
 .../client/internal/ProxyCacheCloseOp.java      |   21 +-
 .../cache/client/internal/ProxyRegion.java      |   21 +-
 .../gemfire/cache/client/internal/PutAllOp.java |   21 +-
 .../gemfire/cache/client/internal/PutOp.java    |   26 +-
 .../gemfire/cache/client/internal/QueryOp.java  |   21 +-
 .../client/internal/QueueConnectionImpl.java    |   21 +-
 .../cache/client/internal/QueueManager.java     |   21 +-
 .../cache/client/internal/QueueManagerImpl.java |   37 +-
 .../cache/client/internal/QueueState.java       |   16 +
 .../cache/client/internal/QueueStateImpl.java   |   27 +-
 .../cache/client/internal/ReadyForEventsOp.java |   21 +-
 .../internal/RegisterDataSerializersOp.java     |   29 +-
 .../internal/RegisterInstantiatorsOp.java       |   29 +-
 .../client/internal/RegisterInterestListOp.java |   21 +-
 .../client/internal/RegisterInterestOp.java     |   21 +-
 .../internal/RegisterInterestTracker.java       |   21 +-
 .../cache/client/internal/RemoveAllOp.java      |   21 +-
 .../cache/client/internal/RollbackOp.java       |   21 +-
 .../cache/client/internal/ServerBlackList.java  |   21 +-
 .../cache/client/internal/ServerProxy.java      |   21 +-
 .../client/internal/ServerRegionDataAccess.java |   23 +-
 .../client/internal/ServerRegionProxy.java      |   65 +-
 .../internal/SingleHopClientExecutor.java       |   21 +-
 .../internal/SingleHopOperationCallable.java    |   21 +-
 .../gemfire/cache/client/internal/SizeOp.java   |   21 +-
 .../cache/client/internal/TXFailoverOp.java     |   21 +-
 .../client/internal/TXSynchronizationOp.java    |   21 +-
 .../internal/UnregisterInterestListOp.java      |   21 +-
 .../client/internal/UnregisterInterestOp.java   |   21 +-
 .../cache/client/internal/UserAttributes.java   |   21 +-
 .../locator/ClientConnectionRequest.java        |   21 +-
 .../locator/ClientConnectionResponse.java       |   21 +-
 .../locator/ClientReplacementRequest.java       |   21 +-
 .../internal/locator/GetAllServersRequest.java  |   20 +-
 .../internal/locator/GetAllServersResponse.java |   20 +-
 .../internal/locator/LocatorListRequest.java    |   21 +-
 .../internal/locator/LocatorListResponse.java   |   21 +-
 .../internal/locator/LocatorStatusRequest.java  |   20 +-
 .../internal/locator/LocatorStatusResponse.java |   20 +-
 .../locator/QueueConnectionRequest.java         |   21 +-
 .../locator/QueueConnectionResponse.java        |   21 +-
 .../internal/locator/SerializationHelper.java   |   21 +-
 .../internal/locator/ServerLocationRequest.java |   23 +-
 .../locator/ServerLocationResponse.java         |   23 +-
 .../locator/wan/LocatorMembershipListener.java  |   21 +-
 .../gemfire/cache/client/internal/package.html  |   16 +
 .../pooling/ConnectionDestroyedException.java   |   21 +-
 .../internal/pooling/ConnectionManager.java     |   21 +-
 .../internal/pooling/ConnectionManagerImpl.java |   30 +-
 .../internal/pooling/PooledConnection.java      |   21 +-
 .../gemstone/gemfire/cache/client/package.html  |   15 +
 .../gemfire/cache/control/RebalanceFactory.java |   21 +-
 .../cache/control/RebalanceOperation.java       |   21 +-
 .../gemfire/cache/control/RebalanceResults.java |   21 +-
 .../gemfire/cache/control/ResourceManager.java  |   21 +-
 .../gemstone/gemfire/cache/control/package.html |   16 +
 .../execute/EmtpyRegionFunctionException.java   |   21 +-
 .../gemfire/cache/execute/Execution.java        |   20 +-
 .../gemfire/cache/execute/Function.java         |   20 +-
 .../gemfire/cache/execute/FunctionAdapter.java  |   20 +-
 .../gemfire/cache/execute/FunctionContext.java  |   20 +-
 .../cache/execute/FunctionException.java        |   20 +-
 .../FunctionInvocationTargetException.java      |   20 +-
 .../gemfire/cache/execute/FunctionService.java  |   20 +-
 .../cache/execute/RegionFunctionContext.java    |   21 +-
 .../gemfire/cache/execute/ResultCollector.java  |   20 +-
 .../gemfire/cache/execute/ResultSender.java     |   20 +-
 .../internal/FunctionServiceManager.java        |   27 +-
 .../gemstone/gemfire/cache/execute/package.html |   15 +
 .../gemfire/cache/hdfs/HDFSIOException.java     |   21 +-
 .../gemstone/gemfire/cache/hdfs/HDFSStore.java  |   21 +-
 .../gemfire/cache/hdfs/HDFSStoreFactory.java    |   21 +-
 .../gemfire/cache/hdfs/HDFSStoreMutator.java    |   21 +-
 .../cache/hdfs/StoreExistsException.java        |   21 +-
 .../cache/hdfs/internal/FailureTracker.java     |   21 +-
 .../cache/hdfs/internal/FlushObserver.java      |   16 +
 .../hdfs/internal/HDFSBucketRegionQueue.java    |   21 +-
 .../cache/hdfs/internal/HDFSEntriesSet.java     |   21 +-
 .../cache/hdfs/internal/HDFSEventListener.java  |   25 +-
 .../hdfs/internal/HDFSEventQueueFilter.java     |   21 +-
 .../hdfs/internal/HDFSGatewayEventImpl.java     |   22 +-
 .../hdfs/internal/HDFSIntegrationUtil.java      |   21 +-
 .../HDFSParallelGatewaySenderQueue.java         |   21 +-
 .../hdfs/internal/HDFSStoreConfigHolder.java    |   23 +-
 .../cache/hdfs/internal/HDFSStoreCreation.java  |   23 +-
 .../hdfs/internal/HDFSStoreFactoryImpl.java     |   21 +-
 .../cache/hdfs/internal/HDFSStoreImpl.java      |   21 +-
 .../hdfs/internal/HDFSStoreMutatorImpl.java     |   23 +-
 .../HDFSWriteOnlyStoreEventListener.java        |   21 +-
 .../hdfs/internal/HoplogListenerForRegion.java  |   21 +-
 .../cache/hdfs/internal/PersistedEventImpl.java |   21 +-
 .../hdfs/internal/QueuedPersistentEvent.java    |   16 +
 .../hdfs/internal/SignalledFlushObserver.java   |   16 +
 .../internal/SortedHDFSQueuePersistedEvent.java |   21 +-
 .../internal/SortedHoplogPersistedEvent.java    |   21 +-
 .../UnsortedHDFSQueuePersistedEvent.java        |   21 +-
 .../internal/UnsortedHoplogPersistedEvent.java  |   21 +-
 .../cache/hdfs/internal/cardinality/Bits.java   |   21 +-
 .../cardinality/CardinalityMergeException.java  |   21 +-
 .../hdfs/internal/cardinality/HyperLogLog.java  |   21 +-
 .../hdfs/internal/cardinality/IBuilder.java     |   21 +-
 .../hdfs/internal/cardinality/ICardinality.java |   21 +-
 .../hdfs/internal/cardinality/MurmurHash.java   |   21 +-
 .../hdfs/internal/cardinality/RegisterSet.java  |   21 +-
 .../hdfs/internal/hoplog/AbstractHoplog.java    |   21 +-
 .../hoplog/AbstractHoplogOrganizer.java         |   21 +-
 .../cache/hdfs/internal/hoplog/BloomFilter.java |   21 +-
 .../hoplog/CloseTmpHoplogsTimerTask.java        |   21 +-
 .../hdfs/internal/hoplog/CompactionStatus.java  |   21 +-
 .../cache/hdfs/internal/hoplog/FlushStatus.java |   16 +
 .../internal/hoplog/HDFSCompactionManager.java  |   21 +-
 .../internal/hoplog/HDFSFlushQueueArgs.java     |   16 +
 .../internal/hoplog/HDFSFlushQueueFunction.java |   16 +
 .../hoplog/HDFSForceCompactionArgs.java         |   21 +-
 .../hoplog/HDFSForceCompactionFunction.java     |   21 +-
 .../HDFSForceCompactionResultCollector.java     |   21 +-
 .../hoplog/HDFSLastCompactionTimeFunction.java  |   21 +-
 .../internal/hoplog/HDFSRegionDirector.java     |   21 +-
 .../hdfs/internal/hoplog/HDFSStoreDirector.java |   21 +-
 .../hoplog/HDFSUnsortedHoplogOrganizer.java     |   21 +-
 .../hdfs/internal/hoplog/HFileSortedOplog.java  |   21 +-
 .../hoplog/HdfsSortedOplogOrganizer.java        |   21 +-
 .../cache/hdfs/internal/hoplog/Hoplog.java      |   21 +-
 .../hdfs/internal/hoplog/HoplogConfig.java      |   21 +-
 .../hdfs/internal/hoplog/HoplogListener.java    |   21 +-
 .../hdfs/internal/hoplog/HoplogOrganizer.java   |   21 +-
 .../hdfs/internal/hoplog/HoplogSetIterator.java |   21 +-
 .../hdfs/internal/hoplog/HoplogSetReader.java   |   21 +-
 .../internal/hoplog/SequenceFileHoplog.java     |   21 +-
 .../hoplog/mapred/AbstractGFRecordReader.java   |   21 +-
 .../internal/hoplog/mapred/GFInputFormat.java   |   21 +-
 .../internal/hoplog/mapred/GFOutputFormat.java  |   21 +-
 .../mapreduce/AbstractGFRecordReader.java       |   21 +-
 .../hoplog/mapreduce/GFInputFormat.java         |   21 +-
 .../hdfs/internal/hoplog/mapreduce/GFKey.java   |   21 +-
 .../hoplog/mapreduce/GFOutputFormat.java        |   21 +-
 .../hoplog/mapreduce/HDFSSplitIterator.java     |   21 +-
 .../internal/hoplog/mapreduce/HoplogUtil.java   |   21 +-
 .../hoplog/mapreduce/RWSplitIterator.java       |   21 +-
 .../hoplog/mapreduce/StreamSplitIterator.java   |   21 +-
 .../org/apache/hadoop/io/SequenceFile.java      |   21 +-
 .../gemfire/cache/lucene/LuceneIndex.java       |   50 -
 .../gemfire/cache/lucene/LuceneQuery.java       |   37 -
 .../cache/lucene/LuceneQueryFactory.java        |  121 -
 .../cache/lucene/LuceneQueryResults.java        |   29 -
 .../cache/lucene/LuceneResultStruct.java        |   59 -
 .../gemfire/cache/lucene/LuceneService.java     |  103 -
 .../cache/lucene/LuceneServiceFactory.java      |   14 -
 .../cache/lucene/LuceneServiceProvider.java     |   36 -
 .../cache/lucene/internal/LuceneIndexImpl.java  |   54 -
 .../lucene/internal/LuceneQueryFactoryImpl.java |   88 -
 .../cache/lucene/internal/LuceneQueryImpl.java  |   62 -
 .../lucene/internal/LuceneQueryResultsImpl.java |   66 -
 .../lucene/internal/LuceneResultStructImpl.java |   45 -
 .../internal/LuceneServiceFactoryImpl.java      |   16 -
 .../lucene/internal/LuceneServiceImpl.java      |   90 -
 .../operations/CloseCQOperationContext.java     |   21 +-
 .../operations/DestroyOperationContext.java     |   21 +-
 .../operations/ExecuteCQOperationContext.java   |   21 +-
 .../ExecuteFunctionOperationContext.java        |   21 +-
 .../GetDurableCQsOperationContext.java          |   21 +-
 .../cache/operations/GetOperationContext.java   |   21 +-
 .../operations/InterestOperationContext.java    |   21 +-
 .../gemfire/cache/operations/InterestType.java  |   21 +-
 .../operations/InvalidateOperationContext.java  |   21 +-
 .../cache/operations/KeyOperationContext.java   |   21 +-
 .../operations/KeySetOperationContext.java      |   21 +-
 .../operations/KeyValueOperationContext.java    |   21 +-
 .../cache/operations/OperationContext.java      |   21 +-
 .../operations/PutAllOperationContext.java      |   21 +-
 .../cache/operations/PutOperationContext.java   |   21 +-
 .../cache/operations/QueryOperationContext.java |   21 +-
 .../operations/RegionClearOperationContext.java |   21 +-
 .../RegionCreateOperationContext.java           |   21 +-
 .../RegionDestroyOperationContext.java          |   21 +-
 .../operations/RegionOperationContext.java      |   21 +-
 .../RegisterInterestOperationContext.java       |   21 +-
 .../operations/RemoveAllOperationContext.java   |   21 +-
 .../operations/StopCQOperationContext.java      |   21 +-
 .../UnregisterInterestOperationContext.java     |   21 +-
 .../internal/GetOperationContextImpl.java       |   18 +-
 .../gemfire/cache/operations/package.html       |   16 +
 .../com/gemstone/gemfire/cache/package.html     |   16 +
 .../cache/partition/PartitionListener.java      |   20 +-
 .../partition/PartitionListenerAdapter.java     |   23 +-
 .../cache/partition/PartitionManager.java       |   22 +-
 .../cache/partition/PartitionMemberInfo.java    |   21 +-
 .../PartitionNotAvailableException.java         |   21 +-
 .../cache/partition/PartitionRebalanceInfo.java |   21 +-
 .../cache/partition/PartitionRegionHelper.java  |   20 +-
 .../cache/partition/PartitionRegionInfo.java    |   21 +-
 .../gemfire/cache/partition/package.html        |   16 +
 .../ConflictingPersistentDataException.java     |   21 +-
 .../persistence/PartitionOfflineException.java  |   21 +-
 .../gemfire/cache/persistence/PersistentID.java |   23 +-
 .../PersistentReplicatesOfflineException.java   |   21 +-
 .../persistence/RevokeFailedException.java      |   21 +-
 .../RevokedPersistentDataException.java         |   21 +-
 .../gemfire/cache/query/Aggregator.java         |   16 +
 .../cache/query/AmbiguousNameException.java     |   21 +-
 .../gemfire/cache/query/CqAttributes.java       |   21 +-
 .../cache/query/CqAttributesFactory.java        |   21 +-
 .../cache/query/CqAttributesMutator.java        |   21 +-
 .../gemfire/cache/query/CqClosedException.java  |   21 +-
 .../gemstone/gemfire/cache/query/CqEvent.java   |   21 +-
 .../gemfire/cache/query/CqException.java        |   21 +-
 .../gemfire/cache/query/CqExistsException.java  |   21 +-
 .../gemfire/cache/query/CqListener.java         |   21 +-
 .../gemstone/gemfire/cache/query/CqQuery.java   |   21 +-
 .../gemstone/gemfire/cache/query/CqResults.java |   21 +-
 .../cache/query/CqServiceStatistics.java        |   21 +-
 .../gemstone/gemfire/cache/query/CqState.java   |   21 +-
 .../gemfire/cache/query/CqStatistics.java       |   21 +-
 .../gemfire/cache/query/CqStatusListener.java   |   21 +-
 .../cache/query/FunctionDomainException.java    |   21 +-
 .../com/gemstone/gemfire/cache/query/Index.java |   21 +-
 .../cache/query/IndexCreationException.java     |   21 +-
 .../cache/query/IndexExistsException.java       |   21 +-
 .../cache/query/IndexInvalidException.java      |   21 +-
 .../cache/query/IndexMaintenanceException.java  |   21 +-
 .../cache/query/IndexNameConflictException.java |   21 +-
 .../gemfire/cache/query/IndexStatistics.java    |   21 +-
 .../gemstone/gemfire/cache/query/IndexType.java |   21 +-
 .../query/MultiIndexCreationException.java      |   16 +
 .../cache/query/NameNotFoundException.java      |   21 +-
 .../cache/query/NameResolutionException.java    |   21 +-
 .../query/ParameterCountInvalidException.java   |   21 +-
 .../com/gemstone/gemfire/cache/query/Query.java |   21 +-
 .../gemfire/cache/query/QueryException.java     |   21 +-
 .../query/QueryExecutionLowMemoryException.java |   21 +-
 .../query/QueryExecutionTimeoutException.java   |   21 +-
 .../cache/query/QueryInvalidException.java      |   21 +-
 .../query/QueryInvocationTargetException.java   |   21 +-
 .../gemfire/cache/query/QueryService.java       |   21 +-
 .../gemfire/cache/query/QueryStatistics.java    |   21 +-
 .../cache/query/RegionNotFoundException.java    |   21 +-
 .../gemfire/cache/query/SelectResults.java      |   21 +-
 .../gemstone/gemfire/cache/query/Struct.java    |   21 +-
 .../cache/query/TypeMismatchException.java      |   21 +-
 .../query/internal/AbstractCompiledValue.java   |   22 +-
 .../internal/AbstractGroupOrRangeJunction.java  |   21 +-
 .../cache/query/internal/AllGroupJunction.java  |   21 +-
 .../query/internal/AttributeDescriptor.java     |   22 +-
 .../gemfire/cache/query/internal/Bag.java       |   21 +-
 .../internal/CompiledAggregateFunction.java     |   16 +
 .../query/internal/CompiledBindArgument.java    |   22 +-
 .../query/internal/CompiledComparison.java      |   22 +-
 .../query/internal/CompiledConstruction.java    |   22 +-
 .../cache/query/internal/CompiledFunction.java  |   22 +-
 .../query/internal/CompiledGroupBySelect.java   |   16 +
 .../cache/query/internal/CompiledID.java        |   22 +-
 .../cache/query/internal/CompiledIn.java        |   22 +-
 .../query/internal/CompiledIndexOperation.java  |   22 +-
 .../query/internal/CompiledIteratorDef.java     |   21 +-
 .../cache/query/internal/CompiledJunction.java  |   22 +-
 .../cache/query/internal/CompiledLike.java      |   21 +-
 .../cache/query/internal/CompiledLiteral.java   |   22 +-
 .../cache/query/internal/CompiledNegation.java  |   22 +-
 .../cache/query/internal/CompiledOperation.java |   26 +-
 .../cache/query/internal/CompiledPath.java      |   22 +-
 .../cache/query/internal/CompiledRegion.java    |   22 +-
 .../cache/query/internal/CompiledSelect.java    |   56 +-
 .../query/internal/CompiledSortCriterion.java   |   21 +-
 .../query/internal/CompiledUnaryMinus.java      |   21 +-
 .../cache/query/internal/CompiledUndefined.java |   22 +-
 .../cache/query/internal/CompiledValue.java     |   22 +-
 .../query/internal/CompositeGroupJunction.java  |   21 +-
 .../gemfire/cache/query/internal/CqEntry.java   |   21 +-
 .../cache/query/internal/CqQueryVsdStats.java   |   21 +-
 .../cache/query/internal/CqStateImpl.java       |   22 +-
 .../internal/CumulativeNonDistinctResults.java  |   16 +
 .../cache/query/internal/DefaultQuery.java      |   31 +-
 .../query/internal/DefaultQueryService.java     |   22 +-
 .../cache/query/internal/ExecutionContext.java  |   22 +-
 .../gemfire/cache/query/internal/Filter.java    |   22 +-
 .../gemfire/cache/query/internal/Functions.java |   22 +-
 .../cache/query/internal/GroupJunction.java     |   21 +-
 .../cache/query/internal/HashingStrategy.java   |   21 +-
 .../gemfire/cache/query/internal/IndexInfo.java |   21 +-
 .../internal/IndexTrackingQueryObserver.java    |   28 +-
 .../cache/query/internal/IndexUpdater.java      |   21 +-
 .../gemfire/cache/query/internal/Indexable.java |   21 +-
 .../cache/query/internal/LinkedResultSet.java   |   21 +-
 .../cache/query/internal/LinkedStructSet.java   |   21 +-
 .../cache/query/internal/MapIndexable.java      |   16 +
 .../cache/query/internal/MethodDispatch.java    |   22 +-
 .../cache/query/internal/NWayMergeResults.java  |   16 +
 .../gemfire/cache/query/internal/Negatable.java |   22 +-
 .../gemfire/cache/query/internal/NullToken.java |   23 +-
 .../cache/query/internal/ObjectIntHashMap.java  |   21 +-
 .../cache/query/internal/OrderByComparator.java |   18 +-
 .../internal/OrderByComparatorUnmapped.java     |   16 +
 .../gemfire/cache/query/internal/Ordered.java   |   16 +
 .../cache/query/internal/OrganizedOperands.java |   21 +-
 .../cache/query/internal/PRQueryTraceInfo.java  |   22 +-
 .../gemfire/cache/query/internal/PathUtils.java |   22 +-
 .../gemfire/cache/query/internal/PlanInfo.java  |   22 +-
 .../cache/query/internal/ProxyQueryService.java |   21 +-
 .../gemfire/cache/query/internal/QCompiler.java |   22 +-
 .../gemfire/cache/query/internal/QRegion.java   |   21 +-
 .../gemfire/cache/query/internal/QScope.java    |   22 +-
 .../QueryExecutionCanceledException.java        |   21 +-
 .../query/internal/QueryExecutionContext.java   |   24 +-
 .../cache/query/internal/QueryExecutor.java     |   21 +-
 .../cache/query/internal/QueryMonitor.java      |   21 +-
 .../cache/query/internal/QueryObserver.java     |   22 +-
 .../query/internal/QueryObserverAdapter.java    |   22 +-
 .../query/internal/QueryObserverHolder.java     |   22 +-
 .../cache/query/internal/QueryUtils.java        |   21 +-
 .../cache/query/internal/RangeJunction.java     |   21 +-
 .../cache/query/internal/ResultsBag.java        |   16 +
 .../ResultsCollectionCopyOnReadWrapper.java     |   21 +-
 ...ResultsCollectionPdxDeserializerWrapper.java |   21 +-
 .../internal/ResultsCollectionWrapper.java      |   21 +-
 .../cache/query/internal/ResultsSet.java        |   21 +-
 .../cache/query/internal/RuntimeIterator.java   |   22 +-
 .../query/internal/SelectResultsComparator.java |   21 +-
 .../cache/query/internal/SortedResultSet.java   |   21 +-
 .../cache/query/internal/SortedResultsBag.java  |   16 +
 .../cache/query/internal/SortedStructBag.java   |   16 +
 .../cache/query/internal/SortedStructSet.java   |   21 +-
 .../gemfire/cache/query/internal/StructBag.java |   21 +-
 .../cache/query/internal/StructFields.java      |   16 +
 .../cache/query/internal/StructImpl.java        |   21 +-
 .../gemfire/cache/query/internal/StructSet.java |   21 +-
 .../gemfire/cache/query/internal/Support.java   |   22 +-
 .../gemfire/cache/query/internal/Undefined.java |   22 +-
 .../internal/aggregate/AbstractAggregator.java  |   16 +
 .../cache/query/internal/aggregate/Avg.java     |   16 +
 .../query/internal/aggregate/AvgBucketNode.java |   16 +
 .../query/internal/aggregate/AvgDistinct.java   |   16 +
 .../aggregate/AvgDistinctPRQueryNode.java       |   16 +
 .../internal/aggregate/AvgPRQueryNode.java      |   16 +
 .../cache/query/internal/aggregate/Count.java   |   16 +
 .../query/internal/aggregate/CountDistinct.java |   16 +
 .../aggregate/CountDistinctPRQueryNode.java     |   16 +
 .../internal/aggregate/CountPRQueryNode.java    |   16 +
 .../internal/aggregate/DistinctAggregator.java  |   16 +
 .../cache/query/internal/aggregate/MaxMin.java  |   16 +
 .../cache/query/internal/aggregate/Sum.java     |   16 +
 .../query/internal/aggregate/SumDistinct.java   |   16 +
 .../aggregate/SumDistinctPRQueryNode.java       |   16 +
 .../cache/query/internal/cq/ClientCQ.java       |   16 +
 .../cache/query/internal/cq/CqService.java      |   16 +
 .../query/internal/cq/CqServiceProvider.java    |   16 +
 .../query/internal/cq/InternalCqQuery.java      |   16 +
 .../query/internal/cq/MissingCqService.java     |   16 +
 .../internal/cq/MissingCqServiceStatistics.java |   16 +
 .../cache/query/internal/cq/ServerCQ.java       |   16 +
 .../query/internal/cq/spi/CqServiceFactory.java |   16 +
 .../query/internal/index/AbstractIndex.java     |   46 +-
 .../query/internal/index/AbstractMapIndex.java  |   21 +-
 .../internal/index/CompactMapRangeIndex.java    |   21 +-
 .../query/internal/index/CompactRangeIndex.java |   21 +-
 .../query/internal/index/DummyQRegion.java      |   23 +-
 .../index/FunctionalIndexCreationHelper.java    |   22 +-
 .../cache/query/internal/index/HashIndex.java   |  199 +-
 .../query/internal/index/HashIndexSet.java      | 1107 ++---
 .../query/internal/index/HashIndexStrategy.java |   81 -
 .../query/internal/index/IMQException.java      |   21 +-
 .../internal/index/IndexConcurrentHashSet.java  |   21 +-
 .../query/internal/index/IndexCreationData.java |   21 +-
 .../internal/index/IndexCreationHelper.java     |   21 +-
 .../cache/query/internal/index/IndexData.java   |   21 +-
 .../query/internal/index/IndexElemArray.java    |   21 +-
 .../query/internal/index/IndexManager.java      |   24 +-
 .../query/internal/index/IndexProtocol.java     |   20 +-
 .../cache/query/internal/index/IndexStats.java  |   21 +-
 .../cache/query/internal/index/IndexStore.java  |   23 +-
 .../cache/query/internal/index/IndexUtils.java  |   21 +-
 .../index/IndexedExpressionEvaluator.java       |   21 +-
 .../query/internal/index/MapIndexStore.java     |   21 +-
 .../query/internal/index/MapRangeIndex.java     |   21 +-
 .../query/internal/index/MemoryIndexStore.java  |   21 +-
 .../query/internal/index/PartitionedIndex.java  |   37 +-
 .../query/internal/index/PrimaryKeyIndex.java   |   21 +-
 .../index/PrimaryKeyIndexCreationHelper.java    |   21 +-
 .../cache/query/internal/index/RangeIndex.java  |   21 +-
 .../cache/query/internal/index/package.html     |   16 +
 .../gemfire/cache/query/internal/package.html   |   16 +
 .../query/internal/parse/ASTAggregateFunc.java  |   16 +
 .../cache/query/internal/parse/ASTAnd.java      |   21 +-
 .../query/internal/parse/ASTCombination.java    |   21 +-
 .../query/internal/parse/ASTCompareOp.java      |   21 +-
 .../query/internal/parse/ASTConstruction.java   |   21 +-
 .../query/internal/parse/ASTConversionExpr.java |   21 +-
 .../cache/query/internal/parse/ASTDummy.java    |   16 +
 .../cache/query/internal/parse/ASTGroupBy.java  |   21 +-
 .../cache/query/internal/parse/ASTHint.java     |   21 +-
 .../query/internal/parse/ASTHintIdentifier.java |   21 +-
 .../query/internal/parse/ASTIdentifier.java     |   21 +-
 .../cache/query/internal/parse/ASTImport.java   |   21 +-
 .../cache/query/internal/parse/ASTIn.java       |   21 +-
 .../query/internal/parse/ASTIteratorDef.java    |   21 +-
 .../cache/query/internal/parse/ASTLike.java     |   21 +-
 .../cache/query/internal/parse/ASTLimit.java    |   23 +-
 .../cache/query/internal/parse/ASTLiteral.java  |   21 +-
 .../internal/parse/ASTMethodInvocation.java     |   21 +-
 .../cache/query/internal/parse/ASTOr.java       |   21 +-
 .../cache/query/internal/parse/ASTOrderBy.java  |   21 +-
 .../query/internal/parse/ASTParameter.java      |   21 +-
 .../cache/query/internal/parse/ASTPostfix.java  |   21 +-
 .../query/internal/parse/ASTProjection.java     |   21 +-
 .../query/internal/parse/ASTRegionPath.java     |   21 +-
 .../cache/query/internal/parse/ASTSelect.java   |   21 +-
 .../query/internal/parse/ASTSortCriterion.java  |   21 +-
 .../cache/query/internal/parse/ASTTrace.java    |   21 +-
 .../cache/query/internal/parse/ASTType.java     |   21 +-
 .../cache/query/internal/parse/ASTTypeCast.java |   21 +-
 .../cache/query/internal/parse/ASTUnary.java    |   21 +-
 .../query/internal/parse/ASTUndefinedExpr.java  |   21 +-
 .../query/internal/parse/ASTUnsupported.java    |   21 +-
 .../cache/query/internal/parse/GemFireAST.java  |   21 +-
 .../cache/query/internal/parse/UtilParser.java  |   21 +-
 .../internal/types/CollectionTypeImpl.java      |   21 +-
 .../types/ExtendedNumericComparator.java        |   22 +-
 .../cache/query/internal/types/MapTypeImpl.java |   21 +-
 .../query/internal/types/NumericComparator.java |   22 +-
 .../query/internal/types/ObjectTypeImpl.java    |   21 +-
 .../query/internal/types/StructTypeImpl.java    |   21 +-
 .../internal/types/TemporalComparator.java      |   22 +-
 .../cache/query/internal/types/TypeUtils.java   |   22 +-
 .../query/internal/utils/LimitIterator.java     |   16 +
 .../cache/query/internal/utils/PDXUtils.java    |   16 +
 .../gemstone/gemfire/cache/query/package.html   |   16 +
 .../cache/query/types/CollectionType.java       |   21 +-
 .../gemfire/cache/query/types/MapType.java      |   21 +-
 .../gemfire/cache/query/types/ObjectType.java   |   21 +-
 .../gemfire/cache/query/types/StructType.java   |   21 +-
 .../gemfire/cache/server/CacheServer.java       |   21 +-
 .../cache/server/ClientSubscriptionConfig.java  |   21 +-
 .../gemfire/cache/server/ServerLoad.java        |   21 +-
 .../gemfire/cache/server/ServerLoadProbe.java   |   31 +-
 .../cache/server/ServerLoadProbeAdapter.java    |   21 +-
 .../gemfire/cache/server/ServerMetrics.java     |   31 +-
 .../server/internal/ConnectionCountProbe.java   |   21 +-
 .../cache/server/internal/LoadMonitor.java      |   33 +-
 .../server/internal/ServerMetricsImpl.java      |   21 +-
 .../gemstone/gemfire/cache/server/package.html  |   15 +
 .../cache/snapshot/CacheSnapshotService.java    |   21 +-
 .../cache/snapshot/RegionSnapshotService.java   |   21 +-
 .../gemfire/cache/snapshot/SnapshotFilter.java  |   21 +-
 .../cache/snapshot/SnapshotIterator.java        |   21 +-
 .../gemfire/cache/snapshot/SnapshotOptions.java |   21 +-
 .../gemfire/cache/snapshot/SnapshotReader.java  |   21 +-
 .../gemfire/cache/snapshot/package.html         |   17 +-
 .../cache/util/BoundedLinkedHashMap.java        |   24 +-
 .../gemfire/cache/util/BridgeClient.java        |  156 -
 .../gemfire/cache/util/BridgeLoader.java        |  607 ---
 .../gemfire/cache/util/BridgeMembership.java    |   55 -
 .../cache/util/BridgeMembershipEvent.java       |   23 -
 .../cache/util/BridgeMembershipListener.java    |   41 -
 .../util/BridgeMembershipListenerAdapter.java   |   43 -
 .../gemfire/cache/util/BridgeServer.java        |  442 --
 .../gemfire/cache/util/BridgeWriter.java        |  795 ---
 .../cache/util/BridgeWriterException.java       |   48 -
 .../cache/util/CacheListenerAdapter.java        |   21 +-
 .../gemfire/cache/util/CacheWriterAdapter.java  |   21 +-
 .../gemfire/cache/util/CqListenerAdapter.java   |   21 +-
 .../util/EndpointDoesNotExistException.java     |   34 -
 .../gemfire/cache/util/EndpointException.java   |   61 -
 .../cache/util/EndpointExistsException.java     |   28 -
 .../cache/util/EndpointInUseException.java      |   27 -
 .../gemstone/gemfire/cache/util/Gateway.java    |   16 +
 .../cache/util/GatewayConflictHelper.java       |   21 +-
 .../cache/util/GatewayConflictResolver.java     |   21 +-
 .../gemfire/cache/util/GatewayEvent.java        |   21 +-
 .../util/IncompatibleVersionException.java      |   47 -
 .../gemfire/cache/util/ObjectSizer.java         |   21 +-
 .../gemfire/cache/util/ObjectSizerImpl.java     |   16 +
 .../util/RegionMembershipListenerAdapter.java   |   21 +-
 .../cache/util/RegionRoleListenerAdapter.java   |   21 +-
 .../util/ServerRefusedConnectionException.java  |   36 -
 .../cache/util/TimestampedEntryEvent.java       |   21 +-
 .../cache/util/TransactionListenerAdapter.java  |   21 +-
 .../UniversalMembershipListenerAdapter.java     |  352 --
 .../cache/util/UnknownVersionException.java     |   39 -
 .../gemfire/cache/util/VersionException.java    |   45 -
 .../gemstone/gemfire/cache/util/package.html    |   18 +-
 .../gemfire/cache/wan/EventSequenceID.java      |   21 +-
 .../gemfire/cache/wan/GatewayEventFilter.java   |   20 +-
 .../wan/GatewayEventSubstitutionFilter.java     |   21 +-
 .../gemfire/cache/wan/GatewayQueueEvent.java    |   21 +-
 .../gemfire/cache/wan/GatewayReceiver.java      |   20 +-
 .../cache/wan/GatewayReceiverFactory.java       |   20 +-
 .../gemfire/cache/wan/GatewaySender.java        |   20 +-
 .../gemfire/cache/wan/GatewaySenderFactory.java |   20 +-
 .../cache/wan/GatewayTransportFilter.java       |   20 +-
 .../compression/CompressionException.java       |   23 +-
 .../gemfire/compression/Compressor.java         |   23 +-
 .../gemfire/compression/SnappyCompressor.java   |   21 +-
 .../gemfire/distributed/AbstractLauncher.java   |   20 +-
 .../distributed/ClientSocketFactory.java        |   21 +-
 .../distributed/DistributedLockService.java     |   21 +-
 .../gemfire/distributed/DistributedMember.java  |   21 +-
 .../gemfire/distributed/DistributedSystem.java  |   98 +-
 .../DistributedSystemDisconnectedException.java |   23 +-
 .../distributed/DurableClientAttributes.java    |   21 +-
 .../distributed/FutureCancelledException.java   |   21 +-
 .../distributed/GatewayCancelledException.java  |   23 +-
 .../distributed/LeaseExpiredException.java      |   21 +-
 .../gemstone/gemfire/distributed/Locator.java   |   26 +-
 .../gemfire/distributed/LocatorLauncher.java    |   22 +-
 .../distributed/LockNotHeldException.java       |   21 +-
 .../LockServiceDestroyedException.java          |   21 +-
 .../distributed/OplogCancelledException.java    |   23 +-
 .../distributed/PoolCancelledException.java     |   23 +-
 .../com/gemstone/gemfire/distributed/Role.java  |   21 +-
 .../gemfire/distributed/ServerLauncher.java     |   26 +-
 .../TXManagerCancelledException.java            |   23 +-
 .../internal/AbstractDistributionConfig.java    |   21 +-
 .../distributed/internal/AdminMessageType.java  |   16 +
 .../internal/AtomicLongWithTerminalState.java   |   21 +-
 .../internal/CollectingReplyProcessor.java      |   21 +-
 .../distributed/internal/ConflationKey.java     |   21 +-
 .../gemfire/distributed/internal/DM.java        |   30 +-
 .../gemfire/distributed/internal/DMStats.java   |  144 +-
 .../gemfire/distributed/internal/DSClock.java   |   18 +-
 .../internal/DirectReplyProcessor.java          |   21 +-
 .../distributed/internal/DistributedState.java  |   21 +-
 .../internal/DistributionAdvisee.java           |   21 +-
 .../internal/DistributionAdvisor.java           |   21 +-
 .../internal/DistributionChannel.java           |   25 +-
 .../internal/DistributionConfig.java            |   29 +-
 .../internal/DistributionConfigImpl.java        |   37 +-
 .../internal/DistributionConfigSnapshot.java    |   21 +-
 .../internal/DistributionException.java         |   21 +-
 .../internal/DistributionManager.java           |  562 +--
 .../internal/DistributionManagerConfig.java     |   93 -
 .../internal/DistributionMessage.java           |   36 +-
 .../internal/DistributionMessageObserver.java   |   23 +-
 .../distributed/internal/DistributionStats.java |  354 +-
 .../distributed/internal/FlowControlParams.java |   21 +-
 .../internal/ForceDisconnectOperation.java      |   21 +-
 .../FunctionExecutionPooledExecutor.java        |   28 +-
 .../distributed/internal/HealthMonitor.java     |   21 +-
 .../distributed/internal/HealthMonitorImpl.java |   21 +-
 .../internal/HighPriorityAckedMessage.java      |   29 +-
 .../HighPriorityDistributionMessage.java        |   21 +-
 .../distributed/internal/IgnoredByManager.java  |   16 +
 .../internal/InternalDistributedSystem.java     |  179 +-
 .../distributed/internal/InternalLocator.java   |  172 +-
 .../internal/LocatorLoadSnapshot.java           |   20 +-
 .../distributed/internal/LocatorStats.java      |   21 +-
 .../internal/LonerDistributionManager.java      |  259 +-
 .../gemfire/distributed/internal/MQueue.java    |   16 +
 .../internal/MembershipListener.java            |   21 +-
 .../distributed/internal/MessageFactory.java    |   21 +-
 .../distributed/internal/MessageWithReply.java  |   21 +-
 .../internal/OverflowQueueWithDMStats.java      |   21 +-
 .../distributed/internal/PoolStatHelper.java    |   21 +-
 .../internal/PooledDistributionMessage.java     |   21 +-
 .../internal/PooledExecutorWithDMStats.java     |   38 +-
 .../distributed/internal/ProcessorKeeper21.java |   21 +-
 .../distributed/internal/ProductUseLog.java     |   21 +-
 .../distributed/internal/ProfileListener.java   |   21 +-
 .../distributed/internal/QueueStatHelper.java   |   21 +-
 .../internal/ReliableReplyException.java        |   21 +-
 .../internal/ReliableReplyProcessor21.java      |   21 +-
 .../distributed/internal/ReplyException.java    |   21 +-
 .../distributed/internal/ReplyMessage.java      |   23 +-
 .../distributed/internal/ReplyProcessor21.java  |   26 +-
 .../distributed/internal/ReplySender.java       |   21 +-
 .../distributed/internal/ResourceEvent.java     |   20 +-
 .../internal/ResourceEventsListener.java        |   20 +-
 .../internal/RuntimeDistributionConfigImpl.java |   21 +-
 .../internal/SerialAckedMessage.java            |   21 +-
 .../internal/SerialDistributionMessage.java     |   21 +-
 .../SerialQueuedExecutorWithDMStats.java        |   21 +-
 .../distributed/internal/ServerLocation.java    |   21 +-
 .../distributed/internal/ServerLocator.java     |   31 +-
 .../internal/SharedConfiguration.java           |   21 +-
 .../distributed/internal/ShutdownMessage.java   |   23 +-
 .../gemfire/distributed/internal/Sizeable.java  |   16 +
 .../distributed/internal/SizeableRunnable.java  |   21 +-
 .../distributed/internal/StartupMessage.java    |   55 +-
 .../internal/StartupMessageData.java            |   21 +-
 .../internal/StartupMessageReplyProcessor.java  |   21 +-
 .../distributed/internal/StartupOperation.java  |   27 +-
 .../internal/StartupResponseMessage.java        |   23 +-
 .../StartupResponseWithVersionMessage.java      |   23 +-
 .../internal/ThrottledMemQueueStatHelper.java   |   21 +-
 .../internal/ThrottledQueueStatHelper.java      |   21 +-
 .../ThrottlingMemLinkedQueueWithDMStats.java    |   21 +-
 .../internal/WaitForViewInstallation.java       |   25 +-
 .../internal/WanLocatorDiscoverer.java          |   16 +
 .../deadlock/DLockDependencyMonitor.java        |   21 +-
 .../internal/deadlock/DeadlockDetector.java     |  206 +-
 .../internal/deadlock/Dependency.java           |   21 +-
 .../internal/deadlock/DependencyGraph.java      |  193 +-
 .../internal/deadlock/DependencyMonitor.java    |   21 +-
 .../deadlock/DependencyMonitorManager.java      |   21 +-
 .../deadlock/GemFireDeadlockDetector.java       |   21 +-
 .../internal/deadlock/LocalLockInfo.java        |   23 +-
 .../internal/deadlock/LocalThread.java          |   25 +-
 .../deadlock/MessageDependencyMonitor.java      |   25 +-
 .../internal/deadlock/ThreadReference.java      |   21 +-
 .../internal/deadlock/UnsafeThreadLocal.java    |   21 +-
 .../internal/direct/DirectChannel.java          |   67 +-
 .../internal/direct/DirectChannelListener.java  |   22 +
 .../internal/direct/MissingStubException.java   |   21 +-
 .../internal/distribution-overview.html         |   16 +
 .../internal/locks/Collaboration.java           |   21 +-
 .../distributed/internal/locks/DLockBatch.java  |   21 +-
 .../internal/locks/DLockBatchId.java            |   21 +-
 .../internal/locks/DLockGrantor.java            |   21 +-
 .../locks/DLockLessorDepartureHandler.java      |   21 +-
 .../internal/locks/DLockQueryProcessor.java     |   21 +-
 .../locks/DLockRecoverGrantorProcessor.java     |   21 +-
 .../internal/locks/DLockReleaseProcessor.java   |   21 +-
 .../internal/locks/DLockRemoteToken.java        |   21 +-
 .../internal/locks/DLockRequestProcessor.java   |   21 +-
 .../internal/locks/DLockService.java            |   21 +-
 .../distributed/internal/locks/DLockStats.java  |   21 +-
 .../distributed/internal/locks/DLockToken.java  |   21 +-
 .../internal/locks/DeposeGrantorProcessor.java  |   21 +-
 .../internal/locks/DistributedLockStats.java    |   21 +-
 .../internal/locks/DistributedMemberLock.java   |   21 +-
 .../internal/locks/DummyDLockStats.java         |   21 +-
 .../internal/locks/ElderInitProcessor.java      |   21 +-
 .../distributed/internal/locks/ElderState.java  |   21 +-
 .../distributed/internal/locks/GrantorInfo.java |   21 +-
 .../internal/locks/GrantorRequestProcessor.java |   21 +-
 .../locks/LockGrantorDestroyedException.java    |   21 +-
 .../internal/locks/LockGrantorId.java           |   21 +-
 .../locks/NonGrantorDestroyedProcessor.java     |   21 +-
 .../internal/locks/RemoteThread.java            |   21 +-
 .../distributed/internal/locks/package.html     |   16 +
 .../DistributedMembershipListener.java          |   34 +-
 .../membership/InternalDistributedMember.java   |  351 +-
 .../internal/membership/InternalRole.java       |   21 +-
 .../internal/membership/MemberAttributes.java   |   41 +-
 .../internal/membership/MemberFactory.java      |   54 +-
 .../internal/membership/MemberServices.java     |   49 +-
 .../internal/membership/MembershipManager.java  |   51 +-
 .../internal/membership/MembershipTestHook.java |   21 +-
 .../internal/membership/NetMember.java          |   80 +-
 .../internal/membership/NetView.java            |  596 ++-
 .../internal/membership/QuorumChecker.java      |   33 +-
 .../internal/membership/gms/GMSMember.java      |  449 ++
 .../membership/gms/GMSMemberFactory.java        |  133 +
 .../internal/membership/gms/GMSUtil.java        |  159 +
 .../internal/membership/gms/NetLocator.java     |   32 +
 .../internal/membership/gms/ServiceConfig.java  |  174 +
 .../internal/membership/gms/Services.java       |  387 ++
 .../internal/membership/gms/SuspectMember.java  |   53 +
 .../membership/gms/auth/GMSAuthenticator.java   |  235 +
 .../membership/gms/fd/GMSHealthMonitor.java     | 1396 ++++++
 .../gms/interfaces/Authenticator.java           |   27 +
 .../gms/interfaces/HealthMonitor.java           |   65 +
 .../membership/gms/interfaces/JoinLeave.java    |   67 +
 .../membership/gms/interfaces/Locator.java      |   32 +
 .../membership/gms/interfaces/Manager.java      |  121 +
 .../gms/interfaces/MessageHandler.java          |   30 +
 .../membership/gms/interfaces/Messenger.java    |   59 +
 .../membership/gms/interfaces/Service.java      |   82 +
 .../gms/locator/FindCoordinatorRequest.java     |  115 +
 .../gms/locator/FindCoordinatorResponse.java    |  162 +
 .../membership/gms/locator/GMSLocator.java      |  373 ++
 .../membership/gms/locator/GetViewRequest.java  |   49 +
 .../membership/gms/locator/GetViewResponse.java |   64 +
 .../gms/locator/PeerLocatorRequest.java         |   25 +
 .../membership/gms/membership/GMSJoinLeave.java | 2167 +++++++++
 .../membership/gms/messages/HasMemberID.java    |   25 +
 .../gms/messages/HeartbeatMessage.java          |   74 +
 .../gms/messages/HeartbeatRequestMessage.java   |   80 +
 .../gms/messages/InstallViewMessage.java        |  106 +
 .../gms/messages/JoinRequestMessage.java        |   97 +
 .../gms/messages/JoinResponseMessage.java       |  129 +
 .../gms/messages/LeaveRequestMessage.java       |   94 +
 .../gms/messages/NetworkPartitionMessage.java   |   44 +
 .../gms/messages/RemoveMemberMessage.java       |   96 +
 .../gms/messages/SuspectMembersMessage.java     |   91 +
 .../membership/gms/messages/SuspectRequest.java |   72 +
 .../membership/gms/messages/ViewAckMessage.java |  103 +
 .../gms/messenger/AddressManager.java           |  130 +
 .../membership/gms/messenger/GMSPingPonger.java |   57 +
 .../gms/messenger/GMSQuorumChecker.java         |  274 ++
 .../membership/gms/messenger/JGAddress.java     |  230 +
 .../gms/messenger/JGroupsMessenger.java         | 1089 +++++
 .../membership/gms/messenger/StatRecorder.java  |  162 +
 .../membership/gms/messenger/Transport.java     |  142 +
 .../gms/mgr/GMSMembershipManager.java           | 2779 +++++++++++
 .../membership/gms/mgr/LocalViewMessage.java    |   85 +
 .../internal/membership/gms/package.html        |   57 +
 .../membership/jgroup/GFJGBasicAdapter.java     |  602 ---
 .../membership/jgroup/GFJGPeerAdapter.java      |  484 --
 .../membership/jgroup/JGroupMember.java         |  251 -
 .../membership/jgroup/JGroupMemberFactory.java  |  119 -
 .../jgroup/JGroupMembershipManager.java         | 4232 ----------------
 .../internal/membership/jgroup/LocatorImpl.java |  474 --
 .../membership/jgroup/QuorumCheckerImpl.java    |  291 --
 .../internal/membership/jgroup/ViewMessage.java |   73 -
 .../internal/membership/jgroup/package.html     |   43 -
 .../gemfire/distributed/internal/package.html   |   16 +
 .../internal/streaming/StreamingOperation.java  |   21 +-
 .../internal/tcpserver/InfoRequest.java         |   20 +-
 .../internal/tcpserver/InfoResponse.java        |   20 +-
 .../internal/tcpserver/ShutdownRequest.java     |   20 +-
 .../internal/tcpserver/ShutdownResponse.java    |   20 +-
 .../internal/tcpserver/TcpClient.java           |   76 +-
 .../internal/tcpserver/TcpHandler.java          |   16 +
 .../internal/tcpserver/TcpServer.java           |  121 +-
 .../internal/tcpserver/VersionRequest.java      |   16 +
 .../internal/tcpserver/VersionResponse.java     |   16 +
 .../unsafe/RegisterSignalHandlerSupport.java    |   20 +-
 .../gemstone/gemfire/distributed/package.html   |   16 +
 .../gemstone/gemfire/i18n/LogWriterI18n.java    |   29 +-
 .../com/gemstone/gemfire/i18n/StringId.java     |  161 +
 .../com/gemstone/gemfire/i18n/StringIdImpl.java |  153 -
 .../gemfire/internal/AbstractConfig.java        |   27 +-
 .../internal/AbstractStatisticsFactory.java     |   21 +-
 .../gemfire/internal/ArchiveSplitter.java       |   21 +-
 .../com/gemstone/gemfire/internal/Assert.java   |   21 +-
 .../gemfire/internal/AvailablePort.java         |   94 +-
 .../com/gemstone/gemfire/internal/Banner.java   |   49 +-
 .../gemfire/internal/ByteArrayDataInput.java    |   21 +-
 .../internal/ByteBufferOutputStream.java        |   21 +-
 .../gemfire/internal/ByteBufferWriter.java      |   16 +
 .../gemfire/internal/ClassLoadUtil.java         |   21 +-
 .../gemfire/internal/ClassPathLoader.java       |   21 +-
 .../com/gemstone/gemfire/internal/Config.java   |   21 +-
 .../gemstone/gemfire/internal/ConfigSource.java |   21 +-
 .../gemfire/internal/ConnectionWatcher.java     |   41 +
 .../gemfire/internal/CopyOnWriteHashSet.java    |   21 +-
 .../com/gemstone/gemfire/internal/DSCODE.java   |   21 +-
 .../gemstone/gemfire/internal/DSFIDFactory.java |   73 +-
 .../internal/DSFIDNotFoundException.java        |   21 +-
 .../internal/DataSerializableFixedID.java       |   73 +-
 .../gemfire/internal/DistributionLocator.java   |   21 +-
 .../internal/DummyStatisticsFactory.java        |   21 +-
 .../gemfire/internal/DummyStatisticsImpl.java   |   21 +-
 .../gemfire/internal/ExternalizableDSFID.java   |   21 +-
 .../com/gemstone/gemfire/internal/FileUtil.java |   21 +-
 .../gemfire/internal/GemFireStatSampler.java    |   21 +-
 .../gemfire/internal/GemFireUtilLauncher.java   |   21 +-
 .../gemfire/internal/GemFireVersion.java        |   21 +-
 .../internal/GfeConsoleReaderFactory.java       |   20 +-
 .../gemfire/internal/HeapDataOutputStream.java  |   21 +-
 .../gemfire/internal/HistogramStats.java        |   21 +-
 .../gemfire/internal/HostStatHelper.java        |   21 +-
 .../gemfire/internal/HostStatSampler.java       |   25 +-
 .../InsufficientDiskSpaceException.java         |   21 +-
 .../internal/InternalDataSerializer.java        |  203 +-
 .../gemfire/internal/InternalEntity.java        |   16 +
 .../gemfire/internal/InternalInstantiator.java  |   23 +-
 .../InternalStatisticsDisabledException.java    |   21 +-
 .../gemfire/internal/JarClassLoader.java        |   20 +-
 .../gemstone/gemfire/internal/JarDeployer.java  |   20 +-
 .../gemfire/internal/LinuxProcFsStatistics.java |   21 +-
 .../gemfire/internal/LinuxProcessStats.java     |   21 +-
 .../gemfire/internal/LinuxSystemStats.java      |   21 +-
 .../gemfire/internal/LocalStatListener.java     |   21 +-
 .../internal/LocalStatisticsFactory.java        |   21 +-
 .../gemfire/internal/LocalStatisticsImpl.java   |   21 +-
 .../gemstone/gemfire/internal/ManagerInfo.java  |   20 +-
 .../gemfire/internal/MigrationClient.java       |   21 +-
 .../gemfire/internal/MigrationServer.java       |   21 +-
 .../gemstone/gemfire/internal/NanoTimer.java    |   58 +-
 .../gemfire/internal/NullDataOutputStream.java  |   21 +-
 .../gemstone/gemfire/internal/OSProcess.java    |   19 +-
 .../gemfire/internal/OSXProcessStats.java       |   21 +-
 .../gemfire/internal/OSXSystemStats.java        |   21 +-
 .../gemfire/internal/ObjIdConcurrentMap.java    |   21 +-
 .../com/gemstone/gemfire/internal/ObjIdMap.java |   21 +-
 .../internal/ObjToByteArraySerializer.java      |   21 +-
 .../gemfire/internal/OneTaskOnlyExecutor.java   |   21 +-
 .../gemfire/internal/OsStatisticsFactory.java   |   21 +-
 .../gemfire/internal/PdxSerializerObject.java   |   21 +-
 .../gemfire/internal/ProcessOutputReader.java   |   21 +-
 .../gemstone/gemfire/internal/ProcessStats.java |   21 +-
 .../gemstone/gemfire/internal/PureJavaMode.java |   21 +-
 ...cheduledThreadPoolExecutorWithKeepAlive.java |   21 +-
 .../com/gemstone/gemfire/internal/Sendable.java |   18 +-
 .../gemfire/internal/SerializationVersions.java |   21 +-
 .../com/gemstone/gemfire/internal/SetUtils.java |   20 +-
 .../gemfire/internal/SharedLibrary.java         |   41 +-
 .../gemfire/internal/SimpleStatSampler.java     |   21 +-
 .../com/gemstone/gemfire/internal/SmHelper.java |   21 +-
 .../gemstone/gemfire/internal/SocketCloser.java |  257 +
 .../gemfire/internal/SocketCreator.java         |  162 +-
 .../gemfire/internal/SolarisProcessStats.java   |   21 +-
 .../gemfire/internal/SolarisSystemStats.java    |   21 +-
 .../gemfire/internal/StatArchiveFormat.java     |   21 +-
 .../gemfire/internal/StatArchiveReader.java     |   21 +-
 .../gemfire/internal/StatArchiveWriter.java     |   49 +-
 .../gemfire/internal/StatSamplerStats.java      |   21 +-
 .../internal/StatisticDescriptorImpl.java       |   21 +-
 .../gemfire/internal/StatisticsImpl.java        |   21 +-
 .../gemfire/internal/StatisticsManager.java     |   21 +-
 .../internal/StatisticsTypeFactoryImpl.java     |   21 +-
 .../gemfire/internal/StatisticsTypeImpl.java    |   21 +-
 .../gemfire/internal/StatisticsTypeXml.java     |   21 +-
 .../gemstone/gemfire/internal/SystemAdmin.java  |   36 +-
 .../gemfire/internal/SystemFailureTestHook.java |   21 +-
 .../gemstone/gemfire/internal/SystemTimer.java  |   21 +-
 .../gemfire/internal/UniqueIdGenerator.java     |   21 +-
 .../com/gemstone/gemfire/internal/VMStats.java  |   21 +-
 .../gemfire/internal/VMStatsContract.java       |   21 +-
 .../internal/VMStatsContractFactory.java        |   21 +-
 .../com/gemstone/gemfire/internal/Version.java  |   45 +-
 .../internal/VersionedDataInputStream.java      |   21 +-
 .../internal/VersionedDataOutputStream.java     |   21 +-
 .../internal/VersionedDataSerializable.java     |   16 +
 .../gemfire/internal/VersionedDataStream.java   |   21 +-
 .../gemfire/internal/VersionedObjectInput.java  |   21 +-
 .../gemfire/internal/VersionedObjectOutput.java |   21 +-
 .../gemfire/internal/WindowsProcessStats.java   |   21 +-
 .../gemfire/internal/WindowsSystemStats.java    |   21 +-
 .../internal/admin/AdminBridgeServer.java       |   16 +
 .../gemstone/gemfire/internal/admin/Alert.java  |   21 +-
 .../gemfire/internal/admin/AlertListener.java   |   21 +-
 .../gemfire/internal/admin/ApplicationVM.java   |   21 +-
 .../gemfire/internal/admin/CacheCollector.java  |   21 +-
 .../gemfire/internal/admin/CacheInfo.java       |   21 +-
 .../gemfire/internal/admin/CacheSnapshot.java   |   21 +-
 .../admin/ClientHealthMonitoringRegion.java     |   21 +-
 .../internal/admin/ClientMembershipMessage.java |   21 +-
 .../internal/admin/ClientStatsManager.java      |   28 +-
 .../internal/admin/CompoundEntrySnapshot.java   |   21 +-
 .../internal/admin/CompoundRegionSnapshot.java  |   21 +-
 .../gemfire/internal/admin/DLockInfo.java       |   21 +-
 .../gemfire/internal/admin/EntrySnapshot.java   |   21 +-
 .../gemfire/internal/admin/EntryValueNode.java  |   23 +-
 .../gemfire/internal/admin/GemFireVM.java       |   23 +-
 .../gemfire/internal/admin/GfManagerAgent.java  |   21 +-
 .../internal/admin/GfManagerAgentConfig.java    |   21 +-
 .../internal/admin/GfManagerAgentFactory.java   |   21 +-
 .../gemfire/internal/admin/GfObject.java        |   21 +-
 .../gemfire/internal/admin/HealthListener.java  |   21 +-
 .../internal/admin/JoinLeaveListener.java       |   21 +-
 .../gemfire/internal/admin/ListenerIdMap.java   |   21 +-
 .../gemfire/internal/admin/RegionSnapshot.java  |   21 +-
 .../gemfire/internal/admin/SSLConfig.java       |   21 +-
 .../gemfire/internal/admin/SnapshotClient.java  |   21 +-
 .../gemstone/gemfire/internal/admin/Stat.java   |   21 +-
 .../gemfire/internal/admin/StatAlert.java       |   21 +-
 .../internal/admin/StatAlertDefinition.java     |   21 +-
 .../internal/admin/StatAlertsManager.java       |   21 +-
 .../gemfire/internal/admin/StatListener.java    |   21 +-
 .../gemfire/internal/admin/StatResource.java    |   21 +-
 .../gemfire/internal/admin/TransportConfig.java |   21 +-
 .../gemfire/internal/admin/package.html         |   16 +
 .../admin/remote/AddHealthListenerRequest.java  |   21 +-
 .../admin/remote/AddHealthListenerResponse.java |   21 +-
 .../admin/remote/AddStatListenerRequest.java    |   21 +-
 .../admin/remote/AddStatListenerResponse.java   |   21 +-
 .../remote/AdminConsoleDisconnectMessage.java   |   21 +-
 .../admin/remote/AdminConsoleMessage.java       |   21 +-
 .../admin/remote/AdminFailureResponse.java      |   21 +-
 .../remote/AdminMultipleReplyProcessor.java     |   21 +-
 .../internal/admin/remote/AdminRegion.java      |   21 +-
 .../admin/remote/AdminReplyProcessor.java       |   21 +-
 .../internal/admin/remote/AdminRequest.java     |   21 +-
 .../internal/admin/remote/AdminResponse.java    |   23 +-
 .../internal/admin/remote/AdminWaiters.java     |   21 +-
 .../admin/remote/AlertLevelChangeMessage.java   |   26 +-
 .../admin/remote/AlertListenerMessage.java      |   23 +-
 .../admin/remote/AlertsNotificationMessage.java |   21 +-
 .../admin/remote/AppCacheSnapshotMessage.java   |   21 +-
 .../admin/remote/BridgeServerRequest.java       |   21 +-
 .../admin/remote/BridgeServerResponse.java      |   39 +-
 .../admin/remote/CacheConfigRequest.java        |   21 +-
 .../admin/remote/CacheConfigResponse.java       |   21 +-
 .../internal/admin/remote/CacheDisplay.java     |   21 +-
 .../internal/admin/remote/CacheInfoRequest.java |   21 +-
 .../admin/remote/CacheInfoResponse.java         |   21 +-
 .../admin/remote/CancelStatListenerRequest.java |   21 +-
 .../remote/CancelStatListenerResponse.java      |   21 +-
 .../internal/admin/remote/Cancellable.java      |   21 +-
 .../admin/remote/CancellationMessage.java       |   23 +-
 .../admin/remote/CancellationRegistry.java      |   23 +-
 .../remote/ChangeRefreshIntervalMessage.java    |   21 +-
 .../internal/admin/remote/CliLegacyMessage.java |   16 +
 .../admin/remote/ClientHealthStats.java         |   21 +-
 .../internal/admin/remote/CompactRequest.java   |   21 +-
 .../internal/admin/remote/CompactResponse.java  |   21 +-
 .../admin/remote/DestroyEntryMessage.java       |   23 +-
 .../admin/remote/DestroyRegionMessage.java      |   23 +-
 .../admin/remote/DistributionLocatorId.java     |   21 +-
 .../internal/admin/remote/DummyEntry.java       |   21 +-
 .../admin/remote/DurableClientInfoRequest.java  |   20 +-
 .../admin/remote/DurableClientInfoResponse.java |   24 +-
 .../admin/remote/EntryValueNodeImpl.java        |   23 +-
 .../admin/remote/FetchDistLockInfoRequest.java  |   21 +-
 .../admin/remote/FetchDistLockInfoResponse.java |   21 +-
 .../remote/FetchHealthDiagnosisRequest.java     |   21 +-
 .../remote/FetchHealthDiagnosisResponse.java    |   21 +-
 .../internal/admin/remote/FetchHostRequest.java |   21 +-
 .../admin/remote/FetchHostResponse.java         |   21 +-
 .../remote/FetchResourceAttributesRequest.java  |   21 +-
 .../remote/FetchResourceAttributesResponse.java |   21 +-
 .../admin/remote/FetchStatsRequest.java         |   21 +-
 .../admin/remote/FetchStatsResponse.java        |   22 +-
 .../admin/remote/FetchSysCfgRequest.java        |   23 +-
 .../admin/remote/FetchSysCfgResponse.java       |   21 +-
 .../remote/FlushAppCacheSnapshotMessage.java    |   21 +-
 .../admin/remote/HealthListenerMessage.java     |   23 +-
 .../remote/InspectionClasspathManager.java      |   21 +-
 .../admin/remote/LicenseInfoRequest.java        |   21 +-
 .../admin/remote/LicenseInfoResponse.java       |   21 +-
 .../remote/MissingPersistentIDsRequest.java     |   21 +-
 .../remote/MissingPersistentIDsResponse.java    |   21 +-
 .../admin/remote/ObjectDetailsRequest.java      |   21 +-
 .../admin/remote/ObjectDetailsResponse.java     |   21 +-
 .../admin/remote/ObjectNamesRequest.java        |   21 +-
 .../admin/remote/ObjectNamesResponse.java       |   21 +-
 .../PrepareRevokePersistentIDRequest.java       |   21 +-
 .../remote/RefreshMemberSnapshotRequest.java    |   21 +-
 .../remote/RefreshMemberSnapshotResponse.java   |   21 +-
 .../admin/remote/RegionAdminMessage.java        |   21 +-
 .../admin/remote/RegionAdminRequest.java        |   21 +-
 .../admin/remote/RegionAttributesRequest.java   |   21 +-
 .../admin/remote/RegionAttributesResponse.java  |   21 +-
 .../internal/admin/remote/RegionRequest.java    |   21 +-
 .../internal/admin/remote/RegionResponse.java   |   21 +-
 .../admin/remote/RegionSizeRequest.java         |   21 +-
 .../admin/remote/RegionSizeResponse.java        |   21 +-
 .../admin/remote/RegionStatisticsRequest.java   |   21 +-
 .../admin/remote/RegionStatisticsResponse.java  |   21 +-
 .../remote/RegionSubRegionSizeRequest.java      |   20 +-
 .../remote/RegionSubRegionsSizeResponse.java    |   20 +-
 .../internal/admin/remote/RemoteAlert.java      |   21 +-
 .../admin/remote/RemoteApplicationVM.java       |   21 +-
 .../admin/remote/RemoteBridgeServer.java        |   31 +-
 .../internal/admin/remote/RemoteCacheInfo.java  |   27 +-
 .../admin/remote/RemoteCacheStatistics.java     |   21 +-
 .../internal/admin/remote/RemoteDLockInfo.java  |   21 +-
 .../admin/remote/RemoteEntrySnapshot.java       |   21 +-
 .../internal/admin/remote/RemoteGemFireVM.java  |   23 +-
 .../admin/remote/RemoteGfManagerAgent.java      |   58 +-
 .../internal/admin/remote/RemoteObjectName.java |   21 +-
 .../admin/remote/RemoteRegionAttributes.java    |   23 +-
 .../admin/remote/RemoteRegionSnapshot.java      |   21 +-
 .../internal/admin/remote/RemoteStat.java       |   21 +-
 .../admin/remote/RemoteStatResource.java        |   21 +-
 .../admin/remote/RemoteTransportConfig.java     |  148 +-
 .../remote/RemoveHealthListenerRequest.java     |   21 +-
 .../remote/RemoveHealthListenerResponse.java    |   21 +-
 .../admin/remote/ResetHealthStatusRequest.java  |   21 +-
 .../admin/remote/ResetHealthStatusResponse.java |   21 +-
 .../admin/remote/RevokePersistentIDRequest.java |   21 +-
 .../remote/RevokePersistentIDResponse.java      |   21 +-
 .../admin/remote/RootRegionRequest.java         |   21 +-
 .../admin/remote/RootRegionResponse.java        |   21 +-
 .../remote/ShutdownAllGatewayHubsRequest.java   |   16 +
 .../admin/remote/ShutdownAllRequest.java        |   27 +-
 .../admin/remote/ShutdownAllResponse.java       |   23 +-
 .../admin/remote/SnapshotResultMessage.java     |   23 +-
 .../remote/StatAlertsManagerAssignMessage.java  |   21 +-
 .../admin/remote/StatListenerMessage.java       |   23 +-
 .../admin/remote/StoreSysCfgRequest.java        |   21 +-
 .../admin/remote/StoreSysCfgResponse.java       |   21 +-
 .../internal/admin/remote/SubRegionRequest.java |   21 +-
 .../admin/remote/SubRegionResponse.java         |   21 +-
 .../internal/admin/remote/TailLogRequest.java   |   21 +-
 .../internal/admin/remote/TailLogResponse.java  |   21 +-
 .../remote/UpdateAlertDefinitionMessage.java    |   21 +-
 .../admin/remote/VersionInfoRequest.java        |   21 +-
 .../admin/remote/VersionInfoResponse.java       |   21 +-
 .../admin/remote/VersionMismatchAlert.java      |   21 +-
 .../gemfire/internal/admin/remote/package.html  |   16 +
 .../admin/statalerts/BaseDecoratorImpl.java     |   21 +-
 .../statalerts/DummyStatisticInfoImpl.java      |   21 +-
 .../admin/statalerts/FunctionDecoratorImpl.java |   21 +-
 .../admin/statalerts/FunctionHelper.java        |   20 +-
 .../statalerts/GaugeThresholdDecoratorImpl.java |   21 +-
 .../statalerts/MultiAttrDefinitionImpl.java     |   21 +-
 .../NumberThresholdDecoratorImpl.java           |   21 +-
 .../statalerts/SingleAttrDefinitionImpl.java    |   21 +-
 .../admin/statalerts/StatisticInfo.java         |   21 +-
 .../admin/statalerts/StatisticInfoImpl.java     |   21 +-
 .../internal/cache/AbstractBridgeServer.java    |  425 --
 .../cache/AbstractBucketRegionQueue.java        |   21 +-
 .../internal/cache/AbstractCacheServer.java     |  407 ++
 .../cache/AbstractDiskLRURegionEntry.java       |   21 +-
 .../internal/cache/AbstractDiskRegion.java      |   21 +-
 .../internal/cache/AbstractDiskRegionEntry.java |   21 +-
 .../internal/cache/AbstractLRURegionEntry.java  |   21 +-
 .../internal/cache/AbstractLRURegionMap.java    |   21 +-
 .../cache/AbstractOplogDiskRegionEntry.java     |   21 +-
 .../gemfire/internal/cache/AbstractRegion.java  |   98 +-
 .../internal/cache/AbstractRegionEntry.java     |   50 +-
 .../internal/cache/AbstractRegionMap.java       |   37 +-
 .../internal/cache/AbstractUpdateOperation.java |   25 +-
 .../gemfire/internal/cache/AcceptHelper.java    |   21 +-
 .../cache/AddCacheServerProfileMessage.java     |   21 +-
 .../gemfire/internal/cache/BackupLock.java      |   21 +-
 .../gemfire/internal/cache/BridgeObserver.java  |   89 -
 .../internal/cache/BridgeObserverAdapter.java   |  107 -
 .../internal/cache/BridgeObserverHolder.java    |   53 -
 .../internal/cache/BridgeRegionEventImpl.java   |  108 -
 .../internal/cache/BridgeServerAdvisor.java     |  165 -
 .../internal/cache/BridgeServerImpl.java        |  816 ----
 .../gemfire/internal/cache/BucketAdvisor.java   |   23 +-
 .../gemfire/internal/cache/BucketDump.java      |   21 +-
 .../internal/cache/BucketNotFoundException.java |   21 +-
 .../cache/BucketPersistenceAdvisor.java         |   21 +-
 .../gemfire/internal/cache/BucketRegion.java    |   23 +-
 .../internal/cache/BucketRegionEvictior.java    |   21 +-
 .../internal/cache/BucketRegionQueue.java       |   21 +-
 .../internal/cache/BucketServerLocation.java    |   20 +-
 .../internal/cache/BucketServerLocation66.java  |   20 +-
 .../cache/BytesAndBitsForCompactor.java         |   23 +-
 .../internal/cache/CacheClientStatus.java       |   21 +-
 .../gemfire/internal/cache/CacheConfig.java     |   29 +-
 .../cache/CacheDistributionAdvisee.java         |   21 +-
 .../cache/CacheDistributionAdvisor.java         |   21 +-
 .../internal/cache/CacheLifecycleListener.java  |   21 +-
 .../gemfire/internal/cache/CacheObserver.java   |   21 +-
 .../internal/cache/CacheObserverAdapter.java    |   21 +-
 .../internal/cache/CacheObserverHolder.java     |   21 +-
 .../gemfire/internal/cache/CachePerfStats.java  |   21 +-
 .../internal/cache/CacheServerAdvisor.java      |  173 +
 .../gemfire/internal/cache/CacheServerImpl.java |  821 ++++
 .../internal/cache/CacheServerLauncher.java     |   21 +-
 .../gemfire/internal/cache/CacheService.java    |   42 +
 .../internal/cache/CacheStatisticsImpl.java     |   21 +-
 .../internal/cache/CachedDeserializable.java    |   21 +-
 .../cache/CachedDeserializableFactory.java      |   21 +-
 .../internal/cache/ClientRegionEventImpl.java   |  117 +
 .../internal/cache/ClientServerObserver.java    |   99 +
 .../cache/ClientServerObserverAdapter.java      |  116 +
 .../cache/ClientServerObserverHolder.java       |   62 +
 .../cache/ClientSubscriptionConfigImpl.java     |   21 +-
 .../internal/cache/CloseCacheMessage.java       |   23 +-
 .../cache/ClusterConfigurationLoader.java       |   16 +
 .../internal/cache/ColocationHelper.java        |   20 +-
 .../internal/cache/CommitReplyException.java    |   21 +-
 .../internal/cache/CompactableOplog.java        |   21 +-
 .../gemfire/internal/cache/Conflatable.java     |   21 +-
 .../internal/cache/ControllerAdvisor.java       |   21 +-
 .../internal/cache/CountingDataInputStream.java |   21 +-
 .../internal/cache/CreateRegionProcessor.java   |   27 +-
 .../internal/cache/CustomEntryExpiryTask.java   |   16 +
 .../cache/CustomEvictionAttributesImpl.java     |   21 +-
 .../internal/cache/DataLocationException.java   |   21 +-
 .../internal/cache/DestroyOperation.java        |   21 +-
 .../cache/DestroyPartitionedRegionMessage.java  |   21 +-
 .../internal/cache/DestroyRegionOperation.java  |   27 +-
 .../gemfire/internal/cache/DestroyedEntry.java  |   21 +-
 .../internal/cache/DirectReplyMessage.java      |   21 +-
 .../gemfire/internal/cache/DirectoryHolder.java |   21 +-
 .../internal/cache/DiskDirectoryStats.java      |   21 +-
 .../gemfire/internal/cache/DiskEntry.java       |   29 +-
 .../gemstone/gemfire/internal/cache/DiskId.java |   21 +-
 .../gemfire/internal/cache/DiskInitFile.java    |   21 +-
 .../gemfire/internal/cache/DiskRegion.java      |   21 +-
 .../gemfire/internal/cache/DiskRegionStats.java |   21 +-
 .../internal/cache/DiskStoreAttributes.java     |   21 +-
 .../gemfire/internal/cache/DiskStoreBackup.java |   21 +-
 .../internal/cache/DiskStoreFactoryImpl.java    |   21 +-
 .../gemfire/internal/cache/DiskStoreImpl.java   |   27 +-
 .../internal/cache/DiskStoreMonitor.java        |   21 +-
 .../internal/cache/DiskStoreObserver.java       |   21 +-
 .../gemfire/internal/cache/DiskStoreStats.java  |   21 +-
 .../gemfire/internal/cache/DiskStoreTask.java   |   21 +-
 .../internal/cache/DiskWriteAttributesImpl.java |   21 +-
 .../internal/cache/DistPeerTXStateStub.java     |   16 +
 .../cache/DistTXAdjunctCommitMessage.java       |   16 +
 .../internal/cache/DistTXCommitMessage.java     |   17 +-
 .../cache/DistTXCoordinatorInterface.java       |   21 +-
 .../internal/cache/DistTXPrecommitMessage.java  |   17 +-
 .../internal/cache/DistTXRollbackMessage.java   |   17 +-
 .../gemfire/internal/cache/DistTXState.java     |   16 +
 .../cache/DistTXStateOnCoordinator.java         |   16 +
 .../internal/cache/DistTXStateProxyImpl.java    |   16 +
 .../DistTXStateProxyImplOnCoordinator.java      |   18 +-
 .../cache/DistTXStateProxyImplOnDatanode.java   |   16 +
 .../cache/DistributedCacheOperation.java        |   36 +-
 .../cache/DistributedClearOperation.java        |   27 +-
 .../cache/DistributedPutAllOperation.java       |   21 +-
 .../internal/cache/DistributedRegion.java       |   31 +-
 ...stributedRegionFunctionStreamingMessage.java |   21 +-
 .../cache/DistributedRemoveAllOperation.java    |   21 +-
 .../cache/DistributedTombstoneOperation.java    |   21 +-
 .../internal/cache/DummyCachePerfStats.java     |   21 +-
 .../internal/cache/DynamicRegionAttributes.java |   21 +-
 .../cache/DynamicRegionFactoryImpl.java         |   21 +-
 .../gemfire/internal/cache/EntriesMap.java      |   21 +-
 .../gemfire/internal/cache/EntriesSet.java      |   21 +-
 .../gemfire/internal/cache/EntryBits.java       |   21 +-
 .../gemfire/internal/cache/EntryEventImpl.java  |   67 +-
 .../gemfire/internal/cache/EntryExpiryTask.java |   24 +-
 .../internal/cache/EntryOperationImpl.java      |   23 +-
 .../gemfire/internal/cache/EntrySnapshot.java   |   23 +-
 .../internal/cache/EnumListenerEvent.java       |   20 +-
 .../gemfire/internal/cache/EventID.java         |   21 +-
 .../internal/cache/EventStateHelper.java        |   21 +-
 .../gemfire/internal/cache/EventTracker.java    |   21 +-
 .../internal/cache/EvictionAttributesImpl.java  |   23 +-
 .../gemfire/internal/cache/EvictorService.java  |   21 +-
 .../internal/cache/ExpirationScheduler.java     |   21 +-
 .../gemfire/internal/cache/ExpiryTask.java      |   81 +-
 .../internal/cache/ExportDiskRegion.java        |   16 +
 .../gemfire/internal/cache/FilterProfile.java   |   21 +-
 .../internal/cache/FilterRoutingInfo.java       |   21 +-
 .../cache/FindDurableQueueProcessor.java        |   25 +-
 .../internal/cache/FindRemoteTXMessage.java     |   23 +-
 .../internal/cache/FindVersionTagOperation.java |   21 +-
 .../cache/FixedPartitionAttributesImpl.java     |   21 +-
 .../internal/cache/ForceReattemptException.java |   21 +-
 .../cache/ForceableLinkedBlockingQueue.java     |   21 +-
 .../FunctionStreamingOrderedReplyMessage.java   |   21 +-
 .../cache/FunctionStreamingReplyMessage.java    |   21 +-
 .../internal/cache/GatewayEventFilter.java      |   16 +
 .../internal/cache/GemFireCacheImpl.java        |  244 +-
 .../internal/cache/GemfireCacheHelper.java      |   23 +-
 .../gemfire/internal/cache/GridAdvisor.java     |   27 +-
 .../gemfire/internal/cache/HARegion.java        |   21 +-
 .../internal/cache/HDFSLRURegionMap.java        |   21 +-
 .../gemfire/internal/cache/HDFSRegionMap.java   |   16 +
 .../internal/cache/HDFSRegionMapDelegate.java   |   21 +-
 .../internal/cache/HDFSRegionMapImpl.java       |   21 +-
 .../internal/cache/HasCachePerfStats.java       |   16 +
 .../gemfire/internal/cache/ImageState.java      |   21 +-
 .../cache/InMemoryPersistentMemberView.java     |   21 +-
 .../internal/cache/IncomingGatewayStatus.java   |   21 +-
 .../internal/cache/InitialImageFlowControl.java |   23 +-
 .../internal/cache/InitialImageOperation.java   |   21 +-
 .../gemfire/internal/cache/InlineKeyHelper.java |   21 +-
 .../gemfire/internal/cache/InterestEvent.java   |   21 +-
 .../gemfire/internal/cache/InterestFilter.java  |   21 +-
 .../cache/InterestRegistrationEventImpl.java    |   21 +-
 .../gemfire/internal/cache/InternalCache.java   |   23 +-
 .../internal/cache/InternalCacheEvent.java      |   21 +-
 .../internal/cache/InternalDataView.java        |   21 +-
 .../internal/cache/InternalRegionArguments.java |   21 +-
 .../internal/cache/InvalidateOperation.java     |   21 +-
 .../InvalidatePartitionedRegionMessage.java     |   21 +-
 .../cache/InvalidateRegionOperation.java        |   21 +-
 .../cache/JtaAfterCompletionMessage.java        |   21 +-
 .../cache/JtaBeforeCompletionMessage.java       |   21 +-
 .../gemfire/internal/cache/KeyInfo.java         |   21 +-
 .../internal/cache/KeyWithRegionContext.java    |   21 +-
 .../gemfire/internal/cache/ListOfDeltas.java    |   21 +-
 .../internal/cache/LoaderHelperFactory.java     |   21 +-
 .../internal/cache/LoaderHelperImpl.java        |   23 +-
 .../gemfire/internal/cache/LocalDataSet.java    |   21 +-
 .../gemfire/internal/cache/LocalRegion.java     |  101 +-
 .../internal/cache/LocalRegionDataView.java     |   21 +-
 .../cache/MemberFunctionStreamingMessage.java   |   30 +-
 .../cache/MinimumSystemRequirements.java        |   25 +-
 .../cache/NetSearchExpirationCalculator.java    |   21 +-
 .../gemstone/gemfire/internal/cache/Node.java   |   20 +-
 .../internal/cache/NonLocalRegionEntry.java     |   21 +-
 .../cache/NonLocalRegionEntryWithStats.java     |   23 +-
 .../internal/cache/OffHeapRegionEntry.java      |   16 +
 .../cache/OfflineCompactionDiskRegion.java      |   21 +-
 .../gemstone/gemfire/internal/cache/OpType.java |   21 +-
 .../gemstone/gemfire/internal/cache/Oplog.java  |   28 +-
 .../gemfire/internal/cache/OplogSet.java        |   16 +
 .../internal/cache/OrderedTombstoneMap.java     |   21 +-
 .../gemfire/internal/cache/OverflowOplog.java   |   21 +-
 .../internal/cache/OverflowOplogSet.java        |   21 +-
 .../internal/cache/PRContainsValueFunction.java |   21 +-
 .../internal/cache/PRHARedundancyProvider.java  |   22 +-
 .../internal/cache/PRQueryProcessor.java        |   20 +-
 .../internal/cache/PRSystemPropertyGetter.java  |   21 +-
 .../internal/cache/PartitionAttributesImpl.java |   21 +-
 .../internal/cache/PartitionRegionConfig.java   |   21 +-
 .../cache/PartitionRegionConfigValidator.java   |   21 +-
 .../internal/cache/PartitionedRegion.java       |   59 +-
 .../PartitionedRegionBucketMgmtHelper.java      |   20 +-
 .../cache/PartitionedRegionDataStore.java       |   24 +-
 .../cache/PartitionedRegionDataView.java        |   21 +-
 .../cache/PartitionedRegionException.java       |   21 +-
 .../internal/cache/PartitionedRegionHelper.java |   22 +-
 .../cache/PartitionedRegionQueryEvaluator.java  |   20 +-
 .../internal/cache/PartitionedRegionStats.java  |   21 +-
 .../internal/cache/PartitionedRegionStatus.java |   21 +-
 .../gemfire/internal/cache/PeerTXStateStub.java |   21 +-
 .../internal/cache/PersistentOplogSet.java      |   21 +-
 .../internal/cache/PlaceHolderDiskRegion.java   |   23 +-
 .../gemfire/internal/cache/PoolFactoryImpl.java |  192 +-
 .../gemfire/internal/cache/PoolManagerImpl.java |   40 +-
 .../gemfire/internal/cache/PoolStats.java       |   21 +-
 .../cache/PreferBytesCachedDeserializable.java  |   21 +-
 .../internal/cache/PrimaryBucketException.java  |   21 +-
 .../cache/ProfileExchangeProcessor.java         |   21 +-
 .../internal/cache/ProxyBucketRegion.java       |   21 +-
 .../gemfire/internal/cache/ProxyRegionMap.java  |   21 +-
 .../cache/PutAllPartialResultException.java     |   21 +-
 .../gemfire/internal/cache/QueuedOperation.java |   21 +-
 .../internal/cache/RegionClearedException.java  |   21 +-
 .../gemfire/internal/cache/RegionEntry.java     |   21 +-
 .../internal/cache/RegionEntryContext.java      |   21 +-
 .../internal/cache/RegionEntryFactory.java      |   21 +-
 .../gemfire/internal/cache/RegionEventImpl.java |   23 +-
 .../internal/cache/RegionEvictorTask.java       |   21 +-
 .../internal/cache/RegionExpiryTask.java        |   24 +-
 .../internal/cache/RegionFactoryImpl.java       |   21 +-
 .../internal/cache/RegionIdleExpiryTask.java    |   21 +-
 .../gemfire/internal/cache/RegionListener.java  |   46 +
 .../gemfire/internal/cache/RegionMap.java       |   21 +-
 .../internal/cache/RegionMapFactory.java        |   21 +-
 .../gemfire/internal/cache/RegionQueue.java     |   21 +-
 .../internal/cache/RegionQueueException.java    |   21 +-
 .../gemfire/internal/cache/RegionStatus.java    |   21 +-
 .../internal/cache/RegionTTLExpiryTask.java     |   21 +-
 .../internal/cache/ReleaseClearLockMessage.java |   23 +-
 .../cache/ReliableDistributionData.java         |   21 +-
 .../internal/cache/ReliableMessageQueue.java    |   21 +-
 .../cache/ReliableMessageQueueFactory.java      |   21 +-
 .../cache/ReliableMessageQueueFactoryImpl.java  |   21 +-
 .../cache/RemoteContainsKeyValueMessage.java    |   20 +-
 .../internal/cache/RemoteDestroyMessage.java    |   21 +-
 .../internal/cache/RemoteFetchEntryMessage.java |   20 +-
 .../cache/RemoteFetchVersionMessage.java        |   20 +-
 .../internal/cache/RemoteGetMessage.java        |   21 +-
 .../internal/cache/RemoteInvalidateMessage.java |   20 +-
 .../cache/RemoteOperationException.java         |   21 +-
 .../internal/cache/RemoteOperationMessage.java  |   21 +-
 .../RemoteOperationMessageWithDirectReply.java  |   21 +-
 .../internal/cache/RemotePutAllMessage.java     |   21 +-
 .../internal/cache/RemotePutMessage.java        |   77 +-
 .../internal/cache/RemoteRegionOperation.java   |   20 +-
 .../internal/cache/RemoteRemoveAllMessage.java  |   21 +-
 .../gemfire/internal/cache/RoleEventImpl.java   |   21 +-
 .../cache/SearchLoadAndWriteProcessor.java      |   21 +-
 .../internal/cache/SendQueueOperation.java      |   21 +-
 .../internal/cache/SerializationHelper.java     |   16 +
 .../internal/cache/ServerPingMessage.java       |   16 +
 .../internal/cache/StateFlushOperation.java     |   26 +-
 .../cache/StoreAllCachedDeserializable.java     |   21 +-
 .../internal/cache/TXBucketRegionState.java     |   21 +-
 .../gemfire/internal/cache/TXCommitMessage.java |   66 +-
 .../gemfire/internal/cache/TXEntry.java         |   21 +-
 .../gemfire/internal/cache/TXEntryState.java    |   40 +-
 .../internal/cache/TXEntryStateFactory.java     |   21 +-
 .../internal/cache/TXEntryUserAttrState.java    |   21 +-
 .../gemfire/internal/cache/TXEvent.java         |   21 +-
 .../internal/cache/TXFarSideCMTracker.java      |   21 +-
 .../gemstone/gemfire/internal/cache/TXId.java   |   22 +-
 .../gemfire/internal/cache/TXLockRequest.java   |   21 +-
 .../gemfire/internal/cache/TXManagerImpl.java   |   21 +-
 .../gemfire/internal/cache/TXMessage.java       |   21 +-
 .../internal/cache/TXRegionLockRequestImpl.java |   21 +-
 .../gemfire/internal/cache/TXRegionState.java   |   21 +-
 .../internal/cache/TXRemoteCommitMessage.java   |   21 +-
 .../internal/cache/TXRemoteRollbackMessage.java |   21 +-
 .../internal/cache/TXReservationMgr.java        |   21 +-
 .../gemfire/internal/cache/TXRmtEvent.java      |   21 +-
 .../gemfire/internal/cache/TXState.java         |   23 +-
 .../internal/cache/TXStateInterface.java        |   21 +-
 .../gemfire/internal/cache/TXStateProxy.java    |   21 +-
 .../internal/cache/TXStateProxyImpl.java        |   25 +-
 .../gemfire/internal/cache/TXStateStub.java     |   21 +-
 .../cache/TXSynchronizationRunnable.java        |   21 +-
 .../cache/TestHeapThresholdObserver.java        |   21 +-
 .../cache/TimestampedEntryEventImpl.java        |   21 +-
 .../gemstone/gemfire/internal/cache/Token.java  |   21 +-
 .../internal/cache/TombstoneService.java        |   24 +-
 .../internal/cache/TransactionMessage.java      |   21 +-
 .../gemfire/internal/cache/TxEntryFactory.java  |   16 +
 .../internal/cache/UnsharedImageState.java      |   21 +-
 .../cache/UpdateAttributesProcessor.java        |   23 +-
 .../cache/UpdateEntryVersionOperation.java      |   22 +-
 .../gemfire/internal/cache/UpdateOperation.java |   20 +-
 .../cache/UserSpecifiedDiskStoreAttributes.java |   21 +-
 .../cache/UserSpecifiedRegionAttributes.java    |   21 +-
 .../internal/cache/VMCachedDeserializable.java  |   21 +-
 .../gemfire/internal/cache/VMLRURegionMap.java  |   21 +-
 .../gemfire/internal/cache/VMRegionMap.java     |   21 +-
 .../cache/VMStatsDiskLRURegionEntry.java        |   21 +-
 .../cache/VMStatsDiskLRURegionEntryHeap.java    |   21 +-
 .../VMStatsDiskLRURegionEntryHeapIntKey.java    |   21 +-
 .../VMStatsDiskLRURegionEntryHeapLongKey.java   |   21 +-
 .../VMStatsDiskLRURegionEntryHeapObjectKey.java |   21 +-
 ...VMStatsDiskLRURegionEntryHeapStringKey1.java |   21 +-
 ...VMStatsDiskLRURegionEntryHeapStringKey2.java |   21 +-
 .../VMStatsDiskLRURegionEntryHeapUUIDKey.java   |   21 +-
 .../cache/VMStatsDiskLRURegionEntryOffHeap.java |   21 +-
 .../VMStatsDiskLRURegionEntryOffHeapIntKey.java |   21 +-
 ...VMStatsDiskLRURegionEntryOffHeapLongKey.java |   21 +-
 ...StatsDiskLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...tatsDiskLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...tatsDiskLRURegionEntryOffHeapStringKey2.java |   21 +-
 ...VMStatsDiskLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../internal/cache/VMStatsDiskRegionEntry.java  |   21 +-
 .../cache/VMStatsDiskRegionEntryHeap.java       |   21 +-
 .../cache/VMStatsDiskRegionEntryHeapIntKey.java |   21 +-
 .../VMStatsDiskRegionEntryHeapLongKey.java      |   21 +-
 .../VMStatsDiskRegionEntryHeapObjectKey.java    |   21 +-
 .../VMStatsDiskRegionEntryHeapStringKey1.java   |   21 +-
 .../VMStatsDiskRegionEntryHeapStringKey2.java   |   21 +-
 .../VMStatsDiskRegionEntryHeapUUIDKey.java      |   21 +-
 .../cache/VMStatsDiskRegionEntryOffHeap.java    |   21 +-
 .../VMStatsDiskRegionEntryOffHeapIntKey.java    |   21 +-
 .../VMStatsDiskRegionEntryOffHeapLongKey.java   |   21 +-
 .../VMStatsDiskRegionEntryOffHeapObjectKey.java |   21 +-
 ...VMStatsDiskRegionEntryOffHeapStringKey1.java |   21 +-
 ...VMStatsDiskRegionEntryOffHeapStringKey2.java |   21 +-
 .../VMStatsDiskRegionEntryOffHeapUUIDKey.java   |   21 +-
 .../internal/cache/VMStatsLRURegionEntry.java   |   21 +-
 .../cache/VMStatsLRURegionEntryHeap.java        |   21 +-
 .../cache/VMStatsLRURegionEntryHeapIntKey.java  |   21 +-
 .../cache/VMStatsLRURegionEntryHeapLongKey.java |   21 +-
 .../VMStatsLRURegionEntryHeapObjectKey.java     |   21 +-
 .../VMStatsLRURegionEntryHeapStringKey1.java    |   21 +-
 .../VMStatsLRURegionEntryHeapStringKey2.java    |   21 +-
 .../cache/VMStatsLRURegionEntryHeapUUIDKey.java |   21 +-
 .../cache/VMStatsLRURegionEntryOffHeap.java     |   21 +-
 .../VMStatsLRURegionEntryOffHeapIntKey.java     |   21 +-
 .../VMStatsLRURegionEntryOffHeapLongKey.java    |   21 +-
 .../VMStatsLRURegionEntryOffHeapObjectKey.java  |   21 +-
 .../VMStatsLRURegionEntryOffHeapStringKey1.java |   21 +-
 .../VMStatsLRURegionEntryOffHeapStringKey2.java |   21 +-
 .../VMStatsLRURegionEntryOffHeapUUIDKey.java    |   21 +-
 .../internal/cache/VMStatsRegionEntry.java      |   21 +-
 .../internal/cache/VMStatsRegionEntryHeap.java  |   21 +-
 .../cache/VMStatsRegionEntryHeapIntKey.java     |   21 +-
 .../cache/VMStatsRegionEntryHeapLongKey.java    |   21 +-
 .../cache/VMStatsRegionEntryHeapObjectKey.java  |   21 +-
 .../cache/VMStatsRegionEntryHeapStringKey1.java |   21 +-
 .../cache/VMStatsRegionEntryHeapStringKey2.java |   21 +-
 .../cache/VMStatsRegionEntryHeapUUIDKey.java    |   21 +-
 .../cache/VMStatsRegionEntryOffHeap.java        |   21 +-
 .../cache/VMStatsRegionEntryOffHeapIntKey.java  |   21 +-
 .../cache/VMStatsRegionEntryOffHeapLongKey.java |   21 +-
 .../VMStatsRegionEntryOffHeapObjectKey.java     |   21 +-
 .../VMStatsRegionEntryOffHeapStringKey1.java    |   21 +-
 .../VMStatsRegionEntryOffHeapStringKey2.java    |   21 +-
 .../cache/VMStatsRegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VMThinDiskLRURegionEntry.java         |   21 +-
 .../cache/VMThinDiskLRURegionEntryHeap.java     |   21 +-
 .../VMThinDiskLRURegionEntryHeapIntKey.java     |   21 +-
 .../VMThinDiskLRURegionEntryHeapLongKey.java    |   21 +-
 .../VMThinDiskLRURegionEntryHeapObjectKey.java  |   21 +-
 .../VMThinDiskLRURegionEntryHeapStringKey1.java |   21 +-
 .../VMThinDiskLRURegionEntryHeapStringKey2.java |   21 +-
 .../VMThinDiskLRURegionEntryHeapUUIDKey.java    |   21 +-
 .../cache/VMThinDiskLRURegionEntryOffHeap.java  |   21 +-
 .../VMThinDiskLRURegionEntryOffHeapIntKey.java  |   21 +-
 .../VMThinDiskLRURegionEntryOffHeapLongKey.java |   21 +-
 ...MThinDiskLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...ThinDiskLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...ThinDiskLRURegionEntryOffHeapStringKey2.java |   21 +-
 .../VMThinDiskLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../internal/cache/VMThinDiskRegionEntry.java   |   21 +-
 .../cache/VMThinDiskRegionEntryHeap.java        |   21 +-
 .../cache/VMThinDiskRegionEntryHeapIntKey.java  |   21 +-
 .../cache/VMThinDiskRegionEntryHeapLongKey.java |   21 +-
 .../VMThinDiskRegionEntryHeapObjectKey.java     |   21 +-
 .../VMThinDiskRegionEntryHeapStringKey1.java    |   21 +-
 .../VMThinDiskRegionEntryHeapStringKey2.java    |   21 +-
 .../cache/VMThinDiskRegionEntryHeapUUIDKey.java |   21 +-
 .../cache/VMThinDiskRegionEntryOffHeap.java     |   21 +-
 .../VMThinDiskRegionEntryOffHeapIntKey.java     |   21 +-
 .../VMThinDiskRegionEntryOffHeapLongKey.java    |   21 +-
 .../VMThinDiskRegionEntryOffHeapObjectKey.java  |   21 +-
 .../VMThinDiskRegionEntryOffHeapStringKey1.java |   21 +-
 .../VMThinDiskRegionEntryOffHeapStringKey2.java |   21 +-
 .../VMThinDiskRegionEntryOffHeapUUIDKey.java    |   21 +-
 .../internal/cache/VMThinLRURegionEntry.java    |   21 +-
 .../cache/VMThinLRURegionEntryHeap.java         |   21 +-
 .../cache/VMThinLRURegionEntryHeapIntKey.java   |   21 +-
 .../cache/VMThinLRURegionEntryHeapLongKey.java  |   21 +-
 .../VMThinLRURegionEntryHeapObjectKey.java      |   21 +-
 .../VMThinLRURegionEntryHeapStringKey1.java     |   21 +-
 .../VMThinLRURegionEntryHeapStringKey2.java     |   21 +-
 .../cache/VMThinLRURegionEntryHeapUUIDKey.java  |   21 +-
 .../cache/VMThinLRURegionEntryOffHeap.java      |   21 +-
 .../VMThinLRURegionEntryOffHeapIntKey.java      |   21 +-
 .../VMThinLRURegionEntryOffHeapLongKey.java     |   21 +-
 .../VMThinLRURegionEntryOffHeapObjectKey.java   |   21 +-
 .../VMThinLRURegionEntryOffHeapStringKey1.java  |   21 +-
 .../VMThinLRURegionEntryOffHeapStringKey2.java  |   21 +-
 .../VMThinLRURegionEntryOffHeapUUIDKey.java     |   21 +-
 .../internal/cache/VMThinRegionEntry.java       |   26 +-
 .../internal/cache/VMThinRegionEntryHeap.java   |   21 +-
 .../cache/VMThinRegionEntryHeapIntKey.java      |   21 +-
 .../cache/VMThinRegionEntryHeapLongKey.java     |   21 +-
 .../cache/VMThinRegionEntryHeapObjectKey.java   |   21 +-
 .../cache/VMThinRegionEntryHeapStringKey1.java  |   21 +-
 .../cache/VMThinRegionEntryHeapStringKey2.java  |   21 +-
 .../cache/VMThinRegionEntryHeapUUIDKey.java     |   21 +-
 .../cache/VMThinRegionEntryOffHeap.java         |   16 +
 .../cache/VMThinRegionEntryOffHeapIntKey.java   |   21 +-
 .../cache/VMThinRegionEntryOffHeapLongKey.java  |   21 +-
 .../VMThinRegionEntryOffHeapObjectKey.java      |   21 +-
 .../VMThinRegionEntryOffHeapStringKey1.java     |   21 +-
 .../VMThinRegionEntryOffHeapStringKey2.java     |   21 +-
 .../cache/VMThinRegionEntryOffHeapUUIDKey.java  |   21 +-
 .../internal/cache/ValidatingDiskRegion.java    |   21 +-
 .../internal/cache/ValueByteWrapper.java        |   21 +-
 .../internal/cache/VersionTimestamp.java        |   21 +-
 .../cache/VersionedStatsDiskLRURegionEntry.java |   21 +-
 .../VersionedStatsDiskLRURegionEntryHeap.java   |   21 +-
 ...sionedStatsDiskLRURegionEntryHeapIntKey.java |   21 +-
 ...ionedStatsDiskLRURegionEntryHeapLongKey.java |   21 +-
 ...nedStatsDiskLRURegionEntryHeapObjectKey.java |   21 +-
 ...edStatsDiskLRURegionEntryHeapStringKey1.java |   21 +-
 ...edStatsDiskLRURegionEntryHeapStringKey2.java |   21 +-
 ...ionedStatsDiskLRURegionEntryHeapUUIDKey.java |   21 +-
 ...VersionedStatsDiskLRURegionEntryOffHeap.java |   21 +-
 ...nedStatsDiskLRURegionEntryOffHeapIntKey.java |   21 +-
 ...edStatsDiskLRURegionEntryOffHeapLongKey.java |   21 +-
 ...StatsDiskLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...tatsDiskLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...tatsDiskLRURegionEntryOffHeapStringKey2.java |   21 +-
 ...edStatsDiskLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedStatsDiskRegionEntry.java    |   21 +-
 .../VersionedStatsDiskRegionEntryHeap.java      |   21 +-
 ...VersionedStatsDiskRegionEntryHeapIntKey.java |   21 +-
 ...ersionedStatsDiskRegionEntryHeapLongKey.java |   21 +-
 ...sionedStatsDiskRegionEntryHeapObjectKey.java |   21 +-
 ...ionedStatsDiskRegionEntryHeapStringKey1.java |   21 +-
 ...ionedStatsDiskRegionEntryHeapStringKey2.java |   21 +-
 ...ersionedStatsDiskRegionEntryHeapUUIDKey.java |   21 +-
 .../VersionedStatsDiskRegionEntryOffHeap.java   |   21 +-
 ...sionedStatsDiskRegionEntryOffHeapIntKey.java |   21 +-
 ...ionedStatsDiskRegionEntryOffHeapLongKey.java |   21 +-
 ...nedStatsDiskRegionEntryOffHeapObjectKey.java |   21 +-
 ...edStatsDiskRegionEntryOffHeapStringKey1.java |   21 +-
 ...edStatsDiskRegionEntryOffHeapStringKey2.java |   21 +-
 ...ionedStatsDiskRegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedStatsLRURegionEntry.java     |   21 +-
 .../cache/VersionedStatsLRURegionEntryHeap.java |   21 +-
 .../VersionedStatsLRURegionEntryHeapIntKey.java |   21 +-
 ...VersionedStatsLRURegionEntryHeapLongKey.java |   21 +-
 ...rsionedStatsLRURegionEntryHeapObjectKey.java |   21 +-
 ...sionedStatsLRURegionEntryHeapStringKey1.java |   21 +-
 ...sionedStatsLRURegionEntryHeapStringKey2.java |   21 +-
 ...VersionedStatsLRURegionEntryHeapUUIDKey.java |   21 +-
 .../VersionedStatsLRURegionEntryOffHeap.java    |   21 +-
 ...rsionedStatsLRURegionEntryOffHeapIntKey.java |   21 +-
 ...sionedStatsLRURegionEntryOffHeapLongKey.java |   21 +-
 ...onedStatsLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...nedStatsLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...nedStatsLRURegionEntryOffHeapStringKey2.java |   21 +-
 ...sionedStatsLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedStatsRegionEntry.java        |   21 +-
 .../cache/VersionedStatsRegionEntryHeap.java    |   21 +-
 .../VersionedStatsRegionEntryHeapIntKey.java    |   21 +-
 .../VersionedStatsRegionEntryHeapLongKey.java   |   21 +-
 .../VersionedStatsRegionEntryHeapObjectKey.java |   21 +-
 ...VersionedStatsRegionEntryHeapStringKey1.java |   21 +-
 ...VersionedStatsRegionEntryHeapStringKey2.java |   21 +-
 .../VersionedStatsRegionEntryHeapUUIDKey.java   |   21 +-
 .../cache/VersionedStatsRegionEntryOffHeap.java |   21 +-
 .../VersionedStatsRegionEntryOffHeapIntKey.java |   21 +-
 ...VersionedStatsRegionEntryOffHeapLongKey.java |   21 +-
 ...rsionedStatsRegionEntryOffHeapObjectKey.java |   21 +-
 ...sionedStatsRegionEntryOffHeapStringKey1.java |   21 +-
 ...sionedStatsRegionEntryOffHeapStringKey2.java |   21 +-
 ...VersionedStatsRegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedThinDiskLRURegionEntry.java  |   21 +-
 .../VersionedThinDiskLRURegionEntryHeap.java    |   21 +-
 ...rsionedThinDiskLRURegionEntryHeapIntKey.java |   21 +-
 ...sionedThinDiskLRURegionEntryHeapLongKey.java |   21 +-
 ...onedThinDiskLRURegionEntryHeapObjectKey.java |   21 +-
 ...nedThinDiskLRURegionEntryHeapStringKey1.java |   21 +-
 ...nedThinDiskLRURegionEntryHeapStringKey2.java |   21 +-
 ...sionedThinDiskLRURegionEntryHeapUUIDKey.java |   21 +-
 .../VersionedThinDiskLRURegionEntryOffHeap.java |   21 +-
 ...onedThinDiskLRURegionEntryOffHeapIntKey.java |   21 +-
 ...nedThinDiskLRURegionEntryOffHeapLongKey.java |   21 +-
 ...dThinDiskLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...ThinDiskLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...ThinDiskLRURegionEntryOffHeapStringKey2.java |   21 +-
 ...nedThinDiskLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedThinDiskRegionEntry.java     |   21 +-
 .../cache/VersionedThinDiskRegionEntryHeap.java |   21 +-
 .../VersionedThinDiskRegionEntryHeapIntKey.java |   21 +-
 ...VersionedThinDiskRegionEntryHeapLongKey.java |   21 +-
 ...rsionedThinDiskRegionEntryHeapObjectKey.java |   21 +-
 ...sionedThinDiskRegionEntryHeapStringKey1.java |   21 +-
 ...sionedThinDiskRegionEntryHeapStringKey2.java |   21 +-
 ...VersionedThinDiskRegionEntryHeapUUIDKey.java |   21 +-
 .../VersionedThinDiskRegionEntryOffHeap.java    |   21 +-
 ...rsionedThinDiskRegionEntryOffHeapIntKey.java |   21 +-
 ...sionedThinDiskRegionEntryOffHeapLongKey.java |   21 +-
 ...onedThinDiskRegionEntryOffHeapObjectKey.java |   21 +-
 ...nedThinDiskRegionEntryOffHeapStringKey1.java |   21 +-
 ...nedThinDiskRegionEntryOffHeapStringKey2.java |   21 +-
 ...sionedThinDiskRegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedThinLRURegionEntry.java      |   21 +-
 .../cache/VersionedThinLRURegionEntryHeap.java  |   21 +-
 .../VersionedThinLRURegionEntryHeapIntKey.java  |   21 +-
 .../VersionedThinLRURegionEntryHeapLongKey.java |   21 +-
 ...ersionedThinLRURegionEntryHeapObjectKey.java |   21 +-
 ...rsionedThinLRURegionEntryHeapStringKey1.java |   21 +-
 ...rsionedThinLRURegionEntryHeapStringKey2.java |   21 +-
 .../VersionedThinLRURegionEntryHeapUUIDKey.java |   21 +-
 .../VersionedThinLRURegionEntryOffHeap.java     |   21 +-
 ...ersionedThinLRURegionEntryOffHeapIntKey.java |   21 +-
 ...rsionedThinLRURegionEntryOffHeapLongKey.java |   21 +-
 ...ionedThinLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...onedThinLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...onedThinLRURegionEntryOffHeapStringKey2.java |   21 +-
 ...rsionedThinLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedThinRegionEntry.java         |   21 +-
 .../cache/VersionedThinRegionEntryHeap.java     |   21 +-
 .../VersionedThinRegionEntryHeapIntKey.java     |   21 +-
 .../VersionedThinRegionEntryHeapLongKey.java    |   21 +-
 .../VersionedThinRegionEntryHeapObjectKey.java  |   21 +-
 .../VersionedThinRegionEntryHeapStringKey1.java |   21 +-
 .../VersionedThinRegionEntryHeapStringKey2.java |   21 +-
 .../VersionedThinRegionEntryHeapUUIDKey.java    |   21 +-
 .../cache/VersionedThinRegionEntryOffHeap.java  |   21 +-
 .../VersionedThinRegionEntryOffHeapIntKey.java  |   21 +-
 .../VersionedThinRegionEntryOffHeapLongKey.java |   21 +-
 ...ersionedThinRegionEntryOffHeapObjectKey.java |   21 +-
 ...rsionedThinRegionEntryOffHeapStringKey1.java |   21 +-
 ...rsionedThinRegionEntryOffHeapStringKey2.java |   21 +-
 .../VersionedThinRegionEntryOffHeapUUIDKey.java |   21 +-
 .../internal/cache/WrappedCallbackArgument.java |   21 +-
 .../cache/WrappedRegionMembershipListener.java  |   21 +-
 .../CompressedCachedDeserializable.java         |   23 +-
 .../SnappyCompressedCachedDeserializable.java   |   23 +-
 .../internal/cache/control/FilterByPath.java    |   21 +-
 .../cache/control/HeapMemoryMonitor.java        |   22 +-
 .../cache/control/InternalResourceManager.java  |   34 +-
 .../internal/cache/control/MemoryEvent.java     |   20 +-
 .../cache/control/MemoryThresholds.java         |   16 +
 .../cache/control/OffHeapMemoryMonitor.java     |  212 +-
 .../control/PartitionRebalanceDetailsImpl.java  |   21 +-
 .../cache/control/RebalanceOperationImpl.java   |   21 +-
 .../cache/control/RebalanceResultsImpl.java     |   21 +-
 .../internal/cache/control/RegionFilter.java    |   21 +-
 .../internal/cache/control/ResourceAdvisor.java |   21 +-
 .../internal/cache/control/ResourceEvent.java   |   20 +-
 .../cache/control/ResourceListener.java         |   21 +-
 .../cache/control/ResourceManagerStats.java     |   21 +-
 .../internal/cache/control/ResourceMonitor.java |   16 +
 .../gemfire/internal/cache/delta/Delta.java     |   21 +-
 .../internal/cache/doc-files/properties.html    |   42 +-
 .../cache/execute/AbstractExecution.java        |   21 +-
 .../cache/execute/BucketMovedException.java     |   21 +-
 .../cache/execute/DefaultResultCollector.java   |   20 +-
 .../DistributedRegionFunctionExecutor.java      |   20 +-
 .../DistributedRegionFunctionResultSender.java  |   21 +-
 .../DistributedRegionFunctionResultWaiter.java  |   21 +-
 .../cache/execute/FunctionContextImpl.java      |   21 +-
 .../execute/FunctionExecutionNodePruner.java    |   21 +-
 .../cache/execute/FunctionRemoteContext.java    |   21 +-
 .../cache/execute/FunctionServiceStats.java     |   24 +-
 .../internal/cache/execute/FunctionStats.java   |   21 +-
 .../FunctionStreamingResultCollector.java       |   21 +-
 .../cache/execute/InternalExecution.java        |   21 +-
 .../execute/InternalFunctionException.java      |   21 +-
 ...ternalFunctionInvocationTargetException.java |   21 +-
 .../cache/execute/InternalFunctionService.java  |   24 +-
 .../execute/InternalRegionFunctionContext.java  |   21 +-
 .../cache/execute/InternalResultSender.java     |   21 +-
 .../cache/execute/LocalResultCollector.java     |   20 +-
 .../cache/execute/LocalResultCollectorImpl.java |   21 +-
 .../cache/execute/MemberFunctionExecutor.java   |   21 +-
 .../execute/MemberFunctionResultSender.java     |   20 +-
 .../execute/MemberFunctionResultWaiter.java     |   21 +-
 .../cache/execute/MemberMappedArgument.java     |   20 +-
 .../execute/MultiRegionFunctionContext.java     |   21 +-
 .../execute/MultiRegionFunctionContextImpl.java |   21 +-
 .../execute/MultiRegionFunctionExecutor.java    |   21 +-
 .../MultiRegionFunctionResultWaiter.java        |   21 +-
 .../internal/cache/execute/NoResult.java        |   20 +-
 .../PartitionedRegionFunctionExecutor.java      |   21 +-
 .../PartitionedRegionFunctionResultSender.java  |   20 +-
 .../PartitionedRegionFunctionResultWaiter.java  |   21 +-
 .../execute/RegionFunctionContextImpl.java      |   21 +-
 .../cache/execute/ServerFunctionExecutor.java   |   39 +-
 .../execute/ServerRegionFunctionExecutor.java   |   23 +-
 .../ServerToClientFunctionResultSender.java     |   20 +-
 .../ServerToClientFunctionResultSender65.java   |   20 +-
 .../execute/StreamingFunctionOperation.java     |   21 +-
 .../execute/TransactionFunctionService.java     |  184 -
 .../cache/execute/util/CommitFunction.java      |   22 +-
 .../util/FindRestEnabledServersFunction.java    |   23 +-
 .../execute/util/NestedTransactionFunction.java |   21 +-
 .../cache/execute/util/RollbackFunction.java    |   22 +-
 .../internal/cache/extension/Extensible.java    |   21 +-
 .../internal/cache/extension/Extension.java     |   21 +-
 .../cache/extension/ExtensionPoint.java         |   21 +-
 .../cache/extension/SimpleExtensionPoint.java   |   23 +-
 .../internal/cache/ha/HAContainerMap.java       |   21 +-
 .../internal/cache/ha/HAContainerRegion.java    |   21 +-
 .../internal/cache/ha/HAContainerWrapper.java   |   21 +-
 .../internal/cache/ha/HARegionQueue.java        |   28 +-
 .../cache/ha/HARegionQueueAttributes.java       |   21 +-
 .../internal/cache/ha/HARegionQueueStats.java   |   21 +-
 .../internal/cache/ha/QueueRemovalMessage.java  |   21 +-
 .../internal/cache/ha/ThreadIdentifier.java     |   21 +-
 .../locks/GFEAbstractQueuedSynchronizer.java    |   21 +-
 .../locks/ReentrantReadWriteWriteShareLock.java |   21 +-
 .../cache/locks/TXLessorDepartureHandler.java   |   21 +-
 .../internal/cache/locks/TXLockBatch.java       |   21 +-
 .../gemfire/internal/cache/locks/TXLockId.java  |   21 +-
 .../internal/cache/locks/TXLockIdImpl.java      |   21 +-
 .../internal/cache/locks/TXLockService.java     |   21 +-
 .../internal/cache/locks/TXLockServiceImpl.java |   21 +-
 .../internal/cache/locks/TXLockToken.java       |   21 +-
 .../locks/TXLockUpdateParticipantsMessage.java  |   21 +-
 .../locks/TXOriginatorRecoveryProcessor.java    |   21 +-
 .../locks/TXRecoverGrantorMessageProcessor.java |   21 +-
 .../cache/locks/TXRegionLockRequest.java        |   21 +-
 .../gemfire/internal/cache/lru/EnableLRU.java   |   21 +-
 .../gemfire/internal/cache/lru/HeapEvictor.java |   21 +-
 .../cache/lru/HeapLRUCapacityController.java    |   21 +-
 .../internal/cache/lru/HeapLRUStatistics.java   |   21 +-
 .../internal/cache/lru/LRUAlgorithm.java        |   21 +-
 .../cache/lru/LRUCapacityController.java        |   21 +-
 .../internal/cache/lru/LRUClockNode.java        |   21 +-
 .../gemfire/internal/cache/lru/LRUEntry.java    |   21 +-
 .../internal/cache/lru/LRUMapCallbacks.java     |   21 +-
 .../internal/cache/lru/LRUStatistics.java       |   21 +-
 .../cache/lru/MemLRUCapacityController.java     |   21 +-
 .../internal/cache/lru/NewLIFOClockHand.java    |   21 +-
 .../internal/cache/lru/NewLRUClockHand.java     |   21 +-
 .../internal/cache/lru/OffHeapEvictor.java      |   21 +-
 .../gemfire/internal/cache/lru/Sizeable.java    |   21 +-
 .../operations/ContainsKeyOperationContext.java |   21 +-
 .../gemfire/internal/cache/package.html         |   16 +
 .../AllBucketProfilesUpdateMessage.java         |   21 +-
 .../partitioned/BecomePrimaryBucketMessage.java |   21 +-
 .../internal/cache/partitioned/Bucket.java      |   21 +-
 .../cache/partitioned/BucketBackupMessage.java  |   20 +-
 .../cache/partitioned/BucketCountLoadProbe.java |   75 +
 .../partitioned/BucketProfileUpdateMessage.java |   23 +-
 .../cache/partitioned/BucketSizeMessage.java    |   21 +-
 .../partitioned/ContainsKeyValueMessage.java    |   20 +-
 .../cache/partitioned/CreateBucketMessage.java  |   21 +-
 .../partitioned/CreateMissingBucketsTask.java   |   23 +-
 .../partitioned/DeposePrimaryBucketMessage.java |   21 +-
 .../cache/partitioned/DestroyMessage.java       |   21 +-
 .../DestroyRegionOnDataStoreMessage.java        |   20 +-
 .../partitioned/DumpAllPRConfigMessage.java     |   20 +-
 .../cache/partitioned/DumpB2NRegion.java        |   20 +-
 .../cache/partitioned/DumpBucketsMessage.java   |   21 +-
 .../partitioned/EndBucketCreationMessage.java   |   21 +-
 .../partitioned/FetchBulkEntriesMessage.java    |   21 +-
 .../cache/partitioned/FetchEntriesMessage.java  |   32 +-
 .../cache/partitioned/FetchEntryMessage.java    |   20 +-
 .../cache/partitioned/FetchKeysMessage.java     |   21 +-
 .../FetchPartitionDetailsMessage.java           |   21 +-
 .../cache/partitioned/FlushMessage.java         |   21 +-
 .../internal/cache/partitioned/GetMessage.java  |   21 +-
 .../partitioned/IdentityRequestMessage.java     |   20 +-
 .../partitioned/IdentityUpdateMessage.java      |   21 +-
 .../cache/partitioned/IndexCreationMsg.java     |   21 +-
 .../cache/partitioned/InterestEventMessage.java |   21 +-
 .../cache/partitioned/InternalPRInfo.java       |   21 +-
 .../partitioned/InternalPartitionDetails.java   |   21 +-
 .../cache/partitioned/InvalidateMessage.java    |   20 +-
 .../internal/cache/partitioned/LoadProbe.java   |   21 +-
 .../internal/cache/partitioned/LockObject.java  |   21 +-
 .../partitioned/ManageBackupBucketMessage.java  |   21 +-
 .../cache/partitioned/ManageBucketMessage.java  |   21 +-
 .../cache/partitioned/MoveBucketMessage.java    |   21 +-
 .../cache/partitioned/OfflineMemberDetails.java |   23 +-
 .../partitioned/OfflineMemberDetailsImpl.java   |   21 +-
 .../cache/partitioned/PREntriesIterator.java    |   21 +-
 .../PRFunctionStreamingResultCollector.java     |   21 +-
 .../internal/cache/partitioned/PRLoad.java      |   21 +-
 .../PRLocallyDestroyedException.java            |   21 +-
 .../cache/partitioned/PRSanityCheckMessage.java |   21 +-
 .../cache/partitioned/PRTombstoneMessage.java   |   24 +-
 .../PRUpdateEntryVersionMessage.java            |   22 +-
 .../partitioned/PartitionMemberInfoImpl.java    |   21 +-
 .../cache/partitioned/PartitionMessage.java     |   23 +-
 .../PartitionMessageWithDirectReply.java        |   21 +-
 .../partitioned/PartitionRegionInfoImpl.java    |   21 +-
 ...rtitionedRegionFunctionStreamingMessage.java |   21 +-
 .../partitioned/PartitionedRegionObserver.java  |   21 +-
 .../PartitionedRegionObserverAdapter.java       |   21 +-
 .../PartitionedRegionObserverHolder.java        |   23 +-
 .../PartitionedRegionRebalanceOp.java           |   73 +-
 .../partitioned/PrimaryRequestMessage.java      |   21 +-
 .../cache/partitioned/PutAllPRMessage.java      |   21 +-
 .../internal/cache/partitioned/PutMessage.java  |   42 +-
 .../cache/partitioned/QueryMessage.java         |   21 +-
 .../cache/partitioned/RecoveryRunnable.java     |   23 +-
 .../RedundancyAlreadyMetException.java          |   21 +-
 .../cache/partitioned/RedundancyLogger.java     |   21 +-
 .../cache/partitioned/RegionAdvisor.java        |   20 +-
 .../partitioned/RemoteFetchKeysMessage.java     |   21 +-
 .../cache/partitioned/RemoteSizeMessage.java    |   21 +-
 .../cache/partitioned/RemoveAllPRMessage.java   |   21 +-
 .../cache/partitioned/RemoveBucketMessage.java  |   21 +-
 .../cache/partitioned/RemoveIndexesMessage.java |   21 +-
 .../internal/cache/partitioned/SizeMessage.java |   21 +-
 .../cache/partitioned/SizedBasedLoadProbe.java  |   21 +-
 .../StreamingPartitionOperation.java            |   24 +-
 .../partitioned/rebalance/BucketOperator.java   |   21 +-
 .../rebalance/CompositeDirector.java            |   21 +-
 .../rebalance/ExplicitMoveDirector.java         |   21 +-
 .../partitioned/rebalance/FPRDirector.java      |   21 +-
 .../partitioned/rebalance/MoveBuckets.java      |   21 +-
 .../partitioned/rebalance/MovePrimaries.java    |   21 +-
 .../partitioned/rebalance/MovePrimariesFPR.java |   21 +-
 .../rebalance/ParallelBucketOperator.java       |   22 +
 .../rebalance/PartitionedRegionLoadModel.java   |   21 +-
 .../rebalance/PercentageMoveDirector.java       |   21 +-
 .../rebalance/RebalanceDirector.java            |   21 +-
 .../rebalance/RebalanceDirectorAdapter.java     |   21 +-
 .../rebalance/RemoveOverRedundancy.java         |   21 +-
 .../rebalance/SatisfyRedundancy.java            |   21 +-
 .../rebalance/SatisfyRedundancyFPR.java         |   21 +-
 .../rebalance/SimulatedBucketOperator.java      |   21 +-
 .../cache/persistence/BackupInspector.java      |   21 +-
 .../cache/persistence/BackupManager.java        |   21 +-
 .../cache/persistence/BytesAndBits.java         |   21 +-
 .../cache/persistence/CanonicalIdHolder.java    |   21 +-
 .../CreatePersistentRegionProcessor.java        |   21 +-
 .../cache/persistence/DiskExceptionHandler.java |   21 +-
 .../persistence/DiskInitFileInterpreter.java    |   21 +-
 .../cache/persistence/DiskInitFileParser.java   |   21 +-
 .../cache/persistence/DiskRecoveryStore.java    |   23 +-
 .../cache/persistence/DiskRegionView.java       |   21 +-
 .../cache/persistence/DiskStoreFilter.java      |   23 +-
 .../internal/cache/persistence/DiskStoreID.java |   21 +-
 .../persistence/MembershipFlushRequest.java     |   21 +-
 .../persistence/MembershipViewRequest.java      |   21 +-
 .../internal/cache/persistence/OplogType.java   |   18 +-
 .../cache/persistence/PRPersistentConfig.java   |   21 +-
 .../cache/persistence/PersistenceAdvisor.java   |   21 +-
 .../persistence/PersistenceAdvisorImpl.java     |   21 +-
 .../persistence/PersistenceObserverHolder.java  |   21 +-
 .../cache/persistence/PersistentMemberID.java   |   21 +-
 .../persistence/PersistentMemberManager.java    |   21 +-
 .../persistence/PersistentMemberPattern.java    |   23 +-
 .../persistence/PersistentMemberState.java      |   21 +-
 .../cache/persistence/PersistentMemberView.java |   21 +-
 .../persistence/PersistentMembershipView.java   |   23 +-
 .../persistence/PersistentStateListener.java    |   23 +-
 .../PersistentStateQueryMessage.java            |   21 +-
 .../PersistentStateQueryResults.java            |   21 +-
 .../PrepareNewPersistentMemberMessage.java      |   21 +-
 .../RemovePersistentMemberMessage.java          |   21 +-
 .../cache/persistence/RestoreScript.java        |   21 +-
 .../persistence/UninterruptibleFileChannel.java |   18 +-
 .../UninterruptibleRandomAccessFile.java        |   16 +
 .../persistence/query/CloseableIterator.java    |   18 +-
 .../persistence/query/IdentityExtractor.java    |   16 +
 .../cache/persistence/query/IndexMap.java       |   21 +-
 .../cache/persistence/query/ResultBag.java      |   21 +-
 .../cache/persistence/query/ResultList.java     |   21 +-
 .../cache/persistence/query/ResultMap.java      |   21 +-
 .../cache/persistence/query/ResultSet.java      |   21 +-
 .../persistence/query/SortKeyExtractor.java     |   16 +
 .../query/TemporaryResultSetFactory.java        |   21 +-
 .../persistence/query/mock/ByteComparator.java  |   21 +-
 .../mock/CachedDeserializableComparator.java    |   23 +-
 .../persistence/query/mock/IndexMapImpl.java    |   21 +-
 .../persistence/query/mock/ItrAdapter.java      |   23 +-
 .../query/mock/NaturalComparator.java           |   16 +
 .../cache/persistence/query/mock/Pair.java      |   21 +-
 .../persistence/query/mock/PairComparator.java  |   21 +-
 .../persistence/query/mock/ResultListImpl.java  |   21 +-
 .../query/mock/ReverseComparator.java           |   21 +-
 .../query/mock/SortedResultBagImpl.java         |   21 +-
 .../query/mock/SortedResultMapImpl.java         |   21 +-
 .../query/mock/SortedResultSetImpl.java         |   21 +-
 .../persistence/soplog/AbstractCompactor.java   |  524 --
 .../soplog/AbstractKeyValueIterator.java        |   67 -
 .../soplog/AbstractSortedReader.java            |  126 -
 .../soplog/ArraySerializedComparator.java       |  135 -
 .../persistence/soplog/ByteComparator.java      |   21 +-
 .../cache/persistence/soplog/Compactor.java     |  165 -
 .../soplog/CompositeSerializedComparator.java   |   48 -
 .../persistence/soplog/CursorIterator.java      |   21 +-
 .../soplog/DelegatingSerializedComparator.java  |   21 +-
 .../soplog/HFileStoreStatistics.java            |   21 +-
 .../soplog/IndexSerializedComparator.java       |  118 -
 .../persistence/soplog/KeyValueIterator.java    |   21 +-
 .../cache/persistence/soplog/LevelTracker.java  |  111 -
 .../soplog/LexicographicalComparator.java       |  451 --
 .../cache/persistence/soplog/NonCompactor.java  |  101 -
 .../soplog/ReversingSerializedComparator.java   |   58 -
 .../persistence/soplog/SizeTieredCompactor.java |  189 -
 .../cache/persistence/soplog/SoplogToken.java   |  107 -
 .../cache/persistence/soplog/SortedBuffer.java  |  358 --
 .../cache/persistence/soplog/SortedOplog.java   |  149 -
 .../persistence/soplog/SortedOplogFactory.java  |  269 -
 .../persistence/soplog/SortedOplogSet.java      |  109 -
 .../persistence/soplog/SortedOplogSetImpl.java  |  771 ---
 .../soplog/SortedOplogStatistics.java           |   21 +-
 .../cache/persistence/soplog/SortedReader.java  |   21 +-
 .../persistence/soplog/TrackedReference.java    |   21 +-
 .../soplog/hfile/BlockCacheHolder.java          |   30 -
 .../soplog/hfile/HFileSortedOplog.java          |  685 ---
 .../soplog/hfile/HFileSortedOplogFactory.java   |   71 -
 .../soplog/nofile/NoFileSortedOplog.java        |  235 -
 .../soplog/nofile/NoFileSortedOplogFactory.java |   32 -
 .../snapshot/CacheSnapshotServiceImpl.java      |   21 +-
 .../internal/cache/snapshot/ClientExporter.java |   21 +-
 .../cache/snapshot/ExportedRegistry.java        |   21 +-
 .../internal/cache/snapshot/FlowController.java |   21 +-
 .../internal/cache/snapshot/GFSnapshot.java     |   21 +-
 .../internal/cache/snapshot/LocalExporter.java  |   21 +-
 .../snapshot/RegionSnapshotServiceImpl.java     |   21 +-
 .../cache/snapshot/SnapshotFileMapper.java      |   21 +-
 .../cache/snapshot/SnapshotOptionsImpl.java     |   21 +-
 .../internal/cache/snapshot/SnapshotPacket.java |   21 +-
 .../cache/snapshot/WindowedExporter.java        |   21 +-
 .../gemfire/internal/cache/tier/Acceptor.java   |   21 +-
 .../internal/cache/tier/BatchException.java     |   21 +-
 .../internal/cache/tier/CachedRegionHelper.java |   21 +-
 .../internal/cache/tier/ClientHandShake.java    |   21 +-
 .../gemfire/internal/cache/tier/Command.java    |   21 +-
 .../internal/cache/tier/ConnectionProxy.java    |  181 +-
 .../internal/cache/tier/InterestType.java       |   21 +-
 .../cache/tier/InternalBridgeMembership.java    |  715 ---
 .../cache/tier/InternalClientMembership.java    |  626 +++
 .../internal/cache/tier/MessageType.java        |   21 +-
 .../gemfire/internal/cache/tier/package.html    |   16 +
 .../cache/tier/sockets/AcceptorImpl.java        |   81 +-
 .../cache/tier/sockets/BaseCommand.java         |   40 +-
 .../cache/tier/sockets/BaseCommandQuery.java    |   16 +
 .../cache/tier/sockets/CacheClientNotifier.java |   55 +-
 .../tier/sockets/CacheClientNotifierStats.java  |   21 +-
 .../cache/tier/sockets/CacheClientProxy.java    |   60 +-
 .../tier/sockets/CacheClientProxyStats.java     |   21 +-
 .../cache/tier/sockets/CacheClientUpdater.java  |   43 +-
 .../cache/tier/sockets/CacheServerHelper.java   |   21 +-
 .../cache/tier/sockets/CacheServerStats.java    |   21 +-
 .../cache/tier/sockets/ChunkedMessage.java      |   21 +-
 .../tier/sockets/ClientBlacklistProcessor.java  |   25 +-
 .../sockets/ClientDataSerializerMessage.java    |   21 +-
 .../cache/tier/sockets/ClientHealthMonitor.java |   21 +-
 .../tier/sockets/ClientInstantiatorMessage.java |   20 +-
 .../tier/sockets/ClientInterestMessageImpl.java |   21 +-
 .../tier/sockets/ClientMarkerMessageImpl.java   |   21 +-
 .../cache/tier/sockets/ClientMessage.java       |   21 +-
 .../tier/sockets/ClientPingMessageImpl.java     |   21 +-
 .../tier/sockets/ClientProxyMembershipID.java   |   32 +-
 .../tier/sockets/ClientTombstoneMessage.java    |   21 +-
 .../cache/tier/sockets/ClientUpdateMessage.java |   21 +-
 .../tier/sockets/ClientUpdateMessageImpl.java   |   21 +-
 .../cache/tier/sockets/ClientUserAuths.java     |   21 +-
 .../cache/tier/sockets/CommandInitializer.java  |   25 +-
 .../cache/tier/sockets/ConnectionListener.java  |   21 +-
 .../tier/sockets/ConnectionListenerAdapter.java |   21 +-
 .../cache/tier/sockets/HAEventWrapper.java      |   21 +-
 .../internal/cache/tier/sockets/HandShake.java  |   39 +-
 .../tier/sockets/InterestResultPolicyImpl.java  |   21 +-
 .../internal/cache/tier/sockets/Message.java    |   44 +-
 .../cache/tier/sockets/MessageStats.java        |   21 +-
 .../cache/tier/sockets/ObjectPartList.java      |   21 +-
 .../cache/tier/sockets/ObjectPartList651.java   |   21 +-
 .../internal/cache/tier/sockets/Part.java       |   25 +-
 .../RemoveClientFromBlacklistMessage.java       |   27 +-
 .../tier/sockets/SerializedObjectPartList.java  |   21 +-
 .../cache/tier/sockets/ServerConnection.java    |   31 +-
 .../tier/sockets/ServerHandShakeProcessor.java  |   21 +-
 .../cache/tier/sockets/ServerQueueStatus.java   |   21 +-
 .../tier/sockets/ServerResponseMatrix.java      |   20 +-
 .../tier/sockets/UnregisterAllInterest.java     |   21 +-
 .../cache/tier/sockets/UserAuthAttributes.java  |   21 +-
 .../cache/tier/sockets/VersionedObjectList.java |   21 +-
 .../cache/tier/sockets/command/AddPdxEnum.java  |   21 +-
 .../cache/tier/sockets/command/AddPdxType.java  |   21 +-
 .../cache/tier/sockets/command/ClearRegion.java |   21 +-
 .../cache/tier/sockets/command/ClientReady.java |   21 +-
 .../tier/sockets/command/CloseConnection.java   |   21 +-
 .../tier/sockets/command/CommitCommand.java     |   25 +-
 .../cache/tier/sockets/command/ContainsKey.java |   21 +-
 .../tier/sockets/command/ContainsKey66.java     |   21 +-
 .../tier/sockets/command/CreateRegion.java      |   21 +-
 .../cache/tier/sockets/command/Default.java     |   21 +-
 .../cache/tier/sockets/command/Destroy.java     |   21 +-
 .../cache/tier/sockets/command/Destroy65.java   |   21 +-
 .../cache/tier/sockets/command/Destroy70.java   |   23 +-
 .../tier/sockets/command/DestroyRegion.java     |   21 +-
 .../tier/sockets/command/ExecuteFunction.java   |   21 +-
 .../tier/sockets/command/ExecuteFunction65.java |   20 +-
 .../tier/sockets/command/ExecuteFunction66.java |   22 +-
 .../tier/sockets/command/ExecuteFunction70.java |   21 +-
 .../sockets/command/ExecuteRegionFunction.java  |   20 +-
 .../command/ExecuteRegionFunction61.java        |   20 +-
 .../command/ExecuteRegionFunction65.java        |   20 +-
 .../command/ExecuteRegionFunction66.java        |   22 +-
 .../command/ExecuteRegionFunctionSingleHop.java |   23 +-
 .../sockets/command/GatewayReceiverCommand.java |   23 +-
 .../cache/tier/sockets/command/Get70.java       |   23 +-
 .../cache/tier/sockets/command/GetAll.java      |   21 +-
 .../cache/tier/sockets/command/GetAll651.java   |   21 +-
 .../cache/tier/sockets/command/GetAll70.java    |   23 +-
 .../cache/tier/sockets/command/GetAllForRI.java |   21 +-
 .../sockets/command/GetAllWithCallback.java     |   21 +-
 .../command/GetClientPRMetadataCommand.java     |   20 +-
 .../command/GetClientPRMetadataCommand66.java   |   20 +-
 .../GetClientPartitionAttributesCommand.java    |   20 +-
 .../GetClientPartitionAttributesCommand66.java  |   20 +-
 .../cache/tier/sockets/command/GetEntry70.java  |   23 +-
 .../tier/sockets/command/GetEntryCommand.java   |   21 +-
 .../sockets/command/GetFunctionAttribute.java   |   21 +-
 .../tier/sockets/command/GetPDXEnumById.java    |   21 +-
 .../tier/sockets/command/GetPDXIdForEnum.java   |   21 +-
 .../tier/sockets/command/GetPDXIdForType.java   |   21 +-
 .../tier/sockets/command/GetPDXTypeById.java    |   21 +-
 .../tier/sockets/command/GetPdxEnums70.java     |   21 +-
 .../tier/sockets/command/GetPdxTypes70.java     |   21 +-
 .../cache/tier/sockets/command/Invalid.java     |   21 +-
 .../cache/tier/sockets/command/Invalidate.java  |   21 +-
 .../tier/sockets/command/Invalidate70.java      |   23 +-
 .../cache/tier/sockets/command/KeySet.java      |   21 +-
 .../cache/tier/sockets/command/MakePrimary.java |   21 +-
 .../tier/sockets/command/ManagementCommand.java |   21 +-
 .../cache/tier/sockets/command/PeriodicAck.java |   21 +-
 .../cache/tier/sockets/command/Ping.java        |   36 +-
 .../cache/tier/sockets/command/Put.java         |   23 +-
 .../cache/tier/sockets/command/Put61.java       |   21 +-
 .../cache/tier/sockets/command/Put65.java       |   21 +-
 .../cache/tier/sockets/command/Put70.java       |   21 +-
 .../cache/tier/sockets/command/PutAll.java      |   21 +-
 .../cache/tier/sockets/command/PutAll70.java    |   21 +-
 .../cache/tier/sockets/command/PutAll80.java    |   21 +-
 .../sockets/command/PutAllWithCallback.java     |   21 +-
 .../sockets/command/PutUserCredentials.java     |   21 +-
 .../cache/tier/sockets/command/Query.java       |   21 +-
 .../cache/tier/sockets/command/Query651.java    |   21 +-
 .../command/RegisterDataSerializers.java        |   21 +-
 .../sockets/command/RegisterInstantiators.java  |   21 +-
 .../tier/sockets/command/RegisterInterest.java  |   23 +-
 .../sockets/command/RegisterInterest61.java     |   23 +-
 .../sockets/command/RegisterInterestList.java   |   23 +-
 .../sockets/command/RegisterInterestList61.java |   23 +-
 .../sockets/command/RegisterInterestList66.java |   23 +-
 .../cache/tier/sockets/command/RemoveAll.java   |   21 +-
 .../tier/sockets/command/RemoveUserAuth.java    |   21 +-
 .../cache/tier/sockets/command/Request.java     |   23 +-
 .../tier/sockets/command/RequestEventValue.java |   20 +-
 .../tier/sockets/command/RollbackCommand.java   |   21 +-
 .../cache/tier/sockets/command/Size.java        |   21 +-
 .../tier/sockets/command/TXFailoverCommand.java |   21 +-
 .../command/TXSynchronizationCommand.java       |   21 +-
 .../sockets/command/UnregisterInterest.java     |   23 +-
 .../sockets/command/UnregisterInterestList.java |   23 +-
 .../command/UpdateClientNotification.java       |   21 +-
 .../doc-files/communication-architecture.fig    |    4 +-
 .../internal/cache/tier/sockets/package.html    |   16 +
 .../cache/tx/AbstractPeerTXRegionStub.java      |   21 +-
 .../internal/cache/tx/ClientTXRegionStub.java   |   21 +-
 .../internal/cache/tx/ClientTXStateStub.java    |   21 +-
 .../cache/tx/DistClientTXStateStub.java         |   19 +-
 .../internal/cache/tx/DistTxEntryEvent.java     |   16 +
 .../internal/cache/tx/DistTxKeyInfo.java        |   18 +-
 .../cache/tx/DistributedTXRegionStub.java       |   21 +-
 .../cache/tx/PartitionedTXRegionStub.java       |   21 +-
 .../gemfire/internal/cache/tx/TXRegionStub.java |   21 +-
 .../cache/tx/TransactionalOperation.java        |   24 +-
 .../cache/versions/CompactVersionHolder.java    |   21 +-
 .../ConcurrentCacheModificationException.java   |   23 +-
 .../cache/versions/DiskRegionVersionVector.java |   21 +-
 .../internal/cache/versions/DiskVersionTag.java |   21 +-
 .../internal/cache/versions/RVVException.java   |   21 +-
 .../internal/cache/versions/RVVExceptionB.java  |   23 +-
 .../internal/cache/versions/RVVExceptionT.java  |   23 +-
 .../cache/versions/RegionVersionHolder.java     |   21 +-
 .../cache/versions/RegionVersionVector.java     |   27 +-
 .../cache/versions/VMRegionVersionVector.java   |   21 +-
 .../internal/cache/versions/VMVersionTag.java   |   21 +-
 .../internal/cache/versions/VersionHolder.java  |   21 +-
 .../internal/cache/versions/VersionSource.java  |   21 +-
 .../internal/cache/versions/VersionStamp.java   |   21 +-
 .../internal/cache/versions/VersionTag.java     |   21 +-
 .../internal/cache/vmotion/VMotionObserver.java |   20 +-
 .../cache/vmotion/VMotionObserverAdapter.java   |   20 +-
 .../cache/vmotion/VMotionObserverHolder.java    |   20 +-
 .../cache/wan/AbstractGatewaySender.java        |   50 +-
 .../AbstractGatewaySenderEventProcessor.java    |   21 +-
 .../AsyncEventQueueConfigurationException.java  |   21 +-
 .../internal/cache/wan/BatchException70.java    |   21 +-
 .../cache/wan/DistributedSystemListener.java    |   16 +
 .../cache/wan/GatewayEventFilterImpl.java       |   21 +-
 .../cache/wan/GatewayReceiverException.java     |   21 +-
 .../cache/wan/GatewayReceiverStats.java         |   21 +-
 .../cache/wan/GatewaySenderAdvisor.java         |   21 +-
 .../cache/wan/GatewaySenderAttributes.java      |   26 +-
 .../GatewaySenderConfigurationException.java    |   21 +-
 .../wan/GatewaySenderEventCallbackArgument.java |   20 +-
 .../GatewaySenderEventCallbackDispatcher.java   |   22 +-
 .../cache/wan/GatewaySenderEventDispatcher.java |   21 +-
 .../cache/wan/GatewaySenderEventImpl.java       |   30 +-
 .../cache/wan/GatewaySenderException.java       |   21 +-
 .../internal/cache/wan/GatewaySenderStats.java  |   21 +-
 .../cache/wan/InternalGatewaySenderFactory.java |   16 +
 .../cache/wan/TransportFilterServerSocket.java  |   21 +-
 .../cache/wan/TransportFilterSocket.java        |   21 +-
 .../cache/wan/TransportFilterSocketFactory.java |   21 +-
 .../internal/cache/wan/WANServiceProvider.java  |   16 +
 .../BucketRegionQueueUnavailableException.java  |   16 +
 ...rentParallelGatewaySenderEventProcessor.java |   23 +-
 .../ConcurrentParallelGatewaySenderQueue.java   |   21 +-
 .../ParallelGatewaySenderEventProcessor.java    |   23 +-
 .../parallel/ParallelGatewaySenderQueue.java    |   84 +-
 .../ParallelQueueBatchRemovalMessage.java       |   21 +-
 .../parallel/ParallelQueueRemovalMessage.java   |   21 +-
 .../cache/wan/parallel/RREventIDResolver.java   |   21 +-
 .../cache/wan/serial/BatchDestroyOperation.java |   21 +-
 ...urrentSerialGatewaySenderEventProcessor.java |   21 +-
 .../SerialGatewaySenderEventProcessor.java      |   24 +-
 .../wan/serial/SerialGatewaySenderQueue.java    |   23 +-
 .../serial/SerialSecondaryGatewayListener.java  |   21 +-
 .../internal/cache/wan/spi/WANFactory.java      |   16 +
 .../cache/xmlcache/AbstractXmlParser.java       |   21 +-
 .../cache/xmlcache/AsyncEventQueueCreation.java |   21 +-
 .../cache/xmlcache/BindingCreation.java         |   21 +-
 .../cache/xmlcache/BridgeServerCreation.java    |  249 -
 .../internal/cache/xmlcache/CacheCreation.java  |  168 +-
 .../cache/xmlcache/CacheServerCreation.java     |  257 +
 .../CacheTransactionManagerCreation.java        |   21 +-
 .../internal/cache/xmlcache/CacheXml.java       |   21 +-
 .../cache/xmlcache/CacheXmlGenerator.java       |   66 +-
 .../internal/cache/xmlcache/CacheXmlParser.java |   45 +-
 .../xmlcache/CacheXmlPropertyResolver.java      |   21 +-
 .../CacheXmlPropertyResolverHelper.java         |   21 +-
 .../cache/xmlcache/CacheXmlVersion.java         |   17 +-
 .../cache/xmlcache/ClientCacheCreation.java     |   26 +-
 .../cache/xmlcache/ClientHaQueueCreation.java   |   21 +-
 .../internal/cache/xmlcache/Declarable2.java    |   21 +-
 .../cache/xmlcache/DefaultEntityResolver2.java  |   23 +-
 .../xmlcache/DiskStoreAttributesCreation.java   |   21 +-
 .../cache/xmlcache/FunctionServiceCreation.java |   21 +-
 .../cache/xmlcache/GatewayReceiverCreation.java |   21 +-
 .../cache/xmlcache/GeodeEntityResolver.java     |   58 +
 .../cache/xmlcache/IndexCreationData.java       |   21 +-
 .../ParallelAsyncEventQueueCreation.java        |   21 +-
 .../xmlcache/ParallelGatewaySenderCreation.java |   21 +-
 .../cache/xmlcache/PivotalEntityResolver.java   |   23 +-
 .../cache/xmlcache/PropertyResolver.java        |   21 +-
 .../xmlcache/RegionAttributesCreation.java      |   30 +-
 .../internal/cache/xmlcache/RegionCreation.java |   21 +-
 .../cache/xmlcache/ResourceManagerCreation.java |   21 +-
 .../xmlcache/SerialAsyncEventQueueCreation.java |   21 +-
 .../xmlcache/SerialGatewaySenderCreation.java   |   21 +-
 .../cache/xmlcache/SerializerCreation.java      |   21 +-
 .../internal/cache/xmlcache/XmlGenerator.java   |   21 +-
 .../cache/xmlcache/XmlGeneratorUtils.java       |   21 +-
 .../internal/cache/xmlcache/XmlParser.java      |   21 +-
 .../internal/cache/xmlcache/package.html        |   16 +
 .../gemfire/internal/concurrent/AL.java         |   21 +-
 .../internal/concurrent/AtomicLong5.java        |   21 +-
 .../gemfire/internal/concurrent/Atomics.java    |   21 +-
 .../concurrent/CompactConcurrentHashSet2.java   |   16 +
 .../internal/concurrent/ConcurrentHashSet.java  |   21 +-
 .../gemfire/internal/concurrent/LI.java         |   21 +-
 .../internal/concurrent/MapCallback.java        |   21 +-
 .../internal/concurrent/MapCallbackAdapter.java |   21 +-
 .../gemfire/internal/concurrent/MapResult.java  |   21 +-
 .../internal/datasource/AbstractDataSource.java |   21 +-
 .../internal/datasource/AbstractPoolCache.java  |   25 +-
 .../ClientConnectionFactoryWrapper.java         |   21 +-
 .../internal/datasource/ConfigProperty.java     |   21 +-
 .../ConfiguredDataSourceProperties.java         |   21 +-
 .../ConnectionEventListenerAdaptor.java         |   21 +-
 .../datasource/ConnectionPoolCache.java         |   21 +-
 .../datasource/ConnectionPoolCacheImpl.java     |   21 +-
 .../internal/datasource/ConnectionProvider.java |   21 +-
 .../datasource/ConnectionProviderException.java |   21 +-
 .../datasource/DataSourceCreateException.java   |   21 +-
 .../internal/datasource/DataSourceFactory.java  |   21 +-
 .../datasource/DataSourceResources.java         |   21 +-
 .../FacetsJCAConnectionManagerImpl.java         |   21 +-
 .../datasource/GemFireBasicDataSource.java      |   23 +-
 .../datasource/GemFireConnPooledDataSource.java |   23 +-
 .../GemFireConnectionPoolManager.java           |   21 +-
 .../GemFireTransactionDataSource.java           |   21 +-
 .../datasource/JCAConnectionManagerImpl.java    |   21 +-
 .../datasource/ManagedPoolCacheImpl.java        |   21 +-
 .../internal/datasource/PoolException.java      |   21 +-
 .../internal/datasource/TranxPoolCacheImpl.java |   21 +-
 .../i18n/AbstractStringIdResourceBundle.java    |   26 +-
 .../gemfire/internal/i18n/LocalizedStrings.java | 4008 +++++++--------
 .../internal/i18n/ParentLocalizedStrings.java   | 4595 +++++++++---------
 .../gemfire/internal/i18n/StringId.java         |   58 +
 .../internal/io/CompositeOutputStream.java      |   21 +-
 .../internal/io/CompositePrintStream.java       |   21 +-
 .../gemfire/internal/io/TeeOutputStream.java    |   21 +-
 .../gemfire/internal/io/TeePrintStream.java     |   21 +-
 .../gemfire/internal/jndi/ContextImpl.java      |   21 +-
 .../jndi/InitialContextFactoryImpl.java         |   21 +-
 .../gemfire/internal/jndi/JNDIInvoker.java      |   21 +-
 .../gemfire/internal/jndi/NameParserImpl.java   |   21 +-
 .../gemfire/internal/jta/GlobalTransaction.java |   21 +-
 .../gemfire/internal/jta/TransactionImpl.java   |   21 +-
 .../internal/jta/TransactionManagerImpl.java    |   21 +-
 .../gemfire/internal/jta/TransactionUtils.java  |   21 +-
 .../internal/jta/UserTransactionImpl.java       |   21 +-
 .../gemstone/gemfire/internal/jta/XidImpl.java  |   21 +-
 .../gemfire/internal/lang/ClassUtils.java       |   20 +-
 .../gemstone/gemfire/internal/lang/Filter.java  |   18 +-
 .../gemfire/internal/lang/InOutParameter.java   |   20 +-
 .../gemfire/internal/lang/Initable.java         |   20 +-
 .../gemfire/internal/lang/Initializer.java      |   21 +-
 .../internal/lang/MutableIdentifiable.java      |   24 +-
 .../gemfire/internal/lang/ObjectUtils.java      |   20 +-
 .../gemfire/internal/lang/Orderable.java        |   20 +-
 .../gemstone/gemfire/internal/lang/Ordered.java |   20 +-
 .../gemfire/internal/lang/StringUtils.java      |   20 +-
 .../gemfire/internal/lang/SystemUtils.java      |   55 +-
 .../gemfire/internal/lang/ThreadUtils.java      |   20 +-
 .../gemfire/internal/logging/DateFormatter.java |   16 +
 .../internal/logging/DebugLogWriter.java        |   25 +-
 .../internal/logging/GemFireFormatter.java      |   21 +-
 .../internal/logging/GemFireHandler.java        |   21 +-
 .../gemfire/internal/logging/GemFireLevel.java  |   21 +-
 .../internal/logging/InternalLogWriter.java     |   21 +-
 .../internal/logging/LocalLogWriter.java        |   21 +-
 .../gemfire/internal/logging/LogConfig.java     |   16 +
 .../gemfire/internal/logging/LogFileParser.java |   29 +-
 .../gemfire/internal/logging/LogService.java    |  142 +-
 .../internal/logging/LogWriterFactory.java      |   16 +
 .../gemfire/internal/logging/LogWriterImpl.java |   23 +-
 .../internal/logging/LoggingThreadGroup.java    |   18 +-
 .../internal/logging/ManagerLogWriter.java      |   21 +-
 .../gemfire/internal/logging/MergeLogFiles.java |   21 +-
 .../gemfire/internal/logging/PureLogWriter.java |   23 +-
 .../logging/SecurityLocalLogWriter.java         |   23 +-
 .../internal/logging/SecurityLogConfig.java     |   16 +
 .../internal/logging/SecurityLogWriter.java     |   23 +-
 .../logging/SecurityManagerLogWriter.java       |   23 +-
 .../gemfire/internal/logging/SortLogFile.java   |   21 +-
 .../internal/logging/StandardErrorPrinter.java  |   16 +
 .../internal/logging/StandardOutputPrinter.java |   16 +
 .../internal/logging/log4j/AlertAppender.java   |   31 +-
 .../internal/logging/log4j/AppenderContext.java |   16 +
 .../internal/logging/log4j/ConfigLocator.java   |   16 +
 .../internal/logging/log4j/Configurator.java    |   29 +
 .../internal/logging/log4j/FastLogger.java      |   16 +
 .../internal/logging/log4j/GemFireLogger.java   |   16 +
 .../logging/log4j/LocalizedMessage.java         |   19 +-
 .../internal/logging/log4j/LogMarker.java       |   17 +-
 .../logging/log4j/LogWriterAppender.java        |   18 +-
 .../logging/log4j/LogWriterAppenders.java       |   16 +
 .../internal/logging/log4j/LogWriterLogger.java |   18 +-
 .../logging/log4j/ThreadIdPatternConverter.java |   16 +
 .../gemfire/internal/memcached/Command.java     |   21 +-
 .../internal/memcached/CommandProcessor.java    |   21 +-
 .../internal/memcached/ConnectionHandler.java   |   21 +-
 .../gemfire/internal/memcached/KeyWrapper.java  |   21 +-
 .../gemfire/internal/memcached/Reply.java       |   21 +-
 .../internal/memcached/RequestReader.java       |   21 +-
 .../internal/memcached/ResponseStatus.java      |   21 +-
 .../internal/memcached/ValueWrapper.java        |   21 +-
 .../memcached/commands/AbstractCommand.java     |   21 +-
 .../internal/memcached/commands/AddCommand.java |   21 +-
 .../memcached/commands/AddQCommand.java         |   16 +
 .../memcached/commands/AppendCommand.java       |   21 +-
 .../memcached/commands/AppendQCommand.java      |   16 +
 .../internal/memcached/commands/CASCommand.java |   21 +-
 .../memcached/commands/ClientError.java         |   21 +-
 .../memcached/commands/DecrementCommand.java    |   21 +-
 .../memcached/commands/DecrementQCommand.java   |   16 +
 .../memcached/commands/DeleteCommand.java       |   21 +-
 .../memcached/commands/DeleteQCommand.java      |   16 +
 .../memcached/commands/FlushAllCommand.java     |   21 +-
 .../memcached/commands/FlushAllQCommand.java    |   16 +
 .../internal/memcached/commands/GATCommand.java |   16 +
 .../memcached/commands/GATQCommand.java         |   16 +
 .../internal/memcached/commands/GetCommand.java |   28 +-
 .../memcached/commands/GetKCommand.java         |   16 +
 .../memcached/commands/GetKQCommand.java        |   16 +
 .../memcached/commands/GetQCommand.java         |   21 +-
 .../memcached/commands/IncrementCommand.java    |   21 +-
 .../memcached/commands/IncrementQCommand.java   |   16 +
 .../memcached/commands/NoOpCommand.java         |   21 +-
 .../memcached/commands/NotSupportedCommand.java |   21 +-
 .../memcached/commands/PrependCommand.java      |   21 +-
 .../memcached/commands/PrependQCommand.java     |   16 +
 .../memcached/commands/QuitCommand.java         |   21 +-
 .../memcached/commands/QuitQCommand.java        |   16 +
 .../memcached/commands/ReplaceCommand.java      |   21 +-
 .../memcached/commands/ReplaceQCommand.java     |   16 +
 .../internal/memcached/commands/SetCommand.java |   21 +-
 .../memcached/commands/SetQCommand.java         |   16 +
 .../memcached/commands/StatsCommand.java        |   21 +-
 .../memcached/commands/StorageCommand.java      |   21 +-
 .../memcached/commands/TouchCommand.java        |   21 +-
 .../memcached/commands/VerbosityCommand.java    |   21 +-
 .../memcached/commands/VersionCommand.java      |   21 +-
 .../modules/util/RegionConfiguration.java       |   21 +-
 .../gemfire/internal/net/SocketUtils.java       |   20 +-
 .../internal/offheap/AbstractStoredObject.java  |  107 +
 .../internal/offheap/ByteArrayMemoryChunk.java  |   16 +
 .../internal/offheap/ByteBufferMemoryChunk.java |   16 +
 .../gemfire/internal/offheap/Chunk.java         |  792 +++
 .../gemfire/internal/offheap/ChunkFactory.java  |   51 +
 .../gemfire/internal/offheap/ChunkType.java     |   30 +
 .../internal/offheap/ChunkWithHeapForm.java     |   40 +
 .../gemfire/internal/offheap/DataAsAddress.java |  120 +
 .../gemfire/internal/offheap/DataType.java      |   16 +
 .../gemfire/internal/offheap/Fragment.java      |  139 +
 .../internal/offheap/FreeListManager.java       |  821 ++++
 .../gemfire/internal/offheap/GemFireChunk.java  |   47 +
 .../internal/offheap/GemFireChunkFactory.java   |   52 +
 .../internal/offheap/GemFireChunkSlice.java     |   44 +
 .../internal/offheap/LifecycleListener.java     |   98 +
 .../internal/offheap/MemoryAllocator.java       |   19 +-
 .../gemfire/internal/offheap/MemoryBlock.java   |   17 +-
 .../internal/offheap/MemoryBlockNode.java       |  170 +
 .../gemfire/internal/offheap/MemoryChunk.java   |   16 +
 .../offheap/MemoryChunkWithRefCount.java        |   16 +
 .../internal/offheap/MemoryInspector.java       |   22 +-
 .../internal/offheap/MemoryUsageListener.java   |   16 +
 .../offheap/OffHeapCachedDeserializable.java    |   82 +-
 .../gemfire/internal/offheap/OffHeapHelper.java |   28 +-
 .../internal/offheap/OffHeapMemoryStats.java    |   16 +
 .../internal/offheap/OffHeapReference.java      |   56 -
 .../offheap/OffHeapRegionEntryHelper.java       |   24 +-
 .../internal/offheap/OffHeapStorage.java        |   46 +-
 .../offheap/OutOfOffHeapMemoryListener.java     |   16 +
 .../internal/offheap/RefCountChangeInfo.java    |  112 +
 .../internal/offheap/ReferenceCountHelper.java  |  252 +
 .../gemfire/internal/offheap/Releasable.java    |   16 +
 .../offheap/SimpleMemoryAllocatorImpl.java      | 3602 +-------------
 .../gemfire/internal/offheap/StoredObject.java  |   82 +-
 .../internal/offheap/SyncChunkStack.java        |  130 +
 .../internal/offheap/UnsafeMemoryChunk.java     |   26 +
 .../offheap/annotations/OffHeapIdentifier.java  |   16 +
 .../internal/offheap/annotations/Released.java  |   16 +
 .../internal/offheap/annotations/Retained.java  |   16 +
 .../offheap/annotations/Unretained.java         |   16 +
 .../com/gemstone/gemfire/internal/package.html  |   16 +
 .../internal/process/AttachProcessUtils.java    |   16 +
 .../process/BlockingProcessStreamReader.java    |   16 +
 ...usterConfigurationNotAvailableException.java |   16 +
 .../process/ConnectionFailedException.java      |   21 +-
 .../internal/process/ControlFileWatchdog.java   |   16 +
 .../process/ControlNotificationHandler.java     |   16 +
 .../internal/process/ControllableProcess.java   |   16 +
 .../process/FileAlreadyExistsException.java     |   21 +-
 .../process/FileControllerParameters.java       |   16 +
 .../internal/process/FileProcessController.java |   16 +
 .../process/LocalProcessController.java         |   21 +-
 .../internal/process/LocalProcessLauncher.java  |   21 +-
 .../process/MBeanControllerParameters.java      |   16 +
 .../process/MBeanInvocationFailedException.java |   21 +-
 .../process/MBeanProcessController.java         |   16 +
 .../internal/process/NativeProcessUtils.java    |   16 +
 .../process/NonBlockingProcessStreamReader.java |   16 +
 .../gemfire/internal/process/PidFile.java       |   16 +
 .../process/PidUnavailableException.java        |   21 +-
 .../internal/process/ProcessController.java     |   16 +
 .../process/ProcessControllerFactory.java       |   16 +
 .../process/ProcessControllerParameters.java    |   16 +
 .../process/ProcessLauncherContext.java         |   21 +-
 .../internal/process/ProcessStreamReader.java   |   21 +-
 .../ProcessTerminatedAbnormallyException.java   |   20 +-
 .../gemfire/internal/process/ProcessType.java   |   16 +
 .../gemfire/internal/process/ProcessUtils.java  |   21 +-
 .../gemfire/internal/process/StartupStatus.java |   18 +-
 .../internal/process/StartupStatusListener.java |   16 +
 .../UnableToControlProcessException.java        |   16 +
 .../AbstractSignalNotificationHandler.java      |   20 +-
 .../gemfire/internal/process/signal/Signal.java |   20 +-
 .../internal/process/signal/SignalEvent.java    |   20 +-
 .../internal/process/signal/SignalListener.java |   20 +-
 .../internal/process/signal/SignalType.java     |   20 +-
 .../internal/redis/ByteArrayWrapper.java        |   16 +
 .../internal/redis/ByteToCommandDecoder.java    |   16 +
 .../gemstone/gemfire/internal/redis/Coder.java  |  210 +-
 .../gemfire/internal/redis/Command.java         |   16 +
 .../gemfire/internal/redis/DoubleWrapper.java   |   16 +
 .../internal/redis/ExecutionHandlerContext.java |   19 +-
 .../gemfire/internal/redis/Executor.java        |   16 +
 .../gemfire/internal/redis/Extendable.java      |   16 +
 .../redis/RedisCommandParserException.java      |   16 +
 .../internal/redis/RedisCommandType.java        |   16 +
 .../gemfire/internal/redis/RedisConstants.java  |   16 +
 .../gemfire/internal/redis/RedisDataType.java   |   18 +-
 .../redis/RedisDataTypeMismatchException.java   |   16 +
 .../internal/redis/RegionCreationException.java |   16 +
 .../gemfire/internal/redis/RegionProvider.java  |   71 +-
 .../redis/executor/AbstractExecutor.java        |   16 +
 .../redis/executor/AbstractScanExecutor.java    |   16 +
 .../internal/redis/executor/AuthExecutor.java   |   16 +
 .../internal/redis/executor/DBSizeExecutor.java |   16 +
 .../internal/redis/executor/DelExecutor.java    |   16 +
 .../internal/redis/executor/EchoExecutor.java   |   16 +
 .../internal/redis/executor/ExistsExecutor.java |   16 +
 .../redis/executor/ExpirationExecutor.java      |   16 +
 .../redis/executor/ExpireAtExecutor.java        |   16 +
 .../internal/redis/executor/ExpireExecutor.java |   16 +
 .../redis/executor/FlushAllExecutor.java        |   16 +
 .../internal/redis/executor/KeysExecutor.java   |   16 +
 .../internal/redis/executor/ListQuery.java      |   16 +
 .../redis/executor/PExpireAtExecutor.java       |   16 +
 .../redis/executor/PExpireExecutor.java         |   16 +
 .../internal/redis/executor/PTTLExecutor.java   |   16 +
 .../redis/executor/PersistExecutor.java         |   16 +
 .../internal/redis/executor/PingExecutor.java   |   16 +
 .../internal/redis/executor/QuitExecutor.java   |   16 +
 .../internal/redis/executor/ScanExecutor.java   |   16 +
 .../redis/executor/ShutDownExecutor.java        |   16 +
 .../internal/redis/executor/SortedSetQuery.java |   16 +
 .../internal/redis/executor/TTLExecutor.java    |   16 +
 .../internal/redis/executor/TimeExecutor.java   |   16 +
 .../internal/redis/executor/TypeExecutor.java   |   16 +
 .../internal/redis/executor/UnkownExecutor.java |   16 +
 .../redis/executor/hash/HDelExecutor.java       |   16 +
 .../redis/executor/hash/HExistsExecutor.java    |   16 +
 .../redis/executor/hash/HGetAllExecutor.java    |   16 +
 .../redis/executor/hash/HGetExecutor.java       |   16 +
 .../redis/executor/hash/HIncrByExecutor.java    |   16 +
 .../executor/hash/HIncrByFloatExecutor.java     |   16 +
 .../redis/executor/hash/HKeysExecutor.java      |   16 +
 .../redis/executor/hash/HLenExecutor.java       |   16 +
 .../redis/executor/hash/HMGetExecutor.java      |   16 +
 .../redis/executor/hash/HMSetExecutor.java      |   16 +
 .../redis/executor/hash/HScanExecutor.java      |   16 +
 .../redis/executor/hash/HSetExecutor.java       |   16 +
 .../redis/executor/hash/HSetNXExecutor.java     |   16 +
 .../redis/executor/hash/HValsExecutor.java      |   16 +
 .../redis/executor/hash/HashExecutor.java       |   18 +-
 .../internal/redis/executor/hll/Bits.java       |   16 +
 .../executor/hll/CardinalityMergeException.java |   18 +-
 .../redis/executor/hll/HllExecutor.java         |   16 +
 .../redis/executor/hll/HyperLogLog.java         |   16 +
 .../redis/executor/hll/HyperLogLogPlus.java     |   18 +-
 .../internal/redis/executor/hll/IBuilder.java   |   18 +-
 .../redis/executor/hll/ICardinality.java        |   16 +
 .../internal/redis/executor/hll/MurmurHash.java |   18 +-
 .../redis/executor/hll/PFAddExecutor.java       |   16 +
 .../redis/executor/hll/PFCountExecutor.java     |   16 +
 .../redis/executor/hll/PFMergeExecutor.java     |   16 +
 .../redis/executor/hll/RegisterSet.java         |   18 +-
 .../internal/redis/executor/hll/Varint.java     |   18 +-
 .../redis/executor/list/LIndexExecutor.java     |   16 +
 .../redis/executor/list/LInsertExecutor.java    |   16 +
 .../redis/executor/list/LLenExecutor.java       |   16 +
 .../redis/executor/list/LPopExecutor.java       |   16 +
 .../redis/executor/list/LPushExecutor.java      |   16 +
 .../redis/executor/list/LPushXExecutor.java     |   16 +
 .../redis/executor/list/LRangeExecutor.java     |   16 +
 .../redis/executor/list/LRemExecutor.java       |   16 +
 .../redis/executor/list/LSetExecutor.java       |   16 +
 .../redis/executor/list/LTrimExecutor.java      |   16 +
 .../redis/executor/list/ListExecutor.java       |   16 +
 .../redis/executor/list/PopExecutor.java        |   16 +
 .../redis/executor/list/PushExecutor.java       |   16 +
 .../redis/executor/list/PushXExecutor.java      |   16 +
 .../redis/executor/list/RPopExecutor.java       |   16 +
 .../redis/executor/list/RPushExecutor.java      |   16 +
 .../redis/executor/list/RPushXExecutor.java     |   16 +
 .../redis/executor/set/SAddExecutor.java        |   16 +
 .../redis/executor/set/SCardExecutor.java       |   16 +
 .../redis/executor/set/SDiffExecutor.java       |   16 +
 .../redis/executor/set/SDiffStoreExecutor.java  |   16 +
 .../redis/executor/set/SInterExecutor.java      |   16 +
 .../redis/executor/set/SInterStoreExecutor.java |   16 +
 .../redis/executor/set/SIsMemberExecutor.java   |   16 +
 .../redis/executor/set/SMembersExecutor.java    |   16 +
 .../redis/executor/set/SMoveExecutor.java       |   16 +
 .../redis/executor/set/SPopExecutor.java        |   16 +
 .../redis/executor/set/SRandMemberExecutor.java |   16 +
 .../redis/executor/set/SRemExecutor.java        |   16 +
 .../redis/executor/set/SScanExecutor.java       |   16 +
 .../redis/executor/set/SUnionExecutor.java      |   16 +
 .../redis/executor/set/SUnionStoreExecutor.java |   16 +
 .../redis/executor/set/SetExecutor.java         |   16 +
 .../redis/executor/set/SetOpExecutor.java       |   16 +
 .../executor/sortedset/SortedSetExecutor.java   |   16 +
 .../redis/executor/sortedset/ZAddExecutor.java  |   16 +
 .../redis/executor/sortedset/ZCardExecutor.java |   16 +
 .../executor/sortedset/ZCountExecutor.java      |   16 +
 .../executor/sortedset/ZIncrByExecutor.java     |   16 +
 .../executor/sortedset/ZLexCountExecutor.java   |   16 +
 .../executor/sortedset/ZRangeByLexExecutor.java |   16 +
 .../sortedset/ZRangeByScoreExecutor.java        |   16 +
 .../executor/sortedset/ZRangeExecutor.java      |   16 +
 .../redis/executor/sortedset/ZRankExecutor.java |   16 +
 .../redis/executor/sortedset/ZRemExecutor.java  |   16 +
 .../sortedset/ZRemRangeByLexExecutor.java       |   16 +
 .../sortedset/ZRemRangeByRankExecutor.java      |   16 +
 .../sortedset/ZRemRangeByScoreExecutor.java     |   16 +
 .../sortedset/ZRevRangeByScoreExecutor.java     |   16 +
 .../executor/sortedset/ZRevRangeExecutor.java   |   16 +
 .../executor/sortedset/ZRevRankExecutor.java    |   16 +
 .../redis/executor/sortedset/ZScanExecutor.java |   16 +
 .../executor/sortedset/ZScoreExecutor.java      |   16 +
 .../redis/executor/string/AppendExecutor.java   |   16 +
 .../redis/executor/string/BitCountExecutor.java |   16 +
 .../redis/executor/string/BitOpExecutor.java    |   16 +
 .../redis/executor/string/BitPosExecutor.java   |   16 +
 .../redis/executor/string/DecrByExecutor.java   |   16 +
 .../redis/executor/string/DecrExecutor.java     |   16 +
 .../redis/executor/string/GetBitExecutor.java   |   16 +
 .../redis/executor/string/GetExecutor.java      |   16 +
 .../redis/executor/string/GetRangeExecutor.java |   16 +
 .../redis/executor/string/GetSetExecutor.java   |   16 +
 .../redis/executor/string/IncrByExecutor.java   |   16 +
 .../executor/string/IncrByFloatExecutor.java    |   16 +
 .../redis/executor/string/IncrExecutor.java     |   16 +
 .../redis/executor/string/MGetExecutor.java     |   16 +
 .../redis/executor/string/MSetExecutor.java     |   16 +
 .../redis/executor/string/MSetNXExecutor.java   |   16 +
 .../redis/executor/string/PSetEXExecutor.java   |   16 +
 .../redis/executor/string/SetBitExecutor.java   |   16 +
 .../redis/executor/string/SetEXExecutor.java    |   16 +
 .../redis/executor/string/SetExecutor.java      |   16 +
 .../redis/executor/string/SetNXExecutor.java    |   16 +
 .../redis/executor/string/SetRangeExecutor.java |   16 +
 .../redis/executor/string/StringExecutor.java   |   18 +-
 .../redis/executor/string/StrlenExecutor.java   |   16 +
 .../executor/transactions/DiscardExecutor.java  |   16 +
 .../executor/transactions/ExecExecutor.java     |   16 +
 .../executor/transactions/MultiExecutor.java    |   16 +
 .../transactions/TransactionExecutor.java       |   16 +
 .../executor/transactions/UnwatchExecutor.java  |   16 +
 .../executor/transactions/WatchExecutor.java    |   16 +
 .../internal/security/AuthorizeRequest.java     |   21 +-
 .../internal/security/AuthorizeRequestPP.java   |   21 +-
 .../security/FilterPostAuthorization.java       |   21 +-
 .../security/FilterPreAuthorization.java        |   21 +-
 .../internal/security/ObjectWithAuthz.java      |   21 +-
 .../gemfire/internal/security/package.html      |   16 +
 .../internal/sequencelog/EntryLogger.java       |   23 +-
 .../gemfire/internal/sequencelog/GraphType.java |   21 +-
 .../internal/sequencelog/MembershipLogger.java  |   21 +-
 .../internal/sequencelog/MessageLogger.java     |   21 +-
 .../internal/sequencelog/RegionLogger.java      |   21 +-
 .../internal/sequencelog/SequenceLogger.java    |   21 +-
 .../sequencelog/SequenceLoggerImpl.java         |   21 +-
 .../internal/sequencelog/Transition.java        |   21 +-
 .../gemfire/internal/sequencelog/io/Filter.java |   21 +-
 .../sequencelog/io/GemfireLogConverter.java     |   21 +-
 .../internal/sequencelog/io/GraphReader.java    |   21 +-
 .../sequencelog/io/InputStreamReader.java       |   21 +-
 .../sequencelog/io/OutputStreamAppender.java    |   21 +-
 .../internal/sequencelog/model/Edge.java        |   23 +-
 .../internal/sequencelog/model/Graph.java       |   21 +-
 .../internal/sequencelog/model/GraphID.java     |   21 +-
 .../sequencelog/model/GraphReaderCallback.java  |   23 +-
 .../internal/sequencelog/model/GraphSet.java    |   21 +-
 .../internal/sequencelog/model/Vertex.java      |   21 +-
 .../visualization/text/TextDisplay.java         |   21 +-
 .../gemfire/internal/shared/NativeCalls.java    |   21 +-
 .../internal/shared/NativeCallsJNAImpl.java     |   21 +-
 .../internal/shared/NativeErrorException.java   |   21 +-
 .../gemfire/internal/shared/OSType.java         |   21 +-
 .../internal/shared/StringPrintWriter.java      |   21 +-
 .../internal/shared/TCPSocketOptions.java       |   21 +-
 .../internal/size/CachingSingleObjectSizer.java |   21 +-
 .../size/InstrumentationSingleObjectSizer.java  |   21 +-
 .../gemfire/internal/size/ObjectGraphSizer.java |   28 +-
 .../gemfire/internal/size/ObjectTraverser.java  |   28 +-
 .../internal/size/ReflectionObjectSizer.java    |   21 +-
 .../size/ReflectionSingleObjectSizer.java       |   25 +-
 .../internal/size/SingleObjectSizer.java        |   16 +
 .../internal/size/SizeClassOnceObjectSizer.java |   21 +-
 .../gemfire/internal/size/SizeOfUtil0.java      |   21 +-
 .../internal/size/WellKnownClassSizer.java      |   21 +-
 .../internal/statistics/CounterMonitor.java     |   21 +-
 .../internal/statistics/GaugeMonitor.java       |   21 +-
 .../statistics/IgnoreResourceException.java     |   21 +-
 .../MapBasedStatisticsNotification.java         |   21 +-
 .../internal/statistics/ResourceInstance.java   |   21 +-
 .../internal/statistics/ResourceType.java       |   21 +-
 .../internal/statistics/SampleCollector.java    |   25 +-
 .../internal/statistics/SampleHandler.java      |   21 +-
 .../internal/statistics/SimpleStatisticId.java  |   21 +-
 .../statistics/StatArchiveDescriptor.java       |   21 +-
 .../internal/statistics/StatArchiveHandler.java |   21 +-
 .../statistics/StatArchiveHandlerConfig.java    |   21 +-
 .../internal/statistics/StatMonitorHandler.java |   21 +-
 .../internal/statistics/StatisticId.java        |   21 +-
 .../statistics/StatisticNotFoundException.java  |   21 +-
 .../internal/statistics/StatisticsListener.java |   23 +-
 .../internal/statistics/StatisticsMonitor.java  |   21 +-
 .../statistics/StatisticsNotification.java      |   21 +-
 .../internal/statistics/StatisticsSampler.java  |   21 +-
 .../internal/statistics/ValueMonitor.java       |   21 +-
 .../gemfire/internal/statistics/package.html    |   16 +
 .../stats50/Atomic50StatisticsImpl.java         |   21 +-
 .../gemfire/internal/stats50/VMStats50.java     |   21 +-
 .../gemfire/internal/tcp/BaseMsgStreamer.java   |   21 +-
 .../gemstone/gemfire/internal/tcp/Buffers.java  |   21 +-
 .../internal/tcp/ByteBufferInputStream.java     |   23 +-
 .../gemfire/internal/tcp/ConnectExceptions.java |   21 +-
 .../gemfire/internal/tcp/Connection.java        |  211 +-
 .../internal/tcp/ConnectionException.java       |   21 +-
 .../gemfire/internal/tcp/ConnectionTable.java   |  127 +-
 .../gemfire/internal/tcp/DirectReplySender.java |   21 +-
 .../tcp/ImmutableByteBufferInputStream.java     |   23 +-
 .../internal/tcp/MemberShunnedException.java    |   21 +-
 .../gemfire/internal/tcp/MsgDestreamer.java     |   21 +-
 .../gemfire/internal/tcp/MsgIdGenerator.java    |   21 +-
 .../gemfire/internal/tcp/MsgOutputStream.java   |   25 +-
 .../gemfire/internal/tcp/MsgReader.java         |   21 +-
 .../gemfire/internal/tcp/MsgStreamer.java       |   21 +-
 .../gemfire/internal/tcp/MsgStreamerList.java   |   21 +-
 .../gemfire/internal/tcp/NIOMsgReader.java      |   21 +-
 .../gemfire/internal/tcp/OioMsgReader.java      |   21 +-
 .../internal/tcp/ReenteredConnectException.java |   21 +-
 .../gemfire/internal/tcp/ServerDelegate.java    |   21 +-
 .../com/gemstone/gemfire/internal/tcp/Stub.java |   21 +-
 .../gemfire/internal/tcp/TCPConduit.java        |   31 +-
 .../tcp/VersionedByteBufferInputStream.java     |   21 +-
 .../internal/tcp/VersionedMsgStreamer.java      |   21 +-
 .../gemstone/gemfire/internal/tcp/package.html  |   16 +
 .../gfsh/aggregator/AggregateFunction.java      |   67 -
 .../tools/gfsh/aggregator/AggregateResults.java |  126 -
 .../aggregator/AggregatorPartitionFunction.java |   54 -
 .../tools/gfsh/app/CommandExecutable.java       |    8 -
 .../gemfire/internal/tools/gfsh/app/Gfsh.java   | 2027 --------
 .../internal/tools/gfsh/app/GfshVersion.java    |  455 --
 .../internal/tools/gfsh/app/Nextable.java       |    8 -
 .../tools/gfsh/app/ServerExecutable.java        |   11 -
 .../app/aggregator/AggregateFunctionTask.java   |   62 -
 .../tools/gfsh/app/aggregator/Aggregator.java   |  549 ---
 .../app/aggregator/AggregatorException.java     |   50 -
 .../gfsh/app/aggregator/AggregatorPeer.java     |  254 -
 .../functions/util/LocalRegionInfoFunction.java |  235 -
 .../functions/util/RegionCreateFunction.java    |   81 -
 .../functions/util/RegionDestroyFunction.java   |   86 -
 .../tools/gfsh/app/cache/CacheBase.java         |  178 -
 .../gfsh/app/cache/InstantiatorClassLoader.java |  101 -
 .../tools/gfsh/app/cache/data/GenericMap.java   |  196 -
 .../gfsh/app/cache/data/GenericMessage.java     |  807 ---
 .../app/cache/data/InvalidTypeException.java    |   28 -
 .../tools/gfsh/app/cache/data/ListMap.java      |  197 -
 .../gfsh/app/cache/data/ListMapMessage.java     |  615 ---
 .../tools/gfsh/app/cache/data/ListMessage.java  |  594 ---
 .../tools/gfsh/app/cache/data/Listable.java     |   45 -
 .../tools/gfsh/app/cache/data/MapMessage.java   |  629 ---
 .../tools/gfsh/app/cache/data/Mappable.java     |   50 -
 .../tools/gfsh/app/cache/index/EntryMap.java    |   80 -
 .../tools/gfsh/app/cache/index/IndexInfo.java   |   52 -
 .../tools/gfsh/app/cache/index/Indexer.java     |   13 -
 .../gfsh/app/cache/index/IndexerManager.java    |   32 -
 .../gfsh/app/cache/index/LookupService.java     |  352 --
 .../app/cache/index/LookupServiceException.java |   64 -
 .../gfsh/app/cache/index/task/ForceGCTask.java  |   32 -
 .../app/cache/index/task/IndexInfoTask.java     |  131 -
 .../app/cache/index/task/QuerySizeTask.java     |  134 -
 .../gfsh/app/cache/index/task/QueryTask.java    |  323 --
 .../tools/gfsh/app/command/CommandClient.java   |  417 --
 .../gfsh/app/command/CommandException.java      |   31 -
 .../app/command/CommandResultsListener.java     |   15 -
 .../tools/gfsh/app/command/task/EchoTask.java   |   63 -
 .../task/PartitionedRegionAttributeTask.java    |  205 -
 .../gfsh/app/command/task/QueryResults.java     |  123 -
 .../tools/gfsh/app/command/task/QueryTask.java  |  522 --
 .../task/RefreshAggregatorRegionTask.java       |   53 -
 .../gfsh/app/command/task/RegionClearTask.java  |   85 -
 .../gfsh/app/command/task/RegionCreateTask.java |  162 -
 .../app/command/task/RegionDestroyTask.java     |   97 -
 .../gfsh/app/command/task/RegionPathTask.java   |  156 -
 .../gfsh/app/command/task/RegionSizeTask.java   |   70 -
 .../gfsh/app/command/task/data/MemberInfo.java  |   79 -
 .../task/data/PartitionAttributeInfo.java       |  124 -
 .../command/task/data/RegionAttributeInfo.java  |  286 --
 .../internal/tools/gfsh/app/commands/bcp.java   |  527 --
 .../internal/tools/gfsh/app/commands/cd.java    |   79 -
 .../tools/gfsh/app/commands/classloader.java    |  323 --
 .../internal/tools/gfsh/app/commands/clear.java |  197 -
 .../tools/gfsh/app/commands/connect.java        |  124 -
 .../internal/tools/gfsh/app/commands/db.java    |  312 --
 .../internal/tools/gfsh/app/commands/debug.java |   48 -
 .../tools/gfsh/app/commands/deploy.java         |  271 --
 .../internal/tools/gfsh/app/commands/echo.java  |   59 -
 .../internal/tools/gfsh/app/commands/fetch.java |   48 -
 .../internal/tools/gfsh/app/commands/gc.java    |   93 -
 .../internal/tools/gfsh/app/commands/get.java   |  143 -
 .../internal/tools/gfsh/app/commands/help.java  |   40 -
 .../internal/tools/gfsh/app/commands/index.java |  395 --
 .../internal/tools/gfsh/app/commands/key.java   |   67 -
 .../internal/tools/gfsh/app/commands/local.java |  101 -
 .../internal/tools/gfsh/app/commands/ls.java    |  584 ---
 .../internal/tools/gfsh/app/commands/mkdir.java |  253 -
 .../internal/tools/gfsh/app/commands/next.java  |   63 -
 .../tools/gfsh/app/commands/optional/look.java  |  166 -
 .../tools/gfsh/app/commands/optional/perf.java  |  150 -
 .../internal/tools/gfsh/app/commands/pr.java    |  209 -
 .../tools/gfsh/app/commands/property.java       |   85 -
 .../internal/tools/gfsh/app/commands/put.java   |  490 --
 .../internal/tools/gfsh/app/commands/pwd.java   |   37 -
 .../tools/gfsh/app/commands/rebalance.java      |  186 -
 .../tools/gfsh/app/commands/refresh.java        |   67 -
 .../internal/tools/gfsh/app/commands/rm.java    |  175 -
 .../internal/tools/gfsh/app/commands/rmdir.java |  249 -
 .../tools/gfsh/app/commands/select.java         |  139 -
 .../internal/tools/gfsh/app/commands/show.java  |  240 -
 .../internal/tools/gfsh/app/commands/size.java  |  281 --
 .../internal/tools/gfsh/app/commands/value.java |   48 -
 .../internal/tools/gfsh/app/commands/which.java |  189 -
 .../internal/tools/gfsh/app/commands/zone.java  |   49 -
 .../tools/gfsh/app/function/GfshData.java       |   86 -
 .../tools/gfsh/app/function/GfshFunction.java   |  149 -
 .../tools/gfsh/app/function/command/clear.java  |   95 -
 .../tools/gfsh/app/function/command/deploy.java |  111 -
 .../tools/gfsh/app/function/command/gc.java     |   49 -
 .../tools/gfsh/app/function/command/index.java  |  199 -
 .../tools/gfsh/app/function/command/ls.java     |   80 -
 .../tools/gfsh/app/function/command/pr.java     |  118 -
 .../gfsh/app/function/command/rebalance.java    |  129 -
 .../tools/gfsh/app/function/command/rm.java     |   43 -
 .../tools/gfsh/app/function/command/which.java  |  159 -
 .../tools/gfsh/app/misc/util/ClassFinder.java   |  209 -
 .../gfsh/app/misc/util/DataSerializerEx.java    |   98 -
 .../app/misc/util/QueueDispatcherListener.java  |   11 -
 .../app/misc/util/QueueDispatcherThread.java    |   87 -
 .../gfsh/app/misc/util/ReflectionUtil.java      |  303 --
 .../tools/gfsh/app/misc/util/StringUtil.java    |  142 -
 .../app/misc/util/SystemClassPathManager.java   |  171 -
 .../gfsh/app/pogo/InvalidKeyException.java      |   34 -
 .../internal/tools/gfsh/app/pogo/KeyType.java   |   96 -
 .../tools/gfsh/app/pogo/KeyTypeManager.java     |  199 -
 .../internal/tools/gfsh/app/pogo/MapLite.java   | 1136 -----
 .../tools/gfsh/app/pogo/MapLiteSerializer.java  |  338 --
 .../internal/tools/gfsh/app/util/DBUtil.java    | 1094 -----
 .../tools/gfsh/app/util/DBUtilException.java    |   48 -
 .../tools/gfsh/app/util/GfshResultsBag.java     |  600 ---
 .../tools/gfsh/app/util/ObjectUtil.java         |  164 -
 .../tools/gfsh/app/util/OutputUtil.java         |  255 -
 .../internal/tools/gfsh/app/util/PrintUtil.java | 1683 -------
 .../tools/gfsh/app/util/SimplePrintUtil.java    | 1366 ------
 .../tools/gfsh/command/AbstractCommandTask.java |  136 -
 .../tools/gfsh/command/CommandResults.java      |  119 -
 .../tools/gfsh/command/CommandTask.java         |   23 -
 .../internal/tools/gfsh/util/RegionUtil.java    |  829 ----
 .../internal/util/AbortableTaskService.java     |   21 +-
 .../gemfire/internal/util/ArrayUtils.java       |   21 +-
 .../gemfire/internal/util/BlobHelper.java       |   23 +-
 .../gemfire/internal/util/Breadcrumbs.java      |   23 +-
 .../gemstone/gemfire/internal/util/Bytes.java   |   21 +-
 .../gemfire/internal/util/Callable.java         |   21 +-
 .../gemfire/internal/util/CollectionUtils.java  |   21 +-
 .../gemfire/internal/util/DebuggerSupport.java  |   25 +-
 .../gemfire/internal/util/DelayedAction.java    |   21 +-
 .../com/gemstone/gemfire/internal/util/Hex.java |   21 +-
 .../gemstone/gemfire/internal/util/IOUtils.java |   21 +-
 .../internal/util/JavaCommandBuilder.java       |   21 +-
 .../gemfire/internal/util/LogFileUtils.java     |   21 +-
 .../internal/util/ObjectIntProcedure.java       |   16 +
 .../gemfire/internal/util/ObjectProcedure.java  |   14 -
 .../gemfire/internal/util/PasswordUtil.java     |   20 +-
 .../gemfire/internal/util/PluckStacks.java      |   95 +-
 .../gemfire/internal/util/PrimeFinder.java      |  159 -
 .../internal/util/SingletonCallable.java        |   16 +
 .../gemfire/internal/util/SingletonValue.java   |   24 +-
 .../internal/util/StackTraceCollector.java      |   21 +-
 .../gemfire/internal/util/StopWatch.java        |   21 +-
 .../internal/util/SunAPINotFoundException.java  |   20 +-
 .../gemfire/internal/util/TransformUtils.java   |   21 +-
 .../gemfire/internal/util/Transformer.java      |   21 +-
 .../gemfire/internal/util/Versionable.java      |   21 +-
 .../internal/util/VersionedArrayList.java       |   20 +-
 .../util/concurrent/CopyOnWriteHashMap.java     |   67 +-
 .../util/concurrent/CopyOnWriteWeakHashMap.java |   33 +-
 .../CustomEntryConcurrentHashMap.java           |   29 +-
 .../internal/util/concurrent/FutureResult.java  |   21 +-
 .../util/concurrent/ReentrantSemaphore.java     |   21 +-
 .../util/concurrent/SemaphoreReadWriteLock.java |   21 +-
 .../util/concurrent/StoppableCondition.java     |   21 +-
 .../concurrent/StoppableCountDownLatch.java     |   21 +-
 .../concurrent/StoppableCountDownOrUpLatch.java |   21 +-
 .../concurrent/StoppableNonReentrantLock.java   |   21 +-
 .../util/concurrent/StoppableReadWriteLock.java |   16 +
 .../util/concurrent/StoppableReentrantLock.java |   21 +-
 .../StoppableReentrantReadWriteLock.java        |   21 +-
 .../lang/AttachAPINotFoundException.java        |   20 +-
 .../com/gemstone/gemfire/lang/Identifiable.java |   20 +-
 .../management/AlreadyRunningException.java     |   21 +-
 .../management/AsyncEventQueueMXBean.java       |   20 +-
 .../gemfire/management/CacheServerMXBean.java   |   20 +-
 .../gemfire/management/ClientHealthStatus.java  |   20 +-
 .../gemfire/management/ClientQueueDetail.java   |   21 +-
 .../DependenciesNotFoundException.java          |   20 +-
 .../gemfire/management/DiskBackupResult.java    |   20 +-
 .../gemfire/management/DiskBackupStatus.java    |   20 +-
 .../gemfire/management/DiskMetrics.java         |   20 +-
 .../gemfire/management/DiskStoreMXBean.java     |   20 +-
 .../DistributedLockServiceMXBean.java           |   20 +-
 .../management/DistributedRegionMXBean.java     |   20 +-
 .../management/DistributedSystemMXBean.java     |   20 +-
 .../management/EvictionAttributesData.java      |   20 +-
 .../FixedPartitionAttributesData.java           |   21 +-
 .../management/GatewayReceiverMXBean.java       |   20 +-
 .../gemfire/management/GatewaySenderMXBean.java |   20 +-
 .../gemfire/management/GemFireProperties.java   |   20 +-
 .../gemfire/management/JMXNotificationType.java |   28 +-
 .../management/JMXNotificationUserData.java     |   28 +-
 .../gemstone/gemfire/management/JVMMetrics.java |   20 +-
 .../gemfire/management/LocatorMXBean.java       |   20 +-
 .../gemfire/management/LockServiceMXBean.java   |   20 +-
 .../gemfire/management/ManagementException.java |   20 +-
 .../gemfire/management/ManagementService.java   |   21 +-
 .../gemfire/management/ManagerMXBean.java       |   20 +-
 .../gemfire/management/MemberMXBean.java        |   20 +-
 .../management/MembershipAttributesData.java    |   20 +-
 .../gemfire/management/NetworkMetrics.java      |   20 +-
 .../gemstone/gemfire/management/OSMetrics.java  |   20 +-
 .../management/PartitionAttributesData.java     |   20 +-
 .../management/PersistentMemberDetails.java     |   20 +-
 .../management/RegionAttributesData.java        |   20 +-
 .../gemfire/management/RegionMXBean.java        |   20 +-
 .../gemfire/management/ServerLoadData.java      |   20 +-
 .../gemfire/management/cli/CliMetaData.java     |   20 +-
 .../cli/CommandProcessingException.java         |   20 +-
 .../gemfire/management/cli/CommandService.java  |   20 +-
 .../management/cli/CommandServiceException.java |   20 +-
 .../management/cli/CommandStatement.java        |   20 +-
 .../gemfire/management/cli/ConverterHint.java   |   21 +-
 .../gemstone/gemfire/management/cli/Result.java |   21 +-
 .../gemfire/management/cli/package.html         |   16 +
 .../gemfire/management/internal/AgentUtil.java  |   56 +-
 .../management/internal/AlertDetails.java       |   21 +-
 .../management/internal/ArrayConverter.java     |   20 +-
 .../internal/BaseManagementService.java         |   21 +-
 .../internal/CollectionConverter.java           |   22 +-
 .../management/internal/CompositeConverter.java |   20 +-
 .../management/internal/EnumConverter.java      |   20 +-
 .../management/internal/FederatingManager.java  |   21 +-
 .../internal/FederationComponent.java           |   20 +-
 .../management/internal/FilterChain.java        |   20 +-
 .../management/internal/FilterParam.java        |   22 +-
 .../management/internal/IdentityConverter.java  |   20 +-
 .../management/internal/JettyHelper.java        |   21 +-
 .../management/internal/JmxManagerAdvisee.java  |   21 +-
 .../management/internal/JmxManagerAdvisor.java  |   23 +-
 .../management/internal/JmxManagerLocator.java  |   21 +-
 .../internal/JmxManagerLocatorRequest.java      |   31 +-
 .../internal/JmxManagerLocatorResponse.java     |   20 +-
 .../management/internal/LocalFilterChain.java   |   20 +-
 .../management/internal/LocalManager.java       |   20 +-
 .../management/internal/MBeanJMXAdapter.java    |   22 +-
 .../management/internal/MBeanProxyFactory.java  |   22 +-
 .../internal/MBeanProxyInfoRepository.java      |   22 +-
 .../internal/MBeanProxyInvocationHandler.java   |   23 +-
 .../internal/MXBeanProxyInvocationHandler.java  |   20 +-
 .../management/internal/ManagementAgent.java    |   21 +-
 .../internal/ManagementCacheListener.java       |   22 +-
 .../internal/ManagementConstants.java           |   20 +-
 .../management/internal/ManagementFunction.java |   20 +-
 .../internal/ManagementMembershipListener.java  |   20 +-
 .../internal/ManagementResourceRepo.java        |   20 +-
 .../management/internal/ManagementStrings.java  |  133 +-
 .../gemfire/management/internal/Manager.java    |   21 +-
 .../internal/ManagerStartupMessage.java         |   21 +-
 .../management/internal/MemberMessenger.java    |   20 +-
 .../internal/MonitoringRegionCacheListener.java |   20 +-
 .../internal/NotificationBroadCasterProxy.java  |   20 +-
 .../internal/NotificationCacheListener.java     |   20 +-
 .../management/internal/NotificationHub.java    |   22 +-
 .../internal/NotificationHubClient.java         |   20 +-
 .../management/internal/NotificationKey.java    |   20 +-
 .../gemfire/management/internal/OpenMethod.java |   22 +-
 .../management/internal/OpenTypeConverter.java  |   20 +-
 .../management/internal/OpenTypeUtil.java       |   20 +-
 .../gemfire/management/internal/ProxyInfo.java  |   20 +-
 .../management/internal/ProxyInterface.java     |   20 +-
 .../management/internal/ProxyListener.java      |   20 +-
 .../management/internal/RemoteFilterChain.java  |   20 +-
 .../gemfire/management/internal/RestAgent.java  |   21 +-
 .../gemfire/management/internal/SSLUtil.java    |   16 +
 .../management/internal/StringBasedFilter.java  |   20 +-
 .../internal/SystemManagementService.java       |   21 +-
 .../management/internal/TableConverter.java     |   22 +-
 .../internal/beans/AggregateHandler.java        |   20 +-
 .../internal/beans/AsyncEventQueueMBean.java    |   20 +-
 .../beans/AsyncEventQueueMBeanBridge.java       |   20 +-
 .../internal/beans/BeanUtilFuncs.java           |   20 +-
 .../internal/beans/CacheServerBridge.java       |   40 +-
 .../internal/beans/CacheServerMBean.java        |   20 +-
 .../internal/beans/DiskRegionBridge.java        |   22 +-
 .../internal/beans/DiskStoreMBean.java          |   20 +-
 .../internal/beans/DiskStoreMBeanBridge.java    |   20 +-
 .../beans/DistributedLockServiceBridge.java     |   20 +-
 .../beans/DistributedLockServiceMBean.java      |   20 +-
 .../internal/beans/DistributedRegionBridge.java |   20 +-
 .../internal/beans/DistributedRegionMBean.java  |   20 +-
 .../internal/beans/DistributedSystemBridge.java |   20 +-
 .../internal/beans/DistributedSystemMBean.java  |   20 +-
 .../internal/beans/GatewayReceiverMBean.java    |   20 +-
 .../beans/GatewayReceiverMBeanBridge.java       |   24 +-
 .../internal/beans/GatewaySenderMBean.java      |   20 +-
 .../beans/GatewaySenderMBeanBridge.java         |   20 +-
 .../internal/beans/HDFSRegionBridge.java        |   20 +-
 .../management/internal/beans/LocatorMBean.java |   20 +-
 .../internal/beans/LocatorMBeanBridge.java      |   20 +-
 .../internal/beans/LockServiceMBean.java        |   20 +-
 .../internal/beans/LockServiceMBeanBridge.java  |   22 +-
 .../internal/beans/MBeanAggregator.java         |   20 +-
 .../internal/beans/ManagementAdapter.java       |   64 +-
 .../internal/beans/ManagementListener.java      |   20 +-
 .../management/internal/beans/ManagerMBean.java |   20 +-
 .../internal/beans/ManagerMBeanBridge.java      |   20 +-
 .../management/internal/beans/MemberMBean.java  |   20 +-
 .../internal/beans/MemberMBeanBridge.java       |   25 +-
 .../internal/beans/MetricsCalculator.java       |   20 +-
 .../internal/beans/PartitionedRegionBridge.java |   20 +-
 .../internal/beans/QueryDataFunction.java       |   20 +-
 .../management/internal/beans/RegionMBean.java  |   20 +-
 .../internal/beans/RegionMBeanBridge.java       |   20 +-
 .../beans/RegionMBeanCompositeDataFactory.java  |   20 +-
 .../internal/beans/SequenceNumber.java          |   20 +-
 .../management/internal/beans/ServerBridge.java |   27 +-
 .../stats/AggregateRegionStatsMonitor.java      |   20 +-
 .../internal/beans/stats/GCStatsMonitor.java    |   20 +-
 .../GatewayReceiverClusterStatsMonitor.java     |   20 +-
 .../stats/GatewaySenderClusterStatsMonitor.java |   20 +-
 .../stats/IntegerStatsDeltaAggregator.java      |   23 +-
 .../beans/stats/LongStatsDeltaAggregator.java   |   23 +-
 .../internal/beans/stats/MBeanStatsMonitor.java |   20 +-
 .../beans/stats/MemberClusterStatsMonitor.java  |   20 +-
 .../beans/stats/MemberLevelDiskMonitor.java     |   20 +-
 .../beans/stats/RegionClusterStatsMonitor.java  |   20 +-
 .../beans/stats/ServerClusterStatsMonitor.java  |   20 +-
 .../internal/beans/stats/StatType.java          |   20 +-
 .../internal/beans/stats/StatsAggregator.java   |   20 +-
 .../beans/stats/StatsAverageLatency.java        |   20 +-
 .../internal/beans/stats/StatsKey.java          |   20 +-
 .../internal/beans/stats/StatsLatency.java      |   20 +-
 .../internal/beans/stats/StatsRate.java         |   20 +-
 .../internal/beans/stats/VMStatsMonitor.java    |   20 +-
 .../cli/AbstractCliAroundInterceptor.java       |   20 +-
 .../internal/cli/CliAroundInterceptor.java      |   20 +-
 .../management/internal/cli/CliUtil.java        |   20 +-
 .../management/internal/cli/CommandManager.java |   20 +-
 .../management/internal/cli/CommandRequest.java |   21 +-
 .../internal/cli/CommandResponse.java           |   22 +-
 .../internal/cli/CommandResponseBuilder.java    |   20 +-
 .../internal/cli/CommandResponseWriter.java     |   20 +-
 .../internal/cli/GfshParseResult.java           |   20 +-
 .../management/internal/cli/GfshParser.java     |   37 +-
 .../management/internal/cli/Launcher.java       |   20 +-
 .../management/internal/cli/LogWrapper.java     |   20 +-
 .../internal/cli/MultipleValueAdapter.java      |   21 +-
 .../internal/cli/MultipleValueConverter.java    |   21 +-
 .../internal/cli/annotation/CliArgument.java    |   20 +-
 .../cli/commands/AbstractCommandsSupport.java   |   20 +-
 .../internal/cli/commands/ClientCommands.java   |   64 +-
 .../internal/cli/commands/ConfigCommands.java   |   20 +-
 .../CreateAlterDestroyRegionCommands.java       |   41 +-
 .../internal/cli/commands/DataCommands.java     |   20 +-
 .../internal/cli/commands/DeployCommands.java   |   20 +-
 .../cli/commands/DiskStoreCommands.java         |   46 +-
 .../cli/commands/DurableClientCommands.java     |   20 +-
 ...ExportImportSharedConfigurationCommands.java |   21 +-
 .../internal/cli/commands/FunctionCommands.java |   20 +-
 .../internal/cli/commands/GfshHelpCommands.java |   20 +-
 .../cli/commands/HDFSStoreCommands.java         |  695 ---
 .../internal/cli/commands/IndexCommands.java    |   20 +-
 .../cli/commands/LauncherLifecycleCommands.java |   20 +-
 .../internal/cli/commands/MemberCommands.java   |   20 +-
 .../cli/commands/MiscellaneousCommands.java     |   58 +-
 .../internal/cli/commands/PDXCommands.java      |   16 +
 .../internal/cli/commands/QueueCommands.java    |   20 +-
 .../internal/cli/commands/RegionCommands.java   |   20 +-
 .../internal/cli/commands/ShellCommands.java    |   27 +-
 .../internal/cli/commands/StatusCommands.java   |   21 +-
 .../internal/cli/commands/WanCommands.java      |   21 +-
 .../cli/commands/dto/RegionAttributesInfo.java  |   21 +-
 .../cli/commands/dto/RegionDetails.java         |   21 +-
 .../cli/commands/dto/RegionMemberDetails.java   |   21 +-
 .../cli/converters/BooleanConverter.java        |   22 +-
 .../ClusterMemberIdNameConverter.java           |   20 +-
 .../converters/ConnectionEndpointConverter.java |   21 +-
 .../internal/cli/converters/DirConverter.java   |   22 +-
 .../cli/converters/DirPathConverter.java        |   20 +-
 .../cli/converters/DiskStoreNameConverter.java  |   20 +-
 .../internal/cli/converters/EnumConverter.java  |   20 +-
 .../cli/converters/FilePathConverter.java       |   20 +-
 .../cli/converters/FilePathStringConverter.java |   20 +-
 .../converters/GatewayReceiverIdsConverter.java |   21 +-
 .../converters/GatewaySenderIdConverter.java    |   20 +-
 .../cli/converters/HdfsStoreNameConverter.java  |   88 -
 .../internal/cli/converters/HelpConverter.java  |   20 +-
 .../cli/converters/HintTopicConverter.java      |   20 +-
 .../cli/converters/IndexTypeConverter.java      |   21 +-
 .../LocatorDiscoveryConfigConverter.java        |   20 +-
 .../cli/converters/LocatorIdNameConverter.java  |   20 +-
 .../cli/converters/LogLevelConverter.java       |   20 +-
 .../cli/converters/MemberGroupConverter.java    |   20 +-
 .../cli/converters/MemberIdNameConverter.java   |   20 +-
 .../cli/converters/RegionPathConverter.java     |   20 +-
 .../cli/converters/StringArrayConverter.java    |   20 +-
 .../cli/converters/StringListConverter.java     |   20 +-
 .../cli/domain/AsyncEventQueueDetails.java      |   21 +-
 .../internal/cli/domain/CacheServerInfo.java    |   21 +-
 .../cli/domain/ConnectToLocatorResult.java      |   20 +-
 .../internal/cli/domain/DataCommandRequest.java |   21 +-
 .../internal/cli/domain/DataCommandResult.java  |   21 +-
 .../internal/cli/domain/DiskStoreDetails.java   |   18 +-
 .../cli/domain/DurableCqNamesResult.java        |   21 +-
 .../cli/domain/EvictionAttributesInfo.java      |   21 +-
 .../domain/FixedPartitionAttributesInfo.java    |   21 +-
 .../internal/cli/domain/IndexDetails.java       |   20 +-
 .../internal/cli/domain/IndexInfo.java          |   21 +-
 .../cli/domain/MemberConfigurationInfo.java     |   21 +-
 .../internal/cli/domain/MemberInformation.java  |   20 +-
 .../internal/cli/domain/MemberResult.java       |   21 +-
 .../cli/domain/PartitionAttributesInfo.java     |   21 +-
 .../cli/domain/RegionAttributesInfo.java        |   21 +-
 .../internal/cli/domain/RegionDescription.java  |   21 +-
 .../cli/domain/RegionDescriptionPerMember.java  |   21 +-
 .../internal/cli/domain/RegionInformation.java  |   20 +-
 .../cli/domain/StackTracesPerMember.java        |   21 +-
 .../cli/domain/SubscriptionQueueSizeResult.java |   21 +-
 .../cli/exceptions/CliCommandException.java     |   20 +-
 .../exceptions/CliCommandInvalidException.java  |   20 +-
 .../CliCommandMultiModeOptionException.java     |   16 +
 .../CliCommandNotAvailableException.java        |   20 +-
 .../exceptions/CliCommandOptionException.java   |   20 +-
 ...CommandOptionHasMultipleValuesException.java |   20 +-
 .../CliCommandOptionInvalidException.java       |   20 +-
 .../CliCommandOptionMissingException.java       |   20 +-
 .../CliCommandOptionNotApplicableException.java |   20 +-
 ...liCommandOptionValueConversionException.java |   20 +-
 .../CliCommandOptionValueException.java         |   20 +-
 .../CliCommandOptionValueMissingException.java  |   20 +-
 .../internal/cli/exceptions/CliException.java   |   20 +-
 .../exceptions/CreateSubregionException.java    |   20 +-
 .../cli/exceptions/ExceptionGenerator.java      |   20 +-
 .../cli/exceptions/ExceptionHandler.java        |   20 +-
 .../cli/exceptions/IndexNotFoundException.java  |   16 +
 .../cli/functions/AlterHDFSStoreFunction.java   |  228 -
 .../functions/AlterRuntimeConfigFunction.java   |   21 +-
 .../cli/functions/ChangeLogLevelFunction.java   |   20 +-
 .../cli/functions/CliFunctionResult.java        |   21 +-
 .../functions/CloseDurableClientFunction.java   |   21 +-
 .../cli/functions/CloseDurableCqFunction.java   |   21 +-
 .../cli/functions/ContunuousQueryFunction.java  |   24 +-
 .../CreateAsyncEventQueueFunction.java          |   21 +-
 .../functions/CreateDefinedIndexesFunction.java |   16 +
 .../cli/functions/CreateDiskStoreFunction.java  |   21 +-
 .../cli/functions/CreateHDFSStoreFunction.java  |  122 -
 .../cli/functions/CreateIndexFunction.java      |   21 +-
 .../cli/functions/DataCommandFunction.java      |   27 +-
 .../internal/cli/functions/DeployFunction.java  |   21 +-
 .../functions/DescribeDiskStoreFunction.java    |   18 +-
 .../functions/DescribeHDFSStoreFunction.java    |   16 +
 .../cli/functions/DestroyDiskStoreFunction.java |   21 +-
 .../cli/functions/DestroyHDFSStoreFunction.java |  100 -
 .../cli/functions/DestroyIndexFunction.java     |   21 +-
 .../cli/functions/ExportConfigFunction.java     |   23 +-
 .../cli/functions/ExportDataFunction.java       |   21 +-
 .../ExportSharedConfigurationFunction.java      |   21 +-
 .../FetchRegionAttributesFunction.java          |   20 +-
 .../FetchSharedConfigurationStatusFunction.java |   21 +-
 .../functions/GarbageCollectionFunction.java    |   21 +-
 .../GatewayReceiverCreateFunction.java          |   21 +-
 .../functions/GatewayReceiverFunctionArgs.java  |   21 +-
 .../functions/GatewaySenderCreateFunction.java  |   21 +-
 .../functions/GatewaySenderFunctionArgs.java    |   21 +-
 .../GetMemberConfigInformationFunction.java     |   21 +-
 .../functions/GetMemberInformationFunction.java |   25 +-
 .../functions/GetRegionDescriptionFunction.java |   20 +-
 .../cli/functions/GetRegionsFunction.java       |   20 +-
 .../cli/functions/GetStackTracesFunction.java   |   21 +-
 .../GetSubscriptionQueueSizeFunction.java       |   21 +-
 .../cli/functions/ImportDataFunction.java       |   21 +-
 ...ortSharedConfigurationArtifactsFunction.java |   21 +-
 .../functions/ListAsyncEventQueuesFunction.java |   18 +-
 .../cli/functions/ListDeployedFunction.java     |   21 +-
 .../cli/functions/ListDiskStoresFunction.java   |   18 +-
 .../functions/ListDurableCqNamesFunction.java   |   20 +-
 .../cli/functions/ListFunctionFunction.java     |   21 +-
 .../cli/functions/ListHDFSStoresFunction.java   |  102 -
 .../cli/functions/ListIndexFunction.java        |   20 +-
 .../LoadSharedConfigurationFunction.java        |   21 +-
 .../internal/cli/functions/LogFileFunction.java |   23 +-
 .../cli/functions/MemberRegionFunction.java     |   23 +-
 .../cli/functions/MembersForRegionFunction.java |   20 +-
 .../internal/cli/functions/NetstatFunction.java |   20 +-
 .../cli/functions/RebalanceFunction.java        |   23 +-
 .../cli/functions/RegionAlterFunction.java      |   20 +-
 .../cli/functions/RegionCreateFunction.java     |   33 +-
 .../cli/functions/RegionDestroyFunction.java    |   20 +-
 .../cli/functions/RegionFunctionArgs.java       |  109 +-
 .../cli/functions/ShutDownFunction.java         |   23 +-
 .../cli/functions/UndeployFunction.java         |   21 +-
 .../cli/functions/UnregisterFunction.java       |   23 +-
 .../cli/functions/UserFunctionExecution.java    |   24 +-
 .../management/internal/cli/help/CliTopic.java  |   20 +-
 .../internal/cli/help/format/Block.java         |   20 +-
 .../internal/cli/help/format/DataNode.java      |   20 +-
 .../internal/cli/help/format/Help.java          |   20 +-
 .../internal/cli/help/format/NewHelp.java       |   20 +-
 .../internal/cli/help/format/Row.java           |   20 +-
 .../internal/cli/help/utils/FormatOutput.java   |   16 +
 .../internal/cli/help/utils/HelpUtils.java      |   20 +-
 .../internal/cli/i18n/CliStrings.java           |  195 +-
 .../internal/cli/json/GfJsonArray.java          |   20 +-
 .../internal/cli/json/GfJsonException.java      |   22 +-
 .../internal/cli/json/GfJsonObject.java         |   20 +-
 .../management/internal/cli/json/TypedJson.java |   20 +-
 .../internal/cli/modes/CommandModes.java        |   18 +-
 .../cli/multistep/CLIMultiStepHelper.java       |   27 +-
 .../internal/cli/multistep/CLIRemoteStep.java   |   16 +
 .../internal/cli/multistep/CLIStep.java         |   16 +
 .../cli/multistep/CLIStepExecption.java         |   21 +-
 .../cli/multistep/MultiStepCommand.java         |   16 +
 .../internal/cli/parser/Argument.java           |   20 +-
 .../internal/cli/parser/AvailabilityTarget.java |   20 +-
 .../internal/cli/parser/CommandTarget.java      |   22 +-
 .../internal/cli/parser/GfshMethodTarget.java   |   30 +-
 .../internal/cli/parser/GfshOptionParser.java   |   22 +-
 .../internal/cli/parser/MethodParameter.java    |   20 +-
 .../management/internal/cli/parser/Option.java  |   20 +-
 .../internal/cli/parser/OptionSet.java          |   20 +-
 .../internal/cli/parser/Parameter.java          |   20 +-
 .../internal/cli/parser/ParserUtils.java        |   20 +-
 .../internal/cli/parser/SyntaxConstants.java    |   22 +-
 .../cli/parser/jopt/JoptOptionParser.java       |   40 +-
 .../preprocessor/EnclosingCharacters.java       |   20 +-
 .../cli/parser/preprocessor/Preprocessor.java   |   20 +-
 .../parser/preprocessor/PreprocessorUtils.java  |   31 +-
 .../internal/cli/parser/preprocessor/Stack.java |   20 +-
 .../cli/parser/preprocessor/TrimmedInput.java   |   20 +-
 .../cli/remote/CommandExecutionContext.java     |   20 +-
 .../internal/cli/remote/CommandProcessor.java   |   20 +-
 .../cli/remote/CommandStatementImpl.java        |   20 +-
 .../cli/remote/MemberCommandService.java        |   20 +-
 .../cli/remote/RemoteExecutionStrategy.java     |   24 +-
 .../internal/cli/remote/WrapperThreadLocal.java |   20 +-
 .../internal/cli/result/AbstractResultData.java |   20 +-
 .../cli/result/CliJsonSerializable.java         |   20 +-
 .../cli/result/CliJsonSerializableFactory.java  |   21 +-
 .../cli/result/CliJsonSerializableIds.java      |   20 +-
 .../internal/cli/result/CommandResult.java      |   20 +-
 .../cli/result/CommandResultException.java      |   21 +-
 .../cli/result/CompositeResultData.java         |   20 +-
 .../internal/cli/result/ErrorResultData.java    |   20 +-
 .../internal/cli/result/FileResult.java         |   20 +-
 .../internal/cli/result/InfoResultData.java     |   20 +-
 .../internal/cli/result/ObjectResultData.java   |   20 +-
 .../internal/cli/result/ResultBuilder.java      |   20 +-
 .../internal/cli/result/ResultData.java         |   20 +-
 .../cli/result/ResultDataException.java         |   20 +-
 .../internal/cli/result/TableBuilder.java       |   20 +-
 .../internal/cli/result/TableBuilderHelper.java |   21 +-
 .../internal/cli/result/TabularResultData.java  |   20 +-
 .../management/internal/cli/shell/Gfsh.java     |   20 +-
 .../internal/cli/shell/GfshConfig.java          |   20 +-
 .../cli/shell/GfshExecutionStrategy.java        |   24 +-
 .../cli/shell/JMXConnectionException.java       |   20 +-
 .../cli/shell/JMXInvocationException.java       |   20 +-
 .../internal/cli/shell/JmxOperationInvoker.java |   20 +-
 .../internal/cli/shell/MultiCommandHelper.java  |   16 +
 .../internal/cli/shell/OperationInvoker.java    |   20 +-
 .../internal/cli/shell/jline/ANSIHandler.java   |   20 +-
 .../cli/shell/jline/CygwinMinttyTerminal.java   |   21 +-
 .../internal/cli/shell/jline/GfshHistory.java   |   20 +-
 .../shell/jline/GfshUnsupportedTerminal.java    |   20 +-
 .../cli/shell/unsafe/GfshSignalHandler.java     |   21 +-
 .../internal/cli/util/CLIConsoleBufferUtil.java |   21 +-
 .../internal/cli/util/CauseFinder.java          |   20 +-
 .../cli/util/ClasspathScanLoadHelper.java       |   20 +-
 .../internal/cli/util/CommandStringBuilder.java |   20 +-
 .../internal/cli/util/CommentSkipHelper.java    |   22 +-
 .../internal/cli/util/ConnectionEndpoint.java   |   21 +-
 .../internal/cli/util/DiskStoreCompacter.java   |   20 +-
 .../cli/util/DiskStoreNotFoundException.java    |   18 +-
 .../internal/cli/util/DiskStoreUpgrader.java    |   21 +-
 .../internal/cli/util/DiskStoreValidater.java   |   21 +-
 .../cli/util/EvictionAttributesInfo.java        |   21 +-
 .../cli/util/FixedPartitionAttributesInfo.java  |   21 +-
 .../internal/cli/util/GfshConsoleReader.java    |   22 +-
 .../cli/util/HDFSStoreNotFoundException.java    |   18 +-
 .../cli/util/JConsoleNotFoundException.java     |   20 +-
 .../management/internal/cli/util/JsonUtil.java  |   20 +-
 .../internal/cli/util/MemberInformation.java    |   20 +-
 .../cli/util/MemberNotFoundException.java       |   18 +-
 .../management/internal/cli/util/MergeLogs.java |   21 +-
 .../internal/cli/util/ReadWriteFile.java        |   21 +-
 .../cli/util/RegionAttributesDefault.java       |   21 +-
 .../cli/util/RegionAttributesNames.java         |   21 +-
 .../internal/cli/util/RegionPath.java           |   20 +-
 .../cli/util/VisualVmNotFoundException.java     |   20 +-
 .../internal/cli/util/spring/Assert.java        |   27 -
 .../internal/cli/util/spring/ObjectUtils.java   |  291 --
 .../cli/util/spring/ReflectionUtils.java        |  123 -
 .../internal/cli/util/spring/StringUtils.java   |  220 -
 .../SharedConfigurationWriter.java              |   21 +-
 .../callbacks/ConfigurationChangeListener.java  |   21 +-
 .../configuration/domain/CacheElement.java      |   21 +-
 .../configuration/domain/Configuration.java     |   21 +-
 .../domain/ConfigurationChangeResult.java       |   21 +-
 .../domain/SharedConfigurationStatus.java       |   16 +
 .../configuration/domain/XmlEntity.java         |   21 +-
 .../configuration/functions/AddJarFunction.java |   21 +-
 .../functions/AddXmlEntityFunction.java         |   21 +-
 .../functions/DeleteJarFunction.java            |   21 +-
 .../functions/DeleteXmlEntityFunction.java      |   21 +-
 .../functions/GetAllJarsFunction.java           |   21 +-
 .../functions/ModifyPropertiesFunction.java     |   21 +-
 .../handlers/ConfigurationRequestHandler.java   |   21 +-
 ...SharedConfigurationStatusRequestHandler.java |   21 +-
 .../messages/ConfigurationRequest.java          |   21 +-
 .../messages/ConfigurationResponse.java         |   21 +-
 .../SharedConfigurationStatusRequest.java       |   21 +-
 .../SharedConfigurationStatusResponse.java      |   21 +-
 .../configuration/utils/DtdResolver.java        |   16 +
 .../configuration/utils/XmlConstants.java       |   21 +-
 .../internal/configuration/utils/XmlUtils.java  |   21 +-
 .../internal/configuration/utils/ZipUtils.java  |   21 +-
 .../internal/messages/CompactRequest.java       |   20 +-
 .../internal/messages/CompactResponse.java      |   20 +-
 .../internal/security/AccessControl.java        |   16 +
 .../internal/security/AccessControlContext.java |   16 +
 .../internal/security/AccessControlMXBean.java  |   16 +
 .../internal/security/CLIOperationContext.java  |   16 +
 .../internal/security/JMXOperationContext.java  |   16 +
 .../internal/security/JSONAuthorization.java    |   16 +
 .../internal/security/MBeanServerWrapper.java   |   16 +
 .../security/ManagementInterceptor.java         |   16 +
 .../management/internal/security/Resource.java  |   16 +
 .../internal/security/ResourceConstants.java    |   16 +
 .../internal/security/ResourceOperation.java    |   16 +
 .../security/ResourceOperationContext.java      |   16 +
 .../unsafe/ReadOpFileAccessController.java      |   21 +-
 .../controllers/AbstractCommandsController.java |   28 +-
 .../AbstractMultiPartCommandsController.java    |   21 +-
 .../controllers/ClientCommandsController.java   |   21 +-
 .../controllers/ClusterCommandsController.java  |   21 +-
 .../controllers/ConfigCommandsController.java   |   21 +-
 .../web/controllers/DataCommandsController.java |   21 +-
 .../controllers/DeployCommandsController.java   |   21 +-
 .../DiskStoreCommandsController.java            |   21 +-
 .../DurableClientCommandsController.java        |   21 +-
 .../controllers/FunctionCommandsController.java |   21 +-
 .../HDFSStoreCommandsController.java            |  229 -
 .../controllers/IndexCommandsController.java    |   21 +-
 .../LauncherLifecycleCommandsController.java    |   21 +-
 .../controllers/MemberCommandsController.java   |   21 +-
 .../MiscellaneousCommandsController.java        |   21 +-
 .../web/controllers/PdxCommandsController.java  |   16 +
 .../controllers/QueueCommandsController.java    |   21 +-
 .../controllers/RegionCommandsController.java   |   21 +-
 .../controllers/ShellCommandsController.java    |  225 +-
 .../web/controllers/WanCommandsController.java  |   23 +-
 .../EnvironmentVariablesHandlerInterceptor.java |   21 +-
 .../support/MemberMXBeanAdapter.java            |   21 +-
 .../management/internal/web/domain/Link.java    |   21 +-
 .../internal/web/domain/LinkIndex.java          |   21 +-
 .../web/domain/QueryParameterSource.java        |   21 +-
 .../internal/web/http/ClientHttpRequest.java    |   21 +-
 .../internal/web/http/HttpHeader.java           |   21 +-
 .../internal/web/http/HttpMethod.java           |   21 +-
 .../SerializableObjectHttpMessageConverter.java |   21 +-
 .../web/http/support/SimpleHttpRequester.java   |   21 +-
 .../internal/web/io/MultipartFileAdapter.java   |   21 +-
 .../web/io/MultipartFileResourceAdapter.java    |   21 +-
 .../web/shell/AbstractHttpOperationInvoker.java |   21 +-
 .../web/shell/HttpOperationInvoker.java         |   16 +
 .../web/shell/MBeanAccessException.java         |   21 +-
 .../RestApiCallForCommandNotFoundException.java |   21 +-
 .../web/shell/RestHttpOperationInvoker.java     |   21 +-
 .../web/shell/SimpleHttpOperationInvoker.java   |   21 +-
 .../shell/support/HttpInvocationHandler.java    |   21 +-
 .../shell/support/HttpMBeanProxyFactory.java    |   21 +-
 .../internal/web/util/ConvertUtils.java         |   21 +-
 .../management/internal/web/util/UriUtils.java  |   21 +-
 .../management/membership/ClientMembership.java |   29 +-
 .../membership/ClientMembershipEvent.java       |   21 +-
 .../membership/ClientMembershipListener.java    |   21 +-
 .../ClientMembershipListenerAdapter.java        |   21 +-
 .../management/membership/MembershipEvent.java  |   21 +-
 .../membership/MembershipListener.java          |   21 +-
 .../UniversalMembershipListenerAdapter.java     |   21 +-
 .../gemstone/gemfire/management/package.html    |   16 +
 .../memcached/GemFireMemcachedServer.java       |   21 +-
 .../main/java/com/gemstone/gemfire/package.html |   16 +
 .../com/gemstone/gemfire/pdx/FieldType.java     |   21 +-
 .../com/gemstone/gemfire/pdx/JSONFormatter.java |   16 +
 .../gemfire/pdx/JSONFormatterException.java     |   23 +-
 .../gemfire/pdx/NonPortableClassException.java  |   16 +
 .../gemfire/pdx/PdxConfigurationException.java  |   21 +-
 .../pdx/PdxFieldAlreadyExistsException.java     |   21 +-
 .../pdx/PdxFieldDoesNotExistException.java      |   21 +-
 .../pdx/PdxFieldTypeMismatchException.java      |   21 +-
 .../gemfire/pdx/PdxInitializationException.java |   21 +-
 .../com/gemstone/gemfire/pdx/PdxInstance.java   |   21 +-
 .../gemfire/pdx/PdxInstanceFactory.java         |   21 +-
 .../com/gemstone/gemfire/pdx/PdxReader.java     |   21 +-
 .../pdx/PdxRegistryMismatchException.java       |   24 +-
 .../gemstone/gemfire/pdx/PdxSerializable.java   |   21 +-
 .../gemfire/pdx/PdxSerializationException.java  |   21 +-
 .../com/gemstone/gemfire/pdx/PdxSerializer.java |   21 +-
 .../gemstone/gemfire/pdx/PdxUnreadFields.java   |   21 +-
 .../com/gemstone/gemfire/pdx/PdxWriter.java     |   21 +-
 .../pdx/ReflectionBasedAutoSerializer.java      |   21 +-
 .../gemfire/pdx/WritablePdxInstance.java        |   21 +-
 .../pdx/internal/AutoSerializableManager.java   |   21 +-
 .../pdx/internal/CheckTypeRegistryState.java    |   21 +-
 .../pdx/internal/ClientTypeRegistration.java    |   21 +-
 .../gemfire/pdx/internal/ComparableEnum.java    |   16 +
 .../pdx/internal/ConvertableToBytes.java        |   16 +
 .../gemstone/gemfire/pdx/internal/DataSize.java |   21 +-
 .../gemfire/pdx/internal/DefaultPdxField.java   |   21 +-
 .../gemstone/gemfire/pdx/internal/EnumId.java   |   21 +-
 .../gemstone/gemfire/pdx/internal/EnumInfo.java |   21 +-
 .../pdx/internal/FieldNotFoundInPdxVersion.java |   16 +
 .../gemfire/pdx/internal/InternalPdxReader.java |   21 +-
 .../pdx/internal/LonerTypeRegistration.java     |   21 +-
 .../pdx/internal/NullTypeRegistration.java      |   21 +-
 .../gemstone/gemfire/pdx/internal/PdxField.java |   21 +-
 .../gemfire/pdx/internal/PdxInputStream.java    |   23 +-
 .../gemfire/pdx/internal/PdxInstanceEnum.java   |   21 +-
 .../pdx/internal/PdxInstanceFactoryImpl.java    |   21 +-
 .../gemfire/pdx/internal/PdxInstanceImpl.java   |   21 +-
 .../pdx/internal/PdxInstanceInputStream.java    |   21 +-
 .../gemfire/pdx/internal/PdxOutputStream.java   |   21 +-
 .../gemfire/pdx/internal/PdxReaderImpl.java     |   21 +-
 .../gemfire/pdx/internal/PdxString.java         |   23 +-
 .../gemstone/gemfire/pdx/internal/PdxType.java  |   21 +-
 .../gemfire/pdx/internal/PdxUnreadData.java     |   21 +-
 .../gemfire/pdx/internal/PdxWriterImpl.java     |   21 +-
 .../pdx/internal/PeerTypeRegistration.java      |   21 +-
 .../pdx/internal/TrackingPdxReaderImpl.java     |   21 +-
 .../gemfire/pdx/internal/TypeRegistration.java  |   21 +-
 .../gemfire/pdx/internal/TypeRegistry.java      |   36 +-
 .../gemfire/pdx/internal/UnreadPdxType.java     |   21 +-
 .../internal/WeakConcurrentIdentityHashMap.java |   21 +-
 .../pdx/internal/WritablePdxInstanceImpl.java   |   21 +-
 .../gemfire/pdx/internal/json/JsonHelper.java   |   21 +-
 .../pdx/internal/json/PdxInstanceHelper.java    |   23 +-
 .../pdx/internal/json/PdxListHelper.java        |   23 +-
 .../gemfire/pdx/internal/json/PdxToJSON.java    |   23 +-
 .../pdx/internal/unsafe/UnsafeWrapper.java      |   21 +-
 .../java/com/gemstone/gemfire/pdx/package.html  |   15 +
 .../com/gemstone/gemfire/ra/GFConnection.java   |   16 +
 .../gemfire/ra/GFConnectionFactory.java         |   16 +
 .../gemfire/redis/GemFireRedisServer.java       |   29 +-
 .../gemfire/security/AccessControl.java         |   21 +-
 .../gemfire/security/AuthInitialize.java        |   21 +-
 .../security/AuthenticationFailedException.java |   21 +-
 .../AuthenticationRequiredException.java        |   21 +-
 .../gemfire/security/Authenticator.java         |   21 +-
 .../security/GemFireSecurityException.java      |   21 +-
 .../security/NotAuthorizedException.java        |   21 +-
 .../com/gemstone/gemfire/security/package.html  |   16 +
 .../util/concurrent/SynchronousQueueNoSpin.java | 1144 -----
 .../src/main/java/external-overview.html        |   16 +
 .../src/main/java/internal-overview.html        |   16 +
 .../gemfire/cache/cache-8.1.xsd                 |   16 +
 .../gemfire/cache/cache-9.0.xsd                 |   42 +-
 .../services/org.xml.sax.ext.EntityResolver2    |    1 +
 .../gemstone/gemfire/admin/doc-files/ds4_0.dtd  |   15 +
 .../gemstone/gemfire/admin/doc-files/ds5_0.dtd  |   15 +
 .../gemfire/cache/doc-files/cache3_0.dtd        |   15 +
 .../gemfire/cache/doc-files/cache4_0.dtd        |   15 +
 .../gemfire/cache/doc-files/cache4_1.dtd        |   15 +
 .../gemfire/cache/doc-files/cache5_0.dtd        |   15 +
 .../gemfire/cache/doc-files/cache5_1.dtd        |   15 +
 .../gemfire/cache/doc-files/cache5_5.dtd        |   15 +
 .../gemfire/cache/doc-files/cache5_7.dtd        |   15 +
 .../gemfire/cache/doc-files/cache5_8.dtd        |   15 +
 .../gemfire/cache/doc-files/cache6_0.dtd        |   15 +
 .../gemfire/cache/doc-files/cache6_1.dtd        |   15 +
 .../gemfire/cache/doc-files/cache6_5.dtd        |   15 +
 .../gemfire/cache/doc-files/cache6_6.dtd        |   15 +
 .../gemfire/cache/doc-files/cache7_0.dtd        |   15 +
 .../gemfire/cache/doc-files/cache8_0.dtd        |   15 +
 .../distributed/internal/javagroups-config.txt  |  101 -
 .../distributed/internal/javagroups-mcast.txt   |   97 -
 .../membership/gms/messenger/jgroups-config.xml |   72 +
 .../membership/gms/messenger/jgroups-mcast.xml  |   98 +
 .../internal/logging/log4j/log4j2-cli.xml       |   17 -
 .../internal/logging/log4j/log4j2-default.xml   |   21 -
 .../internal/cli/commands/support/gfmon.html    |   20 +-
 .../com/gemstone/gemfire/statisticsType.dtd     |   17 +-
 gemfire-core/src/main/resources/log4j2-cli.xml  |   17 +
 gemfire-core/src/main/resources/log4j2.xml      |   23 +
 .../batterytest/greplogs/ExpectedStrings.java   |   25 +-
 .../java/batterytest/greplogs/LogConsumer.java  |   66 +-
 .../src/test/java/cacheRunner/Portfolio.java    |   16 +
 .../src/test/java/cacheRunner/Position.java     |   16 +
 .../src/test/java/com/company/app/Customer.java |   21 +-
 .../src/test/java/com/company/app/DBLoader.java |   21 +-
 .../com/company/app/OrdersCacheListener.java    |   21 +-
 .../java/com/company/data/DatabaseLoader.java   |   21 +-
 .../java/com/company/data/MyDeclarable.java     |   16 +
 .../src/test/java/com/company/data/MySizer.java |   21 +-
 .../com/company/data/MyTransactionListener.java |   21 +-
 .../src/test/java/com/examples/LinkNode.java    |   21 +-
 .../src/test/java/com/examples/SuperClass.java  |   21 +-
 .../src/test/java/com/examples/TestObject.java  |   21 +-
 .../src/test/java/com/examples/ds/Address.java  |   16 +
 .../src/test/java/com/examples/ds/Company.java  |   21 +-
 .../java/com/examples/ds/CompanySerializer.java |   21 +-
 .../src/test/java/com/examples/ds/Employee.java |   21 +-
 .../com/examples/ds/PutDataSerializables.java   |   21 +-
 .../src/test/java/com/examples/ds/User.java     |   21 +-
 .../com/examples/snapshot/MyDataSerializer.java |   21 +-
 .../java/com/examples/snapshot/MyObject.java    |   23 +-
 .../snapshot/MyObjectDataSerializable.java      |   23 +-
 .../java/com/examples/snapshot/MyObjectPdx.java |   16 +
 .../snapshot/MyObjectPdxSerializable.java       |   21 +-
 .../com/examples/snapshot/MyPdxSerializer.java  |   21 +-
 .../java/com/gemstone/gemfire/AppObject.java    |   18 +-
 .../test/java/com/gemstone/gemfire/BadTest.java |   21 +-
 .../com/gemstone/gemfire/CopyJUnitTest.java     |   21 +-
 .../com/gemstone/gemfire/DeltaTestImpl.java     |   21 +-
 .../gemfire/DiskInstantiatorsJUnitTest.java     |   21 +-
 .../com/gemstone/gemfire/GemFireTestCase.java   |   21 +-
 .../java/com/gemstone/gemfire/Invariant.java    |   22 +-
 .../com/gemstone/gemfire/InvariantResult.java   |   22 +-
 .../com/gemstone/gemfire/JUnitTestSetup.java    |   21 +-
 .../gemfire/JtaNoninvolvementJUnitTest.java     |   21 +-
 .../gemfire/LocalStatisticsJUnitTest.java       |   21 +-
 .../com/gemstone/gemfire/LonerDMJUnitTest.java  |   21 +-
 .../gemstone/gemfire/StatisticsTestCase.java    |   21 +-
 .../gemfire/StatisticsTypeJUnitTest.java        |   21 +-
 .../com/gemstone/gemfire/TXExpiryJUnitTest.java |  569 ++-
 .../java/com/gemstone/gemfire/TXJUnitTest.java  |   21 +-
 .../com/gemstone/gemfire/TXWriterJUnitTest.java |   21 +-
 .../gemstone/gemfire/TXWriterOOMEJUnitTest.java |   21 +-
 .../com/gemstone/gemfire/TXWriterTestCase.java  |   16 +
 .../gemstone/gemfire/TestDataSerializer.java    |   21 +-
 .../com/gemstone/gemfire/TimingTestCase.java    |   22 +-
 .../com/gemstone/gemfire/UnitTestDoclet.java    |   21 +-
 .../gemstone/gemfire/admin/AdminTestHelper.java |   16 +
 .../BindDistributedSystemJUnitTest.java         |  177 +-
 .../internal/CacheHealthEvaluatorJUnitTest.java |   21 +-
 .../internal/DistributedSystemTestCase.java     |   21 +-
 .../admin/internal/HealthEvaluatorTestCase.java |   21 +-
 .../MemberHealthEvaluatorJUnitTest.java         |   21 +-
 .../cache/AttributesFactoryJUnitTest.java       |   39 +-
 .../gemfire/cache/Bug36619JUnitTest.java        |   21 +-
 .../gemfire/cache/Bug42039JUnitTest.java        |   44 +-
 .../gemfire/cache/Bug52289JUnitTest.java        |   89 +
 .../gemfire/cache/CacheListenerJUnitTest.java   |   21 +-
 .../cache/CacheRegionClearStatsDUnitTest.java   |   21 +-
 .../gemstone/gemfire/cache/ClientHelper.java    |   22 +-
 .../cache/ClientServerTimeSyncDUnitTest.java    |   25 +-
 .../cache/ConnectionPoolAndLoaderDUnitTest.java |   25 +-
 .../cache/ConnectionPoolFactoryJUnitTest.java   |   21 +-
 .../gemfire/cache/OperationJUnitTest.java       |   21 +-
 .../gemfire/cache/PoolManagerJUnitTest.java     |   21 +-
 .../gemstone/gemfire/cache/ProxyJUnitTest.java  |   21 +-
 .../gemfire/cache/RegionFactoryJUnitTest.java   |   21 +-
 .../gemfire/cache/RoleExceptionJUnitTest.java   |   21 +-
 .../SerialAsyncEventQueueImplJUnitTest.java     |   62 +
 .../client/ClientCacheFactoryJUnitTest.java     |   21 +-
 .../client/ClientRegionFactoryJUnitTest.java    |   21 +-
 .../ClientServerRegisterInterestsDUnitTest.java |   21 +-
 .../internal/AutoConnectionSourceDUnitTest.java |   50 +-
 .../AutoConnectionSourceImplJUnitTest.java      |   21 +-
 .../AutoConnectionSourceWithUDPDUnitTest.java   |  140 -
 .../internal/CacheServerSSLConnectionDUnit.java |  648 ---
 .../CacheServerSSLConnectionDUnitTest.java      |  426 ++
 .../internal/ConnectionPoolImplJUnitTest.java   |   32 +-
 .../internal/LocatorLoadBalancingDUnitTest.java |   31 +-
 .../cache/client/internal/LocatorTestBase.java  |   33 +-
 .../internal/OpExecutorImplJUnitTest.java       |   21 +-
 .../client/internal/QueueManagerJUnitTest.java  |   21 +-
 .../internal/SSLNoClientAuthDUnitTest.java      |  280 ++
 .../internal/ServerBlackListJUnitTest.java      |   21 +-
 .../locator/LocatorStatusResponseJUnitTest.java |   20 +-
 .../pooling/ConnectionManagerJUnitTest.java     |   21 +-
 .../ColocatedRegionWithHDFSDUnitTest.java       |  189 -
 .../hdfs/internal/HDFSConfigJUnitTest.java      |  520 --
 .../hdfs/internal/HDFSEntriesSetJUnitTest.java  |  227 -
 .../internal/HdfsStoreMutatorJUnitTest.java     |  191 -
 .../hdfs/internal/RegionRecoveryDUnitTest.java  |  415 --
 .../internal/RegionWithHDFSBasicDUnitTest.java  | 1594 ------
 .../RegionWithHDFSOffHeapBasicDUnitTest.java    |  114 -
 ...RegionWithHDFSPersistenceBasicDUnitTest.java |   77 -
 .../hdfs/internal/RegionWithHDFSTestBase.java   |  715 ---
 .../SignalledFlushObserverJUnitTest.java        |   23 +-
 .../SortedListForAsyncQueueJUnitTest.java       |   31 +-
 .../internal/hoplog/BaseHoplogTestCase.java     |  389 --
 .../hoplog/CardinalityEstimatorJUnitTest.java   |  188 -
 .../hoplog/HDFSCacheLoaderJUnitTest.java        |  106 -
 .../hoplog/HDFSCompactionManagerJUnitTest.java  |  449 --
 .../hoplog/HDFSRegionDirectorJUnitTest.java     |   97 -
 .../internal/hoplog/HDFSStatsJUnitTest.java     |  250 -
 .../HDFSUnsortedHoplogOrganizerJUnitTest.java   |  297 --
 .../HdfsSortedOplogOrganizerJUnitTest.java      | 1045 ----
 .../hoplog/HfileSortedOplogJUnitTest.java       |  540 --
 .../hoplog/SortedOplogListIterJUnitTest.java    |  178 -
 .../hoplog/TieredCompactionJUnitTest.java       |  904 ----
 .../hoplog/mapreduce/GFKeyJUnitTest.java        |   50 -
 .../mapreduce/HDFSSplitIteratorJUnitTest.java   |  265 -
 .../hoplog/mapreduce/HoplogUtilJUnitTest.java   |  305 --
 .../management/MXMemoryPoolListenerExample.java |   21 +-
 .../management/MemoryThresholdsDUnitTest.java   |  186 +-
 .../MemoryThresholdsOffHeapDUnitTest.java       |  267 +-
 .../management/ResourceManagerDUnitTest.java    |   21 +-
 .../ExceptionHandlingJUnitTest.java             |   21 +-
 .../mapInterface/MapFunctionalJUnitTest.java    |   21 +-
 .../mapInterface/PutAllGlobalLockJUnitTest.java |   21 +-
 .../PutOperationContextJUnitTest.java           |   16 +
 .../GetOperationContextImplJUnitTest.java       |   16 +
 .../partition/PartitionManagerDUnitTest.java    |   96 +-
 .../PartitionRegionHelperDUnitTest.java         |   21 +-
 .../BaseLineAndCompareQueryPerfJUnitTest.java   |   21 +-
 .../query/Bug32947ValueConstraintJUnitTest.java |   21 +-
 .../gemfire/cache/query/BugJUnitTest.java       |   21 +-
 .../gemfire/cache/query/CacheUtils.java         |   21 +-
 .../cache/query/PdxStringQueryJUnitTest.java    |   21 +-
 .../gemstone/gemfire/cache/query/PerfQuery.java |   22 +-
 .../gemfire/cache/query/QueryJUnitTest.java     |  112 +-
 .../cache/query/QueryServiceJUnitTest.java      |   21 +-
 .../gemfire/cache/query/QueryTestUtils.java     |   21 +-
 .../cache/query/QueryTestUtilsJUnitTest.java    |   21 +-
 .../gemfire/cache/query/RegionJUnitTest.java    |   21 +-
 .../cache/query/TypedIteratorJUnitTest.java     |   21 +-
 .../com/gemstone/gemfire/cache/query/Utils.java |   21 +-
 .../query/cq/dunit/CqQueryTestListener.java     |   21 +-
 .../gemfire/cache/query/data/Address.java       |   21 +-
 .../gemstone/gemfire/cache/query/data/City.java |   22 +-
 .../cache/query/data/CollectionHolder.java      |   22 +-
 .../cache/query/data/ComparableWrapper.java     |   22 +-
 .../gemfire/cache/query/data/Country.java       |   21 +-
 .../gemstone/gemfire/cache/query/data/Data.java |   22 +-
 .../gemfire/cache/query/data/District.java      |   22 +-
 .../gemfire/cache/query/data/Employee.java      |   21 +-
 .../gemfire/cache/query/data/Inventory.java     |   21 +-
 .../gemfire/cache/query/data/Keywords.java      |   21 +-
 .../gemfire/cache/query/data/Manager.java       |   21 +-
 .../gemfire/cache/query/data/Numbers.java       |   21 +-
 .../gemfire/cache/query/data/PhoneNo.java       |   21 +-
 .../gemfire/cache/query/data/Portfolio.java     |   22 +-
 .../gemfire/cache/query/data/PortfolioData.java |   21 +-
 .../gemfire/cache/query/data/PortfolioNoDS.java |   16 +
 .../gemfire/cache/query/data/PortfolioPdx.java  |   25 +-
 .../gemfire/cache/query/data/Position.java      |   21 +-
 .../gemfire/cache/query/data/PositionNoDS.java  |   16 +
 .../gemfire/cache/query/data/PositionPdx.java   |   21 +-
 .../query/data/ProhibitedSecurityQuote.java     |   21 +-
 .../gemfire/cache/query/data/Quote.java         |   21 +-
 .../gemfire/cache/query/data/Restricted.java    |   21 +-
 .../cache/query/data/SecurityMaster.java        |   21 +-
 .../gemfire/cache/query/data/State.java         |   21 +-
 .../gemfire/cache/query/data/Street.java        |   21 +-
 .../gemfire/cache/query/data/Student.java       |   23 +-
 .../gemfire/cache/query/data/Vehicle.java       |   21 +-
 .../gemfire/cache/query/data/Village.java       |   21 +-
 .../query/dunit/CloseCacheAuthorization.java    |   16 +
 .../query/dunit/CompactRangeIndexDUnitTest.java |   21 +-
 .../cache/query/dunit/CqTimeTestListener.java   |   21 +-
 .../cache/query/dunit/GroupByDUnitImpl.java     |   16 +
 .../dunit/GroupByPartitionedQueryDUnitTest.java |   16 +
 .../query/dunit/GroupByQueryDUnitTest.java      |   16 +
 .../cache/query/dunit/HashIndexDUnitTest.java   |   21 +-
 .../cache/query/dunit/HelperTestCase.java       |   16 +
 .../dunit/NonDistinctOrderByDUnitImpl.java      |   16 +
 .../NonDistinctOrderByPartitionedDUnitTest.java |   16 +
 .../query/dunit/PdxStringQueryDUnitTest.java    |   37 +-
 .../dunit/QueryAPITestPartitionResolver.java    |   22 +-
 .../cache/query/dunit/QueryAuthorization.java   |   21 +-
 .../dunit/QueryDataInconsistencyDUnitTest.java  |   24 +-
 .../dunit/QueryIndexUsingXMLDUnitTest.java      |   21 +-
 .../QueryParamsAuthorizationDUnitTest.java      |   21 +-
 .../QueryUsingFunctionContextDUnitTest.java     |   47 +-
 .../query/dunit/QueryUsingPoolDUnitTest.java    |   63 +-
 .../cache/query/dunit/RemoteQueryDUnitTest.java |  138 +-
 ...esourceManagerWithQueryMonitorDUnitTest.java |   62 +-
 .../query/dunit/SelectStarQueryDUnitTest.java   |   21 +-
 .../cache/query/facets/lang/Address.java        |   23 +-
 .../gemfire/cache/query/facets/lang/Course.java |   23 +-
 .../cache/query/facets/lang/Department.java     |   23 +-
 .../query/facets/lang/DerivedEmployee.java      |   22 +-
 .../cache/query/facets/lang/Employee.java       |   21 +-
 .../cache/query/facets/lang/Faculty.java        |   21 +-
 .../cache/query/facets/lang/G_Student.java      |   23 +-
 .../gemfire/cache/query/facets/lang/Person.java |   23 +-
 .../cache/query/facets/lang/Student.java        |   23 +-
 .../cache/query/facets/lang/UG_Student.java     |   23 +-
 .../gemfire/cache/query/facets/lang/Utils.java  |   20 +-
 .../ComparisonOperatorsJUnitTest.java           |   21 +-
 .../query/functional/ConstantsJUnitTest.java    |   21 +-
 .../query/functional/CountStarJUnitTest.java    |   21 +-
 .../CustomerOptimizationsJUnitTest.java         |   21 +-
 .../DistinctAndNonDistinctQueryJUnitTest.java   |   29 +-
 ...ctResultsWithDupValuesInRegionJUnitTest.java |   21 +-
 .../query/functional/FunctionJUnitTest.java     |   21 +-
 .../functional/GroupByPartitionedJUnitTest.java |   16 +
 .../functional/GroupByReplicatedJUnitTest.java  |   16 +
 .../cache/query/functional/GroupByTestImpl.java |   23 +-
 .../query/functional/GroupByTestInterface.java  |   16 +
 .../query/functional/INOperatorJUnitTest.java   |   21 +-
 .../functional/IUM6Bug32345ReJUnitTest.java     |   21 +-
 .../cache/query/functional/IUMJUnitTest.java    |   21 +-
 .../IUMRCompositeIteratorJUnitTest.java         |   21 +-
 .../IUMRMultiIndexesMultiRegionJUnitTest.java   |   21 +-
 .../IUMRShuffleIteratorsJUnitTest.java          |   21 +-
 .../functional/IUMRSingleRegionJUnitTest.java   |   21 +-
 ...ependentOperandsInWhereClause2JUnitTest.java |   21 +-
 .../IndexCreationDeadLockJUnitTest.java         |   21 +-
 .../functional/IndexCreationJUnitTest.java      |   23 +-
 .../IndexMaintenanceAsynchJUnitTest.java        |   21 +-
 .../functional/IndexOperatorJUnitTest.java      |   21 +-
 .../IndexPrimaryKeyUsageJUnitTest.java          |   21 +-
 .../IndexUsageInNestedQueryJUnitTest.java       |   21 +-
 .../IndexUsageWithAliasAsProjAtrbt.java         |   21 +-
 ...IndexUsageWithAliasAsProjAtrbtJUnitTest.java |   21 +-
 .../IndexUseMultFrmSnglCondJUnitTest.java       |   21 +-
 ...ndexWithSngleFrmAndMultCondQryJUnitTest.java |   21 +-
 .../functional/IteratorTypeDefEmpJUnitTest.java |   21 +-
 .../functional/IteratorTypeDefJUnitTest.java    |   21 +-
 .../IteratorTypeDefaultTypesJUnitTest.java      |   21 +-
 .../functional/IumMultConditionJUnitTest.java   |   21 +-
 .../functional/JavaSerializationJUnitTest.java  |   21 +-
 .../functional/LikePredicateJUnitTest.java      |   21 +-
 .../query/functional/LimitClauseJUnitTest.java  |   21 +-
 .../functional/LogicalOperatorsJUnitTest.java   |   21 +-
 .../cache/query/functional/MiscJUnitTest.java   |   21 +-
 .../functional/MultiIndexCreationJUnitTest.java |   16 +
 .../MultiRegionIndexUsageJUnitTest.java         |   21 +-
 .../functional/MultipleRegionsJUnitTest.java    |   21 +-
 .../NegativeNumberQueriesJUnitTest.java         |   21 +-
 .../query/functional/NestedQueryJUnitTest.java  |   21 +-
 .../NonDistinctOrderByPartitionedJUnitTest.java |   16 +
 .../NonDistinctOrderByReplicatedJUnitTest.java  |   16 +
 .../NonDistinctOrderByTestImplementation.java   |   21 +-
 .../query/functional/NumericQueryJUnitTest.java |   21 +-
 .../functional/OrderByPartitionedJUnitTest.java |   16 +
 .../functional/OrderByReplicatedJUnitTest.java  |   16 +
 .../functional/OrderByTestImplementation.java   |   21 +-
 .../functional/ParameterBindingJUnitTest.java   |   21 +-
 .../PdxGroupByPartitionedJUnitTest.java         |   16 +
 .../PdxGroupByReplicatedJUnitTest.java          |   16 +
 .../query/functional/PdxGroupByTestImpl.java    |   16 +
 .../query/functional/PdxOrderByJUnitTest.java   |   16 +
 .../functional/QRegionInterfaceJUnitTest.java   |   21 +-
 .../QueryREUpdateInProgressJUnitTest.java       |   21 +-
 .../functional/QueryUndefinedJUnitTest.java     |   21 +-
 .../functional/ReservedKeywordsJUnitTest.java   |   21 +-
 .../ResultsDataSerializabilityJUnitTest.java    |   21 +-
 .../query/functional/SelectToDateJUnitTest.java |   21 +-
 .../functional/StructMemberAccessJUnitTest.java |   21 +-
 .../query/functional/StructSetOrResultsSet.java |  281 +-
 .../query/functional/TestNewFunctionSSorRS.java |   21 +-
 .../CompiledAggregateFunctionJUnitTest.java     |   16 +
 .../CompiledGroupBySelectJUnitTest.java         |   16 +
 .../CompiledJunctionInternalsJUnitTest.java     |   21 +-
 .../internal/CopyOnReadQueryJUnitTest.java      |   21 +-
 .../internal/ExecutionContextJUnitTest.java     |   21 +-
 .../query/internal/IndexManagerJUnitTest.java   |   21 +-
 .../internal/NWayMergeResultsJUnitTest.java     |   16 +
 .../internal/OrderByComparatorJUnitTest.java    |   16 +
 .../internal/ProjectionAttributeJUnitTest.java  |   21 +-
 .../query/internal/QCompilerJUnitTest.java      |   21 +-
 ...ueryFromClauseCanonicalizationJUnitTest.java |   21 +-
 .../QueryObjectSerializationJUnitTest.java      |   21 +-
 .../QueryObserverCallbackJUnitTest.java         |   21 +-
 .../query/internal/QueryTraceJUnitTest.java     |   21 +-
 .../query/internal/QueryUtilsJUnitTest.java     |   21 +-
 .../query/internal/ResultsBagJUnitTest.java     |   21 +-
 .../ResultsBagLimitBehaviourJUnitTest.java      |   21 +-
 .../ResultsCollectionWrapperLimitJUnitTest.java |   21 +-
 .../SelectResultsComparatorJUnitTest.java       |   21 +-
 .../StructBagLimitBehaviourJUnitTest.java       |   21 +-
 .../query/internal/StructSetJUnitTest.java      |   21 +-
 .../internal/aggregate/AggregatorJUnitTest.java |   16 +
 ...syncIndexUpdaterThreadShutdownJUnitTest.java |   21 +-
 .../index/AsynchIndexMaintenanceJUnitTest.java  |   21 +-
 .../CompactRangeIndexIndexMapJUnitTest.java     |   21 +-
 .../index/CompactRangeIndexJUnitTest.java       |   21 +-
 ...rrentIndexInitOnOverflowRegionDUnitTest.java |   25 +-
 ...ndexOperationsOnOverflowRegionDUnitTest.java |   21 +-
 ...pdateWithInplaceObjectModFalseDUnitTest.java |   21 +-
 ...ConcurrentIndexUpdateWithoutWLDUnitTest.java |   21 +-
 .../index/CopyOnReadIndexDUnitTest.java         |   96 +-
 .../index/CopyOnReadIndexJUnitTest.java         |   21 +-
 .../DeclarativeIndexCreationJUnitTest.java      |   21 +-
 .../internal/index/HashIndexJUnitTest.java      |   44 +-
 .../internal/index/HashIndexSetJUnitTest.java   |  504 ++
 .../index/IndexCreationInternalsJUnitTest.java  |   21 +-
 .../internal/index/IndexElemArrayJUnitTest.java |   21 +-
 .../internal/index/IndexHintJUnitTest.java      |   16 +
 .../query/internal/index/IndexJUnitTest.java    |   21 +-
 .../index/IndexMaintainceJUnitTest.java         |   21 +-
 .../index/IndexMaintenanceJUnitTest.java        |   21 +-
 .../index/IndexStatisticsJUnitTest.java         |   21 +-
 .../IndexTrackingQueryObserverDUnitTest.java    |   21 +-
 .../IndexTrackingQueryObserverJUnitTest.java    |   21 +-
 .../query/internal/index/IndexUseJUnitTest.java |   73 +-
 .../IndexedMergeEquiJoinScenariosJUnitTest.java |   21 +-
 ...itializeIndexEntryDestroyQueryDUnitTest.java |   21 +-
 .../internal/index/MapIndexStoreJUnitTest.java  |   21 +-
 .../MapRangeIndexMaintenanceJUnitTest.java      |   21 +-
 .../index/MultiIndexCreationDUnitTest.java      |   23 +-
 .../NewDeclarativeIndexCreationJUnitTest.java   |   21 +-
 .../index/PdxCopyOnReadQueryJUnitTest.java      |   17 +
 ...gRegionCreationIndexUpdateTypeJUnitTest.java |   21 +-
 .../PutAllWithIndexPerfDUnitDisabledTest.java   |   24 +-
 .../internal/index/RangeIndexAPIJUnitTest.java  |   23 +-
 .../PRBasicIndexCreationDUnitTest.java          |   20 +-
 .../PRBasicIndexCreationDeadlockDUnitTest.java  |   20 +-
 .../PRBasicMultiIndexCreationDUnitTest.java     |   20 +-
 .../partitioned/PRBasicQueryDUnitTest.java      |   20 +-
 .../PRBasicRemoveIndexDUnitTest.java            |   21 +-
 .../PRColocatedEquiJoinDUnitTest.java           |   21 +-
 .../partitioned/PRIndexStatisticsJUnitTest.java |   21 +-
 .../partitioned/PRInvalidQueryDUnitTest.java    |   20 +-
 .../partitioned/PRInvalidQueryJUnitTest.java    |   21 +-
 .../partitioned/PRQueryCacheCloseDUnitTest.java |   20 +-
 .../PRQueryCacheClosedJUnitTest.java            |   21 +-
 .../query/partitioned/PRQueryDUnitHelper.java   |   20 +-
 .../query/partitioned/PRQueryDUnitTest.java     |   20 +-
 .../query/partitioned/PRQueryJUnitTest.java     |   21 +-
 .../partitioned/PRQueryNumThreadsJUnitTest.java |   21 +-
 .../query/partitioned/PRQueryPerfDUnitTest.java |   20 +-
 .../PRQueryRegionCloseDUnitTest.java            |   20 +-
 .../PRQueryRegionClosedJUnitTest.java           |   21 +-
 .../PRQueryRegionDestroyedDUnitTest.java        |   20 +-
 .../PRQueryRegionDestroyedJUnitTest.java        |   21 +-
 .../PRQueryRemoteNodeExceptionDUnitTest.java    |   21 +-
 .../gemfire/cache/query/transaction/Person.java |   21 +-
 .../query/transaction/QueryAndJtaJUnitTest.java |   21 +-
 .../internal/ConnectionCountProbeJUnitTest.java |   21 +-
 .../cache/snapshot/CacheSnapshotJUnitTest.java  |   21 +-
 .../snapshot/ParallelSnapshotDUnitTest.java     |   21 +-
 .../gemfire/cache/snapshot/RegionGenerator.java |   21 +-
 .../cache/snapshot/RegionSnapshotJUnitTest.java |   21 +-
 .../snapshot/SnapshotByteArrayDUnitTest.java    |   21 +-
 .../cache/snapshot/SnapshotDUnitTest.java       |   21 +-
 .../snapshot/SnapshotPerformanceDUnitTest.java  |   21 +-
 .../cache/snapshot/SnapshotTestCase.java        |   21 +-
 .../cache/snapshot/WanSnapshotJUnitTest.java    |   21 +-
 .../cache/util/PasswordUtilJUnitTest.java       |   21 +-
 .../cache30/BridgeMembershipDUnitTest.java      | 1660 -------
 .../BridgeMembershipSelectorDUnitTest.java      |   16 -
 .../gemfire/cache30/BridgeTestCase.java         |  376 --
 .../gemfire/cache30/BridgeWriterDUnitTest.java  |  418 --
 .../cache30/BridgeWriterSelectorDUnitTest.java  |   16 -
 .../gemfire/cache30/Bug34387DUnitTest.java      |   21 +-
 .../gemfire/cache30/Bug34948DUnitTest.java      |   21 +-
 .../gemfire/cache30/Bug35214DUnitTest.java      |   21 +-
 .../gemfire/cache30/Bug38013DUnitTest.java      |   21 +-
 .../gemfire/cache30/Bug38741DUnitTest.java      |   31 +-
 .../cache30/Bug40255JUnitDisabledTest.java      |   22 +-
 .../cache30/Bug40662JUnitDisabledTest.java      |   21 +-
 .../gemfire/cache30/Bug44418JUnitTest.java      |   21 +-
 .../gemfire/cache30/CacheCloseDUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheListenerTestCase.java  |   21 +-
 .../gemfire/cache30/CacheLoaderTestCase.java    |   21 +-
 .../gemfire/cache30/CacheLogRollDUnitTest.java  |   21 +-
 .../gemfire/cache30/CacheMapTxnDUnitTest.java   |   22 +-
 ...cheRegionsReliablityStatsCheckDUnitTest.java |   21 +-
 .../cache30/CacheSerializableRunnable.java      |   21 +-
 .../cache30/CacheStatisticsDUnitTest.java       |   21 +-
 .../gemstone/gemfire/cache30/CacheTestCase.java |  148 +-
 .../gemfire/cache30/CacheWriterTestCase.java    |   21 +-
 .../cache30/CacheXMLPartitionResolver.java      |   21 +-
 .../gemfire/cache30/CacheXml30DUnitTest.java    |  148 +-
 .../gemfire/cache30/CacheXml40DUnitTest.java    |   45 +-
 .../gemfire/cache30/CacheXml41DUnitTest.java    |   49 +-
 .../gemfire/cache30/CacheXml45DUnitTest.java    |   25 +-
 .../gemfire/cache30/CacheXml51DUnitTest.java    |   29 +-
 .../gemfire/cache30/CacheXml55DUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheXml57DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXml58DUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheXml60DUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheXml61DUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheXml65DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXml66DUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheXml70DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXml80DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXml81DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXml90DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXmlTestCase.java       |   16 +
 .../cache30/CachedAllEventsDUnitTest.java       |   21 +-
 .../gemfire/cache30/CallbackArgDUnitTest.java   |   21 +-
 .../cache30/CertifiableTestCacheListener.java   |   24 +-
 .../cache30/ClearMultiVmCallBkDUnitTest.java    |   22 +-
 .../gemfire/cache30/ClearMultiVmDUnitTest.java  |   22 +-
 .../cache30/ClientMembershipDUnitTest.java      | 1660 +++++++
 .../ClientMembershipSelectorDUnitTest.java      |   32 +
 .../ClientRegisterInterestDUnitTest.java        |  427 ++
 ...ClientRegisterInterestSelectorDUnitTest.java |   32 +
 .../cache30/ClientServerCCEDUnitTest.java       |   23 +-
 .../gemfire/cache30/ClientServerTestCase.java   |  385 ++
 .../ConcurrentLeaveDuringGIIDUnitTest.java      |   21 +-
 ...ibutedNoAckAsyncOverflowRegionDUnitTest.java |   22 +-
 ...iskDistributedNoAckAsyncRegionDUnitTest.java |   22 +-
 .../DiskDistributedNoAckRegionTestCase.java     |   22 +-
 ...ributedNoAckSyncOverflowRegionDUnitTest.java |   22 +-
 .../gemfire/cache30/DiskRegionDUnitTest.java    |   21 +-
 .../gemfire/cache30/DiskRegionTestImpl.java     |   22 +-
 .../cache30/DistAckMapMethodsDUnitTest.java     |   22 +-
 ...ckOverflowRegionCCECompressionDUnitTest.java |   61 -
 ...istributedAckOverflowRegionCCEDUnitTest.java |   21 +-
 ...tedAckOverflowRegionCCEOffHeapDUnitTest.java |   16 +
 ...PersistentRegionCCECompressionDUnitTest.java |   62 -
 ...tributedAckPersistentRegionCCEDUnitTest.java |   40 +-
 ...dAckPersistentRegionCCEOffHeapDUnitTest.java |   16 +
 .../DistributedAckRegionCCEDUnitTest.java       |   30 +-
 ...DistributedAckRegionCCEOffHeapDUnitTest.java |   16 +
 ...istributedAckRegionCompressionDUnitTest.java |   21 +-
 .../cache30/DistributedAckRegionDUnitTest.java  |   23 +-
 .../DistributedAckRegionOffHeapDUnitTest.java   |   16 +
 .../DistributedNoAckRegionCCEDUnitTest.java     |   39 +-
 ...stributedNoAckRegionCCEOffHeapDUnitTest.java |   16 +
 ...tributedNoAckRegionCompressionDUnitTest.java |   62 -
 .../DistributedNoAckRegionDUnitTest.java        |   21 +-
 .../DistributedNoAckRegionOffHeapDUnitTest.java |   16 +
 .../gemfire/cache30/DynamicRegionDUnitTest.java |   21 +-
 .../gemfire/cache30/GlobalLockingDUnitTest.java |   21 +-
 .../cache30/GlobalRegionCCEDUnitTest.java       |   25 +-
 .../GlobalRegionCCEOffHeapDUnitTest.java        |   16 +
 .../GlobalRegionCompressionDUnitTest.java       |   61 -
 .../gemfire/cache30/GlobalRegionDUnitTest.java  |   21 +-
 .../cache30/GlobalRegionOffHeapDUnitTest.java   |   16 +
 .../cache30/LRUEvictionControllerDUnitTest.java |   21 +-
 .../gemfire/cache30/LocalRegionDUnitTest.java   |   21 +-
 .../MemLRUEvictionControllerDUnitTest.java      |   21 +-
 .../gemfire/cache30/MultiVMRegionTestCase.java  |  352 +-
 .../gemfire/cache30/MyGatewayEventFilter1.java  |   21 +-
 .../gemfire/cache30/MyGatewayEventFilter2.java  |   23 +-
 .../cache30/MyGatewayTransportFilter1.java      |   21 +-
 .../cache30/MyGatewayTransportFilter2.java      |   21 +-
 .../OffHeapLRUEvictionControllerDUnitTest.java  |   21 +-
 .../PRBucketSynchronizationDUnitTest.java       |   26 +-
 .../PartitionedRegionCompressionDUnitTest.java  |   21 +-
 .../cache30/PartitionedRegionDUnitTest.java     |   21 +-
 ...tionedRegionMembershipListenerDUnitTest.java |   22 +-
 .../PartitionedRegionOffHeapDUnitTest.java      |   16 +
 .../cache30/PreloadedRegionTestCase.java        |   21 +-
 .../gemfire/cache30/ProxyDUnitTest.java         |   21 +-
 .../cache30/PutAllCallBkRemoteVMDUnitTest.java  |   22 +-
 .../cache30/PutAllCallBkSingleVMDUnitTest.java  |   22 +-
 .../gemfire/cache30/PutAllMultiVmDUnitTest.java |   22 +-
 .../gemfire/cache30/QueueMsgDUnitTest.java      |   21 +-
 .../cache30/RRSynchronizationDUnitTest.java     |   21 +-
 .../gemfire/cache30/ReconnectDUnitTest.java     |  171 +-
 .../ReconnectedCacheServerDUnitTest.java        |   31 +-
 .../cache30/RegionAttributesTestCase.java       |   21 +-
 .../cache30/RegionExpirationDUnitTest.java      |   21 +-
 .../RegionMembershipListenerDUnitTest.java      |   42 +-
 .../RegionReliabilityDistAckDUnitTest.java      |   21 +-
 .../RegionReliabilityDistNoAckDUnitTest.java    |   21 +-
 .../RegionReliabilityGlobalDUnitTest.java       |   21 +-
 .../RegionReliabilityListenerDUnitTest.java     |   21 +-
 .../cache30/RegionReliabilityTestCase.java      |  105 +-
 .../gemfire/cache30/RegionTestCase.java         |  113 +-
 .../gemfire/cache30/ReliabilityTestCase.java    |   21 +-
 .../cache30/RemoveAllMultiVmDUnitTest.java      |   22 +-
 .../gemfire/cache30/RequiredRolesDUnitTest.java |   21 +-
 .../cache30/RolePerformanceDUnitTest.java       |   21 +-
 .../gemfire/cache30/SearchAndLoadDUnitTest.java |   21 +-
 .../cache30/SlowRecDUnitDisabledTest.java       |   21 +-
 .../gemfire/cache30/TXDistributedDUnitTest.java |  119 +-
 .../gemfire/cache30/TXOrderDUnitTest.java       |   21 +-
 .../cache30/TXRestrictionsDUnitTest.java        |   21 +-
 .../gemfire/cache30/TestCacheCallback.java      |   21 +-
 .../gemfire/cache30/TestCacheListener.java      |   21 +-
 .../gemfire/cache30/TestCacheLoader.java        |   21 +-
 .../gemfire/cache30/TestCacheWriter.java        |   21 +-
 .../gemfire/cache30/TestDiskRegion.java         |   21 +-
 .../gemstone/gemfire/cache30/TestHeapLRU.java   |   21 +-
 .../gemfire/cache30/TestPdxSerializer.java      |   21 +-
 .../cache30/TestTransactionListener.java        |   21 +-
 .../gemfire/cache30/TestTransactionWriter.java  |   21 +-
 .../AnalyzeSerializablesJUnitTest.java          |   76 +-
 .../codeAnalysis/ClassAndMethodDetails.java     |   23 +-
 .../gemfire/codeAnalysis/ClassAndMethods.java   |   23 +-
 .../codeAnalysis/ClassAndVariableDetails.java   |   23 +-
 .../gemfire/codeAnalysis/ClassAndVariables.java |   23 +-
 .../codeAnalysis/CompiledClassUtils.java        |   23 +-
 .../codeAnalysis/decode/CompiledAttribute.java  |   21 +-
 .../codeAnalysis/decode/CompiledClass.java      |   25 +-
 .../codeAnalysis/decode/CompiledCode.java       |   21 +-
 .../codeAnalysis/decode/CompiledField.java      |   21 +-
 .../codeAnalysis/decode/CompiledMethod.java     |   21 +-
 .../gemfire/codeAnalysis/decode/cp/Cp.java      |   21 +-
 .../gemfire/codeAnalysis/decode/cp/CpClass.java |   21 +-
 .../codeAnalysis/decode/cp/CpDouble.java        |   21 +-
 .../codeAnalysis/decode/cp/CpFieldref.java      |   18 +-
 .../gemfire/codeAnalysis/decode/cp/CpFloat.java |   18 +-
 .../codeAnalysis/decode/cp/CpInteger.java       |   18 +-
 .../decode/cp/CpInterfaceMethodref.java         |   18 +-
 .../gemfire/codeAnalysis/decode/cp/CpLong.java  |   21 +-
 .../codeAnalysis/decode/cp/CpMethodref.java     |   18 +-
 .../codeAnalysis/decode/cp/CpNameAndType.java   |   18 +-
 .../codeAnalysis/decode/cp/CpString.java        |   18 +-
 .../gemfire/codeAnalysis/decode/cp/CpUtf8.java  |   21 +-
 .../distributed/AbstractLauncherJUnitTest.java  |   20 +-
 .../AbstractLauncherJUnitTestCase.java          |   16 +
 .../AbstractLauncherServiceStatusJUnitTest.java |   21 +-
 .../AbstractLocatorLauncherJUnitTestCase.java   |   16 +
 .../AbstractServerLauncherJUnitTestCase.java    |   20 +-
 .../gemfire/distributed/AuthInitializer.java    |   23 +-
 .../distributed/CommonLauncherTestSuite.java    |   20 +-
 .../distributed/DistributedMemberDUnitTest.java |   21 +-
 .../DistributedSystemConnectPerf.java           |   21 +-
 .../distributed/DistributedSystemDUnitTest.java |  139 +-
 .../distributed/DistributedTestSuite.java       |   16 +
 .../distributed/HostedLocatorsDUnitTest.java    |   16 +
 .../gemfire/distributed/JGroupsJUnitTest.java   |  171 -
 .../LauncherMemberMXBeanJUnitTest.java          |   16 +
 .../gemfire/distributed/LauncherTestSuite.java  |   16 +
 .../gemfire/distributed/LocatorDUnitTest.java   |  560 +--
 .../gemfire/distributed/LocatorJUnitTest.java   |  117 +-
 .../distributed/LocatorLauncherJUnitTest.java   |   45 +-
 .../LocatorLauncherLocalFileJUnitTest.java      |   16 +
 .../LocatorLauncherLocalJUnitTest.java          |  108 +-
 .../LocatorLauncherRemoteFileJUnitTest.java     |   28 +-
 .../LocatorLauncherRemoteJUnitTest.java         |   77 +-
 .../gemfire/distributed/MyAuthenticator.java    |   23 +-
 .../gemfire/distributed/MyPrincipal.java        |   18 +-
 .../gemfire/distributed/RoleDUnitTest.java      |   21 +-
 .../distributed/ServerLauncherJUnitTest.java    |   78 +-
 .../ServerLauncherLocalFileJUnitTest.java       |   20 +-
 .../ServerLauncherLocalJUnitTest.java           |  187 +-
 .../ServerLauncherRemoteFileJUnitTest.java      |   18 +-
 .../ServerLauncherRemoteJUnitTest.java          |   40 +-
 .../ServerLauncherWithSpringJUnitTest.java      |   18 +
 .../distributed/SystemAdminDUnitTest.java       |   21 +-
 .../AtomicLongWithTerminalStateJUnitTest.java   |   21 +-
 .../distributed/internal/Bug40751DUnitTest.java |   21 +-
 .../ConsoleDistributionManagerDUnitTest.java    |   23 +-
 .../distributed/internal/DateMessage.java       |   21 +-
 .../internal/DistributionAdvisorDUnitTest.java  |   21 +-
 .../internal/DistributionManagerDUnitTest.java  |  116 +-
 ...istributionManagerTimeDUnitDisabledTest.java |  211 -
 .../GemFireTimeSyncServiceDUnitTest.java        |  260 -
 .../InternalDistributedSystemJUnitTest.java     |   52 +-
 .../gemfire/distributed/internal/LDM.java       |   21 +-
 .../internal/LocatorLoadSnapshotJUnitTest.java  |   21 +-
 .../internal/ProduceDateMessages.java           |   21 +-
 .../internal/ProductUseLogDUnitTest.java        |   28 +-
 .../internal/ProductUseLogJUnitTest.java        |   21 +-
 .../internal/ServerLocatorJUnitTest.java        |   20 +-
 .../internal/SharedConfigurationJUnitTest.java  |   21 +-
 .../internal/StartupMessageDataJUnitTest.java   |   21 +-
 .../deadlock/DeadlockDetectorJUnitTest.java     |   21 +-
 .../deadlock/DependencyGraphJUnitTest.java      |   21 +-
 .../GemFireDeadlockDetectorDUnitTest.java       |   68 +-
 .../deadlock/UnsafeThreadLocalJUnitTest.java    |   21 +-
 .../locks/CollaborationJUnitDisabledTest.java   |   21 +-
 .../internal/locks/DLockGrantorHelper.java      |   21 +-
 ...entrantReadWriteWriteShareLockJUnitTest.java |   21 +-
 .../membership/MembershipJUnitTest.java         |  281 +-
 .../internal/membership/NetViewJUnitTest.java   |  194 +
 .../membership/gms/GMSMemberJUnitTest.java      |  148 +
 .../membership/gms/MembershipManagerHelper.java |  171 +
 .../gms/auth/GMSAuthenticatorJUnitTest.java     |  316 ++
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |  576 +++
 .../locator/GMSLocatorRecoveryJUnitTest.java    |  186 +
 .../gms/membership/GMSJoinLeaveJUnitTest.java   |  975 ++++
 .../gms/membership/StatRecorderJUnitTest.java   |  227 +
 .../messenger/GMSQuorumCheckerJUnitTest.java    |  361 ++
 .../membership/gms/messenger/InterceptUDP.java  |  109 +
 .../messenger/JGroupsMessengerJUnitTest.java    |  449 ++
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |  370 ++
 .../jgroup/MembershipManagerHelper.java         |  180 -
 .../StreamingOperationManyDUnitTest.java        |   29 +-
 .../StreamingOperationOneDUnitTest.java         |   29 +-
 .../tcpserver/LocatorVersioningJUnitTest.java   |  205 -
 ...cpServerBackwardCompatDUnitDisabledTest.java |   80 +-
 .../tcpserver/TcpServerJUnitDisabledTest.java   |  203 -
 .../internal/tcpserver/TcpServerJUnitTest.java  |  221 +
 .../support/DistributedSystemAdapter.java       |   21 +-
 .../gemfire/disttx/CacheMapDistTXDUnitTest.java |   16 +
 .../gemfire/disttx/DistTXDebugDUnitTest.java    |   29 +-
 .../disttx/DistTXDistributedTestSuite.java      |   16 +
 .../gemfire/disttx/DistTXExpiryJUnitTest.java   |   16 +
 .../gemfire/disttx/DistTXJUnitTest.java         |   16 +
 .../disttx/DistTXManagerImplJUnitTest.java      |   16 +
 .../gemfire/disttx/DistTXOrderDUnitTest.java    |   16 +
 .../disttx/DistTXPersistentDebugDUnitTest.java  |   19 +-
 .../DistTXReleasesOffHeapOnCloseJUnitTest.java  |   16 +
 .../disttx/DistTXRestrictionsDUnitTest.java     |   16 +
 .../disttx/DistTXWithDeltaDUnitTest.java        |   16 +
 .../gemfire/disttx/DistTXWriterJUnitTest.java   |   16 +
 .../disttx/DistTXWriterOOMEJUnitTest.java       |   16 +
 .../disttx/DistributedTransactionDUnitTest.java |   22 +-
 .../gemfire/disttx/PRDistTXDUnitTest.java       |   16 +
 .../gemfire/disttx/PRDistTXJUnitTest.java       |   16 +
 .../disttx/PRDistTXWithVersionsDUnitTest.java   |   16 +
 ...entPartitionedRegionWithDistTXDUnitTest.java |   16 +
 .../gemfire/internal/ArrayEqualsJUnitTest.java  |   21 +-
 .../gemfire/internal/AvailablePortHelper.java   |   23 +-
 .../internal/AvailablePortJUnitTest.java        |   21 +-
 ...wardCompatibilitySerializationDUnitTest.java |  300 ++
 ...wardCompatibilitySerializationJUnitTest.java |  306 --
 .../gemfire/internal/Bug49856JUnitTest.java     |   21 +-
 .../gemfire/internal/Bug51616JUnitTest.java     |   16 +
 .../gemfire/internal/ByteArrayData.java         |   21 +-
 .../gemstone/gemfire/internal/ClassBuilder.java |   21 +-
 .../ClassNotFoundExceptionDUnitTest.java        |   21 +-
 .../internal/ClassPathLoaderJUnitTest.java      |   21 +-
 .../internal/CopyOnWriteHashSetJUnitTest.java   |   21 +-
 .../internal/DataSerializableJUnitTest.java     |   21 +-
 .../gemstone/gemfire/internal/FDDUnitTest.java  |   53 +-
 .../gemfire/internal/FileUtilJUnitTest.java     |   21 +-
 .../internal/GemFireStatSamplerJUnitTest.java   |   21 +-
 .../GemFireVersionIntegrationJUnitTest.java     |   21 +-
 .../internal/GemFireVersionJUnitTest.java       |   21 +-
 .../internal/HeapDataOutputStreamJUnitTest.java |   21 +-
 .../gemfire/internal/InlineKeyJUnitTest.java    |   21 +-
 .../gemfire/internal/JSSESocketJUnitTest.java   |   23 +-
 .../internal/JarClassLoaderJUnitTest.java       |   22 +-
 .../gemfire/internal/JarDeployerDUnitTest.java  |   22 +-
 .../com/gemstone/gemfire/internal/JavaExec.java |   21 +-
 .../gemfire/internal/LineWrapUnitJUnitTest.java |   21 +-
 .../gemstone/gemfire/internal/LongBuffer.java   |   21 +-
 .../gemfire/internal/NanoTimer2JUnitTest.java   |   79 -
 .../gemfire/internal/NanoTimerJUnitTest.java    |  142 +-
 .../gemfire/internal/ObjIdMapJUnitTest.java     |   21 +-
 .../internal/OneTaskOnlyDecoratorJUnitTest.java |   21 +-
 .../internal/PdxDeleteFieldDUnitTest.java       |   29 +-
 .../internal/PdxDeleteFieldJUnitTest.java       |   16 +
 .../gemfire/internal/PdxRenameDUnitTest.java    |   31 +-
 .../gemfire/internal/PdxRenameJUnitTest.java    |   16 +
 .../PutAllOperationContextJUnitTest.java        |   21 +-
 .../internal/SSLConfigIntegrationJUnitTest.java |   16 +
 .../gemfire/internal/SSLConfigJUnitTest.java    |   20 +-
 ...hreadPoolExecutorWithKeepAliveJUnitTest.java |   21 +-
 .../internal/SimpleStatSamplerJUnitTest.java    |   21 +-
 .../gemfire/internal/SocketCloserJUnitTest.java |  196 +
 .../internal/SocketCloserWithWaitJUnitTest.java |   38 +
 .../StatArchiveWriterReaderJUnitTest.java       |   21 +-
 .../gemfire/internal/StatSamplerJUnitTest.java  |   21 +-
 .../gemfire/internal/StatSamplerTestCase.java   |   21 +-
 .../internal/UniqueIdGeneratorJUnitTest.java    |   21 +-
 .../internal/cache/AbstractRegionJUnitTest.java |   21 +-
 .../gemfire/internal/cache/BackupDUnitTest.java |   21 +-
 .../gemfire/internal/cache/BackupJUnitTest.java |   21 +-
 .../internal/cache/Bug33359DUnitTest.java       |   21 +-
 .../internal/cache/Bug33726DUnitTest.java       |   21 +-
 .../internal/cache/Bug33726JUnitTest.java       |   23 +-
 .../Bug34179TooManyFilesOpenJUnitTest.java      |   21 +-
 .../internal/cache/Bug34583JUnitTest.java       |   21 +-
 .../internal/cache/Bug37241DUnitTest.java       |   21 +-
 .../internal/cache/Bug37244JUnitTest.java       |   21 +-
 .../internal/cache/Bug37377DUnitTest.java       |   21 +-
 .../internal/cache/Bug37500JUnitTest.java       |   21 +-
 .../internal/cache/Bug39079DUnitTest.java       |   27 +-
 .../internal/cache/Bug40299DUnitTest.java       |   21 +-
 .../internal/cache/Bug40632DUnitTest.java       |   21 +-
 .../internal/cache/Bug41091DUnitTest.java       |   21 +-
 .../internal/cache/Bug41733DUnitTest.java       |   21 +-
 .../internal/cache/Bug41957DUnitTest.java       |   35 +-
 .../internal/cache/Bug42010StatsDUnitTest.java  |   21 +-
 .../internal/cache/Bug42055DUnitTest.java       |   21 +-
 .../internal/cache/Bug45164DUnitTest.java       |   21 +-
 .../internal/cache/Bug45934DUnitTest.java       |   21 +-
 .../internal/cache/Bug47667DUnitTest.java       |   21 +-
 .../internal/cache/Bug48182JUnitTest.java       |   16 +
 .../internal/cache/CacheAdvisorDUnitTest.java   |   21 +-
 .../cache/CacheLifecycleListenerJUnitTest.java  |   21 +-
 .../internal/cache/CacheServiceJUnitTest.java   |   59 +
 .../cache/ChunkValueWrapperJUnitTest.java       |   18 +-
 .../internal/cache/ClearDAckDUnitTest.java      |   21 +-
 .../internal/cache/ClearGlobalDUnitTest.java    |   21 +-
 ...ssagesRegionCreationAndDestroyJUnitTest.java |   35 +-
 .../cache/ClientServerGetAllDUnitTest.java      |  100 +-
 ...ServerInvalidAndDestroyedEntryDUnitTest.java |   21 +-
 .../ClientServerTransactionCCEDUnitTest.java    |   21 +-
 .../cache/ClientServerTransactionDUnitTest.java |  233 +-
 .../cache/ComplexDiskRegionJUnitTest.java       |   21 +-
 .../ConcurrentDestroySubRegionDUnitTest.java    |   21 +-
 ...entFlushingAndRegionOperationsJUnitTest.java |   21 +-
 .../cache/ConcurrentMapLocalJUnitTest.java      |   21 +-
 .../cache/ConcurrentMapOpsDUnitTest.java        |   98 +-
 .../ConcurrentRegionOperationsJUnitTest.java    |   21 +-
 ...rentRollingAndRegionOperationsJUnitTest.java |   21 +-
 .../internal/cache/ConflationJUnitTest.java     |   21 +-
 .../cache/ConnectDisconnectDUnitTest.java       |   57 +-
 .../cache/CustomerIDPartitionResolver.java      |   21 +-
 .../internal/cache/DeltaFaultInDUnitTest.java   |   21 +-
 .../cache/DeltaPropagationDUnitTest.java        |   29 +-
 .../cache/DeltaPropagationStatsDUnitTest.java   |   23 +-
 .../internal/cache/DeltaSizingDUnitTest.java    |   21 +-
 .../gemfire/internal/cache/DiskIFJUnitTest.java |   21 +-
 .../gemfire/internal/cache/DiskIdJUnitTest.java |   21 +-
 .../internal/cache/DiskInitFileJUnitTest.java   |   21 +-
 .../cache/DiskOfflineCompactionJUnitTest.java   |   21 +-
 .../internal/cache/DiskOldAPIsJUnitTest.java    |   21 +-
 ...iskRandomOperationsAndRecoveryJUnitTest.java |   21 +-
 .../cache/DiskRegByteArrayDUnitTest.java        |   21 +-
 .../cache/DiskRegCacheXmlJUnitTest.java         |   21 +-
 .../DiskRegCachexmlGeneratorJUnitTest.java      |   21 +-
 .../internal/cache/DiskRegCbkChkJUnitTest.java  |   21 +-
 .../DiskRegOplogSwtchingAndRollerJUnitTest.java |   21 +-
 .../cache/DiskRegRecoveryJUnitTest.java         |   21 +-
 .../cache/DiskRegionAsyncRecoveryJUnitTest.java |   21 +-
 ...RegionChangingRegionAttributesJUnitTest.java |   21 +-
 .../cache/DiskRegionClearJUnitTest.java         |   21 +-
 .../internal/cache/DiskRegionHelperFactory.java |   21 +-
 .../DiskRegionIllegalArguementsJUnitTest.java   |   21 +-
 ...iskRegionIllegalCacheXMLvaluesJUnitTest.java |   21 +-
 .../internal/cache/DiskRegionJUnitTest.java     |   60 +-
 .../internal/cache/DiskRegionProperties.java    |   21 +-
 .../internal/cache/DiskRegionTestingBase.java   |   21 +-
 .../cache/DiskStoreFactoryJUnitTest.java        |   21 +-
 .../cache/DiskWriteAttributesJUnitTest.java     |   21 +-
 ...DistrbutedRegionProfileOffHeapDUnitTest.java |   16 +
 .../cache/DistributedCacheTestCase.java         |   21 +-
 .../cache/EnumListenerEventJUnitTest.java       |   21 +-
 .../internal/cache/EventTrackerDUnitTest.java   |   29 +-
 .../cache/EvictionDUnitDisabledTest.java        |   21 +-
 .../cache/EvictionObjectSizerDUnitTest.java     |   21 +-
 .../internal/cache/EvictionStatsDUnitTest.java  |   21 +-
 .../internal/cache/EvictionTestBase.java        |   21 +-
 .../internal/cache/FaultingInJUnitTest.java     |   21 +-
 .../cache/FixedPRSinglehopDUnitTest.java        |   21 +-
 .../internal/cache/GIIDeltaDUnitTest.java       |   23 +-
 .../internal/cache/GIIFlowControlDUnitTest.java |   21 +-
 .../internal/cache/GridAdvisorDUnitTest.java    |  127 +-
 .../internal/cache/HABug36773DUnitTest.java     |   29 +-
 .../HAOverflowMemObjectSizerDUnitTest.java      |   33 +-
 .../HDFSQueueRegionOperationsJUnitTest.java     |   33 -
 ...FSQueueRegionOperationsOffHeapJUnitTest.java |   54 -
 .../cache/HDFSRegionOperationsJUnitTest.java    |  542 ---
 .../HDFSRegionOperationsOffHeapJUnitTest.java   |   78 -
 .../cache/IncrementalBackupDUnitTest.java       |   21 +-
 .../cache/InterruptClientServerDUnitTest.java   |   21 +-
 .../internal/cache/InterruptDiskJUnitTest.java  |   21 +-
 ...InterruptsConserveSocketsFalseDUnitTest.java |   16 +
 .../internal/cache/InterruptsDUnitTest.java     |   21 +-
 .../internal/cache/IteratorDUnitTest.java       |   21 +-
 .../LIFOEvictionAlgoEnabledRegionJUnitTest.java |   21 +-
 ...victionAlgoMemoryEnabledRegionJUnitTest.java |   21 +-
 .../internal/cache/MapClearGIIDUnitTest.java    |   21 +-
 .../internal/cache/MapInterface2JUnitTest.java  |   21 +-
 .../internal/cache/MapInterfaceJUnitTest.java   |   21 +-
 .../internal/cache/MockCacheService.java        |   24 +
 .../internal/cache/MockCacheServiceImpl.java    |   39 +
 .../MultipleOplogsRollingFeatureJUnitTest.java  |   21 +-
 .../cache/NetSearchMessagingDUnitTest.java      |   51 +-
 .../cache/OffHeapEvictionDUnitTest.java         |   23 +-
 .../cache/OffHeapEvictionStatsDUnitTest.java    |   21 +-
 .../gemfire/internal/cache/OffHeapTestUtil.java |   26 +-
 .../cache/OfflineSnapshotJUnitTest.java         |   21 +-
 .../gemfire/internal/cache/OldVLJUnitTest.java  |   21 +-
 .../cache/OldValueImporterTestBase.java         |  181 +
 .../cache/OplogEntryIdMapJUnitTest.java         |   21 +-
 .../cache/OplogEntryIdSetJUnitTest.java         |   21 +-
 .../gemfire/internal/cache/OplogJUnitTest.java  |   67 +-
 .../internal/cache/OplogRVVJUnitTest.java       |   21 +-
 .../cache/OrderedTombstoneMapJUnitTest.java     |   21 +-
 .../cache/P2PDeltaPropagationDUnitTest.java     |   21 +-
 .../internal/cache/PRBadToDataDUnitTest.java    |   21 +-
 .../cache/PRConcurrentMapOpsJUnitTest.java      |   21 +-
 .../cache/PRDataStoreMemoryJUnitTest.java       |   21 +-
 .../PRDataStoreMemoryOffHeapJUnitTest.java      |   16 +
 .../gemfire/internal/cache/PRTXJUnitTest.java   |   21 +-
 .../cache/PartitionAttributesImplJUnitTest.java |   16 +
 .../cache/PartitionListenerDUnitTest.java       |   21 +-
 ...dRegionAPIConserveSocketsFalseDUnitTest.java |   21 +-
 .../cache/PartitionedRegionAPIDUnitTest.java    |   20 +-
 .../PartitionedRegionAsSubRegionDUnitTest.java  |   20 +-
 ...gionBucketCreationDistributionDUnitTest.java |   20 +-
 .../PartitionedRegionCacheCloseDUnitTest.java   |   20 +-
 ...rtitionedRegionCacheLoaderForRootRegion.java |   21 +-
 ...artitionedRegionCacheLoaderForSubRegion.java |   21 +-
 ...rtitionedRegionCacheXMLExampleDUnitTest.java |   21 +-
 .../PartitionedRegionCreationDUnitTest.java     |   20 +-
 .../PartitionedRegionCreationJUnitTest.java     |   20 +-
 .../cache/PartitionedRegionDUnitTestCase.java   |   20 +-
 .../PartitionedRegionDataStoreJUnitTest.java    |   20 +-
 ...rtitionedRegionDelayedRecoveryDUnitTest.java |   21 +-
 .../PartitionedRegionDestroyDUnitTest.java      |   20 +-
 .../PartitionedRegionEntryCountDUnitTest.java   |   21 +-
 .../PartitionedRegionEvictionDUnitTest.java     |   21 +-
 .../cache/PartitionedRegionHADUnitTest.java     |   20 +-
 ...onedRegionHAFailureAndRecoveryDUnitTest.java |   20 +-
 .../cache/PartitionedRegionHelperJUnitTest.java |   21 +-
 .../PartitionedRegionInvalidateDUnitTest.java   |   21 +-
 ...artitionedRegionLocalMaxMemoryDUnitTest.java |   20 +-
 ...nedRegionLocalMaxMemoryOffHeapDUnitTest.java |   16 +
 .../PartitionedRegionMultipleDUnitTest.java     |   20 +-
 ...rtitionedRegionOffHeapEvictionDUnitTest.java |   22 +-
 .../cache/PartitionedRegionPRIDDUnitTest.java   |   21 +-
 .../cache/PartitionedRegionQueryDUnitTest.java  |   21 +-
 ...artitionedRegionQueryEvaluatorJUnitTest.java |   21 +-
 ...artitionedRegionRedundancyZoneDUnitTest.java |   25 +-
 ...tionedRegionSerializableObjectJUnitTest.java |   21 +-
 .../PartitionedRegionSingleHopDUnitTest.java    |   80 +-
 ...RegionSingleHopWithServerGroupDUnitTest.java |   21 +-
 ...onedRegionSingleNodeOperationsJUnitTest.java |   20 +-
 .../cache/PartitionedRegionSizeDUnitTest.java   |   20 +-
 .../cache/PartitionedRegionStatsDUnitTest.java  |   20 +-
 .../cache/PartitionedRegionStatsJUnitTest.java  |   21 +-
 .../cache/PartitionedRegionTestHelper.java      |   20 +-
 .../PartitionedRegionTestUtilsDUnitTest.java    |   20 +-
 .../PartitionedRegionWithSameNameDUnitTest.java |   21 +-
 .../PersistentPartitionedRegionJUnitTest.java   |   16 +
 .../internal/cache/PutAllDAckDUnitTest.java     |   21 +-
 .../internal/cache/PutAllGlobalDUnitTest.java   |   21 +-
 .../cache/RegionEntryFlagsJUnitTest.java        |   21 +-
 .../internal/cache/RegionListenerJUnitTest.java |   63 +
 .../cache/RemotePutReplyMessageJUnitTest.java   |   53 +
 .../cache/RemoteTransactionCCEDUnitTest.java    |   16 +
 .../cache/RemoteTransactionDUnitTest.java       |   95 +-
 .../internal/cache/RemoveAllDAckDUnitTest.java  |   21 +-
 .../internal/cache/RemoveDAckDUnitTest.java     |   21 +-
 .../internal/cache/RemoveGlobalDUnitTest.java   |   21 +-
 .../internal/cache/RunCacheInOldGemfire.java    |   21 +-
 .../cache/SimpleDiskRegionJUnitTest.java        |   21 +-
 .../internal/cache/SizingFlagDUnitTest.java     |   21 +-
 .../internal/cache/SnapshotTestUtil.java        |   16 +
 .../internal/cache/SystemFailureDUnitTest.java  |   21 +-
 .../internal/cache/TXManagerImplJUnitTest.java  |   21 +-
 .../cache/TXReservationMgrJUnitTest.java        |   21 +-
 .../gemfire/internal/cache/TestDelta.java       |   21 +-
 .../internal/cache/TestHelperForHydraTests.java |   16 +
 .../internal/cache/TestNonSizerObject.java      |   21 +-
 .../internal/cache/TestObjectSizerImpl.java     |   21 +-
 .../gemfire/internal/cache/TestUtils.java       |   21 +-
 .../cache/TombstoneCreationJUnitTest.java       |   21 +-
 .../cache/TransactionsWithDeltaDUnitTest.java   |   21 +-
 .../internal/cache/UnitTestValueHolder.java     |   18 +-
 .../gemfire/internal/cache/UnzipUtil.java       |   21 +-
 .../internal/cache/UpdateVersionJUnitTest.java  |   21 +-
 .../gemfire/internal/cache/VLJUnitTest.java     |   21 +-
 .../cache/control/FilterByPathJUnitTest.java    |   21 +-
 .../cache/control/MemoryMonitorJUnitTest.java   |   21 +-
 .../control/MemoryMonitorOffHeapJUnitTest.java  |   23 +-
 .../control/MemoryThresholdsJUnitTest.java      |   16 +
 .../control/RebalanceOperationDUnitTest.java    |  205 +-
 .../control/TestMemoryThresholdListener.java    |   34 +-
 ...skRegOverflowAsyncGetInMemPerfJUnitTest.java |   21 +-
 ...iskRegOverflowAsyncJUnitPerformanceTest.java |   21 +-
 ...lowSyncGetInMemPerfJUnitPerformanceTest.java |   21 +-
 ...DiskRegOverflowSyncJUnitPerformanceTest.java |   21 +-
 ...egionOverflowAsyncRollingOpLogJUnitTest.java |   21 +-
 ...RegionOverflowSyncRollingOpLogJUnitTest.java |   21 +-
 .../DiskRegionPerfJUnitPerformanceTest.java     |   21 +-
 .../DiskRegionPersistOnlySyncJUnitTest.java     |   21 +-
 ...DiskRegionRollOpLogJUnitPerformanceTest.java |   21 +-
 ...ltiThreadedOplogPerJUnitPerformanceTest.java |   21 +-
 .../cache/execute/Bug51193DUnitTest.java        |   28 +-
 .../ClientServerFunctionExecutionDUnitTest.java |   21 +-
 .../execute/ColocationFailoverDUnitTest.java    |   21 +-
 .../cache/execute/CustomResultCollector.java    |   21 +-
 .../execute/CustomerIDPartitionResolver.java    |   21 +-
 ...ributedRegionFunctionExecutionDUnitTest.java |   21 +-
 .../FunctionExecution_ExceptionDUnitTest.java   |   21 +-
 .../execute/FunctionServiceStatsDUnitTest.java  |   64 +-
 .../cache/execute/LocalDataSetDUnitTest.java    |   21 +-
 .../cache/execute/LocalDataSetFunction.java     |   21 +-
 .../execute/LocalDataSetIndexingDUnitTest.java  |   21 +-
 .../LocalFunctionExecutionDUnitTest.java        |   21 +-
 .../MemberFunctionExecutionDUnitTest.java       |   70 +-
 .../MultiRegionFunctionExecutionDUnitTest.java  |   21 +-
 .../execute/MyFunctionExecutionException.java   |   21 +-
 .../cache/execute/MyTransactionFunction.java    |   21 +-
 .../OnGroupsFunctionExecutionDUnitTest.java     |   51 +-
 ...ntServerFunctionExecutionNoAckDUnitTest.java |   21 +-
 ...tServerRegionFunctionExecutionDUnitTest.java |   21 +-
 ...egionFunctionExecutionFailoverDUnitTest.java |   21 +-
 ...onFunctionExecutionNoSingleHopDUnitTest.java |   21 +-
 ...onExecutionSelectorNoSingleHopDUnitTest.java |   21 +-
 ...gionFunctionExecutionSingleHopDUnitTest.java |   23 +-
 .../cache/execute/PRClientServerTestBase.java   |   21 +-
 .../cache/execute/PRColocationDUnitTest.java    |   21 +-
 .../execute/PRCustomPartitioningDUnitTest.java  |   21 +-
 .../execute/PRFunctionExecutionDUnitTest.java   |   21 +-
 .../PRFunctionExecutionTimeOutDUnitTest.java    |   22 +-
 ...ctionExecutionWithResultSenderDUnitTest.java |   21 +-
 .../execute/PRPerformanceTestDUnitTest.java     |   21 +-
 .../cache/execute/PRTransactionDUnitTest.java   |   21 +-
 .../PRTransactionWithVersionsDUnitTest.java     |   16 +
 .../internal/cache/execute/PerfFunction.java    |   21 +-
 .../internal/cache/execute/PerfTxFunction.java  |   21 +-
 .../cache/execute/PerformanceTestFunction.java  |   21 +-
 .../execute/SingleHopGetAllPutAllDUnitTest.java |   21 +-
 .../internal/cache/execute/TestFunction.java    |   21 +-
 .../internal/cache/execute/data/CustId.java     |   21 +-
 .../internal/cache/execute/data/Customer.java   |   21 +-
 .../internal/cache/execute/data/Order.java      |   21 +-
 .../internal/cache/execute/data/OrderId.java    |   21 +-
 .../internal/cache/execute/data/Shipment.java   |   21 +-
 .../internal/cache/execute/data/ShipmentId.java |   21 +-
 .../SimpleExtensionPointJUnitTest.java          |   21 +-
 .../extension/mock/AbstractMockExtension.java   |   21 +-
 .../mock/AbstractMockExtensionXmlGenerator.java |   21 +-
 .../mock/AlterMockCacheExtensionFunction.java   |   21 +-
 .../mock/AlterMockRegionExtensionFunction.java  |   21 +-
 .../mock/CreateMockCacheExtensionFunction.java  |   21 +-
 .../mock/CreateMockRegionExtensionFunction.java |   21 +-
 .../mock/DestroyMockCacheExtensionFunction.java |   23 +-
 .../DestroyMockRegionExtensionFunction.java     |   21 +-
 .../extension/mock/MockCacheExtension.java      |   21 +-
 .../mock/MockCacheExtensionXmlGenerator.java    |   21 +-
 .../extension/mock/MockExtensionCommands.java   |   21 +-
 .../extension/mock/MockExtensionXmlParser.java  |   21 +-
 .../extension/mock/MockRegionExtension.java     |   21 +-
 .../mock/MockRegionExtensionXmlGenerator.java   |   21 +-
 ...gionFunctionFunctionInvocationException.java |   21 +-
 .../functions/DistributedRegionFunction.java    |   21 +-
 .../cache/functions/LocalDataSetFunction.java   |   21 +-
 .../internal/cache/functions/TestFunction.java  |   31 +-
 .../ha/BlockingHARQAddOperationJUnitTest.java   |   21 +-
 .../cache/ha/BlockingHARQStatsJUnitTest.java    |   21 +-
 .../cache/ha/BlockingHARegionJUnitTest.java     |   21 +-
 .../ha/BlockingHARegionQueueJUnitTest.java      |   21 +-
 .../cache/ha/Bug36853EventsExpiryDUnitTest.java |   29 +-
 .../internal/cache/ha/Bug48571DUnitTest.java    |   27 +-
 .../internal/cache/ha/Bug48879DUnitTest.java    |   23 +-
 .../internal/cache/ha/ConflatableObject.java    |   21 +-
 .../cache/ha/EventIdOptimizationDUnitTest.java  |   33 +-
 .../cache/ha/EventIdOptimizationJUnitTest.java  |   20 +-
 .../internal/cache/ha/FailoverDUnitTest.java    |   43 +-
 .../internal/cache/ha/HABugInPutDUnitTest.java  |   29 +-
 .../internal/cache/ha/HAClearDUnitTest.java     |   31 +-
 .../cache/ha/HAConflationDUnitTest.java         |   29 +-
 .../internal/cache/ha/HADuplicateDUnitTest.java |   31 +-
 .../cache/ha/HAEventIdPropagationDUnitTest.java |   31 +-
 .../internal/cache/ha/HAExpiryDUnitTest.java    |   21 +-
 .../internal/cache/ha/HAGIIBugDUnitTest.java    |   21 +-
 .../internal/cache/ha/HAGIIDUnitTest.java       |   39 +-
 .../gemfire/internal/cache/ha/HAHelper.java     |   21 +-
 .../cache/ha/HARQAddOperationJUnitTest.java     |   21 +-
 .../cache/ha/HARQueueNewImplDUnitTest.java      |   87 +-
 .../internal/cache/ha/HARegionDUnitTest.java    |   21 +-
 .../internal/cache/ha/HARegionJUnitTest.java    |   20 +-
 .../cache/ha/HARegionQueueDUnitTest.java        |   21 +-
 .../cache/ha/HARegionQueueJUnitTest.java        |   20 +-
 ...HARegionQueueStartStopJUnitDisabledTest.java |   21 +-
 .../ha/HARegionQueueStartStopJUnitTest.java     |   21 +-
 .../cache/ha/HARegionQueueStatsJUnitTest.java   |   20 +-
 .../cache/ha/HASlowReceiverDUnitTest.java       |   31 +-
 .../ha/OperationsPropagationDUnitTest.java      |   29 +-
 .../internal/cache/ha/PutAllDUnitTest.java      |   33 +-
 .../cache/ha/StatsBugDUnitDisabledTest.java     |   35 +-
 .../cache/ha/TestBlockingHARegionQueue.java     |   21 +-
 .../cache/ha/ThreadIdentifierJUnitTest.java     |   21 +-
 .../cache/locks/TXLockServiceDUnitTest.java     |   23 +-
 .../internal/cache/lru/LRUClockJUnitTest.java   |   21 +-
 .../cache/partitioned/Bug39356DUnitTest.java    |   21 +-
 .../cache/partitioned/Bug43684DUnitTest.java    |   57 +-
 .../cache/partitioned/Bug47388DUnitTest.java    |   21 +-
 .../cache/partitioned/Bug51400DUnitTest.java    |   27 +-
 .../partitioned/ElidedPutAllDUnitTest.java      |   21 +-
 .../OfflineMembersDetailsJUnitTest.java         |   21 +-
 .../partitioned/PartitionResolverDUnitTest.java |   21 +-
 .../PartitionedRegionLoadModelJUnitTest.java    |   29 +-
 .../PartitionedRegionLoaderWriterDUnitTest.java |   21 +-
 ...rtitionedRegionMetaDataCleanupDUnitTest.java |   21 +-
 .../partitioned/PersistPRKRFDUnitTest.java      |   21 +-
 ...tentColocatedPartitionedRegionDUnitTest.java |   23 +-
 .../PersistentPartitionedRegionDUnitTest.java   |   27 +-
 ...tentPartitionedRegionOldConfigDUnitTest.java |   21 +-
 .../PersistentPartitionedRegionTestBase.java    |   21 +-
 ...rtitionedRegionWithTransactionDUnitTest.java |   21 +-
 .../PutPutReplyMessageJUnitTest.java            |   54 +
 .../cache/partitioned/ShutdownAllDUnitTest.java |   33 +-
 ...treamingPartitionOperationManyDUnitTest.java |   28 +-
 ...StreamingPartitionOperationOneDUnitTest.java |   29 +-
 .../fixed/CustomerFixedPartitionResolver.java   |   21 +-
 .../fixed/FixedPartitioningDUnitTest.java       |   21 +-
 .../fixed/FixedPartitioningTestBase.java        |   27 +-
 ...ngWithColocationAndPersistenceDUnitTest.java |   21 +-
 .../cache/partitioned/fixed/MyDate1.java        |   21 +-
 .../cache/partitioned/fixed/MyDate2.java        |   21 +-
 .../cache/partitioned/fixed/MyDate3.java        |   21 +-
 .../fixed/QuarterPartitionResolver.java         |   21 +-
 .../SingleHopQuarterPartitionResolver.java      |   21 +-
 .../persistence/BackupInspectorJUnitTest.java   |   21 +-
 .../PersistentRVVRecoveryDUnitTest.java         |   21 +-
 .../PersistentRecoveryOrderDUnitTest.java       |   21 +-
 ...rsistentRecoveryOrderOldConfigDUnitTest.java |   21 +-
 .../PersistentReplicatedTestBase.java           |   21 +-
 .../TemporaryResultSetFactoryJUnitTest.java     |   21 +-
 .../cache/persistence/soplog/AppendLog.java     |   56 -
 .../ArraySerializedComparatorJUnitTest.java     |   86 -
 .../CompactionSortedOplogSetTestCase.java       |  125 -
 .../persistence/soplog/CompactionTestCase.java  |  197 -
 .../persistence/soplog/ComparisonTestCase.java  |   68 -
 .../soplog/IndexComparatorJUnitTest.java        |   70 -
 .../LexicographicalComparatorJUnitTest.java     |  195 -
 .../soplog/RecoverableSortedOplogSet.java       |  212 -
 .../soplog/SizeTieredCompactorJUnitTest.java    |  101 -
 .../SizeTieredSortedOplogSetJUnitTest.java      |   27 -
 .../soplog/SortedBufferJUnitTest.java           |   23 -
 .../soplog/SortedOplogSetJUnitTest.java         |  264 -
 .../soplog/SortedReaderTestCase.java            |  286 --
 .../nofile/NoFileSortedOplogJUnitTest.java      |   39 -
 .../GFSnapshotJUnitPerformanceTest.java         |   21 +-
 .../internal/cache/tier/Bug40396DUnitTest.java  |   21 +-
 .../tier/sockets/AcceptorImplJUnitTest.java     |   83 +-
 ...rdCompatibilityCommandDUnitDisabledTest.java |  235 -
 ...CompatibilityHandshakeDUnitDisabledTest.java |  218 -
 ...mpatibilityHigherVersionClientDUnitTest.java |   25 +-
 ...rdCompatibilityMessageDUnitDisabledTest.java |  299 --
 .../BridgeServerMaxConnectionsJUnitTest.java    |  221 -
 ...geServerSelectorMaxConnectionsJUnitTest.java |   19 -
 .../tier/sockets/BridgeWriterMiscDUnitTest.java | 1384 ------
 .../BridgeWriterMiscSelectorDUnitTest.java      |   27 -
 .../cache/tier/sockets/Bug36269DUnitTest.java   |   25 +-
 .../cache/tier/sockets/Bug36457DUnitTest.java   |   39 +-
 .../cache/tier/sockets/Bug36805DUnitTest.java   |   27 +-
 .../cache/tier/sockets/Bug36829DUnitTest.java   |   26 +-
 .../cache/tier/sockets/Bug36995DUnitTest.java   |   34 +-
 .../cache/tier/sockets/Bug37210DUnitTest.java   |   53 +-
 .../cache/tier/sockets/Bug37805DUnitTest.java   |   24 +-
 .../CacheServerMaxConnectionsJUnitTest.java     |  229 +
 ...heServerSelectorMaxConnectionsJUnitTest.java |   35 +
 .../cache/tier/sockets/CacheServerTestUtil.java |   82 +-
 .../CacheServerTransactionsDUnitTest.java       |   25 +-
 ...acheServerTransactionsSelectorDUnitTest.java |   16 +
 .../tier/sockets/ClearPropagationDUnitTest.java |   25 +-
 .../tier/sockets/ClientConflationDUnitTest.java |   41 +-
 .../sockets/ClientHealthMonitorJUnitTest.java   |   27 +-
 .../ClientHealthMonitorSelectorJUnitTest.java   |   16 +
 .../sockets/ClientInterestNotifyDUnitTest.java  |   35 +-
 .../tier/sockets/ClientServerMiscDUnitTest.java | 1392 ++++++
 .../ClientServerMiscSelectorDUnitTest.java      |   36 +
 .../cache/tier/sockets/ConflationDUnitTest.java |   45 +-
 .../tier/sockets/ConnectionProxyJUnitTest.java  |  278 +-
 .../DataSerializerPropogationDUnitTest.java     |   57 +-
 .../cache/tier/sockets/DeltaEOFException.java   |   21 +-
 .../DestroyEntryPropagationDUnitTest.java       |   33 +-
 .../sockets/DurableClientBug39997DUnitTest.java |   21 +-
 .../DurableClientQueueSizeDUnitTest.java        |   37 +-
 .../DurableClientReconnectAutoDUnitTest.java    |   21 +-
 .../DurableClientReconnectDUnitTest.java        |   33 +-
 .../sockets/DurableClientStatsDUnitTest.java    |   43 +-
 .../sockets/DurableRegistrationDUnitTest.java   |   63 +-
 .../sockets/DurableResponseMatrixDUnitTest.java |   31 +-
 .../sockets/EventIDVerificationDUnitTest.java   |   31 +-
 .../EventIDVerificationInP2PDUnitTest.java      |   20 +-
 .../cache/tier/sockets/FaultyDelta.java         |   21 +-
 .../tier/sockets/FilterProfileJUnitTest.java    |   21 +-
 .../ForceInvalidateEvictionDUnitTest.java       |   31 +-
 ...ForceInvalidateOffHeapEvictionDUnitTest.java |   21 +-
 .../cache/tier/sockets/HABug36738DUnitTest.java |   21 +-
 .../cache/tier/sockets/HAInterestBaseTest.java  |   85 +-
 .../sockets/HAInterestDistributedTestCase.java  |   16 +
 .../tier/sockets/HAInterestPart1DUnitTest.java  |   22 +-
 .../tier/sockets/HAInterestPart2DUnitTest.java  |   57 +-
 .../sockets/HAStartupAndFailoverDUnitTest.java  |  102 +-
 .../internal/cache/tier/sockets/HaHelper.java   |   16 +
 .../tier/sockets/InterestListDUnitTest.java     |   25 +-
 .../sockets/InterestListEndpointDUnitTest.java  |   39 +-
 .../InterestListEndpointPRDUnitTest.java        |   21 +-
 .../InterestListEndpointSelectorDUnitTest.java  |   16 +
 .../sockets/InterestListFailoverDUnitTest.java  |   32 +-
 .../sockets/InterestListRecoveryDUnitTest.java  |   31 +-
 .../sockets/InterestRegrListenerDUnitTest.java  |   21 +-
 .../sockets/InterestResultPolicyDUnitTest.java  |   25 +-
 .../sockets/NewRegionAttributesDUnitTest.java   |   21 +-
 .../tier/sockets/ObjectPartListJUnitTest.java   |   21 +-
 .../tier/sockets/RedundancyLevelJUnitTest.java  |   34 +-
 .../sockets/RedundancyLevelPart1DUnitTest.java  |   25 +-
 .../sockets/RedundancyLevelPart2DUnitTest.java  |   21 +-
 .../sockets/RedundancyLevelPart3DUnitTest.java  |   27 +-
 .../tier/sockets/RedundancyLevelTestBase.java   |   72 +-
 .../tier/sockets/RegionCloseDUnitTest.java      |   35 +-
 ...erInterestBeforeRegionCreationDUnitTest.java |   27 +-
 .../sockets/RegisterInterestKeysDUnitTest.java  |   25 +-
 .../RegisterInterestKeysPRDUnitTest.java        |   21 +-
 .../sockets/ReliableMessagingDUnitTest.java     |   45 +-
 .../internal/cache/tier/sockets/TestPut.java    |   53 -
 .../sockets/UnregisterInterestDUnitTest.java    |   24 +-
 .../sockets/UpdatePropagationDUnitTest.java     |   31 +-
 .../sockets/UpdatePropagationPRDUnitTest.java   |   21 +-
 .../VerifyEventIDGenerationInP2PDUnitTest.java  |   20 +-
 ...UpdatesFromNonInterestEndPointDUnitTest.java |   25 +-
 .../tier/sockets/command/CommitCommandTest.java |   55 +
 .../cache/versions/RVVExceptionJUnitTest.java   |   21 +-
 .../versions/RegionVersionHolderJUnitTest.java  |   21 +-
 .../RegionVersionHolderRandomJUnitTest.java     |   21 +-
 ...RegionVersionHolderSmallBitSetJUnitTest.java |   21 +-
 .../versions/RegionVersionVectorJUnitTest.java  |  252 +-
 .../cache/wan/CompressionConstants.java         |   21 +-
 .../cache/wan/CompressionInputStream.java       |   21 +-
 .../cache/wan/CompressionOutputStream.java      |   21 +-
 .../cache/wan/CustomAsyncEventListener.java     |   21 +-
 .../gemfire/internal/cache/wan/Filter70.java    |   21 +-
 .../cache/wan/MyAsyncEventListener.java         |   21 +-
 .../cache/wan/MyAsyncEventListener2.java        |   21 +-
 .../cache/wan/MyDistributedSystemListener.java  |   21 +-
 .../cache/wan/MyGatewaySenderEventListener.java |   21 +-
 .../wan/MyGatewaySenderEventListener2.java      |   21 +-
 .../cache/wan/MyGatewayTransportFilter1.java    |   21 +-
 .../cache/wan/MyGatewayTransportFilter2.java    |   21 +-
 .../cache/wan/MyGatewayTransportFilter3.java    |   21 +-
 .../cache/wan/MyGatewayTransportFilter4.java    |   21 +-
 .../internal/cache/wan/QueueListener.java       |   21 +-
 .../AsyncEventQueueValidationsJUnitTest.java    |   21 +-
 .../xmlcache/AbstractXmlParserJUnitTest.java    |   21 +-
 .../cache/xmlcache/CacheCreationJUnitTest.java  |  209 +
 .../cache/xmlcache/CacheXmlParserJUnitTest.java |   21 +-
 .../xmlcache/CacheXmlVersionJUnitTest.java      |   17 +-
 .../PivotalEntityResolverJUnitTest.java         |   21 +-
 .../cache/xmlcache/RegionCreationJUnitTest.java |   21 +-
 .../xmlcache/XmlGeneratorUtilsJUnitTest.java    |   21 +-
 .../classpathloaderjunittest/DoesExist.java     |   16 +
 .../CompressionCacheConfigDUnitTest.java        |   21 +-
 .../CompressionCacheListenerDUnitTest.java      |   21 +-
 ...ompressionCacheListenerOffHeapDUnitTest.java |   16 +
 .../CompressionRegionConfigDUnitTest.java       |   22 +-
 .../CompressionRegionFactoryDUnitTest.java      |   21 +-
 .../CompressionRegionOperationsDUnitTest.java   |   21 +-
 ...ressionRegionOperationsOffHeapDUnitTest.java |   16 +
 .../compression/CompressionStatsDUnitTest.java  |   21 +-
 .../compression/SnappyCompressorJUnitTest.java  |   25 +-
 .../datasource/AbstractPoolCacheJUnitTest.java  |   21 +-
 .../internal/datasource/CleanUpJUnitTest.java   |   21 +-
 .../ConnectionPoolCacheImplJUnitTest.java       |   21 +-
 .../datasource/ConnectionPoolingJUnitTest.java  |   21 +-
 .../datasource/DataSourceFactoryJUnitTest.java  |   21 +-
 .../internal/datasource/RestartJUnitTest.java   |   21 +-
 .../internal/i18n/BasicI18nJUnitTest.java       |   82 +-
 .../io/CompositeOutputStreamJUnitTest.java      |   21 +-
 .../gemfire/internal/jndi/ContextJUnitTest.java |   21 +-
 .../internal/jta/BlockingTimeOutJUnitTest.java  |   21 +-
 .../gemfire/internal/jta/CacheUtils.java        |   21 +-
 .../internal/jta/DataSourceJTAJUnitTest.java    |   21 +-
 .../internal/jta/ExceptionJUnitTest.java        |   21 +-
 .../jta/GlobalTransactionJUnitTest.java         |   21 +-
 .../gemstone/gemfire/internal/jta/JTAUtils.java |   21 +-
 .../internal/jta/JtaIntegrationJUnitTest.java   |   16 +
 .../gemstone/gemfire/internal/jta/SyncImpl.java |   21 +-
 .../internal/jta/TransactionImplJUnitTest.java  |   21 +-
 .../jta/TransactionManagerImplJUnitTest.java    |   21 +-
 .../jta/TransactionTimeOutJUnitTest.java        |   21 +-
 .../jta/UserTransactionImplJUnitTest.java       |   21 +-
 .../internal/jta/dunit/CommitThread.java        |   21 +-
 .../internal/jta/dunit/ExceptionsDUnitTest.java |   21 +-
 .../jta/dunit/IdleTimeOutDUnitTest.java         |   21 +-
 .../jta/dunit/LoginTimeOutDUnitTest.java        |   21 +-
 .../jta/dunit/MaxPoolSizeDUnitTest.java         |   21 +-
 .../internal/jta/dunit/RollbackThread.java      |   21 +-
 .../jta/dunit/TransactionTimeOutDUnitTest.java  |   21 +-
 .../dunit/TxnManagerMultiThreadDUnitTest.java   |   21 +-
 .../internal/jta/dunit/TxnTimeOutDUnitTest.java |   21 +-
 .../internal/jta/functional/CacheJUnitTest.java |   21 +-
 .../jta/functional/TestXACacheLoader.java       |   21 +-
 .../internal/lang/ClassUtilsJUnitTest.java      |   21 +-
 .../internal/lang/InOutParameterJUnitTest.java  |   21 +-
 .../internal/lang/InitializerJUnitTest.java     |   21 +-
 .../internal/lang/ObjectUtilsJUnitTest.java     |   21 +-
 .../internal/lang/StringUtilsJUnitTest.java     |   21 +-
 .../internal/lang/SystemUtilsJUnitTest.java     |   21 +-
 .../internal/lang/ThreadUtilsJUnitTest.java     |   58 +-
 .../DistributedSystemLogFileJUnitTest.java      |   16 +
 .../logging/LocatorLogFileJUnitTest.java        |   16 +
 .../logging/LogServiceIntegrationJUnitTest.java |  223 +
 .../LogServiceIntegrationTestSupport.java       |   40 +
 .../internal/logging/LogServiceJUnitTest.java   |  204 +-
 .../LogWriterDisabledPerformanceTest.java       |   16 +
 .../logging/LogWriterImplJUnitTest.java         |   16 +
 .../logging/LogWriterPerformanceTest.java       |   16 +
 .../logging/LoggingIntegrationTestSuite.java    |   23 +-
 .../logging/LoggingPerformanceTestCase.java     |   16 +
 .../internal/logging/LoggingUnitTestSuite.java  |   32 +-
 .../logging/MergeLogFilesJUnitTest.java         |   25 +-
 .../gemfire/internal/logging/NullLogWriter.java |   16 +
 .../internal/logging/SortLogFileJUnitTest.java  |   21 +-
 .../internal/logging/TestLogWriterFactory.java  |   16 +
 .../logging/log4j/AlertAppenderJUnitTest.java   |   16 +
 .../logging/log4j/ConfigLocatorJUnitTest.java   |   16 +
 .../log4j/FastLoggerIntegrationJUnitTest.java   |   20 +-
 .../logging/log4j/FastLoggerJUnitTest.java      |   16 +
 .../FastLoggerWithDefaultConfigJUnitTest.java   |   20 +-
 .../log4j/LocalizedMessageJUnitTest.java        |   25 +-
 .../log4j/Log4J2DisabledPerformanceTest.java    |   16 +
 .../logging/log4j/Log4J2PerformanceTest.java    |   16 +
 .../log4j/Log4jIntegrationTestSuite.java        |   16 +
 .../logging/log4j/Log4jUnitTestSuite.java       |   16 +
 .../log4j/LogWriterAppenderJUnitTest.java       |   16 +
 .../LogWriterLoggerDisabledPerformanceTest.java |   16 +
 .../log4j/LogWriterLoggerPerformanceTest.java   |   16 +
 .../internal/net/SocketUtilsJUnitTest.java      |   21 +-
 .../offheap/ByteArrayMemoryChunkJUnitTest.java  |   16 +
 .../offheap/ConcurrentBagJUnitTest.java         |  114 -
 .../internal/offheap/DataTypeJUnitTest.java     |   16 +
 .../DirectByteBufferMemoryChunkJUnitTest.java   |   16 +
 .../offheap/FreeListOffHeapRegionJUnitTest.java |   18 +-
 .../HeapByteBufferMemoryChunkJUnitTest.java     |   16 +
 .../internal/offheap/InlineKeyJUnitTest.java    |   16 +
 .../offheap/MemoryChunkJUnitTestBase.java       |   63 +
 .../offheap/NullOffHeapMemoryStats.java         |   22 +
 .../offheap/NullOutOfOffHeapMemoryListener.java |   22 +
 .../internal/offheap/OffHeapIndexJUnitTest.java |   16 +
 .../internal/offheap/OffHeapRegionBase.java     |   17 +-
 .../offheap/OffHeapStorageJUnitTest.java        |  218 +
 .../offheap/OffHeapValidationJUnitTest.java     |   62 +-
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java |  115 +
 .../OldFreeListOffHeapRegionJUnitTest.java      |   18 +-
 .../offheap/OutOfOffHeapMemoryDUnitTest.java    |   43 +-
 ...mpleMemoryAllocatorFillPatternJUnitTest.java |   18 +-
 .../offheap/SimpleMemoryAllocatorJUnitTest.java |  394 +-
 ...moryAllocatorLifecycleListenerJUnitTest.java |   25 +-
 .../TxReleasesOffHeapOnCloseJUnitTest.java      |   16 +
 .../offheap/UnsafeMemoryChunkJUnitTest.java     |   16 +
 .../BlockingProcessStreamReaderJUnitTest.java   |   16 +
 .../LocalProcessControllerJUnitTest.java        |   21 +-
 .../process/LocalProcessLauncherDUnitTest.java  |   21 +-
 .../process/LocalProcessLauncherJUnitTest.java  |   21 +-
 ...NonBlockingProcessStreamReaderJUnitTest.java |   16 +
 .../internal/process/PidFileJUnitTest.java      |   16 +
 .../ProcessControllerFactoryJUnitTest.java      |   16 +
 .../process/ProcessStreamReaderTestCase.java    |   22 +-
 .../gemfire/internal/process/mbean/Process.java |   16 +
 .../internal/process/mbean/ProcessMBean.java    |   16 +
 ...tractSignalNotificationHandlerJUnitTest.java |   21 +-
 .../internal/size/ObjectSizerJUnitTest.java     |   35 +-
 .../internal/size/ObjectTraverserJUnitTest.java |   21 +-
 .../internal/size/ObjectTraverserPerf.java      |   21 +-
 .../size/SizeClassOnceObjectSizerJUnitTest.java |   21 +-
 .../gemfire/internal/size/SizeTestUtil.java     |   16 +
 .../size/WellKnownClassSizerJUnitTest.java      |   21 +-
 .../internal/statistics/DummyStatistics.java    |   21 +-
 .../statistics/SampleCollectorJUnitTest.java    |   21 +-
 .../statistics/StatMonitorHandlerJUnitTest.java |   21 +-
 .../statistics/StatisticsDUnitTest.java         |   21 +-
 .../statistics/StatisticsMonitorJUnitTest.java  |   21 +-
 .../internal/statistics/TestSampleHandler.java  |   21 +-
 .../statistics/TestStatArchiveWriter.java       |   21 +-
 .../statistics/TestStatisticsManager.java       |   21 +-
 .../statistics/TestStatisticsSampler.java       |   21 +-
 .../statistics/ValueMonitorJUnitTest.java       |   21 +-
 .../internal/stats50/AtomicStatsJUnitTest.java  |   37 +-
 .../internal/tcp/ConnectionJUnitTest.java       |   87 +
 .../util/AbortableTaskServiceJUnitTest.java     |   21 +-
 .../internal/util/ArrayUtilsJUnitTest.java      |   18 +-
 .../gemfire/internal/util/BytesJUnitTest.java   |   21 +-
 .../internal/util/CollectionUtilsJUnitTest.java |   18 +-
 .../internal/util/DelayedActionJUnitTest.java   |   21 +-
 .../gemfire/internal/util/IOUtilsJUnitTest.java |   21 +-
 .../gemfire/internal/util/SerializableImpl.java |   21 +-
 .../util/SerializableImplWithValue.java         |   21 +-
 .../gemfire/internal/util/Valuable.java         |   21 +-
 .../CompactConcurrentHashSetJUnitTest.java      |   16 +
 .../ConcurrentHashMapIteratorJUnitTest.java     |   21 +-
 .../concurrent/CopyOnWriteHashMapJUnitTest.java |  505 ++
 .../concurrent/ReentrantSemaphoreJUnitTest.java |   22 +-
 .../SemaphoreReadWriteLockJUnitTest.java        |   21 +-
 .../cm/ConcurrentHashMapJUnitTest.java          |   21 +-
 .../concurrent/cm/CountedMapLoopsJUnitTest.java |   21 +-
 .../concurrent/cm/IntMapCheckJUnitTest.java     |   21 +-
 .../util/concurrent/cm/LoopHelpers.java         |   21 +-
 .../util/concurrent/cm/MapCheckJUnitTest.java   |   21 +-
 .../util/concurrent/cm/MapLoopsJUnitTest.java   |   21 +-
 .../util/concurrent/cm/RLJBarJUnitTest.java     |   21 +-
 .../concurrent/cm/StringMapLoopsJUnitTest.java  |   21 +-
 .../management/CacheManagementDUnitTest.java    |   22 +-
 .../management/ClientHealthStatsDUnitTest.java  |   23 +-
 .../gemfire/management/CompositeStats.java      |   21 +-
 .../gemfire/management/CompositeTestMBean.java  |   21 +-
 .../gemfire/management/CompositeTestMXBean.java |   16 +
 .../management/CompositeTypeTestDUnitTest.java  |   21 +-
 .../gemfire/management/CustomMBean.java         |   22 +-
 .../gemfire/management/CustomMXBean.java        |   20 +-
 .../management/DLockManagementDUnitTest.java    |   20 +-
 .../DataBrowserJSONValidationJUnitTest.java     |   21 +-
 .../management/DiskManagementDUnitTest.java     |   20 +-
 .../management/DistributedSystemDUnitTest.java  |   81 +-
 .../management/LocatorManagementDUnitTest.java  |   25 +-
 .../gemstone/gemfire/management/MBeanUtil.java  |   20 +-
 .../gemfire/management/ManagementTestBase.java  |   21 +-
 .../MemberMBeanAttributesDUnitTest.java         |   20 +-
 .../management/OffHeapManagementDUnitTest.java  |   32 +-
 .../gemfire/management/QueryDataDUnitTest.java  |   21 +-
 .../management/RegionManagementDUnitTest.java   |   20 +-
 .../gemfire/management/TypedJsonJUnitTest.java  |   21 +-
 ...ersalMembershipListenerAdapterDUnitTest.java |   53 +-
 .../stats/AsyncEventQueueStatsJUnitTest.java    |   21 +-
 .../bean/stats/CacheServerStatsJUnitTest.java   |   21 +-
 .../bean/stats/DiskStatsJUnitTest.java          |   21 +-
 .../stats/DistributedSystemStatsDUnitTest.java  |   21 +-
 .../stats/DistributedSystemStatsJUnitTest.java  |   21 +-
 .../stats/GatewayReceiverStatsJUnitTest.java    |   21 +-
 .../bean/stats/GatewaySenderStatsJUnitTest.java |   21 +-
 .../HDFSRegionMBeanAttributeJUnitTest.java      |  169 -
 .../bean/stats/MBeanStatsTestCase.java          |   21 +-
 .../bean/stats/MemberLevelStatsJUnitTest.java   |   21 +-
 .../bean/stats/RegionStatsJUnitTest.java        |   21 +-
 .../bean/stats/StatsRateJUnitTest.java          |   21 +-
 .../internal/JettyHelperJUnitTest.java          |   21 +-
 .../cli/ClasspathScanLoadHelperJUnitTest.java   |   21 +-
 .../internal/cli/CliUtilDUnitTest.java          |   21 +-
 .../internal/cli/CommandManagerJUnitTest.java   |   21 +-
 .../cli/CommandSeparatorEscapeJUnitTest.java    |   16 +
 .../internal/cli/DataCommandJsonJUnitTest.java  |   16 +
 .../internal/cli/GfshParserJUnitTest.java       |   21 +-
 .../cli/annotations/CliArgumentJUnitTest.java   |   21 +-
 .../AbstractCommandsSupportJUnitTest.java       |   21 +-
 .../commands/DiskStoreCommandsJUnitTest.java    |   21 +-
 .../commands/HDFSStoreCommandsJUnitTest.java    |  838 ----
 .../HTTPServiceSSLSupportJUnitTest.java         |   20 +-
 .../cli/commands/IndexCommandsJUnitTest.java    |   21 +-
 .../RegionPathConverterJUnitTest.java           |   21 +-
 .../internal/cli/domain/AbstractImpl.java       |   16 +
 .../management/internal/cli/domain/Impl1.java   |   16 +
 .../management/internal/cli/domain/Impl12.java  |   16 +
 .../internal/cli/domain/Interface1.java         |   16 +
 .../internal/cli/domain/Interface2.java         |   16 +
 .../management/internal/cli/domain/Stock.java   |   21 +-
 .../management/internal/cli/dto/Car.java        |   16 +
 .../management/internal/cli/dto/Key1.java       |   21 +-
 .../management/internal/cli/dto/Key2.java       |   21 +-
 .../internal/cli/dto/ObjectWithCharAttr.java    |   21 +-
 .../management/internal/cli/dto/Value1.java     |   21 +-
 .../management/internal/cli/dto/Value2.java     |   21 +-
 .../AlterHDFSStoreFunctionJUnitTest.java        |  324 --
 .../CreateHDFSStoreFunctionJUnitTest.java       |  307 --
 .../functions/DataCommandFunctionJUnitTest.java |  132 +
 .../DescribeDiskStoreFunctionJUnitTest.java     |   21 +-
 .../DescribeHDFSStoreFunctionJUnitTest.java     |  364 --
 .../DestroyHDFSStoreFunctionJUnitTest.java      |  305 --
 .../ListDiskStoresFunctionJUnitTest.java        |   21 +-
 .../ListHDFSStoresFunctionJUnitTest.java        |  319 --
 .../functions/ListIndexFunctionJUnitTest.java   |   21 +-
 .../cli/parser/ParserUtilsJUnitTest.java        |   21 +-
 .../preprocessor/PreprocessorJUnitTest.java     |   21 +-
 .../PreprocessorUtilsJUnitTest.java             |   21 +-
 .../cli/shell/GfshConfigInitFileJUnitTest.java  |   16 +
 .../shell/GfshExecutionStrategyJUnitTest.java   |   21 +-
 .../cli/shell/GfshInitFileJUnitTest.java        |   16 +
 .../SharedConfigurationDUnitTest.java           |   21 +-
 .../configuration/ZipUtilsJUnitTest.java        |   21 +-
 .../domain/CacheElementJUnitTest.java           |   21 +-
 .../utils/XmlUtilsAddNewNodeJUnitTest.java      |   21 +-
 .../configuration/utils/XmlUtilsJUnitTest.java  |   21 +-
 .../internal/pulse/TestClientIdsDUnitTest.java  |   22 +-
 .../internal/pulse/TestFunctionsDUnitTest.java  |   22 +-
 .../internal/pulse/TestHeapDUnitTest.java       |   23 +-
 .../internal/pulse/TestLocatorsDUnitTest.java   |   22 +-
 .../pulse/TestSubscriptionsDUnitTest.java       |   20 +-
 .../internal/security/JSONAuthCodeTest.java     |   16 +
 .../security/JSONAuthorizationTest.java         |   16 +
 .../security/ResourceOperationJUnit.java        |   16 +
 .../ReadOpFileAccessControllerJUnitTest.java    |   21 +-
 .../WanCommandsControllerJUnitTest.java         |  140 +
 .../gemfire/management/model/EmptyObject.java   |   21 +-
 .../gemstone/gemfire/management/model/Item.java |   21 +-
 .../gemfire/management/model/Order.java         |   21 +-
 .../gemfire/management/model/SubOrder.java      |   21 +-
 .../DomainObjectsAsValuesJUnitTest.java         |   21 +-
 .../GemcachedBinaryClientJUnitTest.java         |   21 +-
 .../GemcachedDevelopmentJUnitTest.java          |   21 +-
 .../gemfire/memcached/IntegrationJUnitTest.java |   24 +-
 .../gemfire/pdx/AutoSerializableJUnitTest.java  |   21 +-
 .../gemfire/pdx/ByteSourceJUnitTest.java        |   20 +-
 .../ClientsWithVersioningRetryDUnitTest.java    |   21 +-
 .../com/gemstone/gemfire/pdx/DSInsidePdx.java   |   23 +-
 .../pdx/DistributedSystemIdDUnitTest.java       |   21 +-
 .../com/gemstone/gemfire/pdx/DomainObject.java  |   21 +-
 .../gemstone/gemfire/pdx/DomainObjectBad.java   |   16 +
 .../gemfire/pdx/DomainObjectClassLoadable.java  |   16 +
 .../gemfire/pdx/DomainObjectPdxAuto.java        |   21 +-
 ...DomainObjectPdxAutoNoDefaultConstructor.java |   21 +-
 .../java/com/gemstone/gemfire/pdx/Employee.java |   23 +-
 .../pdx/JSONPdxClientServerDUnitTest.java       |   23 +-
 .../com/gemstone/gemfire/pdx/NestedPdx.java     |   21 +-
 .../gemfire/pdx/NonDelegatingLoader.java        |   23 +-
 .../OffHeapByteBufferByteSourceJUnitTest.java   |   18 +-
 .../gemfire/pdx/OffHeapByteSourceJUnitTest.java |   18 +-
 .../pdx/PDXAsyncEventQueueDUnitTest.java        |   22 +-
 .../gemfire/pdx/PdxAttributesJUnitTest.java     |   21 +-
 .../gemfire/pdx/PdxClientServerDUnitTest.java   |   21 +-
 .../pdx/PdxDeserializationDUnitTest.java        |   21 +-
 .../pdx/PdxFormatterPutGetJUnitTest.java        |   25 +-
 .../com/gemstone/gemfire/pdx/PdxInsideDS.java   |   23 +-
 .../pdx/PdxInstanceFactoryJUnitTest.java        |   21 +-
 .../gemfire/pdx/PdxInstanceJUnitTest.java       |   21 +-
 .../gemfire/pdx/PdxSerializableDUnitTest.java   |   21 +-
 .../gemfire/pdx/PdxSerializableJUnitTest.java   |   21 +-
 .../gemfire/pdx/PdxStringJUnitTest.java         |   21 +-
 .../gemfire/pdx/PdxTypeExportDUnitTest.java     |   21 +-
 .../gemfire/pdx/SeparateClassloaderPdx.java     |   23 +-
 .../com/gemstone/gemfire/pdx/SimpleClass.java   |   23 +-
 .../com/gemstone/gemfire/pdx/SimpleClass1.java  |   21 +-
 .../com/gemstone/gemfire/pdx/SimpleClass2.java  |   21 +-
 .../gemfire/pdx/TestObjectForPdxFormatter.java  |   21 +-
 .../gemfire/pdx/VersionClassLoader.java         |   22 +-
 .../gemstone/gemfire/redis/AuthJUnitTest.java   |   18 +-
 .../gemfire/redis/ConcurrentStartTest.java      |   16 +
 .../gemstone/gemfire/redis/HashesJUnitTest.java |   16 +
 .../gemstone/gemfire/redis/ListsJUnitTest.java  |   16 +
 .../gemfire/redis/RedisDistDUnitTest.java       |   50 +-
 .../gemstone/gemfire/redis/SetsJUnitTest.java   |   16 +
 .../gemfire/redis/SortedSetsJUnitTest.java      |   20 +-
 .../gemfire/redis/StringsJunitTest.java         |   16 +
 .../web/controllers/AddFreeItemToOrders.java    |   21 +-
 .../rest/internal/web/controllers/Customer.java |   21 +-
 .../internal/web/controllers/DateTimeUtils.java |   16 +
 .../rest/internal/web/controllers/Gender.java   |   16 +
 .../internal/web/controllers/GetAllEntries.java |   21 +-
 .../web/controllers/GetDeliveredOrders.java     |   21 +-
 .../internal/web/controllers/GetRegions.java    |   21 +-
 .../web/controllers/GetValueForKey.java         |   21 +-
 .../rest/internal/web/controllers/Item.java     |   21 +-
 .../rest/internal/web/controllers/Order.java    |   21 +-
 .../rest/internal/web/controllers/Person.java   |   21 +-
 .../web/controllers/PutKeyFunction.java         |   21 +-
 .../web/controllers/RestAPITestBase.java        |   16 +
 .../internal/web/controllers/RestTestUtils.java |   21 +-
 .../gemfire/test/golden/ExecutableProcess.java  |   16 +
 .../gemfire/test/golden/FailOutputTestCase.java |   16 +
 .../golden/FailWithErrorInOutputJUnitTest.java  |   16 +
 .../FailWithExtraLineInOutputJUnitTest.java     |   16 +
 ...WithLineMissingFromEndOfOutputJUnitTest.java |   16 +
 ...hLineMissingFromMiddleOfOutputJUnitTest.java |   16 +
 .../FailWithLoggerErrorInOutputJUnitTest.java   |   16 +
 .../FailWithLoggerFatalInOutputJUnitTest.java   |   16 +
 .../FailWithLoggerWarnInOutputJUnitTest.java    |   16 +
 .../golden/FailWithProblemInOutputTestCase.java |   16 +
 .../golden/FailWithSevereInOutputJUnitTest.java |   16 +
 ...hTimeoutOfWaitForOutputToMatchJUnitTest.java |   16 +
 .../FailWithWarningInOutputJUnitTest.java       |   16 +
 .../gemfire/test/golden/GoldenComparator.java   |   16 +
 .../test/golden/GoldenStringComparator.java     |   16 +
 .../gemfire/test/golden/GoldenTestCase.java     |   16 +
 .../gemfire/test/golden/GoldenTestSuite.java    |   16 +
 .../gemfire/test/golden/PassJUnitTest.java      |   16 +
 .../golden/PassWithExpectedErrorJUnitTest.java  |   16 +
 .../golden/PassWithExpectedProblemTestCase.java |   16 +
 .../golden/PassWithExpectedSevereJUnitTest.java |   16 +
 .../PassWithExpectedWarningJUnitTest.java       |   16 +
 .../test/golden/RegexGoldenComparator.java      |   16 +
 .../test/golden/StringGoldenComparator.java     |   16 +
 .../gemfire/test/process/MainLauncher.java      |   16 +
 .../test/process/MainLauncherJUnitTest.java     |   16 +
 .../gemfire/test/process/OutputFormatter.java   |   16 +
 .../test/process/ProcessOutputReader.java       |   16 +
 .../test/process/ProcessStreamReader.java       |   16 +
 .../process/ProcessTestFrameworkTestSuite.java  |   16 +
 .../gemfire/test/process/ProcessWrapper.java    |   16 +
 .../test/process/ProcessWrapperJUnitTest.java   |   16 +
 .../gemstone/gemfire/util/JSR166TestCase.java   |   22 +-
 .../gemstone/gemfire/util/test/TestUtil.java    |   16 +
 .../protocols/CacheTimeSlowDownDUnitTest.java   |  204 -
 .../GemFireTimeSyncProtocolDUnitTest.java       |  477 --
 .../JGroupsFailureDetectionJUnitTest.java       |  488 --
 .../protocols/JGroupsVersioningJUnitTest.java   |  343 --
 .../com/gemstone/persistence/admin/Logger.java  |   21 +-
 .../gemstone/persistence/logging/Formatter.java |   21 +-
 .../gemstone/persistence/logging/Handler.java   |   21 +-
 .../com/gemstone/persistence/logging/Level.java |   21 +-
 .../gemstone/persistence/logging/LogRecord.java |   21 +-
 .../gemstone/persistence/logging/Logger.java    |   21 +-
 .../persistence/logging/SimpleFormatter.java    |   21 +-
 .../persistence/logging/StreamHandler.java      |   21 +-
 .../test/java/com/gemstone/sequence/Arrow.java  |   21 +-
 .../java/com/gemstone/sequence/Lifeline.java    |   21 +-
 .../com/gemstone/sequence/LifelineState.java    |   21 +-
 .../java/com/gemstone/sequence/LineMapper.java  |   21 +-
 .../com/gemstone/sequence/SequenceDiagram.java  |   21 +-
 .../com/gemstone/sequence/SequencePanel.java    |   21 +-
 .../com/gemstone/sequence/StateColorMap.java    |   21 +-
 .../java/com/gemstone/sequence/TimeAxis.java    |   21 +-
 .../com/gemstone/sequence/ZoomingPanel.java     |   21 +-
 .../sequence/gemfire/DefaultLineMapper.java     |   21 +-
 .../gemfire/GemfireSequenceDisplay.java         |   21 +-
 .../sequence/gemfire/HydraLineMapper.java       |   21 +-
 .../sequence/gemfire/SelectGraphDialog.java     |   21 +-
 .../com/main/MyDistributedSystemListener.java   |   21 +-
 .../com/main/WANBootStrapping_Site1_Add.java    |   21 +-
 .../com/main/WANBootStrapping_Site1_Remove.java |   21 +-
 .../com/main/WANBootStrapping_Site2_Add.java    |   21 +-
 .../com/main/WANBootStrapping_Site2_Remove.java |   21 +-
 .../src/test/java/dunit/AsyncInvocation.java    |   21 +-
 .../src/test/java/dunit/BounceResult.java       |   16 +
 gemfire-core/src/test/java/dunit/DUnitEnv.java  |   21 +-
 .../test/java/dunit/DistributedTestCase.java    |  150 +-
 gemfire-core/src/test/java/dunit/Host.java      |   21 +-
 .../src/test/java/dunit/RMIException.java       |   21 +-
 .../src/test/java/dunit/RemoteDUnitVMIF.java    |   16 +
 .../src/test/java/dunit/RepeatableRunnable.java |   16 +
 .../test/java/dunit/SerializableCallable.java   |   21 +-
 .../test/java/dunit/SerializableRunnable.java   |   21 +-
 gemfire-core/src/test/java/dunit/VM.java        |   21 +-
 .../src/test/java/dunit/standalone/ChildVM.java |   25 +-
 .../java/dunit/standalone/DUnitLauncher.java    |   49 +-
 .../java/dunit/standalone/ProcessManager.java   |   48 +-
 .../java/dunit/standalone/RemoteDUnitVM.java    |   21 +-
 .../dunit/standalone/StandAloneDUnitEnv.java    |   21 +-
 .../test/java/dunit/tests/BasicDUnitTest.java   |   21 +-
 .../src/test/java/dunit/tests/TestFailure.java  |   21 +-
 .../src/test/java/dunit/tests/VMDUnitTest.java  |   21 +-
 gemfire-core/src/test/java/hydra/GsRandom.java  |   21 +-
 .../test/java/hydra/HydraRuntimeException.java  |   21 +-
 gemfire-core/src/test/java/hydra/Log.java       |   21 +-
 .../src/test/java/hydra/LogVersionHelper.java   |   21 +-
 .../src/test/java/hydra/MethExecutor.java       |   21 +-
 .../src/test/java/hydra/MethExecutorResult.java |   21 +-
 .../src/test/java/hydra/SchedulingOrder.java    |   21 +-
 .../src/test/java/hydra/log/AnyLogWriter.java   |   23 +-
 .../java/hydra/log/CircularOutputStream.java    |   21 +-
 .../parReg/query/unittest/NewPortfolio.java     |   21 +-
 .../java/parReg/query/unittest/Position.java    |   22 +-
 .../src/test/java/perffmwk/Formatter.java       |   22 +-
 .../templates/security/DummyAuthenticator.java  |   21 +-
 .../templates/security/DummyAuthorization.java  |   21 +-
 .../security/FunctionSecurityPrmsHolder.java    |   21 +-
 .../security/LdapUserAuthenticator.java         |   21 +-
 .../java/templates/security/PKCSAuthInit.java   |   21 +-
 .../templates/security/PKCSAuthenticator.java   |   21 +-
 .../java/templates/security/PKCSPrincipal.java  |   21 +-
 .../security/UserPasswordAuthInit.java          |   21 +-
 .../templates/security/UsernamePrincipal.java   |   21 +-
 .../templates/security/XmlAuthorization.java    |   21 +-
 .../templates/security/XmlErrorHandler.java     |   21 +-
 .../src/test/java/util/TestException.java       |   21 +-
 ...gemstone.gemfire.internal.cache.CacheService |    1 +
 .../cache/client/internal/cacheserver.cer       |  Bin 0 -> 782 bytes
 .../cache/client/internal/cacheserver.keystore  |  Bin 0 -> 1253 bytes
 .../client/internal/cacheserver.truststore      |  Bin 0 -> 844 bytes
 .../gemfire/cache/client/internal/client.cer    |  Bin 0 -> 782 bytes
 .../cache/client/internal/client.keystore       |  Bin 0 -> 1251 bytes
 .../cache/client/internal/client.truststore     |  Bin 0 -> 846 bytes
 .../cache/client/internal/default.keystore      |  Bin 0 -> 1115 bytes
 .../cache/client/internal/trusted.keystore      |  Bin 0 -> 1078 bytes
 .../gemfire/codeAnalysis/excludedClasses.txt    |    3 +-
 .../sanctionedDataSerializables.txt             | 1835 ++++---
 .../codeAnalysis/sanctionedSerializables.txt    |  158 +-
 .../tier/sockets/RedundancyLevelJUnitTest.xml   |   21 +-
 .../resources/templates/security/authz5_5.dtd   |   16 +
 .../resources/templates/security/authz6_0.dtd   |   16 +
 gemfire-jgroups/build.gradle                    |   39 -
 .../java/com/gemstone/org/jgroups/Address.java  |   46 -
 .../com/gemstone/org/jgroups/BlockEvent.java    |   15 -
 .../java/com/gemstone/org/jgroups/Channel.java  |  508 --
 .../org/jgroups/ChannelClosedException.java     |   27 -
 .../gemstone/org/jgroups/ChannelException.java  |   28 -
 .../gemstone/org/jgroups/ChannelFactory.java    |   38 -
 .../gemstone/org/jgroups/ChannelListener.java   |   20 -
 .../jgroups/ChannelNotConnectedException.java   |   26 -
 .../java/com/gemstone/org/jgroups/Event.java    |  225 -
 .../com/gemstone/org/jgroups/ExitEvent.java     |   14 -
 .../com/gemstone/org/jgroups/GetStateEvent.java |   22 -
 .../java/com/gemstone/org/jgroups/Global.java   |   44 -
 .../java/com/gemstone/org/jgroups/Header.java   |   51 -
 .../java/com/gemstone/org/jgroups/JChannel.java | 1725 -------
 .../gemstone/org/jgroups/JChannelFactory.java   |  129 -
 .../gemstone/org/jgroups/JGroupsVersion.java    |   95 -
 .../com/gemstone/org/jgroups/Membership.java    |  358 --
 .../org/jgroups/MembershipListener.java         |   53 -
 .../com/gemstone/org/jgroups/MergeView.java     |  166 -
 .../java/com/gemstone/org/jgroups/Message.java  |  784 ---
 .../com/gemstone/org/jgroups/Message.java.old   |  711 ---
 .../gemstone/org/jgroups/MessageListener.java   |   34 -
 .../java/com/gemstone/org/jgroups/Receiver.java |   13 -
 .../gemstone/org/jgroups/ReceiverAdapter.java   |   35 -
 .../com/gemstone/org/jgroups/SetStateEvent.java |   37 -
 .../org/jgroups/ShunnedAddressException.java    |   14 -
 .../com/gemstone/org/jgroups/SuspectEvent.java  |   26 -
 .../com/gemstone/org/jgroups/SuspectMember.java |   42 -
 .../org/jgroups/SuspectedException.java         |   21 -
 .../gemstone/org/jgroups/TimeoutException.java  |   43 -
 .../com/gemstone/org/jgroups/Transport.java     |   22 -
 .../com/gemstone/org/jgroups/UpHandler.java     |   18 -
 .../java/com/gemstone/org/jgroups/View.java     |  528 --
 .../java/com/gemstone/org/jgroups/ViewId.java   |  174 -
 .../org/jgroups/blocks/ConnectionTable.java     | 1051 ----
 .../org/jgroups/blocks/ConnectionTableNIO.java  | 1519 ------
 .../jgroups/blocks/DistributedHashtable.java    |  656 ---
 .../jgroups/blocks/DistributedLockManager.java  |  742 ---
 .../org/jgroups/blocks/DistributedQueue.java    |  758 ---
 .../org/jgroups/blocks/DistributedTree.java     |  756 ---
 .../org/jgroups/blocks/GroupRequest.java        |  636 ---
 .../org/jgroups/blocks/GroupRequest.java.old    |  641 ---
 .../com/gemstone/org/jgroups/blocks/Link.java   |  680 ---
 .../com/gemstone/org/jgroups/blocks/Link.txt    |   48 -
 .../org/jgroups/blocks/LockManager.java         |   87 -
 .../blocks/LockMultiLockedException.java        |   26 -
 .../jgroups/blocks/LockNotGrantedException.java |   24 -
 .../blocks/LockNotReleasedException.java        |   24 -
 .../org/jgroups/blocks/LockingException.java    |   38 -
 .../org/jgroups/blocks/LogicalLink.java         |  340 --
 .../blocks/MembershipListenerAdapter.java       |   91 -
 .../org/jgroups/blocks/MessageDispatcher.java   |  845 ----
 .../jgroups/blocks/MessageListenerAdapter.java  |  123 -
 .../gemstone/org/jgroups/blocks/MethodCall.java |  529 --
 .../org/jgroups/blocks/MethodLookup.java        |   15 -
 .../org/jgroups/blocks/NBMessageForm_NIO.java   |   94 -
 .../org/jgroups/blocks/NotificationBus.java     |  458 --
 .../org/jgroups/blocks/PullPushAdapter.java     |  434 --
 .../org/jgroups/blocks/ReplicatedHashtable.java |  530 --
 .../org/jgroups/blocks/ReplicatedTree.java      | 1121 -----
 .../org/jgroups/blocks/ReplicationData.java     |  148 -
 .../org/jgroups/blocks/ReplicationManager.java  |  345 --
 .../org/jgroups/blocks/ReplicationReceiver.java |   69 -
 .../org/jgroups/blocks/RequestCorrelator.java   |  911 ----
 .../org/jgroups/blocks/RequestHandler.java      |   15 -
 .../org/jgroups/blocks/RpcDispatcher.java       |  392 --
 .../org/jgroups/blocks/RspCollector.java        |   18 -
 .../jgroups/blocks/TwoPhaseVotingAdapter.java   |  173 -
 .../jgroups/blocks/TwoPhaseVotingListener.java  |   35 -
 .../org/jgroups/blocks/UpdateException.java     |   19 -
 .../org/jgroups/blocks/VoteException.java       |   19 -
 .../jgroups/blocks/VoteResponseProcessor.java   |   31 -
 .../org/jgroups/blocks/VotingAdapter.java       |  472 --
 .../org/jgroups/blocks/VotingListener.java      |   26 -
 .../com/gemstone/org/jgroups/blocks/Xid.java    |  164 -
 .../gemstone/org/jgroups/blocks/package.html    |   13 -
 .../org/jgroups/conf/ClassConfigurator.java     |  228 -
 .../com/gemstone/org/jgroups/conf/ClassMap.java |   75 -
 .../jgroups/conf/ClassPathEntityResolver.java   |   63 -
 .../org/jgroups/conf/ConfiguratorFactory.java   |  449 --
 .../org/jgroups/conf/MagicNumberReader.java     |  402 --
 .../org/jgroups/conf/PlainConfigurator.java     |   48 -
 .../gemstone/org/jgroups/conf/ProtocolData.java |  130 -
 .../org/jgroups/conf/ProtocolParameter.java     |   64 -
 .../jgroups/conf/ProtocolStackConfigurator.java |   18 -
 .../org/jgroups/conf/XmlConfigurator.java       |  463 --
 .../gemstone/org/jgroups/conf/XmlValidator.java |  146 -
 .../com/gemstone/org/jgroups/conf/package.html  |    5 -
 .../gemstone/org/jgroups/debug/Debugger.java    |  133 -
 .../org/jgroups/debug/JChannelTestHook.java     |   14 -
 .../gemstone/org/jgroups/debug/Profiler.java    |  160 -
 .../org/jgroups/debug/ProtocolSpecificView.java |   24 -
 .../org/jgroups/debug/ProtocolTester.java       |  142 -
 .../org/jgroups/debug/ProtocolView.java         |   90 -
 .../gemstone/org/jgroups/debug/QUEUEView.java   |   28 -
 .../gemstone/org/jgroups/debug/Simulator.java   |  249 -
 .../com/gemstone/org/jgroups/debug/package.html |    5 -
 .../com/gemstone/org/jgroups/gemstonizing.txt   |   28 -
 .../org/jgroups/oswego/concurrent/Barrier.java  |   65 -
 .../oswego/concurrent/BoundedBuffer.java        |  190 -
 .../oswego/concurrent/BoundedChannel.java       |   37 -
 .../oswego/concurrent/BoundedLinkedQueue.java   |  384 --
 .../oswego/concurrent/BoundedPriorityQueue.java |  123 -
 .../concurrent/BrokenBarrierException.java      |   48 -
 .../org/jgroups/oswego/concurrent/Callable.java |   39 -
 .../org/jgroups/oswego/concurrent/Channel.java  |  309 --
 .../jgroups/oswego/concurrent/ClockDaemon.java  |  403 --
 .../org/jgroups/oswego/concurrent/CondVar.java  |  277 --
 .../jgroups/oswego/concurrent/CountDown.java    |  126 -
 .../oswego/concurrent/CyclicBarrier.java        |  299 --
 .../concurrent/DefaultChannelCapacity.java      |   58 -
 .../oswego/concurrent/DirectExecutor.java       |   36 -
 .../org/jgroups/oswego/concurrent/Executor.java |   70 -
 .../oswego/concurrent/FIFOReadWriteLock.java    |  198 -
 .../oswego/concurrent/FIFOSemaphore.java        |   84 -
 .../org/jgroups/oswego/concurrent/FJTask.java   |  535 --
 .../jgroups/oswego/concurrent/FJTaskRunner.java |  979 ----
 .../oswego/concurrent/FJTaskRunnerGroup.java    |  625 ---
 .../jgroups/oswego/concurrent/FutureResult.java |  209 -
 .../org/jgroups/oswego/concurrent/Heap.java     |  151 -
 .../org/jgroups/oswego/concurrent/Latch.java    |  103 -
 .../jgroups/oswego/concurrent/LayeredSync.java  |   96 -
 .../jgroups/oswego/concurrent/LinkedNode.java   |   29 -
 .../jgroups/oswego/concurrent/LinkedQueue.java  |  192 -
 .../oswego/concurrent/LockedExecutor.java       |   62 -
 .../org/jgroups/oswego/concurrent/Mutex.java    |  173 -
 .../org/jgroups/oswego/concurrent/NullSync.java |   51 -
 .../oswego/concurrent/PooledExecutor.java       |  924 ----
 .../oswego/concurrent/PrioritySemaphore.java    |   96 -
 .../concurrent/PropertyChangeMulticaster.java   |  466 --
 .../org/jgroups/oswego/concurrent/Puttable.java |   68 -
 .../oswego/concurrent/QueuedExecutor.java       |  220 -
 .../oswego/concurrent/QueuedSemaphore.java      |  181 -
 .../oswego/concurrent/ReadWriteLock.java        |   86 -
 .../ReaderPreferenceReadWriteLock.java          |   35 -
 .../oswego/concurrent/ReentrantLock.java        |  151 -
 .../ReentrantWriterPreferenceReadWriteLock.java |  169 -
 .../jgroups/oswego/concurrent/Rendezvous.java   |  422 --
 .../jgroups/oswego/concurrent/Semaphore.java    |  193 -
 .../concurrent/SemaphoreControlledChannel.java  |  164 -
 .../org/jgroups/oswego/concurrent/Slot.java     |   88 -
 .../org/jgroups/oswego/concurrent/Sync.java     |  344 --
 .../oswego/concurrent/SyncCollection.java       |  514 --
 .../org/jgroups/oswego/concurrent/SyncList.java |  327 --
 .../org/jgroups/oswego/concurrent/SyncMap.java  |  314 --
 .../org/jgroups/oswego/concurrent/SyncSet.java  |   82 -
 .../oswego/concurrent/SyncSortedMap.java        |  129 -
 .../oswego/concurrent/SyncSortedSet.java        |  129 -
 .../oswego/concurrent/SynchronizedBoolean.java  |  182 -
 .../oswego/concurrent/SynchronizedByte.java     |  253 -
 .../oswego/concurrent/SynchronizedChar.java     |  176 -
 .../oswego/concurrent/SynchronizedDouble.java   |  181 -
 .../oswego/concurrent/SynchronizedFloat.java    |  181 -
 .../oswego/concurrent/SynchronizedInt.java      |  250 -
 .../oswego/concurrent/SynchronizedLong.java     |  252 -
 .../oswego/concurrent/SynchronizedRef.java      |  107 -
 .../oswego/concurrent/SynchronizedShort.java    |  254 -
 .../oswego/concurrent/SynchronizedVariable.java |  209 -
 .../oswego/concurrent/SynchronousChannel.java   |  379 --
 .../org/jgroups/oswego/concurrent/Takable.java  |   67 -
 .../oswego/concurrent/ThreadFactory.java        |   35 -
 .../oswego/concurrent/ThreadFactoryUser.java    |   62 -
 .../oswego/concurrent/ThreadedExecutor.java     |   40 -
 .../oswego/concurrent/TimedCallable.java        |   68 -
 .../oswego/concurrent/TimeoutException.java     |   53 -
 .../jgroups/oswego/concurrent/TimeoutSync.java  |   75 -
 .../concurrent/VetoableChangeMulticaster.java   |  577 ---
 .../oswego/concurrent/WaitFreeQueue.java        |  208 -
 .../oswego/concurrent/WaitableBoolean.java      |  141 -
 .../jgroups/oswego/concurrent/WaitableByte.java |  238 -
 .../jgroups/oswego/concurrent/WaitableChar.java |  170 -
 .../oswego/concurrent/WaitableDouble.java       |  173 -
 .../oswego/concurrent/WaitableFloat.java        |  173 -
 .../jgroups/oswego/concurrent/WaitableInt.java  |  239 -
 .../jgroups/oswego/concurrent/WaitableLong.java |  239 -
 .../jgroups/oswego/concurrent/WaitableRef.java  |  112 -
 .../oswego/concurrent/WaitableShort.java        |  238 -
 .../concurrent/WaiterPreferenceSemaphore.java   |  155 -
 .../WriterPreferenceReadWriteLock.java          |  337 --
 .../org/jgroups/oswego/concurrent/package.html  |  998 ----
 .../java/com/gemstone/org/jgroups/overview.html |   15 -
 .../java/com/gemstone/org/jgroups/package.html  |    5 -
 .../persistence/CannotConnectException.java     |   55 -
 .../CannotCreateSchemaException.java            |   45 -
 .../persistence/CannotPersistException.java     |   44 -
 .../persistence/CannotRemoveException.java      |   45 -
 .../persistence/CannotRetrieveException.java    |   44 -
 .../persistence/DBPersistenceManager.java       |  877 ----
 .../persistence/FilePersistenceManager.java     |  173 -
 .../jgroups/persistence/PersistenceFactory.java |  207 -
 .../jgroups/persistence/PersistenceManager.java |   69 -
 .../org/jgroups/persistence/package.html        |    5 -
 .../com/gemstone/org/jgroups/primarychanges.txt |   80 -
 .../gemstone/org/jgroups/protocols/AUTH.java    |  438 --
 .../org/jgroups/protocols/AUTOCONF.java         |  249 -
 .../org/jgroups/protocols/AuthHeader.java       |  108 -
 .../gemstone/org/jgroups/protocols/CAUSAL.java  |  296 --
 .../org/jgroups/protocols/COMPRESS.java         |  188 -
 .../org/jgroups/protocols/DEADLOCK.java         |  250 -
 .../gemstone/org/jgroups/protocols/DELAY.java   |  116 -
 .../com/gemstone/org/jgroups/protocols/DESIGN   |  283 --
 .../gemstone/org/jgroups/protocols/DISCARD.java |  150 -
 .../gemstone/org/jgroups/protocols/DUMMY.java   |   85 -
 .../org/jgroups/protocols/DUMMY_TP.java         |   84 -
 .../gemstone/org/jgroups/protocols/Digest.java  |   38 -
 .../org/jgroups/protocols/Discovery.java        |  484 --
 .../gemstone/org/jgroups/protocols/ENCRYPT.java | 1390 ------
 .../org/jgroups/protocols/ENCRYPT1_4.java       |  566 ---
 .../gemstone/org/jgroups/protocols/EXAMPLE.java |  104 -
 .../com/gemstone/org/jgroups/protocols/FC.java  | 1025 ----
 .../com/gemstone/org/jgroups/protocols/FD.java  | 1100 -----
 .../gemstone/org/jgroups/protocols/FD_PID.java  |  640 ---
 .../gemstone/org/jgroups/protocols/FD_PROB.java |  628 ---
 .../org/jgroups/protocols/FD_SIMPLE.java        |  357 --
 .../gemstone/org/jgroups/protocols/FD_SOCK.java | 2654 ----------
 .../org/jgroups/protocols/FD_SOCK.java.new      | 1153 -----
 .../org/jgroups/protocols/FLOWCONTROL.java      |   89 -
 .../org/jgroups/protocols/FLOW_CONTROL.java     |  317 --
 .../gemstone/org/jgroups/protocols/FLUSH.java   |  466 --
 .../gemstone/org/jgroups/protocols/FRAG.java    |  572 ---
 .../gemstone/org/jgroups/protocols/FRAG2.java   |  773 ---
 .../gemstone/org/jgroups/protocols/FRAG3.java   |   35 -
 .../org/jgroups/protocols/FlushRsp.java         |   31 -
 .../org/jgroups/protocols/FragHeader.java       |   80 -
 .../gemstone/org/jgroups/protocols/GMS.java.rmi |  219 -
 .../org/jgroups/protocols/GemFireTimeSync.java  |  720 ---
 .../gemstone/org/jgroups/protocols/HDRS.java    |   53 -
 .../gemstone/org/jgroups/protocols/HTOTAL.java  |  207 -
 .../org/jgroups/protocols/LOOPBACK.java         |  115 -
 .../gemstone/org/jgroups/protocols/LOSS.java    |  118 -
 .../gemstone/org/jgroups/protocols/MERGE.java   |  376 --
 .../gemstone/org/jgroups/protocols/MERGE2.java  |  362 --
 .../gemstone/org/jgroups/protocols/MERGE3.java  |  312 --
 .../org/jgroups/protocols/MERGEFAST.java        |  118 -
 .../protocols/MessageProtocolEXAMPLE.java       |   64 -
 .../protocols/NAKACK.java.MessageProtocol       |  422 --
 .../org/jgroups/protocols/NAKACK.java.separate  |  484 --
 .../org/jgroups/protocols/NakAckHeader.java     |  113 -
 .../org/jgroups/protocols/PARTITIONER.java      |  185 -
 .../gemstone/org/jgroups/protocols/PERF.java    |  284 --
 .../gemstone/org/jgroups/protocols/PERF_TP.java |  172 -
 .../org/jgroups/protocols/PIGGYBACK.java        |  271 --
 .../gemstone/org/jgroups/protocols/PING.java    |  297 --
 .../org/jgroups/protocols/PRINTMETHODS.java     |   64 -
 .../org/jgroups/protocols/PRINTOBJS.java        |  120 -
 .../org/jgroups/protocols/PerfHeader.java       |  451 --
 .../org/jgroups/protocols/PingHeader.java       |   79 -
 .../gemstone/org/jgroups/protocols/PingRsp.java |  107 -
 .../org/jgroups/protocols/PingSender.java       |  109 -
 .../org/jgroups/protocols/PingWaiter.java       |  301 --
 .../gemstone/org/jgroups/protocols/QUEUE.java   |  184 -
 .../jgroups/protocols/RpcProtocolEXAMPLE.java   |   64 -
 .../gemstone/org/jgroups/protocols/SHUFFLE.java |  146 -
 .../gemstone/org/jgroups/protocols/SIZE.java    |  186 -
 .../gemstone/org/jgroups/protocols/SMACK.java   |  392 --
 .../org/jgroups/protocols/STATE_TRANSFER.java   |  448 --
 .../gemstone/org/jgroups/protocols/STATS.java   |  211 -
 .../com/gemstone/org/jgroups/protocols/TCP.java |  308 --
 .../org/jgroups/protocols/TCPGOSSIP.java        |  429 --
 .../gemstone/org/jgroups/protocols/TCPPING.java |  144 -
 .../gemstone/org/jgroups/protocols/TCP_NIO.java |  118 -
 .../gemstone/org/jgroups/protocols/TOTAL.java   | 1055 ----
 .../com/gemstone/org/jgroups/protocols/TP.java  | 2012 --------
 .../gemstone/org/jgroups/protocols/TP.java.mt   | 1522 ------
 .../gemstone/org/jgroups/protocols/TRACE.java   |   47 -
 .../gemstone/org/jgroups/protocols/TUNNEL.java  |  459 --
 .../org/jgroups/protocols/TcpHeader.java        |   55 -
 .../org/jgroups/protocols/TpHeader.java         |   64 -
 .../protocols/TransportedVectorTime.java        |  185 -
 .../org/jgroups/protocols/TunnelHeader.java     |   47 -
 .../com/gemstone/org/jgroups/protocols/UDP.java | 1462 ------
 .../gemstone/org/jgroups/protocols/UDP.java.mt  | 1005 ----
 .../jgroups/protocols/UDP.java.packethandler    |  592 ---
 .../gemstone/org/jgroups/protocols/UDP_NIO.java | 1567 ------
 .../gemstone/org/jgroups/protocols/UNICAST.java |  977 ----
 .../org/jgroups/protocols/UdpHeader.java        |   62 -
 .../org/jgroups/protocols/VERIFY_SUSPECT.java   |  470 --
 .../org/jgroups/protocols/VIEW_ENFORCER.java    |   81 -
 .../org/jgroups/protocols/VIEW_SYNC.java        |  499 --
 .../org/jgroups/protocols/VectorTime.java       |  270 -
 .../gemstone/org/jgroups/protocols/WANPING.java |  106 -
 .../gemstone/org/jgroups/protocols/WANPIPE.java |  441 --
 .../org/jgroups/protocols/WanPipeAddress.java   |  140 -
 .../gemstone/org/jgroups/protocols/dump2.log    |    0
 .../jgroups/protocols/obsolete/ENCRYPT.java.txt |  408 --
 .../org/jgroups/protocols/obsolete/FC.java.txt  |  643 ---
 .../jgroups/protocols/obsolete/FD_RAND.java.txt |  287 --
 .../jgroups/protocols/obsolete/FD_SHUN.java.txt |  323 --
 .../org/jgroups/protocols/obsolete/TCP.java.txt |  493 --
 .../org/jgroups/protocols/obsolete/UDP.java.txt | 1921 --------
 .../jgroups/protocols/obsolete/UNIFORM.java.txt |  349 --
 .../gemstone/org/jgroups/protocols/package.html |    6 -
 .../jgroups/protocols/pbcast/ClientGmsImpl.java |  907 ----
 .../jgroups/protocols/pbcast/CoordGmsImpl.java  | 1103 -----
 .../org/jgroups/protocols/pbcast/DESIGN         |  478 --
 .../org/jgroups/protocols/pbcast/Digest.java    |  534 --
 .../jgroups/protocols/pbcast/Digest.java.old    |  558 ---
 .../org/jgroups/protocols/pbcast/FD.java        |  273 --
 .../org/jgroups/protocols/pbcast/GMS.java       | 2732 -----------
 .../org/jgroups/protocols/pbcast/GmsImpl.java   |  111 -
 .../org/jgroups/protocols/pbcast/Gossip.java    |  134 -
 .../org/jgroups/protocols/pbcast/JoinRsp.java   |  126 -
 .../org/jgroups/protocols/pbcast/MergeData.java |  118 -
 .../org/jgroups/protocols/pbcast/NAKACK.java    | 1631 -------
 .../jgroups/protocols/pbcast/NakAckHeader.java  |  148 -
 .../org/jgroups/protocols/pbcast/PBCAST.java    | 1043 ----
 .../protocols/pbcast/ParticipantGmsImpl.java    |  371 --
 .../jgroups/protocols/pbcast/PbcastHeader.java  |  110 -
 .../org/jgroups/protocols/pbcast/README         |  140 -
 .../org/jgroups/protocols/pbcast/STABLE.java    |  907 ----
 .../jgroups/protocols/pbcast/STABLE.java.latest |  897 ----
 .../jgroups/protocols/pbcast/STABLE.java.new    |  890 ----
 .../jgroups/protocols/pbcast/STABLE.java.old    |  855 ----
 .../protocols/pbcast/STATE_TRANSFER.java        |  461 --
 .../org/jgroups/protocols/pbcast/package.html   |    5 -
 .../org/jgroups/protocols/ring/RingNode.java    |   26 -
 .../protocols/ring/RingNodeFlowControl.java     |  135 -
 .../org/jgroups/protocols/ring/RingToken.java   |  245 -
 .../org/jgroups/protocols/ring/TcpRingNode.java |  205 -
 .../protocols/ring/TokenLostException.java      |   63 -
 .../org/jgroups/protocols/ring/package.html     |    5 -
 .../org/jgroups/spi/GFBasicAdapter.java         |   96 -
 .../gemstone/org/jgroups/spi/GFPeerAdapter.java |  169 -
 .../jgroups/stack/AckMcastReceiverWindow.java   |  169 -
 .../org/jgroups/stack/AckMcastSenderWindow.java |  601 ---
 .../org/jgroups/stack/AckReceiverWindow.java    |  184 -
 .../org/jgroups/stack/AckSenderWindow.java      |  354 --
 .../org/jgroups/stack/BoundedLinkedHashMap.java |   82 -
 .../org/jgroups/stack/Configurator.java         |  764 ---
 .../org/jgroups/stack/GFBasicAdapterImpl.java   |  580 ---
 .../org/jgroups/stack/GFPeerAdapterImpl.java    |  322 --
 .../org/jgroups/stack/GossipClient.java         |  819 ----
 .../gemstone/org/jgroups/stack/GossipData.java  |  232 -
 .../org/jgroups/stack/GossipServer.java         |  633 ---
 .../gemstone/org/jgroups/stack/Interval.java    |   40 -
 .../gemstone/org/jgroups/stack/IpAddress.java   |  780 ---
 .../org/jgroups/stack/LogicalAddress.java       |  370 --
 .../org/jgroups/stack/MessageProtocol.java      |  243 -
 .../org/jgroups/stack/NakReceiverWindow.java    |  914 ----
 .../gemstone/org/jgroups/stack/Protocol.java    |  837 ----
 .../org/jgroups/stack/ProtocolObserver.java     |   81 -
 .../org/jgroups/stack/ProtocolStack.java        |  506 --
 .../org/jgroups/stack/Retransmitter.java        |  480 --
 .../com/gemstone/org/jgroups/stack/Router.java  |  577 ---
 .../gemstone/org/jgroups/stack/RouterStub.java  |  419 --
 .../gemstone/org/jgroups/stack/RpcProtocol.java |  166 -
 .../org/jgroups/stack/SockCreatorImpl.java      |   48 -
 .../org/jgroups/stack/StateTransferInfo.java    |   64 -
 .../com/gemstone/org/jgroups/stack/package.html |    5 -
 .../gemstone/org/jgroups/util/AckCollector.java |  149 -
 .../gemstone/org/jgroups/util/BoundedList.java  |   53 -
 .../com/gemstone/org/jgroups/util/Buffer.java   |   64 -
 .../com/gemstone/org/jgroups/util/Command.java  |   16 -
 .../com/gemstone/org/jgroups/util/CondVar.java  |  139 -
 .../org/jgroups/util/ConnectionWatcher.java     |   25 -
 .../jgroups/util/ContextObjectInputStream.java  |   91 -
 .../util/ExposedBufferedInputStream.java        |   61 -
 .../util/ExposedBufferedOutputStream.java       |   44 -
 .../util/ExposedByteArrayInputStream.java       |   58 -
 .../util/ExposedByteArrayOutputStream.java      |   31 -
 .../jgroups/util/ExposedDataOutputStream.java   |   31 -
 .../org/jgroups/util/ExternalStrings.java       |  823 ----
 .../gemstone/org/jgroups/util/GFLogWriter.java  |   57 -
 .../org/jgroups/util/GFStringIdImpl.java        |   68 -
 .../org/jgroups/util/GemFireTracer.java         |  473 --
 .../org/jgroups/util/GetNetworkInterfaces.java  |   34 -
 .../org/jgroups/util/LinkedListQueue.java       |  422 --
 .../com/gemstone/org/jgroups/util/List.java     |  462 --
 .../jgroups/util/MagicObjectInputStream.java    |   55 -
 .../jgroups/util/MagicObjectOutputStream.java   |   60 -
 .../gemstone/org/jgroups/util/Marshaller.java   |  152 -
 .../org/jgroups/util/NullReadWriteLock.java     |   25 -
 .../com/gemstone/org/jgroups/util/NullSync.java |   26 -
 .../gemstone/org/jgroups/util/PrintXMLTree.java |  168 -
 .../com/gemstone/org/jgroups/util/Promise.java  |  162 -
 .../com/gemstone/org/jgroups/util/Proxy.java    |  870 ----
 .../com/gemstone/org/jgroups/util/Queue.java    |  653 ---
 .../org/jgroups/util/Queue.java.concurrent      |  113 -
 .../com/gemstone/org/jgroups/util/Queue2.java   |  715 ---
 .../org/jgroups/util/QueueClosedException.java  |   29 -
 .../com/gemstone/org/jgroups/util/Range.java    |   58 -
 .../org/jgroups/util/ReentrantLatch.java        |   65 -
 .../org/jgroups/util/ReusableThread.java        |  295 --
 .../java/com/gemstone/org/jgroups/util/Rsp.java |   82 -
 .../com/gemstone/org/jgroups/util/RspList.java  |  193 -
 .../gemstone/org/jgroups/util/Scheduler.java    |  270 -
 .../org/jgroups/util/SchedulerListener.java     |   29 -
 .../gemstone/org/jgroups/util/SockCreator.java  |   20 -
 .../com/gemstone/org/jgroups/util/Stack.java    |  108 -
 .../gemstone/org/jgroups/util/Streamable.java   |   26 -
 .../org/jgroups/util/StreamableFixedID.java     |   19 -
 .../com/gemstone/org/jgroups/util/StringId.java |   49 -
 .../gemstone/org/jgroups/util/ThreadPool.java   |  105 -
 .../org/jgroups/util/TimeScheduler.java         |  720 ---
 .../gemstone/org/jgroups/util/TimedWriter.java  |  295 --
 .../com/gemstone/org/jgroups/util/Util.java     | 1869 -------
 .../org/jgroups/util/VersionedStreamable.java   |   18 -
 .../com/gemstone/org/jgroups/util/package.html  |    5 -
 .../java/com/gemstone/org/jgroups/util/todo.txt |   80 -
 .../gemstone/org/jgroups/conf/jg-magic-map.dtd  |   55 -
 .../gemstone/org/jgroups/conf/jg-magic-map.xml  |  269 -
 .../com/gemstone/org/jgroups/log4j2-default.xml |   21 -
 .../org/jgroups/stack/jboss-service.xml         |  146 -
 .../gemstone/org/jgroups/JChannelJUnitTest.java |  271 --
 .../src/main/java/org/json/JSONObject.java      |    2 +
 .../test/junit/categories/DistributedTest.java  |   16 +
 .../categories/DistributedTransactionsTest.java |   16 +
 .../test/junit/categories/HoplogTest.java       |    7 -
 .../test/junit/categories/IntegrationTest.java  |   16 +
 .../test/junit/categories/PerformanceTest.java  |   16 +
 .../gemfire/test/junit/categories/UnitTest.java |   16 +
 .../gemfire/test/junit/categories/WanTest.java  |   16 +
 .../test/junit/rules/ExpectedTimeoutRule.java   |   16 +
 .../tests/ExpectedTimeoutRuleJUnitTest.java     |   16 +
 gemfire-lucene/build.gradle                     |   29 +
 .../gemfire/cache/lucene/LuceneIndex.java       |   60 +
 .../gemfire/cache/lucene/LuceneQuery.java       |   48 +
 .../cache/lucene/LuceneQueryFactory.java        |  101 +
 .../cache/lucene/LuceneQueryProvider.java       |   45 +
 .../cache/lucene/LuceneQueryResults.java        |   58 +
 .../cache/lucene/LuceneResultStruct.java        |   62 +
 .../gemfire/cache/lucene/LuceneService.java     |  125 +
 .../cache/lucene/LuceneServiceProvider.java     |   46 +
 .../lucene/internal/InternalLuceneIndex.java    |   29 +
 .../lucene/internal/InternalLuceneService.java  |   29 +
 .../lucene/internal/LuceneEventListener.java    |   99 +
 .../LuceneIndexForPartitionedRegion.java        |  136 +
 .../LuceneIndexForReplicatedRegion.java         |   48 +
 .../cache/lucene/internal/LuceneIndexImpl.java  |  107 +
 .../lucene/internal/LuceneQueryFactoryImpl.java |   67 +
 .../cache/lucene/internal/LuceneQueryImpl.java  |   87 +
 .../lucene/internal/LuceneQueryResultsImpl.java |  120 +
 .../lucene/internal/LuceneResultStructImpl.java |   94 +
 .../lucene/internal/LuceneServiceImpl.java      |  273 ++
 .../internal/PartitionedRepositoryManager.java  |  163 +
 .../lucene/internal/StringQueryProvider.java    |  106 +
 .../internal/directory/FileIndexInput.java      |  131 +
 .../internal/directory/RegionDirectory.java     |  119 +
 .../internal/distributed/CollectorManager.java  |   55 +
 .../lucene/internal/distributed/EntryScore.java |   82 +
 .../internal/distributed/LuceneFunction.java    |  137 +
 .../distributed/LuceneFunctionContext.java      |  115 +
 .../lucene/internal/distributed/TopEntries.java |  133 +
 .../distributed/TopEntriesCollector.java        |  102 +
 .../distributed/TopEntriesCollectorManager.java |  178 +
 .../TopEntriesFunctionCollector.java            |  163 +
 .../lucene/internal/filesystem/ChunkKey.java    |  123 +
 .../cache/lucene/internal/filesystem/File.java  |  155 +
 .../internal/filesystem/FileInputStream.java    |  166 +
 .../internal/filesystem/FileOutputStream.java   |  103 +
 .../lucene/internal/filesystem/FileSystem.java  |  156 +
 .../filesystem/SeekableInputStream.java         |   43 +
 .../internal/repository/IndexRepository.java    |   74 +
 .../repository/IndexRepositoryImpl.java         |  113 +
 .../repository/IndexResultCollector.java        |   47 +
 .../internal/repository/RepositoryManager.java  |   45 +
 .../HeterogenousLuceneSerializer.java           |   83 +
 .../repository/serializer/LuceneSerializer.java |   35 +
 .../serializer/PdxLuceneSerializer.java         |   47 +
 .../serializer/ReflectionLuceneSerializer.java  |   74 +
 .../repository/serializer/SerializerUtil.java   |  168 +
 .../internal/xml/LuceneIndexCreation.java       |  111 +
 .../internal/xml/LuceneIndexXmlGenerator.java   |   65 +
 .../internal/xml/LuceneServiceXmlGenerator.java |   39 +
 .../lucene/internal/xml/LuceneXmlConstants.java |   31 +
 .../lucene/internal/xml/LuceneXmlParser.java    |   97 +
 .../geode.apache.org/lucene/lucene-1.0.xsd      |   57 +
 ...gemstone.gemfire.internal.cache.CacheService |    1 +
 ...ne.gemfire.internal.cache.xmlcache.XmlParser |    1 +
 .../internal/LuceneEventListenerJUnitTest.java  |  109 +
 .../LuceneIndexRecoveryHAJUnitTest.java         |  201 +
 .../LuceneQueryFactoryImplJUnitTest.java        |   50 +
 .../internal/LuceneQueryImplJUnitTest.java      |  123 +
 .../LuceneQueryResultsImplJUnitTest.java        |  126 +
 .../LuceneResultStructImpJUnitTest.java         |   51 +
 .../internal/LuceneServiceImplJUnitTest.java    |  226 +
 .../PartitionedRepositoryManagerJUnitTest.java  |  230 +
 .../internal/StringQueryProviderJUnitTest.java  |   90 +
 .../directory/RegionDirectoryJUnitTest.java     |   56 +
 .../DistributedScoringJUnitTest.java            |  155 +
 .../distributed/EntryScoreJUnitTest.java        |   40 +
 .../LuceneFunctionContextJUnitTest.java         |   64 +
 .../distributed/LuceneFunctionJUnitTest.java    |  423 ++
 .../LuceneFunctionReadPathDUnitTest.java        |  241 +
 .../TopEntriesCollectorJUnitTest.java           |  139 +
 .../TopEntriesFunctionCollectorJUnitTest.java   |  323 ++
 .../distributed/TopEntriesJUnitTest.java        |  146 +
 .../internal/filesystem/ChunkKeyJUnitTest.java  |   48 +
 .../internal/filesystem/FileJUnitTest.java      |   53 +
 .../filesystem/FileSystemJUnitTest.java         |  578 +++
 ...IndexRepositoryImplJUnitPerformanceTest.java |  437 ++
 .../IndexRepositoryImplJUnitTest.java           |  208 +
 .../HeterogenousLuceneSerializerJUnitTest.java  |   90 +
 .../serializer/PdxFieldMapperJUnitTest.java     |   85 +
 .../ReflectionFieldMapperJUnitTest.java         |   85 +
 .../internal/repository/serializer/Type1.java   |   48 +
 .../internal/repository/serializer/Type2.java   |   34 +
 ...neIndexXmlGeneratorIntegrationJUnitTest.java |   78 +
 .../xml/LuceneIndexXmlGeneratorJUnitTest.java   |   80 +
 ...uceneIndexXmlParserIntegrationJUnitTest.java |  107 +
 .../xml/LuceneIndexXmlParserJUnitTest.java      |   72 +
 ...erIntegrationJUnitTest.createIndex.cache.xml |   24 +
 ...serIntegrationJUnitTest.parseIndex.cache.xml |   24 +
 gemfire-rebalancer/build.gradle                 |   11 +-
 .../gemfire/cache/util/AutoBalancer.java        |  190 +-
 .../util/AutoBalancerIntegrationJUnitTest.java  |  206 +
 .../cache/util/AutoBalancerJUnitTest.java       |  447 +-
 gemfire-spark-connector/doc/1_building.md       |    2 +
 gemfire-spark-connector/doc/5_rdd_join.md       |    2 +-
 .../connector/internal/RegionMetadata.java      |   16 +
 .../gemfirefunctions/QueryFunction.java         |   18 +-
 .../RetrieveRegionFunction.java                 |   16 +
 .../RetrieveRegionMetadataFunction.java         |   16 +
 .../StructStreamingResultSender.java            |   16 +
 .../gemfire/spark/connector/Employee.java       |   16 +
 .../spark/connector/JavaApiIntegrationTest.java |   16 +
 .../gemfire/spark/connector/Portfolio.java      |   16 +
 .../gemfire/spark/connector/Position.java       |   16 +
 .../spark/connector/BasicIntegrationTest.scala  |   16 +
 .../RDDJoinRegionIntegrationTest.scala          |   16 +
 .../RetrieveRegionIntegrationTest.scala         |   16 +
 .../gemfire/spark/connector/package.scala       |   16 +
 .../connector/testkit/GemFireCluster.scala      |   16 +
 .../spark/connector/testkit/GemFireRunner.scala |   16 +
 .../spark/connector/testkit/IOUtils.scala       |   16 +
 .../spark/streaming/ManualClockHelper.scala     |   16 +
 .../spark/streaming/TestInputDStream.scala      |   16 +
 .../javaapi/GemFireJavaDStreamFunctions.java    |   16 +
 .../GemFireJavaPairDStreamFunctions.java        |   16 +
 .../javaapi/GemFireJavaPairRDDFunctions.java    |   16 +
 .../javaapi/GemFireJavaRDDFunctions.java        |   16 +
 .../javaapi/GemFireJavaSQLContextFunctions.java |   16 +
 .../GemFireJavaSparkContextFunctions.java       |   16 +
 .../connector/javaapi/GemFireJavaUtil.java      |   16 +
 .../spark/connector/GemFireConnection.scala     |   16 +
 .../spark/connector/GemFireConnectionConf.scala |   16 +
 .../connector/GemFireConnectionManager.scala    |   16 +
 .../connector/GemFireFunctionDeployer.scala     |   16 +
 .../connector/GemFireKryoRegistrator.scala      |   16 +
 .../connector/GemFirePairRDDFunctions.scala     |   16 +
 .../spark/connector/GemFireRDDFunctions.scala   |   16 +
 .../connector/GemFireSQLContextFunctions.scala  |   16 +
 .../GemFireSparkContextFunctions.scala          |   16 +
 .../internal/DefaultGemFireConnection.scala     |   16 +
 .../DefaultGemFireConnectionManager.scala       |   16 +
 .../connector/internal/LocatorHelper.scala      |   16 +
 .../StructStreamingResultCollector.scala        |   16 +
 .../connector/internal/oql/QueryParser.scala    |   16 +
 .../spark/connector/internal/oql/QueryRDD.scala |   18 +-
 .../internal/oql/QueryResultCollector.scala     |   18 +-
 .../connector/internal/oql/RDDConverter.scala   |   18 +-
 .../connector/internal/oql/RowBuilder.scala     |   16 +
 .../connector/internal/oql/SchemaBuilder.scala  |   16 +
 .../internal/oql/UndefinedSerializer.scala      |   16 +
 .../connector/internal/rdd/GemFireJoinRDD.scala |   16 +
 .../internal/rdd/GemFireOuterJoinRDD.scala      |   16 +
 .../internal/rdd/GemFireRDDPartition.scala      |   16 +
 .../internal/rdd/GemFireRDDPartitioner.scala    |   16 +
 .../rdd/GemFireRDDPartitionerImpl.scala         |   16 +
 .../internal/rdd/GemFireRDDWriter.scala         |   16 +
 .../internal/rdd/GemFireRegionRDD.scala         |   16 +
 .../javaapi/GemFireJavaRegionRDD.scala          |   16 +
 .../spark/connector/javaapi/JavaAPIHelper.scala |   16 +
 .../gemfire/spark/connector/package.scala       |   16 +
 .../streaming/GemFireDStreamFunctions.scala     |   18 +-
 .../spark/connector/streaming/package.scala     |   16 +
 .../gemfire/spark/connector/JavaAPITest.java    |   18 +-
 .../connector/GemFireFunctionDeployerTest.scala |   16 +
 .../DefaultGemFireConnectionManagerTest.scala   |   16 +
 ...tStreamingResultSenderAndCollectorTest.scala |   16 +
 .../internal/oql/QueryParserTest.scala          |   18 +-
 .../connector/ConnectorImplicitsTest.scala      |   16 +
 .../connector/GemFireConnectionConfTest.scala   |   16 +
 .../connector/GemFireDStreamFunctionsTest.scala |   16 +
 .../connector/GemFireRDDFunctionsTest.scala     |   16 +
 .../spark/connector/LocatorHelperTest.scala     |   16 +
 .../rdd/GemFireRDDPartitionerTest.scala         |   16 +
 .../connector/rdd/GemFireRegionRDDTest.scala    |   16 +
 .../basic-demos/src/main/java/demo/Emp.java     |   16 +
 .../src/main/java/demo/OQLJavaDemo.java         |   16 +
 .../src/main/java/demo/PairRDDSaveJavaDemo.java |   16 +
 .../src/main/java/demo/RDDSaveJavaDemo.java     |   16 +
 .../src/main/java/demo/RegionToRDDJavaDemo.java |   16 +
 .../src/main/scala/demo/NetworkWordCount.scala  |   16 +
 .../project/Dependencies.scala                  |   16 +
 .../project/GemFireSparkBuild.scala             |   16 +
 gemfire-spark-connector/project/Settings.scala  |   16 +
 gemfire-web-api/build.gradle                    |   46 +-
 .../web/controllers/AbstractBaseController.java |   21 +-
 .../web/controllers/BaseControllerAdvice.java   |   21 +-
 .../web/controllers/CommonCrudController.java   |   21 +-
 .../controllers/FunctionAccessController.java   |   21 +-
 .../web/controllers/PdxBasedCrudController.java |   21 +-
 .../web/controllers/QueryAccessController.java  |   21 +-
 .../web/controllers/support/JSONTypes.java      |   16 +
 .../controllers/support/QueryResultTypes.java   |   21 +-
 .../web/controllers/support/RegionData.java     |   21 +-
 .../controllers/support/RegionEntryData.java    |   21 +-
 .../support/RestServersResultCollector.java     |   16 +
 .../web/controllers/support/UpdateOp.java       |   21 +-
 .../DataTypeNotSupportedException.java          |   21 +-
 .../web/exception/GemfireRestException.java     |   21 +-
 .../web/exception/MalformedJsonException.java   |   21 +-
 .../web/exception/RegionNotFoundException.java  |   21 +-
 .../exception/ResourceNotFoundException.java    |   21 +-
 ...stomMappingJackson2HttpMessageConverter.java |   16 +
 .../web/swagger/config/RestApiPathProvider.java |   29 +-
 .../web/swagger/config/SwaggerConfig.java       |   16 +
 .../rest/internal/web/util/ArrayUtils.java      |   21 +-
 .../rest/internal/web/util/DateTimeUtils.java   |   21 +-
 .../internal/web/util/IdentifiableUtils.java    |   21 +-
 .../rest/internal/web/util/JSONUtils.java       |   21 +-
 .../rest/internal/web/util/JsonWriter.java      |   21 +-
 .../rest/internal/web/util/NumberUtils.java     |   21 +-
 .../rest/internal/web/util/ValidationUtils.java |   21 +-
 .../main/webapp/WEB-INF/gemfire-api-servlet.xml |   16 +
 gemfire-web-api/src/main/webapp/WEB-INF/web.xml |   16 +
 gemfire-web/build.gradle                        |   32 +-
 .../src/main/webapp/WEB-INF/gemfire-servlet.xml |   16 +
 gemfire-web/src/main/webapp/WEB-INF/web.xml     |   16 +
 .../internal/web/AbstractWebTestCase.java       |   21 +-
 .../ShellCommandsControllerJUnitTest.java       |   68 +-
 ...entVariablesHandlerInterceptorJUnitTest.java |   21 +-
 .../internal/web/domain/LinkIndexJUnitTest.java |   21 +-
 .../internal/web/domain/LinkJUnitTest.java      |   21 +-
 .../domain/QueryParameterSourceJUnitTest.java   |   21 +-
 .../web/http/ClientHttpRequestJUnitTest.java    |   21 +-
 ...ableObjectHttpMessageConverterJUnitTest.java |   21 +-
 .../RestHttpOperationInvokerJUnitTest.java      |   21 +-
 .../SimpleHttpOperationInvokerJUnitTest.java    |   21 +-
 .../web/util/ConvertUtilsJUnitTest.java         |   21 +-
 .../internal/web/util/UriUtilsJUnitTest.java    |   21 +-
 gradle/dependency-versions.properties           |   67 +
 gradle/wrapper/gradle-wrapper.jar               |  Bin 51018 -> 53637 bytes
 gradle/wrapper/gradle-wrapper.properties        |    4 +-
 gradlew                                         |   12 +-
 settings.gradle                                 |    3 +-
 5835 files changed, 131768 insertions(+), 240936 deletions(-)
----------------------------------------------------------------------



[43/50] [abbrv] incubator-geode git commit: GEODE-637: Additional tests for AsyncEventQueues

Posted by kl...@apache.org.
GEODE-637: Additional tests for AsyncEventQueues


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

Branch: refs/heads/feature/GEODE-217
Commit: 476c6cd3be1da503b2345d23fb2857da27d77127
Parents: 386d1ac
Author: Dan Smith <up...@apache.org>
Authored: Wed Dec 2 09:51:49 2015 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Tue Dec 8 15:40:05 2015 -0800

----------------------------------------------------------------------
 .../cache/wan/AsyncEventQueueTestBase.java      | 1658 +++++++++++++++
 .../asyncqueue/AsyncEventListenerDUnitTest.java | 1911 ++++++++++++++++++
 .../AsyncEventListenerOffHeapDUnitTest.java     |   17 +
 .../AsyncEventQueueStatsDUnitTest.java          |  311 +++
 .../ConcurrentAsyncEventQueueDUnitTest.java     |  330 +++
 ...ncurrentAsyncEventQueueOffHeapDUnitTest.java |   16 +
 .../CommonParallelAsyncEventQueueDUnitTest.java |   53 +
 ...ParallelAsyncEventQueueOffHeapDUnitTest.java |   16 +
 8 files changed, 4312 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/476c6cd3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
new file mode 100644
index 0000000..a800118
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
@@ -0,0 +1,1658 @@
+/*
+ * =========================================================================
+ * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved. This
+ * product is protected by U.S. and international copyright and intellectual
+ * property laws. Pivotal products are covered by one or more patents listed at
+ * http://www.pivotal.io/patents.
+ * =========================================================================
+ */
+package com.gemstone.gemfire.internal.cache.wan;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+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.Properties;
+import java.util.Set;
+import java.util.StringTokenizer;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.CacheLoader;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Declarable;
+import com.gemstone.gemfire.cache.DiskStore;
+import com.gemstone.gemfire.cache.DiskStoreFactory;
+import com.gemstone.gemfire.cache.EntryEvent;
+import com.gemstone.gemfire.cache.LoaderHelper;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.Scope;
+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.cache.asyncqueue.internal.AsyncEventQueueImpl;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueStats;
+import com.gemstone.gemfire.cache.client.internal.LocatorDiscoveryCallbackAdapter;
+import com.gemstone.gemfire.cache.control.RebalanceFactory;
+import com.gemstone.gemfire.cache.control.RebalanceOperation;
+import com.gemstone.gemfire.cache.control.RebalanceResults;
+import com.gemstone.gemfire.cache.control.ResourceManager;
+import com.gemstone.gemfire.cache.persistence.PartitionOfflineException;
+import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
+import com.gemstone.gemfire.cache.wan.GatewayEventFilter;
+import com.gemstone.gemfire.cache.wan.GatewayReceiver;
+import com.gemstone.gemfire.cache.wan.GatewayReceiverFactory;
+import com.gemstone.gemfire.cache.wan.GatewaySender;
+import com.gemstone.gemfire.cache.wan.GatewaySender.OrderPolicy;
+import com.gemstone.gemfire.cache.wan.GatewaySenderFactory;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.cache.ForceReattemptException;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.RegionQueue;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.VM;
+
+public class AsyncEventQueueTestBase extends DistributedTestCase {
+
+  protected static Cache cache;
+
+  protected static VM vm0;
+
+  protected static VM vm1;
+
+  protected static VM vm2;
+
+  protected static VM vm3;
+
+  protected static VM vm4;
+
+  protected static VM vm5;
+
+  protected static VM vm6;
+
+  protected static VM vm7;
+
+  protected static AsyncEventListener eventListener1;
+
+  private static final long MAX_WAIT = 10000;
+
+  protected static GatewayEventFilter eventFilter;
+
+  protected static boolean destroyFlag = false;
+
+  protected static List<Integer> dispatcherThreads = new ArrayList<Integer>(
+      Arrays.asList(1, 3, 5));
+
+  // this will be set for each test method run with one of the values from above
+  // list
+  protected static int numDispatcherThreadsForTheRun = 1;
+
+  public AsyncEventQueueTestBase(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception {
+    super.setUp();
+    final Host host = Host.getHost(0);
+    vm0 = host.getVM(0);
+    vm1 = host.getVM(1);
+    vm2 = host.getVM(2);
+    vm3 = host.getVM(3);
+    vm4 = host.getVM(4);
+    vm5 = host.getVM(5);
+    vm6 = host.getVM(6);
+    vm7 = host.getVM(7);
+    // this is done to vary the number of dispatchers for sender
+    // during every test method run
+    shuffleNumDispatcherThreads();
+    invokeInEveryVM(AsyncEventQueueTestBase.class,
+        "setNumDispatcherThreadsForTheRun",
+        new Object[] { dispatcherThreads.get(0) });
+  }
+
+  public static void shuffleNumDispatcherThreads() {
+    Collections.shuffle(dispatcherThreads);
+  }
+
+  public static void setNumDispatcherThreadsForTheRun(int numThreads) {
+    numDispatcherThreadsForTheRun = numThreads;
+  }
+
+  public static Integer createFirstLocatorWithDSId(int dsId) {
+    if (Locator.hasLocator()) {
+      Locator.getLocator().stop();
+    }
+    AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(testName);
+    int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    //props.setProperty(DistributionConfig.DISTRIBUTED_SYSTEM_ID_NAME, "" + dsId);
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + port
+        + "]");
+    props.setProperty(DistributionConfig.START_LOCATOR_NAME, "localhost["
+        + port + "],server=true,peer=true,hostname-for-clients=localhost");
+    test.getSystem(props);
+    return port;
+  }
+
+  public static Integer createFirstRemoteLocator(int dsId, int remoteLocPort) {
+    AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(testName);
+    int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.DISTRIBUTED_SYSTEM_ID_NAME, "" + dsId);
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + port
+        + "]");
+    props.setProperty(DistributionConfig.START_LOCATOR_NAME, "localhost["
+        + port + "],server=true,peer=true,hostname-for-clients=localhost");
+    props.setProperty(DistributionConfig.REMOTE_LOCATORS_NAME, "localhost["
+        + remoteLocPort + "]");
+    test.getSystem(props);
+    return port;
+  }
+
+  public static void createReplicatedRegionWithAsyncEventQueue(
+      String regionName, String asyncQueueIds, Boolean offHeap) {
+    ExpectedException exp1 = addExpectedException(ForceReattemptException.class
+        .getName());
+    try {
+      AttributesFactory fact = new AttributesFactory();
+      if (asyncQueueIds != null) {
+        StringTokenizer tokenizer = new StringTokenizer(asyncQueueIds, ",");
+        while (tokenizer.hasMoreTokens()) {
+          String asyncQueueId = tokenizer.nextToken();
+          fact.addAsyncEventQueueId(asyncQueueId);
+        }
+      }
+      fact.setDataPolicy(DataPolicy.REPLICATE);
+      fact.setOffHeap(offHeap);
+      RegionFactory regionFactory = cache.createRegionFactory(fact.create());
+      Region r = regionFactory.create(regionName);
+      assertNotNull(r);
+    }
+    finally {
+      exp1.remove();
+    }
+  }
+
+  public static void createReplicatedRegionWithCacheLoaderAndAsyncEventQueue(
+      String regionName, String asyncQueueIds) {
+
+    AttributesFactory fact = new AttributesFactory();
+    if (asyncQueueIds != null) {
+      StringTokenizer tokenizer = new StringTokenizer(asyncQueueIds, ",");
+      while (tokenizer.hasMoreTokens()) {
+        String asyncQueueId = tokenizer.nextToken();
+        fact.addAsyncEventQueueId(asyncQueueId);
+      }
+    }
+    fact.setDataPolicy(DataPolicy.REPLICATE);
+    // set the CacheLoader
+    fact.setCacheLoader(new MyCacheLoader());
+    RegionFactory regionFactory = cache.createRegionFactory(fact.create());
+    Region r = regionFactory.create(regionName);
+    assertNotNull(r);
+  }
+
+  public static void createReplicatedRegionWithSenderAndAsyncEventQueue(
+      String regionName, String senderIds, String asyncChannelId, Boolean offHeap) {
+    ExpectedException exp = addExpectedException(ForceReattemptException.class
+        .getName());
+    try {
+
+      AttributesFactory fact = new AttributesFactory();
+      if (senderIds != null) {
+        StringTokenizer tokenizer = new StringTokenizer(senderIds, ",");
+        while (tokenizer.hasMoreTokens()) {
+          String senderId = tokenizer.nextToken();
+          fact.addGatewaySenderId(senderId);
+        }
+      }
+      fact.setDataPolicy(DataPolicy.REPLICATE);
+      fact.setOffHeap(offHeap);
+      fact.setScope(Scope.DISTRIBUTED_ACK);
+      RegionFactory regionFactory = cache.createRegionFactory(fact.create());
+      regionFactory.addAsyncEventQueueId(asyncChannelId);
+      Region r = regionFactory.create(regionName);
+      assertNotNull(r);
+    }
+    finally {
+      exp.remove();
+    }
+  }
+
+  public static void createAsyncEventQueue(String asyncChannelId,
+      boolean isParallel, Integer maxMemory, Integer batchSize,
+      boolean isConflation, boolean isPersistent, String diskStoreName,
+      boolean isDiskSynchronous) {
+
+    if (diskStoreName != null) {
+      File directory = new File(asyncChannelId + "_disk_"
+          + System.currentTimeMillis() + "_" + VM.getCurrentVMNum());
+      directory.mkdir();
+      File[] dirs1 = new File[] { directory };
+      DiskStoreFactory dsf = cache.createDiskStoreFactory();
+      dsf.setDiskDirs(dirs1);
+      DiskStore ds = dsf.create(diskStoreName);
+    }
+
+    AsyncEventListener asyncEventListener = new MyAsyncEventListener();
+
+    AsyncEventQueueFactory factory = cache.createAsyncEventQueueFactory();
+    factory.setBatchSize(batchSize);
+    factory.setPersistent(isPersistent);
+    factory.setDiskStoreName(diskStoreName);
+    factory.setDiskSynchronous(isDiskSynchronous);
+    factory.setBatchConflationEnabled(isConflation);
+    factory.setMaximumQueueMemory(maxMemory);
+    factory.setParallel(isParallel);
+    // set dispatcher threads
+    factory.setDispatcherThreads(numDispatcherThreadsForTheRun);
+    AsyncEventQueue asyncChannel = factory.create(asyncChannelId,
+        asyncEventListener);
+  }
+
+  public static void createAsyncEventQueueWithListener2(String asyncChannelId,
+      boolean isParallel, Integer maxMemory, Integer batchSize,
+      boolean isPersistent, String diskStoreName) {
+
+    if (diskStoreName != null) {
+      File directory = new File(asyncChannelId + "_disk_"
+          + System.currentTimeMillis() + "_" + VM.getCurrentVMNum());
+      directory.mkdir();
+      File[] dirs1 = new File[] { directory };
+      DiskStoreFactory dsf = cache.createDiskStoreFactory();
+      dsf.setDiskDirs(dirs1);
+      DiskStore ds = dsf.create(diskStoreName);
+    }
+
+    AsyncEventListener asyncEventListener = new MyAsyncEventListener2();
+
+    AsyncEventQueueFactory factory = cache.createAsyncEventQueueFactory();
+    factory.setBatchSize(batchSize);
+    factory.setPersistent(isPersistent);
+    factory.setDiskStoreName(diskStoreName);
+    factory.setMaximumQueueMemory(maxMemory);
+    factory.setParallel(isParallel);
+    // set dispatcher threads
+    factory.setDispatcherThreads(numDispatcherThreadsForTheRun);
+    AsyncEventQueue asyncChannel = factory.create(asyncChannelId,
+        asyncEventListener);
+  }
+
+  public static void createAsyncEventQueue(String asyncChannelId,
+      boolean isParallel, Integer maxMemory, Integer batchSize,
+      boolean isConflation, boolean isPersistent, String diskStoreName,
+      boolean isDiskSynchronous, String asyncListenerClass) throws Exception {
+
+    if (diskStoreName != null) {
+      File directory = new File(asyncChannelId + "_disk_"
+          + System.currentTimeMillis() + "_" + VM.getCurrentVMNum());
+      directory.mkdir();
+      File[] dirs1 = new File[] { directory };
+      DiskStoreFactory dsf = cache.createDiskStoreFactory();
+      dsf.setDiskDirs(dirs1);
+      DiskStore ds = dsf.create(diskStoreName);
+    }
+
+    String packagePrefix = "com.gemstone.gemfire.internal.cache.wan.";
+    String className = packagePrefix + asyncListenerClass;
+    AsyncEventListener asyncEventListener = null;
+    try {
+      Class clazz = Class.forName(className);
+      asyncEventListener = (AsyncEventListener)clazz.newInstance();
+    }
+    catch (ClassNotFoundException e) {
+      throw e;
+    }
+    catch (InstantiationException e) {
+      throw e;
+    }
+    catch (IllegalAccessException e) {
+      throw e;
+    }
+
+    AsyncEventQueueFactory factory = cache.createAsyncEventQueueFactory();
+    factory.setBatchSize(batchSize);
+    factory.setPersistent(isPersistent);
+    factory.setDiskStoreName(diskStoreName);
+    factory.setDiskSynchronous(isDiskSynchronous);
+    factory.setBatchConflationEnabled(isConflation);
+    factory.setMaximumQueueMemory(maxMemory);
+    factory.setParallel(isParallel);
+    // set dispatcher threads
+    factory.setDispatcherThreads(numDispatcherThreadsForTheRun);
+    AsyncEventQueue asyncChannel = factory.create(asyncChannelId,
+        asyncEventListener);
+  }
+
+  public static void createAsyncEventQueueWithCustomListener(
+      String asyncChannelId, boolean isParallel, Integer maxMemory,
+      Integer batchSize, boolean isConflation, boolean isPersistent,
+      String diskStoreName, boolean isDiskSynchronous) {
+    createAsyncEventQueueWithCustomListener(asyncChannelId, isParallel,
+        maxMemory, batchSize, isConflation, isPersistent, diskStoreName,
+        isDiskSynchronous, GatewaySender.DEFAULT_DISPATCHER_THREADS);
+  }
+
+  public static void createAsyncEventQueueWithCustomListener(
+      String asyncChannelId, boolean isParallel, Integer maxMemory,
+      Integer batchSize, boolean isConflation, boolean isPersistent,
+      String diskStoreName, boolean isDiskSynchronous, int nDispatchers) {
+
+    ExpectedException exp = addExpectedException(ForceReattemptException.class
+        .getName());
+
+    try {
+      if (diskStoreName != null) {
+        File directory = new File(asyncChannelId + "_disk_"
+            + System.currentTimeMillis() + "_" + VM.getCurrentVMNum());
+        directory.mkdir();
+        File[] dirs1 = new File[] { directory };
+        DiskStoreFactory dsf = cache.createDiskStoreFactory();
+        dsf.setDiskDirs(dirs1);
+        DiskStore ds = dsf.create(diskStoreName);
+      }
+
+      AsyncEventListener asyncEventListener = new CustomAsyncEventListener();
+
+      AsyncEventQueueFactory factory = cache.createAsyncEventQueueFactory();
+      factory.setBatchSize(batchSize);
+      factory.setPersistent(isPersistent);
+      factory.setDiskStoreName(diskStoreName);
+      factory.setMaximumQueueMemory(maxMemory);
+      factory.setParallel(isParallel);
+      factory.setDispatcherThreads(nDispatchers);
+      AsyncEventQueue asyncChannel = factory.create(asyncChannelId,
+          asyncEventListener);
+    }
+    finally {
+      exp.remove();
+    }
+  }
+
+  public static void createConcurrentAsyncEventQueue(String asyncChannelId,
+      boolean isParallel, Integer maxMemory, Integer batchSize,
+      boolean isConflation, boolean isPersistent, String diskStoreName,
+      boolean isDiskSynchronous, int dispatcherThreads, OrderPolicy policy) {
+
+    if (diskStoreName != null) {
+      File directory = new File(asyncChannelId + "_disk_"
+          + System.currentTimeMillis() + "_" + VM.getCurrentVMNum());
+      directory.mkdir();
+      File[] dirs1 = new File[] { directory };
+      DiskStoreFactory dsf = cache.createDiskStoreFactory();
+      dsf.setDiskDirs(dirs1);
+      DiskStore ds = dsf.create(diskStoreName);
+    }
+
+    AsyncEventListener asyncEventListener = new MyAsyncEventListener();
+
+    AsyncEventQueueFactory factory = cache.createAsyncEventQueueFactory();
+    factory.setBatchSize(batchSize);
+    factory.setPersistent(isPersistent);
+    factory.setDiskStoreName(diskStoreName);
+    factory.setDiskSynchronous(isDiskSynchronous);
+    factory.setBatchConflationEnabled(isConflation);
+    factory.setMaximumQueueMemory(maxMemory);
+    factory.setParallel(isParallel);
+    factory.setDispatcherThreads(dispatcherThreads);
+    factory.setOrderPolicy(policy);
+    AsyncEventQueue asyncChannel = factory.create(asyncChannelId,
+        asyncEventListener);
+  }
+
+  public static String createAsyncEventQueueWithDiskStore(
+      String asyncChannelId, boolean isParallel, Integer maxMemory,
+      Integer batchSize, boolean isPersistent, String diskStoreName) {
+
+    AsyncEventListener asyncEventListener = new MyAsyncEventListener();
+
+    File persistentDirectory = null;
+    if (diskStoreName == null) {
+      persistentDirectory = new File(asyncChannelId + "_disk_"
+          + System.currentTimeMillis() + "_" + VM.getCurrentVMNum());
+    }
+    else {
+      persistentDirectory = new File(diskStoreName);
+    }
+    getLogWriter().info("The ds is : " + persistentDirectory.getName());
+    persistentDirectory.mkdir();
+    DiskStoreFactory dsf = cache.createDiskStoreFactory();
+    File[] dirs1 = new File[] { persistentDirectory };
+
+    AsyncEventQueueFactory factory = cache.createAsyncEventQueueFactory();
+    factory.setBatchSize(batchSize);
+    factory.setParallel(isParallel);
+    if (isPersistent) {
+      factory.setPersistent(isPersistent);
+      factory.setDiskStoreName(dsf.setDiskDirs(dirs1).create(asyncChannelId)
+          .getName());
+    }
+    factory.setMaximumQueueMemory(maxMemory);
+    // set dispatcher threads
+    factory.setDispatcherThreads(numDispatcherThreadsForTheRun);
+    AsyncEventQueue asyncChannel = factory.create(asyncChannelId,
+        asyncEventListener);
+    return persistentDirectory.getName();
+  }
+
+  public static void pauseAsyncEventQueue(String asyncChannelId) {
+    AsyncEventQueue theChannel = null;
+
+    Set<AsyncEventQueue> asyncEventChannels = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncChannel : asyncEventChannels) {
+      if (asyncChannelId.equals(asyncChannel.getId())) {
+        theChannel = asyncChannel;
+      }
+    }
+
+    ((AsyncEventQueueImpl)theChannel).getSender().pause();
+  }
+
+  public static void pauseAsyncEventQueueAndWaitForDispatcherToPause(
+      String asyncChannelId) {
+    AsyncEventQueue theChannel = null;
+
+    Set<AsyncEventQueue> asyncEventChannels = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncChannel : asyncEventChannels) {
+      if (asyncChannelId.equals(asyncChannel.getId())) {
+        theChannel = asyncChannel;
+        break;
+      }
+    }
+
+    ((AsyncEventQueueImpl)theChannel).getSender().pause();
+
+    ((AbstractGatewaySender)((AsyncEventQueueImpl)theChannel).getSender())
+        .getEventProcessor().waitForDispatcherToPause();
+  }
+
+  public static void resumeAsyncEventQueue(String asyncQueueId) {
+    AsyncEventQueue theQueue = null;
+
+    Set<AsyncEventQueue> asyncEventChannels = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncChannel : asyncEventChannels) {
+      if (asyncQueueId.equals(asyncChannel.getId())) {
+        theQueue = asyncChannel;
+      }
+    }
+
+    ((AsyncEventQueueImpl)theQueue).getSender().resume();
+  }
+
+  public static void checkAsyncEventQueueSize(String asyncQueueId,
+      int numQueueEntries) {
+    AsyncEventQueue theAsyncEventQueue = null;
+
+    Set<AsyncEventQueue> asyncEventChannels = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncChannel : asyncEventChannels) {
+      if (asyncQueueId.equals(asyncChannel.getId())) {
+        theAsyncEventQueue = asyncChannel;
+      }
+    }
+
+    GatewaySender sender = ((AsyncEventQueueImpl)theAsyncEventQueue)
+        .getSender();
+
+    if (sender.isParallel()) {
+      Set<RegionQueue> queues = ((AbstractGatewaySender)sender).getQueues();
+      assertEquals(numQueueEntries,
+          queues.toArray(new RegionQueue[queues.size()])[0].getRegion().size());
+    }
+    else {
+      Set<RegionQueue> queues = ((AbstractGatewaySender)sender).getQueues();
+      int size = 0;
+      for (RegionQueue q : queues) {
+        size += q.size();
+      }
+      assertEquals(numQueueEntries, size);
+    }
+  }
+
+  /**
+   * This method verifies the queue size of a ParallelGatewaySender. For
+   * ParallelGatewaySender conflation happens in a separate thread, hence test
+   * code needs to wait for some time for expected result
+   * 
+   * @param asyncQueueId
+   *          Async Queue ID
+   * @param numQueueEntries
+   *          expected number of Queue entries
+   * @throws Exception
+   */
+  public static void waitForAsyncEventQueueSize(String asyncQueueId,
+      final int numQueueEntries) throws Exception {
+    AsyncEventQueue theAsyncEventQueue = null;
+
+    Set<AsyncEventQueue> asyncEventChannels = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncChannel : asyncEventChannels) {
+      if (asyncQueueId.equals(asyncChannel.getId())) {
+        theAsyncEventQueue = asyncChannel;
+      }
+    }
+
+    GatewaySender sender = ((AsyncEventQueueImpl)theAsyncEventQueue)
+        .getSender();
+
+    if (sender.isParallel()) {
+      final Set<RegionQueue> queues = ((AbstractGatewaySender)sender)
+          .getQueues();
+
+      waitForCriterion(new WaitCriterion() {
+
+        public String description() {
+          return "Waiting for EventQueue size to be " + numQueueEntries;
+        }
+
+        public boolean done() {
+          boolean done = numQueueEntries == queues.toArray(new RegionQueue[queues
+              .size()])[0].getRegion().size();
+          return done;
+        }
+
+      }, MAX_WAIT, 500, true);
+
+    }
+    else {
+      throw new Exception(
+          "This method should be used for only ParallelGatewaySender,SerialGatewaySender should use checkAsyncEventQueueSize() method instead");
+
+    }
+  }
+
+  public static void createPartitionedRegion(String regionName,
+      String senderIds, Integer redundantCopies, Integer totalNumBuckets) {
+    ExpectedException exp = addExpectedException(ForceReattemptException.class
+        .getName());
+    ExpectedException exp1 = addExpectedException(PartitionOfflineException.class
+        .getName());
+    try {
+      AttributesFactory fact = new AttributesFactory();
+      if (senderIds != null) {
+        StringTokenizer tokenizer = new StringTokenizer(senderIds, ",");
+        while (tokenizer.hasMoreTokens()) {
+          String senderId = tokenizer.nextToken();
+          // GatewaySender sender = cache.getGatewaySender(senderId);
+          // assertNotNull(sender);
+          fact.addGatewaySenderId(senderId);
+        }
+      }
+      PartitionAttributesFactory pfact = new PartitionAttributesFactory();
+      pfact.setTotalNumBuckets(totalNumBuckets);
+      pfact.setRedundantCopies(redundantCopies);
+      pfact.setRecoveryDelay(0);
+      fact.setPartitionAttributes(pfact.create());
+      Region r = cache.createRegionFactory(fact.create()).create(regionName);
+      assertNotNull(r);
+    }
+    finally {
+      exp.remove();
+      exp1.remove();
+    }
+  }
+
+  public static void createPartitionedRegionWithAsyncEventQueue(
+      String regionName, String asyncEventQueueId, Boolean offHeap) {
+    ExpectedException exp = addExpectedException(ForceReattemptException.class
+        .getName());
+    ExpectedException exp1 = addExpectedException(PartitionOfflineException.class
+        .getName());
+    try {
+      AttributesFactory fact = new AttributesFactory();
+
+      PartitionAttributesFactory pfact = new PartitionAttributesFactory();
+      pfact.setTotalNumBuckets(16);
+      fact.setPartitionAttributes(pfact.create());
+      fact.setOffHeap(offHeap);
+      Region r = cache.createRegionFactory(fact.create())
+          .addAsyncEventQueueId(asyncEventQueueId).create(regionName);
+      assertNotNull(r);
+    }
+    finally {
+      exp.remove();
+      exp1.remove();
+    }
+  }
+
+  public static void createColocatedPartitionedRegionWithAsyncEventQueue(
+      String regionName, String asyncEventQueueId, Integer totalNumBuckets,
+      String colocatedWith) {
+
+    ExpectedException exp = addExpectedException(ForceReattemptException.class
+        .getName());
+    ExpectedException exp1 = addExpectedException(PartitionOfflineException.class
+        .getName());
+    try {
+      AttributesFactory fact = new AttributesFactory();
+
+      PartitionAttributesFactory pfact = new PartitionAttributesFactory();
+      pfact.setTotalNumBuckets(totalNumBuckets);
+      pfact.setColocatedWith(colocatedWith);
+      fact.setPartitionAttributes(pfact.create());
+      Region r = cache.createRegionFactory(fact.create())
+          .addAsyncEventQueueId(asyncEventQueueId).create(regionName);
+      assertNotNull(r);
+    }
+    finally {
+      exp.remove();
+      exp1.remove();
+    }
+  }
+
+  public static void createPartitionedRegionWithCacheLoaderAndAsyncQueue(
+      String regionName, String asyncEventQueueId) {
+
+    AttributesFactory fact = new AttributesFactory();
+
+    PartitionAttributesFactory pfact = new PartitionAttributesFactory();
+    pfact.setTotalNumBuckets(16);
+    fact.setPartitionAttributes(pfact.create());
+    // set the CacheLoader implementation
+    fact.setCacheLoader(new MyCacheLoader());
+    Region r = cache.createRegionFactory(fact.create())
+        .addAsyncEventQueueId(asyncEventQueueId).create(regionName);
+    assertNotNull(r);
+  }
+
+  /**
+   * Create PartitionedRegion with 1 redundant copy
+   */
+  public static void createPRWithRedundantCopyWithAsyncEventQueue(
+      String regionName, String asyncEventQueueId, Boolean offHeap) {
+    ExpectedException exp = addExpectedException(ForceReattemptException.class
+        .getName());
+
+    try {
+      AttributesFactory fact = new AttributesFactory();
+
+      PartitionAttributesFactory pfact = new PartitionAttributesFactory();
+      pfact.setTotalNumBuckets(16);
+      pfact.setRedundantCopies(1);
+      fact.setPartitionAttributes(pfact.create());
+      fact.setOffHeap(offHeap);
+      Region r = cache.createRegionFactory(fact.create())
+          .addAsyncEventQueueId(asyncEventQueueId).create(regionName);
+      assertNotNull(r);
+    }
+    finally {
+      exp.remove();
+    }
+  }
+
+  public static void createPartitionedRegionAccessorWithAsyncEventQueue(
+      String regionName, String asyncEventQueueId) {
+    AttributesFactory fact = new AttributesFactory();
+    PartitionAttributesFactory pfact = new PartitionAttributesFactory();
+    pfact.setTotalNumBuckets(16);
+    pfact.setLocalMaxMemory(0);
+    fact.setPartitionAttributes(pfact.create());
+    Region r = cache.createRegionFactory(fact.create())
+        .addAsyncEventQueueId(asyncEventQueueId).create(regionName);
+    // fact.create()).create(regionName);
+    assertNotNull(r);
+  }
+
+  protected static void createCache(Integer locPort) {
+    AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(testName);
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort
+        + "]");
+    InternalDistributedSystem ds = test.getSystem(props);
+    cache = CacheFactory.create(ds);
+  }
+
+  public static void createCacheWithoutLocator(Integer mCastPort) {
+    AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(testName);
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "" + mCastPort);
+    InternalDistributedSystem ds = test.getSystem(props);
+    cache = CacheFactory.create(ds);
+  }
+
+  public static void checkAsyncEventQueueStats(String queueId,
+      final int queueSize, final int eventsReceived, final int eventsQueued,
+      final int eventsDistributed) {
+    Set<AsyncEventQueue> asyncQueues = cache.getAsyncEventQueues();
+    AsyncEventQueue queue = null;
+    for (AsyncEventQueue q : asyncQueues) {
+      if (q.getId().equals(queueId)) {
+        queue = q;
+        break;
+      }
+    }
+    final AsyncEventQueueStats statistics = ((AsyncEventQueueImpl)queue)
+        .getStatistics();
+    assertEquals(queueSize, statistics.getEventQueueSize());
+    assertEquals(eventsReceived, statistics.getEventsReceived());
+    assertEquals(eventsQueued, statistics.getEventsQueued());
+    assert (statistics.getEventsDistributed() >= eventsDistributed);
+  }
+
+  public static void checkAsyncEventQueueConflatedStats(
+      String asyncEventQueueId, final int eventsConflated) {
+    Set<AsyncEventQueue> queues = cache.getAsyncEventQueues();
+    AsyncEventQueue queue = null;
+    for (AsyncEventQueue q : queues) {
+      if (q.getId().equals(asyncEventQueueId)) {
+        queue = q;
+        break;
+      }
+    }
+    final AsyncEventQueueStats statistics = ((AsyncEventQueueImpl)queue)
+        .getStatistics();
+    assertEquals(eventsConflated, statistics.getEventsNotQueuedConflated());
+  }
+
+  public static void checkAsyncEventQueueStats_Failover(
+      String asyncEventQueueId, final int eventsReceived) {
+    Set<AsyncEventQueue> asyncEventQueues = cache.getAsyncEventQueues();
+    AsyncEventQueue queue = null;
+    for (AsyncEventQueue q : asyncEventQueues) {
+      if (q.getId().equals(asyncEventQueueId)) {
+        queue = q;
+        break;
+      }
+    }
+    final AsyncEventQueueStats statistics = ((AsyncEventQueueImpl)queue)
+        .getStatistics();
+
+    assertEquals(eventsReceived, statistics.getEventsReceived());
+    assertEquals(
+        eventsReceived,
+        (statistics.getEventsQueued()
+            + statistics.getUnprocessedTokensAddedByPrimary() + statistics
+            .getUnprocessedEventsRemovedByPrimary()));
+  }
+
+  public static void checkAsyncEventQueueBatchStats(String asyncQueueId,
+      final int batches) {
+    Set<AsyncEventQueue> queues = cache.getAsyncEventQueues();
+    AsyncEventQueue queue = null;
+    for (AsyncEventQueue q : queues) {
+      if (q.getId().equals(asyncQueueId)) {
+        queue = q;
+        break;
+      }
+    }
+    final AsyncEventQueueStats statistics = ((AsyncEventQueueImpl)queue)
+        .getStatistics();
+    assert (statistics.getBatchesDistributed() >= batches);
+    assertEquals(0, statistics.getBatchesRedistributed());
+  }
+
+  public static void checkAsyncEventQueueUnprocessedStats(String asyncQueueId,
+      int events) {
+    Set<AsyncEventQueue> asyncQueues = cache.getAsyncEventQueues();
+    AsyncEventQueue queue = null;
+    for (AsyncEventQueue q : asyncQueues) {
+      if (q.getId().equals(asyncQueueId)) {
+        queue = q;
+        break;
+      }
+    }
+    final AsyncEventQueueStats statistics = ((AsyncEventQueueImpl)queue)
+        .getStatistics();
+    assertEquals(events,
+        (statistics.getUnprocessedEventsAddedBySecondary() + statistics
+            .getUnprocessedTokensRemovedBySecondary()));
+    assertEquals(events,
+        (statistics.getUnprocessedEventsRemovedByPrimary() + statistics
+            .getUnprocessedTokensAddedByPrimary()));
+  }
+
+  public static void setRemoveFromQueueOnException(String senderId,
+      boolean removeFromQueue) {
+    Set<GatewaySender> senders = cache.getGatewaySenders();
+    GatewaySender sender = null;
+    for (GatewaySender s : senders) {
+      if (s.getId().equals(senderId)) {
+        sender = s;
+        break;
+      }
+    }
+    assertNotNull(sender);
+    ((AbstractGatewaySender)sender)
+        .setRemoveFromQueueOnException(removeFromQueue);
+  }
+
+  public static void unsetRemoveFromQueueOnException(String senderId) {
+    Set<GatewaySender> senders = cache.getGatewaySenders();
+    GatewaySender sender = null;
+    for (GatewaySender s : senders) {
+      if (s.getId().equals(senderId)) {
+        sender = s;
+        break;
+      }
+    }
+    assertNotNull(sender);
+    ((AbstractGatewaySender)sender).setRemoveFromQueueOnException(false);
+  }
+
+  public static void waitForSenderToBecomePrimary(String senderId) {
+    Set<GatewaySender> senders = ((GemFireCacheImpl)cache)
+        .getAllGatewaySenders();
+    final GatewaySender sender = getGatewaySenderById(senders, senderId);
+    WaitCriterion wc = new WaitCriterion() {
+      public boolean done() {
+        if (sender != null && ((AbstractGatewaySender)sender).isPrimary()) {
+          return true;
+        }
+        return false;
+      }
+
+      public String description() {
+        return "Expected sender primary state to be true but is false";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, 10000, 1000, true);
+  }
+
+  private static GatewaySender getGatewaySenderById(Set<GatewaySender> senders,
+      String senderId) {
+    for (GatewaySender s : senders) {
+      if (s.getId().equals(senderId)) {
+        return s;
+      }
+    }
+    // if none of the senders matches with the supplied senderid, return null
+    return null;
+  }
+
+  public static void createSender(String dsName, int remoteDsId,
+      boolean isParallel, Integer maxMemory, Integer batchSize,
+      boolean isConflation, boolean isPersistent, GatewayEventFilter filter,
+      boolean isManulaStart) {
+    final ExpectedException exln = addExpectedException("Could not connect");
+    try {
+      File persistentDirectory = new File(dsName + "_disk_"
+          + System.currentTimeMillis() + "_" + VM.getCurrentVMNum());
+      persistentDirectory.mkdir();
+      DiskStoreFactory dsf = cache.createDiskStoreFactory();
+      File[] dirs1 = new File[] { persistentDirectory };
+      if (isParallel) {
+        GatewaySenderFactory gateway = cache.createGatewaySenderFactory();
+        gateway.setParallel(true);
+        gateway.setMaximumQueueMemory(maxMemory);
+        gateway.setBatchSize(batchSize);
+        gateway.setManualStart(isManulaStart);
+        // set dispatcher threads
+        gateway.setDispatcherThreads(numDispatcherThreadsForTheRun);
+        ((InternalGatewaySenderFactory)gateway)
+            .setLocatorDiscoveryCallback(new MyLocatorCallback());
+        if (filter != null) {
+          eventFilter = filter;
+          gateway.addGatewayEventFilter(filter);
+        }
+        if (isPersistent) {
+          gateway.setPersistenceEnabled(true);
+          gateway.setDiskStoreName(dsf.setDiskDirs(dirs1).create(dsName)
+              .getName());
+        }
+        else {
+          DiskStore store = dsf.setDiskDirs(dirs1).create(dsName);
+          gateway.setDiskStoreName(store.getName());
+        }
+        gateway.setBatchConflationEnabled(isConflation);
+        gateway.create(dsName, remoteDsId);
+
+      }
+      else {
+        GatewaySenderFactory gateway = cache.createGatewaySenderFactory();
+        gateway.setMaximumQueueMemory(maxMemory);
+        gateway.setBatchSize(batchSize);
+        gateway.setManualStart(isManulaStart);
+        // set dispatcher threads
+        gateway.setDispatcherThreads(numDispatcherThreadsForTheRun);
+        ((InternalGatewaySenderFactory)gateway)
+            .setLocatorDiscoveryCallback(new MyLocatorCallback());
+        if (filter != null) {
+          eventFilter = filter;
+          gateway.addGatewayEventFilter(filter);
+        }
+        gateway.setBatchConflationEnabled(isConflation);
+        if (isPersistent) {
+          gateway.setPersistenceEnabled(true);
+          gateway.setDiskStoreName(dsf.setDiskDirs(dirs1).create(dsName)
+              .getName());
+        }
+        else {
+          DiskStore store = dsf.setDiskDirs(dirs1).create(dsName);
+          gateway.setDiskStoreName(store.getName());
+        }
+        gateway.create(dsName, remoteDsId);
+      }
+    }
+    finally {
+      exln.remove();
+    }
+  }
+
+  public static void pauseWaitCriteria(final long millisec) {
+    WaitCriterion wc = new WaitCriterion() {
+      public boolean done() {
+        return false;
+      }
+
+      public String description() {
+        return "Expected to wait for " + millisec + " millisec.";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, millisec, 500, false);
+  }
+
+  public static int createReceiver(int locPort) {
+    AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(testName);
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort
+        + "]");
+
+    InternalDistributedSystem ds = test.getSystem(props);
+    cache = CacheFactory.create(ds);
+    GatewayReceiverFactory fact = cache.createGatewayReceiverFactory();
+    int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
+    fact.setStartPort(port);
+    fact.setEndPort(port);
+    fact.setManualStart(true);
+    GatewayReceiver receiver = fact.create();
+    try {
+      receiver.start();
+    }
+    catch (IOException e) {
+      e.printStackTrace();
+      fail("Test " + test.getName()
+          + " failed to start GatewayRecevier on port " + port);
+    }
+    return port;
+  }
+
+  public static String makePath(String[] strings) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < strings.length; i++) {
+      sb.append(strings[i]);
+      sb.append(File.separator);
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Do a rebalance and verify balance was improved. If evictionPercentage > 0
+   * (the default) then we have heapLRU and this can cause simulate and
+   * rebalance results to differ if eviction kicks in between. (See BUG 44899).
+   */
+  public static void doRebalance() {
+    ResourceManager resMan = cache.getResourceManager();
+    boolean heapEviction = (resMan.getEvictionHeapPercentage() > 0);
+    RebalanceFactory factory = resMan.createRebalanceFactory();
+    try {
+      RebalanceResults simulateResults = null;
+      if (!heapEviction) {
+        getLogWriter().info("Calling rebalance simulate");
+        RebalanceOperation simulateOp = factory.simulate();
+        simulateResults = simulateOp.getResults();
+      }
+
+      getLogWriter().info("Starting rebalancing");
+      RebalanceOperation rebalanceOp = factory.start();
+      RebalanceResults rebalanceResults = rebalanceOp.getResults();
+
+    }
+    catch (InterruptedException e) {
+      fail("Interrupted", e);
+    }
+  }
+
+  public static void doPuts(String regionName, int numPuts) {
+    ExpectedException exp1 = addExpectedException(InterruptedException.class
+        .getName());
+    ExpectedException exp2 = addExpectedException(GatewaySenderException.class
+        .getName());
+    try {
+      Region r = cache.getRegion(Region.SEPARATOR + regionName);
+      assertNotNull(r);
+      for (long i = 0; i < numPuts; i++) {
+        r.put(i, i);
+      }
+    }
+    finally {
+      exp1.remove();
+      exp2.remove();
+    }
+    // for (long i = 0; i < numPuts; i++) {
+    // r.destroy(i);
+    // }
+  }
+
+  /**
+   * To be used for CacheLoader related tests
+   */
+  public static void doGets(String regionName, int numGets) {
+    Region r = cache.getRegion(Region.SEPARATOR + regionName);
+    assertNotNull(r);
+    for (long i = 0; i < numGets; i++) {
+      r.get(i);
+    }
+  }
+
+  public static void doPutsFrom(String regionName, int from, int numPuts) {
+    Region r = cache.getRegion(Region.SEPARATOR + regionName);
+    assertNotNull(r);
+    for (long i = from; i < numPuts; i++) {
+      r.put(i, i);
+    }
+  }
+
+  public static void doPutAll(String regionName, int numPuts, int size) {
+    Region r = cache.getRegion(Region.SEPARATOR + regionName);
+    assertNotNull(r);
+    for (long i = 0; i < numPuts; i++) {
+      Map putAllMap = new HashMap();
+      for (long j = 0; j < size; j++) {
+        putAllMap.put((size * i) + j, i);
+      }
+      r.putAll(putAllMap, "putAllCallback");
+      putAllMap.clear();
+    }
+  }
+
+  public static void putGivenKeyValue(String regionName, Map keyValues) {
+    Region r = cache.getRegion(Region.SEPARATOR + regionName);
+    assertNotNull(r);
+    for (Object key : keyValues.keySet()) {
+      r.put(key, keyValues.get(key));
+    }
+  }
+
+  public static void doNextPuts(String regionName, int start, int numPuts) {
+    // waitForSitesToUpdate();
+    ExpectedException exp = addExpectedException(CacheClosedException.class
+        .getName());
+    try {
+      Region r = cache.getRegion(Region.SEPARATOR + regionName);
+      assertNotNull(r);
+      for (long i = start; i < numPuts; i++) {
+        r.put(i, i);
+      }
+    }
+    finally {
+      exp.remove();
+    }
+  }
+
+  public static void validateRegionSize(String regionName, final int regionSize) {
+    ExpectedException exp = addExpectedException(ForceReattemptException.class
+        .getName());
+    ExpectedException exp1 = addExpectedException(CacheClosedException.class
+        .getName());
+    try {
+
+      final Region r = cache.getRegion(Region.SEPARATOR + regionName);
+      assertNotNull(r);
+      WaitCriterion wc = new WaitCriterion() {
+        public boolean done() {
+          if (r.keySet().size() == regionSize) {
+            return true;
+          }
+          return false;
+        }
+
+        public String description() {
+          return "Expected region entries: " + regionSize
+              + " but actual entries: " + r.keySet().size()
+              + " present region keyset " + r.keySet();
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 240000, 500, true);
+    }
+    finally {
+      exp.remove();
+      exp1.remove();
+    }
+  }
+
+  /**
+   * Validate whether all the attributes set on AsyncEventQueueFactory are set
+   * on the sender underneath the AsyncEventQueue.
+   */
+  public static void validateAsyncEventQueueAttributes(String asyncChannelId,
+      int maxQueueMemory, int batchSize, int batchTimeInterval,
+      boolean isPersistent, String diskStoreName, boolean isDiskSynchronous,
+      boolean batchConflationEnabled) {
+
+    AsyncEventQueue theChannel = null;
+
+    Set<AsyncEventQueue> asyncEventChannels = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncChannel : asyncEventChannels) {
+      if (asyncChannelId.equals(asyncChannel.getId())) {
+        theChannel = asyncChannel;
+      }
+    }
+
+    GatewaySender theSender = ((AsyncEventQueueImpl)theChannel).getSender();
+    assertEquals("maxQueueMemory", maxQueueMemory,
+        theSender.getMaximumQueueMemory());
+    assertEquals("batchSize", batchSize, theSender.getBatchSize());
+    assertEquals("batchTimeInterval", batchTimeInterval,
+        theSender.getBatchTimeInterval());
+    assertEquals("isPersistent", isPersistent, theSender.isPersistenceEnabled());
+    assertEquals("diskStoreName", diskStoreName, theSender.getDiskStoreName());
+    assertEquals("isDiskSynchronous", isDiskSynchronous,
+        theSender.isDiskSynchronous());
+    assertEquals("batchConflation", batchConflationEnabled,
+        theSender.isBatchConflationEnabled());
+  }
+  
+  /**
+   * Validate whether all the attributes set on AsyncEventQueueFactory are set
+   * on the sender underneath the AsyncEventQueue.
+   */
+  public static void validateConcurrentAsyncEventQueueAttributes(String asyncChannelId,
+      int maxQueueMemory, int batchSize, int batchTimeInterval,
+      boolean isPersistent, String diskStoreName, boolean isDiskSynchronous,
+      boolean batchConflationEnabled, int dispatcherThreads, OrderPolicy policy) {
+
+    AsyncEventQueue theChannel = null;
+
+    Set<AsyncEventQueue> asyncEventChannels = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncChannel : asyncEventChannels) {
+      if (asyncChannelId.equals(asyncChannel.getId())) {
+        theChannel = asyncChannel;
+      }
+    }
+
+    GatewaySender theSender = ((AsyncEventQueueImpl)theChannel).getSender();
+    assertEquals("maxQueueMemory", maxQueueMemory, theSender
+        .getMaximumQueueMemory());
+    assertEquals("batchSize", batchSize, theSender.getBatchSize());
+    assertEquals("batchTimeInterval", batchTimeInterval, theSender
+        .getBatchTimeInterval());
+    assertEquals("isPersistent", isPersistent, theSender.isPersistenceEnabled());
+    assertEquals("diskStoreName", diskStoreName, theSender.getDiskStoreName());
+    assertEquals("isDiskSynchronous", isDiskSynchronous, theSender
+        .isDiskSynchronous());
+    assertEquals("batchConflation", batchConflationEnabled, theSender
+        .isBatchConflationEnabled());
+    assertEquals("dispatcherThreads", dispatcherThreads, theSender
+        .getDispatcherThreads());
+    assertEquals("orderPolicy", policy, theSender.getOrderPolicy());
+  }
+
+  public static void validateAsyncEventListener(String asyncQueueId,
+      final int expectedSize) {
+    AsyncEventListener theListener = null;
+
+    Set<AsyncEventQueue> asyncEventQueues = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncQueue : asyncEventQueues) {
+      if (asyncQueueId.equals(asyncQueue.getId())) {
+        theListener = asyncQueue.getAsyncEventListener();
+      }
+    }
+
+    final Map eventsMap = ((MyAsyncEventListener)theListener).getEventsMap();
+    assertNotNull(eventsMap);
+    WaitCriterion wc = new WaitCriterion() {
+      public boolean done() {
+        if (eventsMap.size() == expectedSize) {
+          return true;
+        }
+        return false;
+      }
+
+      public String description() {
+        return "Expected map entries: " + expectedSize
+            + " but actual entries: " + eventsMap.size();
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, 60000, 500, true); // TODO:Yogs
+  }
+
+  public static void validateAsyncEventForOperationDetail(String asyncQueueId,
+      final int expectedSize, boolean isLoad, boolean isPutAll) {
+
+    AsyncEventListener theListener = null;
+
+    Set<AsyncEventQueue> asyncEventQueues = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncQueue : asyncEventQueues) {
+      if (asyncQueueId.equals(asyncQueue.getId())) {
+        theListener = asyncQueue.getAsyncEventListener();
+      }
+    }
+
+    final Map eventsMap = ((MyAsyncEventListener_CacheLoader)theListener)
+        .getEventsMap();
+    assertNotNull(eventsMap);
+    WaitCriterion wc = new WaitCriterion() {
+      public boolean done() {
+        if (eventsMap.size() == expectedSize) {
+          return true;
+        }
+        return false;
+      }
+
+      public String description() {
+        return "Expected map entries: " + expectedSize
+            + " but actual entries: " + eventsMap.size();
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, 60000, 500, true); // TODO:Yogs
+    Collection values = eventsMap.values();
+    Iterator itr = values.iterator();
+    while (itr.hasNext()) {
+      AsyncEvent asyncEvent = (AsyncEvent)itr.next();
+      if (isLoad)
+        assertTrue(asyncEvent.getOperation().isLoad());
+      if (isPutAll)
+        assertTrue(asyncEvent.getOperation().isPutAll());
+    }
+  }
+
+  public static void validateCustomAsyncEventListener(String asyncQueueId,
+      final int expectedSize) {
+    AsyncEventListener theListener = null;
+
+    Set<AsyncEventQueue> asyncEventQueues = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncQueue : asyncEventQueues) {
+      if (asyncQueueId.equals(asyncQueue.getId())) {
+        theListener = asyncQueue.getAsyncEventListener();
+      }
+    }
+
+    final Map eventsMap = ((CustomAsyncEventListener)theListener)
+        .getEventsMap();
+    assertNotNull(eventsMap);
+    WaitCriterion wc = new WaitCriterion() {
+      public boolean done() {
+        if (eventsMap.size() == expectedSize) {
+          return true;
+        }
+        return false;
+      }
+
+      public String description() {
+        return "Expected map entries: " + expectedSize
+            + " but actual entries: " + eventsMap.size();
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, 60000, 500, true); // TODO:Yogs
+
+    Iterator<AsyncEvent> itr = eventsMap.values().iterator();
+    while (itr.hasNext()) {
+      AsyncEvent event = itr.next();
+      assertTrue("possibleDuplicate should be true for event: " + event,
+          event.getPossibleDuplicate());
+    }
+  }
+
+  public static void waitForAsyncQueueToGetEmpty(String asyncQueueId) {
+    AsyncEventQueue theAsyncEventQueue = null;
+
+    Set<AsyncEventQueue> asyncEventChannels = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncChannel : asyncEventChannels) {
+      if (asyncQueueId.equals(asyncChannel.getId())) {
+        theAsyncEventQueue = asyncChannel;
+      }
+    }
+
+    final GatewaySender sender = ((AsyncEventQueueImpl)theAsyncEventQueue)
+        .getSender();
+
+    if (sender.isParallel()) {
+      final Set<RegionQueue> queues = ((AbstractGatewaySender)sender)
+          .getQueues();
+
+      WaitCriterion wc = new WaitCriterion() {
+        public boolean done() {
+          int size = 0;
+          for (RegionQueue q : queues) {
+            size += q.size();
+          }
+          if (size == 0) {
+            return true;
+          }
+          return false;
+        }
+
+        public String description() {
+          int size = 0;
+          for (RegionQueue q : queues) {
+            size += q.size();
+          }
+          return "Expected queue size to be : " + 0 + " but actual entries: "
+              + size;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 60000, 500, true);
+
+    }
+    else {
+      WaitCriterion wc = new WaitCriterion() {
+        public boolean done() {
+          Set<RegionQueue> queues = ((AbstractGatewaySender)sender).getQueues();
+          int size = 0;
+          for (RegionQueue q : queues) {
+            size += q.size();
+          }
+          if (size == 0) {
+            return true;
+          }
+          return false;
+        }
+
+        public String description() {
+          Set<RegionQueue> queues = ((AbstractGatewaySender)sender).getQueues();
+          int size = 0;
+          for (RegionQueue q : queues) {
+            size += q.size();
+          }
+          return "Expected queue size to be : " + 0 + " but actual entries: "
+              + size;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 60000, 500, true);
+    }
+  }
+
+  public static void verifyAsyncEventListenerForPossibleDuplicates(
+      String asyncEventQueueId, Set<Integer> bucketIds, int batchSize) {
+    AsyncEventListener theListener = null;
+
+    Set<AsyncEventQueue> asyncEventQueues = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncQueue : asyncEventQueues) {
+      if (asyncEventQueueId.equals(asyncQueue.getId())) {
+        theListener = asyncQueue.getAsyncEventListener();
+      }
+    }
+
+    final Map<Integer, List<GatewaySenderEventImpl>> bucketToEventsMap = ((MyAsyncEventListener2)theListener)
+        .getBucketToEventsMap();
+    assertNotNull(bucketToEventsMap);
+    assertTrue(bucketIds.size() > 1);
+
+    for (int bucketId : bucketIds) {
+      List<GatewaySenderEventImpl> eventsForBucket = bucketToEventsMap
+          .get(bucketId);
+      getLogWriter().info(
+          "Events for bucket: " + bucketId + " is " + eventsForBucket);
+      assertNotNull(eventsForBucket);
+      for (int i = 0; i < batchSize; i++) {
+        GatewaySenderEventImpl senderEvent = eventsForBucket.get(i);
+        assertTrue(senderEvent.getPossibleDuplicate());
+      }
+    }
+  }
+
+  public static int getAsyncEventListenerMapSize(String asyncEventQueueId) {
+    AsyncEventListener theListener = null;
+
+    Set<AsyncEventQueue> asyncEventQueues = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncQueue : asyncEventQueues) {
+      if (asyncEventQueueId.equals(asyncQueue.getId())) {
+        theListener = asyncQueue.getAsyncEventListener();
+      }
+    }
+
+    final Map eventsMap = ((MyAsyncEventListener)theListener).getEventsMap();
+    assertNotNull(eventsMap);
+    getLogWriter().info("The events map size is " + eventsMap.size());
+    return eventsMap.size();
+  }
+
+  public static int getAsyncEventQueueSize(String asyncEventQueueId) {
+    AsyncEventQueue theQueue = null;
+
+    Set<AsyncEventQueue> asyncEventQueues = cache.getAsyncEventQueues();
+    for (AsyncEventQueue asyncQueue : asyncEventQueues) {
+      if (asyncEventQueueId.equals(asyncQueue.getId())) {
+        theQueue = asyncQueue;
+      }
+    }
+    assertNotNull(theQueue);
+    return theQueue.size();
+  }
+
+  public static String getRegionFullPath(String regionName) {
+    final Region r = cache.getRegion(Region.SEPARATOR + regionName);
+    assertNotNull(r);
+    return r.getFullPath();
+  }
+
+  public static Set<Integer> getAllPrimaryBucketsOnTheNode(String regionName) {
+    PartitionedRegion region = (PartitionedRegion)cache.getRegion(regionName);
+    return region.getDataStore().getAllLocalPrimaryBucketIds();
+  }
+
+  public static void addCacheListenerAndCloseCache(String regionName) {
+    final Region region = cache.getRegion(Region.SEPARATOR + regionName);
+    assertNotNull(region);
+    CacheListenerAdapter cl = new CacheListenerAdapter() {
+      @Override
+      public void afterCreate(EntryEvent event) {
+        if ((Long)event.getKey() == 900) {
+          cache.getLogger().fine(" Gateway sender is killed by a test");
+          cache.close();
+          cache.getDistributedSystem().disconnect();
+        }
+      }
+    };
+    region.getAttributesMutator().addCacheListener(cl);
+  }
+
+  public static Boolean killSender(String senderId) {
+    final ExpectedException exln = addExpectedException("Could not connect");
+    ExpectedException exp = addExpectedException(CacheClosedException.class
+        .getName());
+    ExpectedException exp1 = addExpectedException(ForceReattemptException.class
+        .getName());
+    try {
+      Set<GatewaySender> senders = cache.getGatewaySenders();
+      AbstractGatewaySender sender = null;
+      for (GatewaySender s : senders) {
+        if (s.getId().equals(senderId)) {
+          sender = (AbstractGatewaySender)s;
+          break;
+        }
+      }
+      if (sender.isPrimary()) {
+        getLogWriter().info("Gateway sender is killed by a test");
+        cache.getDistributedSystem().disconnect();
+        return Boolean.TRUE;
+      }
+      return Boolean.FALSE;
+    }
+    finally {
+      exp.remove();
+      exp1.remove();
+      exln.remove();
+    }
+  }
+
+  public static Boolean killAsyncEventQueue(String asyncQueueId) {
+    Set<AsyncEventQueue> queues = cache.getAsyncEventQueues();
+    AsyncEventQueueImpl queue = null;
+    for (AsyncEventQueue q : queues) {
+      if (q.getId().equals(asyncQueueId)) {
+        queue = (AsyncEventQueueImpl)q;
+        break;
+      }
+    }
+    if (queue.isPrimary()) {
+      getLogWriter().info("AsyncEventQueue is killed by a test");
+      cache.getDistributedSystem().disconnect();
+      return Boolean.TRUE;
+    }
+    return Boolean.FALSE;
+  }
+
+  public static void killSender() {
+    getLogWriter().info("Gateway sender is going to be killed by a test");
+    cache.close();
+    cache.getDistributedSystem().disconnect();
+    getLogWriter().info("Gateway sender is killed by a test");
+  }
+
+  public 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();
+    }
+
+    public synchronized void locatorsRemoved(List locators) {
+      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) {
+        wait(remaining);
+        remaining = endTime - System.currentTimeMillis();
+      }
+      return set.contains(locator);
+    }
+
+    public synchronized Set getDiscovered() {
+      return new HashSet(discoveredLocators);
+    }
+
+    public synchronized Set getRemoved() {
+      return new HashSet(removedLocators);
+    }
+  }
+  
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+    cleanupVM();
+    vm0.invoke(AsyncEventQueueTestBase.class, "cleanupVM");
+    vm1.invoke(AsyncEventQueueTestBase.class, "cleanupVM");
+    vm2.invoke(AsyncEventQueueTestBase.class, "cleanupVM");
+    vm3.invoke(AsyncEventQueueTestBase.class, "cleanupVM");
+    vm4.invoke(AsyncEventQueueTestBase.class, "cleanupVM");
+    vm5.invoke(AsyncEventQueueTestBase.class, "cleanupVM");
+    vm6.invoke(AsyncEventQueueTestBase.class, "cleanupVM");
+    vm7.invoke(AsyncEventQueueTestBase.class, "cleanupVM");
+  }
+
+  public static void cleanupVM() throws IOException {
+    closeCache();
+  }
+
+  public static void closeCache() throws IOException {
+    if (cache != null && !cache.isClosed()) {
+      cache.close();
+      cache.getDistributedSystem().disconnect();
+      cache = null;
+    }
+    else {
+      AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(testName);
+      if (test.isConnectedToDS()) {
+        test.getSystem().disconnect();
+      }
+    }
+  }
+
+  public static void shutdownLocator() {
+    AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(testName);
+    test.getSystem().disconnect();
+  }
+
+  public static void printEventListenerMap() {
+    ((MyGatewaySenderEventListener)eventListener1).printMap();
+  }
+  
+
+  @Override
+  public InternalDistributedSystem getSystem(Properties props) {
+    // For now all WANTestBase tests allocate off-heap memory even though
+    // many of them never use it.
+    // The problem is that WANTestBase has static methods that create instances
+    // of WANTestBase (instead of instances of the subclass). So we can't override
+    // this method so that only the off-heap subclasses allocate off heap memory.
+    props.setProperty(DistributionConfig.OFF_HEAP_MEMORY_SIZE_NAME, "300m");
+    return super.getSystem(props);
+  }
+  
+  /**
+   * Returns true if the test should create off-heap regions.
+   * OffHeap tests should over-ride this method and return false.
+   */
+  public boolean isOffHeap() {
+    return false;
+  }
+
+}
+
+class MyAsyncEventListener_CacheLoader implements AsyncEventListener {
+  private final Map eventsMap;
+
+  public MyAsyncEventListener_CacheLoader() {
+    this.eventsMap = new ConcurrentHashMap();
+  }
+
+  public boolean processEvents(List<AsyncEvent> events) {
+    for (AsyncEvent event : events) {
+      this.eventsMap.put(event.getKey(), event);
+    }
+    return true;
+  }
+
+  public Map getEventsMap() {
+    return eventsMap;
+  }
+
+  public void close() {
+  }
+}
+
+class MyCacheLoader implements CacheLoader, Declarable {
+
+  public Object load(LoaderHelper helper) {
+    Long key = (Long)helper.getKey();
+    return "LoadedValue" + "_" + key;
+  }
+
+  public void close() {
+  }
+
+  public void init(Properties props) {
+  }
+
+}


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

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/build.gradle
----------------------------------------------------------------------
diff --cc build.gradle
index 0c24db9,b5465b8..625da36
--- a/build.gradle
+++ b/build.gradle
@@@ -264,35 -290,37 +290,39 @@@ subprojects 
    }
  
    javadoc.classpath += configurations.provided
-   
+ 
+   javadoc {
+     options.addStringOption('Xdoclint:none', '-quiet')
+   }
+ 
    dependencies {
-     compile 'org.springframework:spring-aop:3.2.12.RELEASE'
-     compile 'org.springframework:spring-beans:3.2.12.RELEASE'
-     compile 'org.springframework:spring-context:3.2.12.RELEASE'
-     compile 'org.springframework:spring-context-support:3.2.12.RELEASE'
-     compile 'org.springframework:spring-core:3.2.12.RELEASE'
-     compile 'org.springframework:spring-expression:3.2.12.RELEASE'
-     compile 'org.springframework:spring-web:3.2.12.RELEASE'
-     compile 'org.springframework:spring-webmvc:3.2.12.RELEASE'
- 
-     testCompile 'com.github.stefanbirkner:system-rules:1.9.0'
-     testCompile 'com.jayway.awaitility:awaitility:1.6.3'
-     testCompile 'edu.umd.cs.mtc:multithreadedtc:1.01'
-     testCompile 'eu.codearte.catch-exception:catch-exception:1.4.4'
-     testCompile 'eu.codearte.catch-exception:catch-throwable:1.4.4'
-     testCompile 'junit:junit:4.12'
-     testCompile 'org.assertj:assertj-core:3.1.0'
-     testCompile 'org.easetech:easytest-core:1.3.2'
-     testCompile 'org.hamcrest:hamcrest-all:1.3'
-     testCompile 'org.jmock:jmock:2.8.1'
-     testCompile 'org.jmock:jmock-junit4:2.8.1'
-     testCompile 'org.jmock:jmock-legacy:2.8.1'
-     testCompile 'org.mockito:mockito-core:1.10.19'
-     testCompile 'pl.pragmatists:JUnitParams:1.0.4'
- 
-     testRuntime 'cglib:cglib:3.1'
-     testRuntime 'org.objenesis:objenesis:2.1'
-     testRuntime 'org.ow2.asm:asm:5.0.3'
+     compile 'org.springframework:spring-aop:' + project.'springframework.version'
+     compile 'org.springframework:spring-beans:' + project.'springframework.version'
+     compile 'org.springframework:spring-context:' + project.'springframework.version'
+     compile 'org.springframework:spring-context-support:' + project.'springframework.version'
+     compile 'org.springframework:spring-core:' + project.'springframework.version'
+     compile 'org.springframework:spring-expression:' + project.'springframework.version'
+     compile 'org.springframework:spring-web:' + project.'springframework.version'
+     compile 'org.springframework:spring-webmvc:' + project.'springframework.version'
+     compile 'com.github.stephenc.findbugs:findbugs-annotations:' + project.'stephenc-findbugs.version'
+ 
 -    testCompile 'com.jayway.awaitility:awaitility:' + project.'awaitility.version'
+     testCompile 'com.github.stefanbirkner:system-rules:' + project.'system-rules.version'
++    testCompile 'com.jayway.awaitility:awaitility:' + project.'awaitility.version'
+     testCompile 'edu.umd.cs.mtc:multithreadedtc:' + project.'multithreadedtc.version'
++    testCompile 'eu.codearte.catch-exception:catch-exception:' + project.'catch-exception.version'
++    testCompile 'eu.codearte.catch-exception:catch-throwable:' + project.'catch-throwable.version'
+     testCompile 'junit:junit:' + project.'junit.version'
+     testCompile 'org.assertj:assertj-core:' + project.'assertj-core.version'
 -    testCompile 'org.mockito:mockito-core:' + project.'mockito-core.version'
+     testCompile 'org.hamcrest:hamcrest-all:' + project.'hamcrest-all.version'
+     testCompile 'org.jmock:jmock:' + project.'jmock.version'
+     testCompile 'org.jmock:jmock-junit4:' + project.'jmock.version'
+     testCompile 'org.jmock:jmock-legacy:' + project.'jmock.version'
++    testCompile 'org.mockito:mockito-core:' + project.'mockito-core.version'
+     testCompile 'pl.pragmatists:JUnitParams:' + project.'JUnitParams.version'
+     
+     testRuntime 'cglib:cglib:' + project.'cglib.version'
+     testRuntime 'org.objenesis:objenesis:' + project.'objenesis.version'
+     testRuntime 'org.ow2.asm:asm:' + project.'asm.version'
    }
  
    test {
@@@ -345,12 -373,15 +375,14 @@@
    task distributedTest(type:Test) {
      include '**/*DUnitTest.class'
      
-     // exclde all categories so that dunitTest task is only task executing categorized dunit tests
+     // maxParallelForks = 2
+     // maxParallelForks = Runtime.runtime.availableProcessors()
+     
 -// TODO add @Category(DistributedTest.class) to dunit tests
 -//    useJUnit {
 -//      excludeCategories 'com.gemstone.gemfire.test.junit.categories.UnitTest'
 -//      excludeCategories 'com.gemstone.gemfire.test.junit.categories.IntegrationTest'
 -//      includeCategories 'com.gemstone.gemfire.test.junit.categories.DistributedTest'
 -//    }    
 +    useJUnit {
 +      excludeCategories 'com.gemstone.gemfire.test.junit.categories.UnitTest'
 +      excludeCategories 'com.gemstone.gemfire.test.junit.categories.IntegrationTest'
-       excludeCategories 'com.gemstone.gemfire.test.junit.categories.DistributedTest'
++      includeCategories 'com.gemstone.gemfire.test.junit.categories.DistributedTest'
 +    }    
      
      //I'm hoping this might deal with SOME OOMEs I've seen
      forkEvery 30

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/DebuggerSupport.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/DebuggerSupport.java
index f7d4fcb,49ce32e..b022700
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/DebuggerSupport.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/DebuggerSupport.java
@@@ -8,14 -17,9 +17,12 @@@
  
  package com.gemstone.gemfire.internal.util;
  
 -import com.gemstone.gemfire.i18n.LogWriterI18n;
 +import org.apache.logging.log4j.Logger;
 +
  import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 +import com.gemstone.gemfire.internal.logging.LogService;
 +import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
  
- import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
- 
  /**
   *
   * @author Eric Zoerner
@@@ -29,12 -32,12 +36,12 @@@ public abstract class DebuggerSupport  
    }
    
    /** Debugger support */
 -  public static void waitForJavaDebugger(LogWriterI18n logger) {
 -    waitForJavaDebugger(logger, null);
 +  public static void waitForJavaDebugger() {
 +    waitForJavaDebugger(null);
    }
    
-   @SuppressFBWarnings(value="IL_INFINITE_LOOP", justification="Endless loop is for debugging purposes.") 
+   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IL_INFINITE_LOOP", justification="Endless loop is for debugging purposes.") 
 -  public static void waitForJavaDebugger(LogWriterI18n logger, String extraLogMsg) {
 +  public static void waitForJavaDebugger(String extraLogMsg) {
      boolean cont = false;
      String msg = ":";
      if (extraLogMsg != null)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedMemberDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/HostedLocatorsDUnitTest.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/java/com/gemstone/gemfire/distributed/HostedLocatorsDUnitTest.java
index 4588651,6a622e3..0b5048a
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/HostedLocatorsDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/HostedLocatorsDUnitTest.java
@@@ -1,14 -1,22 +1,30 @@@
+ /*
+  * 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.distributed;
  
 +import static com.gemstone.gemfire.test.dunit.DUnitTestRule.*;
 +import static com.gemstone.gemfire.test.dunit.Invoke.*;
 +import static com.jayway.awaitility.Awaitility.*;
 +import static java.util.concurrent.TimeUnit.*;
 +import static org.assertj.core.api.Assertions.*;
 +import static org.junit.Assert.*;
 +
  import java.io.File;
 +import java.io.Serializable;
  import java.util.Collection;
  import java.util.HashSet;
  import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/BackupDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
index 72b64e6,2c351be..eb67628
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
@@@ -1,10 -1,21 +1,26 @@@
+ /*
+  * 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.offheap;
  
 +import static com.gemstone.gemfire.test.dunit.ExpectedExceptionString.*;
 +import static com.gemstone.gemfire.test.dunit.Invoke.*;
 +import static com.gemstone.gemfire.test.dunit.Wait.*;
 +import static org.junit.Assert.*;
 +
  import java.util.Properties;
  import java.util.concurrent.atomic.AtomicBoolean;
  import java.util.concurrent.atomic.AtomicReference;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestSuite.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestSuite.java
index ef2686e,0000000..6748386
mode 100755,000000..100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestSuite.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestSuite.java
@@@ -1,27 -1,0 +1,43 @@@
++/*
++ * 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.golden;
 +
 +import org.junit.runner.RunWith;
 +import org.junit.runners.Suite;
 +
 +@RunWith(Suite.class)
 +@Suite.SuiteClasses({
 +  FailWithErrorInOutputJUnitTest.class,
 +  FailWithExtraLineInOutputJUnitTest.class,
 +  FailWithLineMissingFromEndOfOutputJUnitTest.class,
 +  FailWithLineMissingFromMiddleOfOutputJUnitTest.class,
 +  FailWithLoggerErrorInOutputJUnitTest.class,
 +  FailWithLoggerFatalInOutputJUnitTest.class,
 +  FailWithLoggerWarnInOutputJUnitTest.class,
 +  FailWithSevereInOutputJUnitTest.class,
 +  FailWithTimeoutOfWaitForOutputToMatchJUnitTest.class,
 +  FailWithWarningInOutputJUnitTest.class,
 +  PassJUnitTest.class,
 +  PassWithExpectedErrorJUnitTest.class,
 +  PassWithExpectedSevereJUnitTest.class,
 +  PassWithExpectedWarningJUnitTest.class,
 +})
 +/**
 + * Suite of tests for the test.golden Golden Test framework classes.
 + */
 +public class GoldenTestSuite {
 +}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-core/src/test/java/dunit/DistributedTestCase.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/java/dunit/DistributedTestCase.java
index d26d3e1,a3d4785..c43f61c
--- a/gemfire-core/src/test/java/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
@@@ -544,20 -559,8 +559,8 @@@ public abstract class DistributedTestCa
     * NOTE: if you use this method be sure that you clean up the VM before the end of your
     * test with disconnectFromDS() or disconnectAllFromDS().
     */
 -  public void crashDistributedSystem(final DistributedSystem msys) {
 +  public static void crashDistributedSystem(final DistributedSystem msys) {
-     MembershipManagerHelper.inhibitForcedDisconnectLogging(true);
-     MembershipManagerHelper.playDead(msys);
-     JChannel c = MembershipManagerHelper.getJChannel(msys);
-     Protocol udp = c.getProtocolStack().findProtocol("UDP");
-     udp.stop();
-     udp.passUp(new Event(Event.EXIT, new RuntimeException("killing member's ds")));
-     try {
-       MembershipManagerHelper.getJChannel(msys).waitForClose();
-     }
-     catch (InterruptedException ie) {
-       Thread.currentThread().interrupt();
-       // attempt rest of work with interrupt bit set
-     }
+     MembershipManagerHelper.crashDistributedSystem(msys);
      MembershipManagerHelper.inhibitForcedDisconnectLogging(false);
      WaitCriterion wc = new WaitCriterion() {
        public boolean done() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-core/src/test/java/dunit/VM.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
index 965b1ab,60ac04d..4d89d01
--- a/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
+++ b/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
@@@ -148,16 -168,14 +168,14 @@@ public class ProcessManager 
        "-D" + DUnitLauncher.WORKSPACE_DIR_PARAM + "=" + new File(".").getAbsolutePath(),
        "-DlogLevel=" + DUnitLauncher.LOG_LEVEL,
        "-Djava.library.path=" + System.getProperty("java.library.path"),
-       "-Xrunjdwp:transport=dt_socket,server=y,suspend=n",
+       "-Xrunjdwp:transport=dt_socket,server=y,suspend=" + jdkSuspend + jdkDebug,
        "-XX:+HeapDumpOnOutOfMemoryError",
        "-Xmx512m",
-       "-XX:MaxPermSize=256M",
        "-Dgemfire.DEFAULT_MAX_OPLOG_SIZE=10",
        "-Dgemfire.disallowMcastDefaults=true",
-       "-XX:MaxPermSize=256M",
        "-ea",
        agent,
 -      "dunit.standalone.ChildVM"
 +      ChildVM.class.getName()
      };
    }
    

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRule.java
----------------------------------------------------------------------
diff --cc gemfire-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRule.java
index bf6456f,0000000..7b6d345
mode 100755,000000..100755
--- a/gemfire-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRule.java
+++ b/gemfire-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRule.java
@@@ -1,164 -1,0 +1,180 @@@
++/*
++ * 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.assertThat;
 +
 +import java.util.concurrent.TimeUnit;
 +
 +import org.hamcrest.Matcher;
 +import org.junit.rules.ExpectedException;
 +import org.junit.rules.TestRule;
 +import org.junit.runner.Description;
 +import org.junit.runners.model.Statement;
 +
 +/**
 + * Expect an Exception within a specified timeout.
 + * 
 + * @author Kirk Lund
 + * @since 8.2
 + */
 +public class ExpectedTimeoutRule implements TestRule {
 +
 +  /**
 +   * @return a Rule that expects no timeout (identical to behavior without this Rule)
 +   */
 +  public static ExpectedTimeoutRule none() {
 +    return new ExpectedTimeoutRule();
 +  }
 +  
 +  private ExpectedException delegate;
 +  private boolean expectsThrowable;
 +  private long minDuration;
 +  private long maxDuration;
 +  private TimeUnit timeUnit;
 +  
 +  private ExpectedTimeoutRule() {
 +    this.delegate = ExpectedException.none();
 +  }
 +
 +  public ExpectedTimeoutRule expectMinimumDuration(final long minDuration) {
 +    this.minDuration = minDuration;
 +    return this;
 +  }
 +  public ExpectedTimeoutRule expectMaximumDuration(final long maxDuration) {
 +    this.maxDuration = maxDuration;
 +    return this;
 +  }
 +  public ExpectedTimeoutRule expectTimeUnit(final TimeUnit timeUnit) {
 +    this.timeUnit = timeUnit;
 +    return this;
 +  }
 +
 +  public ExpectedTimeoutRule handleAssertionErrors() {
 +    this.delegate.handleAssertionErrors();
 +    return this;
 +  }
 +  
 +  public ExpectedTimeoutRule handleAssumptionViolatedExceptions() {
 +    this.delegate.handleAssumptionViolatedExceptions();
 +    return this;
 +  }
 +  
 +  /**
 +   * Adds {@code matcher} to the list of requirements for any thrown
 +   * exception.
 +   */
 +  public void expect(final Matcher<?> matcher) {
 +    this.delegate.expect(matcher);
 +  }
 +
 +  /**
 +   * Adds to the list of requirements for any thrown exception that it should
 +   * be an instance of {@code type}
 +   */
 +  public void expect(final Class<? extends Throwable> type) {
 +    this.delegate.expect(type);
 +    this.expectsThrowable = true;
 +  }
 +
 +  /**
 +   * Adds to the list of requirements for any thrown exception that it should
 +   * <em>contain</em> string {@code substring}
 +   */
 +  public void expectMessage(final String substring) {
 +    this.delegate.expectMessage(substring);
 +  }
 +
 +  /**
 +   * Adds {@code matcher} to the list of requirements for the message returned
 +   * from any thrown exception.
 +   */
 +  public void expectMessage(final Matcher<String> matcher) {
 +    this.delegate.expectMessage(matcher);
 +  }
 +
 +  /**
 +   * Adds {@code matcher} to the list of requirements for the cause of
 +   * any thrown exception.
 +   */
 +  public void expectCause(final Matcher<? extends Throwable> expectedCause) {
 +    this.delegate.expectCause(expectedCause);
 +  }
 +
 +  public boolean expectsTimeout() {
 +    return minDuration > 0 || maxDuration > 0;
 +  }
 +  
 +  public boolean expectsThrowable() {
 +    return expectsThrowable = true;
 +  }
 +  
 +  @Override
 +  public Statement apply(final Statement base, final Description description) {
 +    Statement next = delegate.apply(base, description);
 +    return new ExpectedTimeoutStatement(next);
 +  }
 +  
 +  private void handleTime(final Long duration) {
 +    if (expectsTimeout()) {
 +      assertThat(timeUnit.convert(duration, TimeUnit.NANOSECONDS), new TimeMatcher(timeUnit, minDuration, maxDuration));
 +    }
 +  }
 +  
 +  private static class TimeMatcher extends org.hamcrest.TypeSafeMatcher<Long> {
 +    
 +    private final TimeUnit timeUnit;
 +    private final long minDuration;
 +    private final long maxDuration;
 + 
 +    public TimeMatcher(final TimeUnit timeUnit, final long minDuration, final long maxDuration) {
 +      this.timeUnit = timeUnit;
 +      this.minDuration = minDuration;
 +      this.maxDuration = maxDuration;
 +    }
 + 
 +    @Override
 +    public boolean matchesSafely(final Long duration) {
 +      return duration >= this.minDuration && duration <= this.maxDuration;
 +    }
 +
 +    @Override
 +    public void describeTo(final org.hamcrest.Description description) {
 +      description.appendText("expects duration to be greater than or equal to ")
 +          .appendValue(this.minDuration)
 +          .appendText(" and less than or equal to ")
 +          .appendValue(this.maxDuration)
 +          .appendText(" ")
 +          .appendValue(this.timeUnit);
 +    }
 +  }
 +  
 +  private class ExpectedTimeoutStatement extends Statement {
 +    private final Statement next;
 +
 +    public ExpectedTimeoutStatement(final Statement base) {
 +      next = base;
 +    }
 +
 +    @Override
 +    public void evaluate() throws Throwable {
 +      long start = System.nanoTime();
 +      next.evaluate();
 +      handleTime(System.nanoTime() - start);
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gemfire-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/ExpectedTimeoutRuleJUnitTest.java
----------------------------------------------------------------------
diff --cc gemfire-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/ExpectedTimeoutRuleJUnitTest.java
index 41ef275,0000000..67c285a
mode 100755,000000..100755
--- a/gemfire-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/ExpectedTimeoutRuleJUnitTest.java
+++ b/gemfire-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/ExpectedTimeoutRuleJUnitTest.java
@@@ -1,198 -1,0 +1,214 @@@
++/*
++ * 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.tests;
 +
 +import static com.gemstone.gemfire.test.junit.rules.tests.RunTest.*;
 +import static org.assertj.core.api.Assertions.*;
 +
 +import java.util.List;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.TimeoutException;
 +
 +import org.junit.Rule;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +import org.junit.runner.Result;
 +import org.junit.runner.notification.Failure;
 +
 +import com.gemstone.gemfire.test.junit.categories.UnitTest;
 +import com.gemstone.gemfire.test.junit.rules.ExpectedTimeoutRule;
 +
 +/**
 + * Unit tests for ExpectedTimeout JUnit Rule.
 + * 
 + * @author Kirk Lund
 + * @since 8.2
 + */
 +@Category(UnitTest.class)
 +public class ExpectedTimeoutRuleJUnitTest {
 +
 +  @Test
 +  public void passesUnused() {
 +    Result result = runTest(PassingTestShouldPassWhenUnused.class);
 +    
 +    assertThat(result.wasSuccessful()).isTrue();
 +  }
 +  
 +  @Test
 +  public void failsWithoutExpectedException() {
 +    Result result = runTest(FailsWithoutExpectedException.class);
 +    
 +    assertThat(result.wasSuccessful()).isFalse();
 +    
 +    List<Failure> failures = result.getFailures();
 +    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
 +    
 +    Failure failure = failures.get(0);
 +    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw an instance of " + TimeoutException.class.getName());
 +  }
 +  
 +  @Test
 +  public void failsWithoutExpectedTimeoutException() {
 +    Result result = runTest(FailsWithoutExpectedTimeoutException.class);
 +    
 +    assertThat(result.wasSuccessful()).isFalse();
 +    
 +    List<Failure> failures = result.getFailures();
 +    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
 +    
 +    Failure failure = failures.get(0);
 +    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWithoutExpectedTimeoutException.message + "\")");
 +  }
 +  
 +  @Test
 +  public void failsWithExpectedTimeoutButWrongError() {
 +    Result result = runTest(FailsWithExpectedTimeoutButWrongError.class);
 +    
 +    assertThat(result.wasSuccessful()).isFalse();
 +    
 +    List<Failure> failures = result.getFailures();
 +    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
 +    
 +    Failure failure = failures.get(0);
 +    String expectedMessage = 
 +        "\n" + 
 +        "Expected: (an instance of java.util.concurrent.TimeoutException and exception with message a string containing \"this is a message for FailsWithExpectedTimeoutButWrongError\")" +
 +        "\n" + 
 +        "     " +
 +        "but: an instance of java.util.concurrent.TimeoutException <java.lang.NullPointerException> is a java.lang.NullPointerException";
 +    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessageContaining(expectedMessage);
 +  }
 +  
 +  @Test
 +  public void passesWithExpectedTimeoutAndTimeoutException() {
 +    Result result = runTest(PassesWithExpectedTimeoutAndTimeoutException.class);
 +    
 +    assertThat(result.wasSuccessful()).isTrue();
 +  }
 +  
 +  @Test
 +  public void failsWhenTimeoutIsEarly() {
 +    Result result = runTest(FailsWhenTimeoutIsEarly.class);
 +   
 +    assertThat(result.wasSuccessful()).isFalse();
 +    
 +    List<Failure> failures = result.getFailures();
 +    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
 +    
 +    Failure failure = failures.get(0);
 +    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWhenTimeoutIsEarly.message + "\")");
 +  }
 +  
 +  @Test
 +  public void failsWhenTimeoutIsLate() {
 +    Result result = runTest(FailsWhenTimeoutIsLate.class);
 +    
 +    assertThat(result.wasSuccessful()).isFalse();
 +    
 +    List<Failure> failures = result.getFailures();
 +    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
 +    
 +    Failure failure = failures.get(0);
 +    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWhenTimeoutIsLate.message + "\")");
 +  }
 +  
 +  public static class AbstractExpectedTimeoutRuleTest {
 +    @Rule
 +    public ExpectedTimeoutRule timeout = ExpectedTimeoutRule.none();
 +  }
 +  
 +  public static class PassingTestShouldPassWhenUnused extends AbstractExpectedTimeoutRuleTest {
 +    @Test
 +    public void passesUnused() throws Exception {
 +    }
 +  }
 +  
 +  public static class FailsWithoutExpectedException extends AbstractExpectedTimeoutRuleTest {
 +    @Test
 +    public void failsWithoutExpectedException() throws Exception {
 +      timeout.expect(TimeoutException.class);
 +    }
 +  }
 +  
 +  public static class FailsWithoutExpectedTimeoutException extends AbstractExpectedTimeoutRuleTest {
 +    public static final String message = "this is a message for FailsWithoutExpectedTimeoutException";
 +    @Test
 +    public void failsWithoutExpectedTimeoutAndTimeoutException() throws Exception {
 +      timeout.expect(TimeoutException.class);
 +      timeout.expectMessage(message);
 +      timeout.expectMinimumDuration(10);
 +      timeout.expectMaximumDuration(1000);
 +      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
 +      Thread.sleep(100);
 +    }
 +  }
 +  
 +  public static class FailsWithExpectedTimeoutButWrongError extends AbstractExpectedTimeoutRuleTest {
 +    public static final String message = "this is a message for FailsWithExpectedTimeoutButWrongError";
 +    @Test
 +    public void failsWithExpectedTimeoutButWrongError() throws Exception {
 +      timeout.expect(TimeoutException.class);
 +      timeout.expectMessage(message);
 +      timeout.expectMinimumDuration(10);
 +      timeout.expectMaximumDuration(1000);
 +      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
 +      Thread.sleep(100);
 +      throw new NullPointerException();
 +    }
 +  }
 +
 +  public static class PassesWithExpectedTimeoutAndTimeoutException extends AbstractExpectedTimeoutRuleTest {
 +    public static final String message = "this is a message for PassesWithExpectedTimeoutAndTimeoutException";
 +    public static final Class<TimeoutException> exceptionClass = TimeoutException.class;
 +    @Test
 +    public void passesWithExpectedTimeoutAndTimeoutException() throws Exception {
 +      timeout.expect(exceptionClass);
 +      timeout.expectMessage(message);
 +      timeout.expectMinimumDuration(10);
 +      timeout.expectMaximumDuration(1000);
 +      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
 +      Thread.sleep(100);
 +      throw new TimeoutException(message);
 +    }
 +  }
 +
 +  public static class FailsWhenTimeoutIsEarly extends AbstractExpectedTimeoutRuleTest {
 +    public static final String message = "this is a message for FailsWhenTimeoutIsEarly";
 +    @Test
 +    public void failsWhenTimeoutIsEarly() throws Exception {
 +      timeout.expect(TimeoutException.class);
 +      timeout.expectMessage(message);
 +      timeout.expectMinimumDuration(1000);
 +      timeout.expectMaximumDuration(2000);
 +      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
 +      Thread.sleep(10);
 +    }
 +  }
 +
 +  public static class FailsWhenTimeoutIsLate extends AbstractExpectedTimeoutRuleTest {
 +    public static final String message = "this is a message for FailsWhenTimeoutIsLate";
 +    @Test
 +    public void failsWhenTimeoutIsLate() throws Exception {
 +      timeout.expect(TimeoutException.class);
 +      timeout.expectMessage(message);
 +      timeout.expectMinimumDuration(10);
 +      timeout.expectMaximumDuration(20);
 +      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
 +      Thread.sleep(100);
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4c72833c/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --cc gradle/dependency-versions.properties
index 0000000,3e6b6a5..160ff5b
mode 000000,100644..100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@@ -1,0 -1,65 +1,67 @@@
+ # Buildscript Dependencies
+ gradle-maven-publish-auth.version = 2.0.1
+ 
+ # Dependency versions
+ activation.version = 1.1.1
+ annotations.version = 3.0.0
+ antlr.version = 2.7.7
+ asm.version = 5.0.3
 -assertj-core.version = 2.1.0
++assertj-core.version = 3.1.0
+ awaitility.version = 1.6.5
+ bcel.version = 5.2
++catch-exception.version = 1.4.4
++catch-throwable.version = 1.4.4
+ cglib.version = 3.1
+ classmate.version = 0.9.0
+ commons-collections.version = 3.2.1
+ commons-configuration.version = 1.6
+ commons-fileupload.version = 1.3.1
+ commons-io.version = 2.3
+ commons-lang.version = 2.5
+ commons-logging.version = 1.1.1
+ commons-modeler.version = 2.0
+ derby.version = 10.2.2.0
+ fastutil.version = 7.0.2
+ guava.version = 15.0
+ hadoop.version = 2.4.1
+ hamcrest-all.version = 1.3
+ hbase.version = 0.94.27
+ jackson.version = 2.2.0
+ jackson-module-scala_2.10.version = 2.1.5
+ jansi.version = 1.8
+ javax.mail-api.version = 1.4.5
+ javax.resource-api.version = 1.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
+ jline.version = 1.0.S2-B
+ jmock.version = 2.8.1
+ jna.version = 4.0.0
+ json4s.version = 3.2.4
+ junit.version = 4.12
+ JUnitParams.version = 1.0.4
+ log4j.version = 2.1
+ lucene.version = 5.3.0
+ mockito-core.version = 1.10.19
+ multithreadedtc.version = 1.01
+ mx4j.version = 3.0.1
+ mx4j-remote.version = 3.0.1
+ mx4j-tools.version = 3.0.1
+ netty-all.version = 4.0.4.Final
+ objenesis.version = 2.1
+ paranamer.version = 2.3
+ quartz.version = 2.2.1
+ scala.version = 2.10.0
+ slf4j-api.version = 1.7.7
+ snappy-java.version = 1.1.1.6
+ spring-data-commons.version = 1.9.1.RELEASE
+ spring-data-gemfire.version = 1.5.1.RELEASE
+ spring-hateos.version = 0.16.0.RELEASE
+ spring-shell.version = 1.0.0.RELEASE
+ springframework.version = 3.2.12.RELEASE
+ stephenc-findbugs.version = 1.3.9-1
+ spymemcached.version = 2.9.0
+ swagger.version = 1.3.2
+ swagger-springmvc.version = 0.8.2
+ system-rules.version = 1.12.1


[14/50] [abbrv] incubator-geode git commit: GEODE-184: The gfsh 'locate entry' command fails to find the entry on partitioned regions if the key is not a string

Posted by kl...@apache.org.
GEODE-184: The gfsh 'locate entry' command fails to find the entry on partitioned regions if the key is not a string


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

Branch: refs/heads/feature/GEODE-217
Commit: 74e136401fa0d0a01d562c8e3f3db48b30ad56bc
Parents: f974462
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed Dec 2 08:10:13 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Wed Dec 2 15:10:06 2015 -0800

----------------------------------------------------------------------
 .../cli/functions/DataCommandFunction.java      |   6 +-
 .../functions/DataCommandFunctionJUnitTest.java | 132 +++++++++++++++++++
 2 files changed, 135 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/74e13640/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
index d4dac05..af849f8 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
@@ -536,10 +536,10 @@ public class DataCommandFunction extends FunctionAdapter implements  InternalEnt
         //Following code is adaptation of which.java of old Gfsh
         PartitionedRegion pr = (PartitionedRegion)region;
         Region localRegion = PartitionRegionHelper.getLocalData((PartitionedRegion)region);
-        value = localRegion.get(key);
+        value = localRegion.get(keyObject);
         if(value!=null){
-          DistributedMember primaryMember = PartitionRegionHelper.getPrimaryMemberForKey(region, key);
-          int bucketId = pr.getKeyInfo(key).getBucketId();        
+          DistributedMember primaryMember = PartitionRegionHelper.getPrimaryMemberForKey(region, keyObject);
+          int bucketId = pr.getKeyInfo(keyObject).getBucketId();
           boolean isPrimary = member == primaryMember;
           keyInfo.addLocation(new Object[]{region.getFullPath(),true,getJSONForNonPrimitiveObject(value)[1],isPrimary,""+bucketId});          
         }else{

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/74e13640/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunctionJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunctionJUnitTest.java
new file mode 100644
index 0000000..905a9cd
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunctionJUnitTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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 com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.management.internal.cli.domain.DataCommandResult;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * TODO: Add additional tests for all methods in DataCommandFunction.
+ *
+ * @author Jens Deppe
+ */
+@Category(IntegrationTest.class)
+public class DataCommandFunctionJUnitTest {
+
+  private static Cache cache;
+
+  private static Region region1;
+
+  private static final String PARTITIONED_REGION = "part_region";
+
+  public static class StringCheese {
+    private String cheese;
+
+    public StringCheese() {
+      // Empty constructor
+    }
+
+    public StringCheese(final String cheese) {
+      this.cheese = cheese;
+    }
+
+    public void setCheese(final String cheese) {
+      this.cheese = cheese;
+    }
+
+    @Override
+    public String toString() {
+      return cheese;
+    }
+
+    @Override
+    public int hashCode() {
+      int h = this.cheese.hashCode();
+      return h;
+    }
+
+    public boolean equals(Object other) {
+      if (this == other) {
+        return true;
+      }
+      if (other instanceof StringCheese) {
+        return this.cheese.equals(((StringCheese)other).cheese);
+      }
+      return false;
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    cache = new CacheFactory().
+        set(DistributionConfig.MCAST_PORT_NAME, "0").
+        create();
+    RegionFactory factory = cache.createRegionFactory(RegionShortcut.PARTITION);
+    region1 = factory.create(PARTITIONED_REGION);
+
+    region1.put(new StringCheese("key_1"), "value_1");
+    region1.put("key_2", "value_2");
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    cache.close();
+    cache = null;
+  }
+
+  /*
+   * This test addresses GEODE-184
+   */
+  @Test
+  public void testLocateKeyIsObject() throws Exception {
+    DataCommandFunction dataCmdFn = new DataCommandFunction();
+
+    DataCommandResult result = dataCmdFn.locateEntry("{'cheese': 'key_1'}", StringCheese.class.getName(), String.class.getName(), PARTITIONED_REGION, false);
+
+    assertNotNull(result);
+    result.aggregate(null);
+    List<DataCommandResult.KeyInfo> keyInfos = result.getLocateEntryLocations();
+    assertEquals(1, keyInfos.size());
+  }
+
+  @Test
+  public void testLocateKeyIsString() throws Exception {
+    DataCommandFunction dataCmdFn = new DataCommandFunction();
+
+    DataCommandResult result = dataCmdFn.locateEntry("key_2", String.class.getName(), String.class.getName(), PARTITIONED_REGION, false);
+
+    assertNotNull(result);
+    result.aggregate(null);
+    List<DataCommandResult.KeyInfo> keyInfos = result.getLocateEntryLocations();
+    assertEquals(1, keyInfos.size());
+  }
+}


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

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


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

Branch: refs/heads/feature/GEODE-217
Commit: edc29d0625c6f6b69e7b1fe4c77d90f431affb6d
Parents: 4c72833 6e32ffe
Author: Kirk Lund <kl...@pivotal.io>
Authored: Wed Dec 9 16:24:13 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Wed Dec 9 16:24:13 2015 -0800

----------------------------------------------------------------------
 build.gradle                                    |   18 +-
 gemfire-assembly/build.gradle                   |   13 +
 .../LauncherLifecycleCommandsDUnitTest.java     | 1005 +++++++++
 .../LauncherLifecycleCommandsJUnitTest.java     |  625 ++++++
 .../SharedConfigurationEndToEndDUnitTest.java   |  434 ++++
 .../cache/partition/PartitionManager.java       |  377 ----
 .../internal/DistributionMessage.java           |    2 +-
 .../internal/membership/MemberAttributes.java   |  131 +-
 .../membership/gms/membership/GMSJoinLeave.java |   44 +-
 .../gms/messages/HeartbeatMessage.java          |    2 +-
 .../gms/messages/HeartbeatRequestMessage.java   |    2 +-
 .../gms/messages/InstallViewMessage.java        |    2 +-
 .../gms/messages/JoinResponseMessage.java       |   10 +-
 .../membership/gms/messages/ViewAckMessage.java |    2 +-
 .../gms/messenger/AddressManager.java           |   21 +-
 .../membership/gms/messenger/GMSPingPonger.java |   22 +-
 .../membership/gms/messenger/JGAddress.java     |   23 +-
 .../gms/messenger/JGroupsMessenger.java         |  385 ++--
 .../membership/gms/messenger/Transport.java     |    2 +-
 .../internal/tcpserver/TcpServer.java           |    2 +-
 .../internal/i18n/ParentLocalizedStrings.java   |    4 +-
 .../internal/offheap/OffHeapStorage.java        |    3 +-
 .../internal/offheap/RefCountChangeInfo.java    |   43 +-
 .../partition/PartitionManagerDUnitTest.java    |  443 ----
 .../gemfire/cache30/ReconnectDUnitTest.java     |    2 +-
 .../internal/DistributionManagerDUnitTest.java  |    1 +
 .../membership/MembershipJUnitTest.java         |  116 +-
 .../membership/gms/MembershipManagerHelper.java |    1 +
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |    1 +
 .../messenger/JGroupsMessengerJUnitTest.java    |  481 +++-
 .../fixed/FixedPartitioningTestBase.java        |   83 -
 ...ngWithColocationAndPersistenceDUnitTest.java |  106 -
 .../InstantiatorPropagationDUnitTest.java       | 1761 +++++++++++++++
 .../cache/wan/AsyncEventQueueTestBase.java      | 1658 ++++++++++++++
 .../asyncqueue/AsyncEventListenerDUnitTest.java | 1911 ++++++++++++++++
 .../AsyncEventListenerOffHeapDUnitTest.java     |   17 +
 .../AsyncEventQueueStatsDUnitTest.java          |  311 +++
 .../ConcurrentAsyncEventQueueDUnitTest.java     |  330 +++
 ...ncurrentAsyncEventQueueOffHeapDUnitTest.java |   16 +
 .../CommonParallelAsyncEventQueueDUnitTest.java |   53 +
 ...ParallelAsyncEventQueueOffHeapDUnitTest.java |   16 +
 .../offheap/RefCountChangeInfoJUnitTest.java    |  159 ++
 .../internal/cli/GfshParserJUnitTest.java       |    2 +
 .../management/internal/cli/HeadlessGfsh.java   |  376 ++++
 .../internal/cli/HeadlessGfshJUnitTest.java     |   87 +
 .../management/internal/cli/ResultHandler.java  |   23 +
 .../internal/cli/TableBuilderJUnitTest.java     |  183 ++
 .../cli/commands/CliCommandTestBase.java        |  560 +++++
 .../cli/commands/ConfigCommandsDUnitTest.java   |  497 +++++
 ...eateAlterDestroyRegionCommandsDUnitTest.java | 1148 ++++++++++
 .../cli/commands/DeployCommandsDUnitTest.java   |  480 ++++
 .../commands/DiskStoreCommandsDUnitTest.java    | 1154 ++++++++++
 .../cli/commands/FunctionCommandsDUnitTest.java |  593 +++++
 .../commands/GemfireDataCommandsDUnitTest.java  | 2087 ++++++++++++++++++
 ...WithCacheLoaderDuringCacheMissDUnitTest.java |  371 ++++
 .../cli/commands/IndexCommandsDUnitTest.java    |  817 +++++++
 ...stAndDescribeDiskStoreCommandsDUnitTest.java |  197 ++
 .../ListAndDescribeRegionDUnitTest.java         |  320 +++
 .../cli/commands/ListIndexCommandDUnitTest.java |  672 ++++++
 .../cli/commands/MemberCommandsDUnitTest.java   |  286 +++
 .../MiscellaneousCommandsDUnitTest.java         |  492 +++++
 ...laneousCommandsExportLogsPart1DUnitTest.java |  139 ++
 ...laneousCommandsExportLogsPart2DUnitTest.java |  148 ++
 ...laneousCommandsExportLogsPart3DUnitTest.java |  150 ++
 ...laneousCommandsExportLogsPart4DUnitTest.java |  141 ++
 .../cli/commands/QueueCommandsDUnitTest.java    |  385 ++++
 .../SharedConfigurationCommandsDUnitTest.java   |  338 +++
 .../cli/commands/ShellCommandsDUnitTest.java    |  365 +++
 .../cli/commands/ShowDeadlockDUnitTest.java     |  271 +++
 .../cli/commands/ShowMetricsDUnitTest.java      |  347 +++
 .../cli/commands/ShowStackTraceDUnitTest.java   |  149 ++
 .../cli/commands/UserCommandsDUnitTest.java     |  164 ++
 .../src/test/java/dunit/RemoteDUnitVMIF.java    |    2 +
 .../src/test/java/dunit/standalone/ChildVM.java |   11 +-
 .../java/dunit/standalone/DUnitLauncher.java    |   24 +
 .../java/dunit/standalone/ProcessManager.java   |   14 +-
 .../java/dunit/standalone/RemoteDUnitVM.java    |    7 +-
 gemfire-site/build.gradle                       |   37 -
 .../content/bootstrap/bootstrap.min.css         |    9 +
 gemfire-site/content/community/index.html       |  629 ++++++
 .../content/css/bootflat-extensions.css         |  356 +++
 gemfire-site/content/css/bootflat-square.css    |   69 +
 gemfire-site/content/css/bootflat.css           | 1559 +++++++++++++
 gemfire-site/content/css/font-awesome.min.css   |  405 ++++
 gemfire-site/content/css/geode-site.css         | 1554 +++++++++++++
 gemfire-site/content/css/usergrid-site.css      | 1554 +++++++++++++
 gemfire-site/content/favicon.ico                |  Bin 0 -> 20805 bytes
 gemfire-site/content/font/FontAwesome.otf       |  Bin 0 -> 61896 bytes
 .../content/font/fontawesome-webfont-eot.eot    |  Bin 0 -> 37405 bytes
 .../content/font/fontawesome-webfont-svg.svg    |  399 ++++
 .../content/font/fontawesome-webfont-ttf.ttf    |  Bin 0 -> 79076 bytes
 .../content/font/fontawesome-webfont-woff.woff  |  Bin 0 -> 43572 bytes
 gemfire-site/content/img/apache_geode_logo.png  |  Bin 0 -> 23616 bytes
 .../content/img/apache_geode_logo_white.png     |  Bin 0 -> 22695 bytes
 .../img/apache_geode_logo_white_small.png       |  Bin 0 -> 52948 bytes
 gemfire-site/content/img/check_flat/default.png |  Bin 0 -> 25851 bytes
 gemfire-site/content/img/egg-logo.png           |  Bin 0 -> 9938 bytes
 gemfire-site/content/img/github.png             |  Bin 0 -> 8936 bytes
 gemfire-site/content/index.html                 |  295 +++
 gemfire-site/content/js/bootstrap.min.js        |    8 +
 gemfire-site/content/js/head.js                 |  708 ++++++
 gemfire-site/content/js/html5shiv.js            |    8 +
 gemfire-site/content/js/jquery-1.10.1.min.js    |    6 +
 gemfire-site/content/js/jquery.icheck.js        |  397 ++++
 gemfire-site/content/js/respond.min.js          |    6 +
 gemfire-site/content/js/usergrid-site.js        |   50 +
 gemfire-site/content/releases/index.html        |  239 ++
 gemfire-site/content/static/github-btn.html     |    2 +
 gemfire-site/src/jbake.zip                      |  Bin 207030 -> 0 bytes
 gemfire-site/src/jbake/assets/favicon.ico       |  Bin 1150 -> 0 bytes
 .../src/jbake/assets/images/bg-billboard.png    |  Bin 25538 -> 0 bytes
 .../jbake/assets/images/bg-crystals-home.png    |  Bin 41684 -> 0 bytes
 .../assets/images/bg-crystals-secondary.png     |  Bin 26046 -> 0 bytes
 .../src/jbake/assets/images/egg-logo1.png       |  Bin 8626 -> 0 bytes
 .../jbake/assets/images/events/apachecon.png    |  Bin 4528 -> 0 bytes
 .../src/jbake/assets/images/events/oscon.png    |  Bin 26024 -> 0 bytes
 .../src/jbake/assets/images/geode-banner.png    |  Bin 7916 -> 0 bytes
 .../assets/images/logo-apache-geode-white.png   |  Bin 2336 -> 0 bytes
 .../jbake/assets/images/logo-apache-geode.png   |  Bin 3200 -> 0 bytes
 .../jbake/assets/images/logo-geode-white.png    |  Bin 1620 -> 0 bytes
 .../src/jbake/assets/images/logo-geode.png      |  Bin 3345 -> 0 bytes
 .../src/jbake/assets/javascripts/master.js      |  121 -
 .../src/jbake/assets/javascripts/scale.fix.js   |   20 -
 .../jbake/assets/stylesheets/pygment_trac.css   |   60 -
 .../src/jbake/assets/stylesheets/styles.css     |  319 ---
 gemfire-site/src/jbake/content/404.md           |    9 -
 gemfire-site/src/jbake/content/README.md        |   36 -
 gemfire-site/src/jbake/content/about/index.md   |   31 -
 .../src/jbake/content/community/index.md        |   82 -
 .../src/jbake/content/contribute/index.md       |   47 -
 gemfire-site/src/jbake/content/docs/index.md    |   23 -
 .../src/jbake/content/download/index.md         |   13 -
 .../src/jbake/content/getting-started/index.md  |   88 -
 gemfire-site/src/jbake/content/index.md         |   76 -
 gemfire-site/src/jbake/jbake.properties         |    6 -
 gemfire-site/src/jbake/templates/page.groovy    |   80 -
 gemfire-site/website/.gitignore                 |    1 +
 gemfire-site/website/README.md                  |   54 +
 gemfire-site/website/Rules                      |   52 +
 gemfire-site/website/build.sh                   |    1 +
 .../website/content/bootstrap/bootstrap.min.css |    9 +
 .../website/content/community/index.html        |  286 +++
 .../website/content/css/bootflat-extensions.css |  356 +++
 .../website/content/css/bootflat-square.css     |   69 +
 gemfire-site/website/content/css/bootflat.css   | 1559 +++++++++++++
 .../website/content/css/font-awesome.min.css    |  405 ++++
 gemfire-site/website/content/css/geode-site.css | 1554 +++++++++++++
 gemfire-site/website/content/favicon.ico        |  Bin 0 -> 20805 bytes
 .../website/content/font/FontAwesome.otf        |  Bin 0 -> 61896 bytes
 .../content/font/fontawesome-webfont-eot.eot    |  Bin 0 -> 37405 bytes
 .../content/font/fontawesome-webfont-svg.svg    |  399 ++++
 .../content/font/fontawesome-webfont-ttf.ttf    |  Bin 0 -> 79076 bytes
 .../content/font/fontawesome-webfont-woff.woff  |  Bin 0 -> 43572 bytes
 .../website/content/img/apache_geode_logo.png   |  Bin 0 -> 23616 bytes
 .../content/img/apache_geode_logo_white.png     |  Bin 0 -> 22695 bytes
 .../img/apache_geode_logo_white_small.png       |  Bin 0 -> 52948 bytes
 .../website/content/img/check_flat/default.png  |  Bin 0 -> 25851 bytes
 gemfire-site/website/content/img/egg-logo.png   |  Bin 0 -> 9938 bytes
 gemfire-site/website/content/img/github.png     |  Bin 0 -> 8936 bytes
 gemfire-site/website/content/index.html         |  124 ++
 .../website/content/js/bootstrap.min.js         |    8 +
 gemfire-site/website/content/js/head.js         |  708 ++++++
 gemfire-site/website/content/js/html5shiv.js    |    8 +
 .../website/content/js/jquery-1.10.1.min.js     |    6 +
 .../website/content/js/jquery.icheck.js         |  397 ++++
 gemfire-site/website/content/js/respond.min.js  |    6 +
 .../website/content/js/usergrid-site.js         |   50 +
 .../website/content/releases/index.html         |   65 +
 gemfire-site/website/layouts/community.html     |    1 +
 gemfire-site/website/layouts/default.html       |   44 +
 gemfire-site/website/layouts/docs.html          |    1 +
 gemfire-site/website/layouts/footer.html        |   96 +
 gemfire-site/website/layouts/header.html        |  231 ++
 gemfire-site/website/lib/default.rb             |   43 +
 gemfire-site/website/lib/helpers_.rb            |    0
 gemfire-site/website/lib/pandoc.template        |    4 +
 gemfire-site/website/nanoc.yaml                 |   77 +
 gemfire-site/website/run.sh                     |    1 +
 gemfire-site/website/utilities/map-markers.rb   |   58 +
 gemfire-site/website/utilities/markers.txt      |  440 ++++
 .../website/utilities/snapshot-apigee.rb        |   71 +
 gemfire-site/website/utilities/usergrid.csv     |  290 +++
 182 files changed, 37844 insertions(+), 2569 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/edc29d06/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
----------------------------------------------------------------------


[41/50] [abbrv] incubator-geode git commit: GEODE-637: Additional tests for AsyncEventQueues

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/476c6cd3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerOffHeapDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerOffHeapDUnitTest.java
new file mode 100644
index 0000000..b050ef5
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerOffHeapDUnitTest.java
@@ -0,0 +1,17 @@
+package com.gemstone.gemfire.internal.cache.wan.asyncqueue;
+
+
+@SuppressWarnings("serial")
+public class AsyncEventListenerOffHeapDUnitTest extends
+    AsyncEventListenerDUnitTest {
+
+  public AsyncEventListenerOffHeapDUnitTest(String name) {
+    super(name);
+  }
+
+  @Override
+  public boolean isOffHeap() {
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/476c6cd3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueStatsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueStatsDUnitTest.java
new file mode 100644
index 0000000..cf4a184
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueStatsDUnitTest.java
@@ -0,0 +1,311 @@
+/*=========================================================================
+ * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
+ * This product is protected by U.S. and international copyright
+ * and intellectual property laws. Pivotal products are covered by
+ * one or more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+package com.gemstone.gemfire.internal.cache.wan.asyncqueue;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.gemstone.gemfire.internal.cache.wan.AsyncEventQueueTestBase;
+
+import dunit.AsyncInvocation;
+
+public class AsyncEventQueueStatsDUnitTest extends AsyncEventQueueTestBase {
+
+  private static final long serialVersionUID = 1L;
+  
+  public AsyncEventQueueStatsDUnitTest(String name) {
+    super(name);
+  }
+  
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+  
+  /**
+   * Normal replication scenario
+   */
+  public void testReplicatedSerialPropagation() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, false, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        1000 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });// primary sender
+    pause(2000);//give some time for system to become stable
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueStats", new Object[] {
+        "ln", 0, 1000, 1000, 1000 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueBatchStats",
+        new Object[] { "ln", 10 });
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueStats", new Object[] {
+        "ln", 0, 1000, 0, 0 });
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueBatchStats",
+        new Object[] { "ln", 0 });
+  }
+  
+  /**
+   * Two listeners added to the same RR.
+   */
+  public void testAsyncStatsTwoListeners() throws Exception {
+    Integer lnPort = createFirstLocatorWithDSId(1);
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] {lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] {lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] {lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] {lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln1",
+      false, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln1",
+      false, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln1",
+      false, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln1",
+      false, 100, 100, false, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln2",
+      false, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln2",
+      false, 100, 100, false, false, null, false });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln2",
+      false, 100, 100, false, false, null, false });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln2",
+      false, 100, 100, false, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln1,ln2", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln1,ln2", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln1,ln2", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln1,ln2", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        1000 });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln1", 1000 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln2", 1000 });
+    pause(2000);//give some time for system to become stable
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueStats", new Object[] {
+        "ln1", 0, 1000, 1000, 1000 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueBatchStats",
+        new Object[] { "ln1", 10 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueStats", new Object[] {
+        "ln2", 0, 1000, 1000, 1000 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueBatchStats",
+        new Object[] { "ln2", 10 });
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueStats", new Object[] {
+        "ln1", 0, 1000, 0, 0 });
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueBatchStats",
+        new Object[] { "ln1", 0 });
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueStats", new Object[] {
+        "ln2", 0, 1000, 0, 0 });
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueBatchStats",
+        new Object[] { "ln2", 0 });
+  }
+  
+  /**
+   * HA scenario: kill one vm when puts are in progress on the other vm.
+   */
+  public void testReplicatedSerialPropagationHA() throws Exception {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] {lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] {lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] {lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] {lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+      false, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+      false, 100, 100, false, false, null, false });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR", "ln", isOffHeap() });
+    
+    AsyncInvocation inv1 = vm5.invokeAsync(AsyncEventQueueTestBase.class, "doPuts",
+        new Object[] { testName + "_RR", 10000 });
+    pause(2000);
+    AsyncInvocation inv2 = vm4.invokeAsync(AsyncEventQueueTestBase.class, "killAsyncEventQueue", new Object[] { "ln" });
+    Boolean isKilled = Boolean.FALSE;
+    try {
+      isKilled = (Boolean)inv2.getResult();
+    }
+    catch (Throwable e) {
+      fail("Unexpected exception while killing a AsyncEventQueue");
+    }
+    AsyncInvocation inv3 = null; 
+    if(!isKilled){
+      inv3 = vm5.invokeAsync(AsyncEventQueueTestBase.class, "killSender", new Object[] { "ln" });
+      inv3.join();
+    }
+    inv1.join();
+    inv2.join();
+    pause(2000);//give some time for system to become stable
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueStats_Failover", new Object[] {"ln", 10000});
+  }
+
+  /**
+   * Two regions attached to same AsyncEventQueue
+   */
+  public void testReplicatedSerialPropagationUNPorcessedEvents() throws Exception {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+      false, 100, 100, false, false, null, false });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+      false, 100, 100, false, false, null, false });
+
+    //create one RR (RR_1) on local site
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR_1", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR_1", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR_1", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR_1", "ln", isOffHeap() });
+
+    //create another RR (RR_2) on local site
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR_2", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR_2", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR_2", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue", new Object[] {
+        testName + "_RR_2", "ln", isOffHeap() });
+    
+    //start puts in RR_1 in another thread
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR_1", 1000 });
+    //do puts in RR_2 in main thread
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPutsFrom", new Object[] { testName + "_RR_2", 1000, 1500 });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1500 });
+        
+    pause(2000);//give some time for system to become stable
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueStats", new Object[] {"ln",
+      0, 1500, 1500, 1500});
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueUnprocessedStats", new Object[] {"ln", 0});
+    
+    
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueStats", new Object[] {"ln",
+      0, 1500, 0, 0});
+    vm5.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueUnprocessedStats", new Object[] {"ln", 1500});
+  }
+  
+  /**
+   * Test with conflation enabled
+   */
+  public void testSerialPropagationConflation() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+        false, 100, 100, true, false, null, false });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    
+    vm4
+        .invoke(AsyncEventQueueTestBase.class, "pauseAsyncEventQueue",
+            new Object[] { "ln" });
+    //pause at least for the batchTimeInterval to make sure that the AsyncEventQueue is actually paused
+    pause(2000);
+
+    final Map keyValues = new HashMap();
+    final Map updateKeyValues = new HashMap();
+    for(int i=0; i< 1000; i++) {
+      keyValues.put(i, i);
+    }
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] { testName + "_RR", keyValues });
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] { "ln", keyValues.size() });
+    
+    for(int i=0;i<500;i++) {
+      updateKeyValues.put(i, i+"_updated");
+    }
+    
+    // Put the update events and check the queue size.
+    // There should be no conflation with the previous create events.
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] { testName + "_RR", updateKeyValues });    
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] { "ln", keyValues.size() + updateKeyValues.size() });
+    
+    // Put the update events again and check the queue size.
+    // There should be conflation with the previous update events.
+    vm4.invoke(AsyncEventQueueTestBase.class, "putGivenKeyValue", new Object[] { testName + "_RR", updateKeyValues });    
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueSize", new Object[] { "ln", keyValues.size() + updateKeyValues.size() });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });
+  
+    vm4.invoke(AsyncEventQueueTestBase.class, "resumeAsyncEventQueue", new Object[] { "ln" });
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 1000 });
+    
+    pause(2000);// give some time for system to become stable
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueStats", new Object[] {
+        "ln", 0, 2000, 2000, 1000 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "checkAsyncEventQueueConflatedStats",
+        new Object[] { "ln", 500 });
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/476c6cd3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueDUnitTest.java
new file mode 100644
index 0000000..2fb7496
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueDUnitTest.java
@@ -0,0 +1,330 @@
+/*=========================================================================
+ * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
+ * This product is protected by U.S. and international copyright
+ * and intellectual property laws. Pivotal products are covered by
+ * one or more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+/**
+ * 
+ */
+package com.gemstone.gemfire.internal.cache.wan.concurrent;
+
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
+import com.gemstone.gemfire.cache.wan.GatewaySender.OrderPolicy;
+import com.gemstone.gemfire.internal.cache.wan.AsyncEventQueueTestBase;
+
+import dunit.AsyncInvocation;
+
+/**
+ * @author skumar
+ *
+ */
+public class ConcurrentAsyncEventQueueDUnitTest extends AsyncEventQueueTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public ConcurrentAsyncEventQueueDUnitTest(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+  
+  public void testConcurrentSerialAsyncEventQueueAttributes() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 150, true, true, "testDS", true, 5, OrderPolicy.THREAD });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateConcurrentAsyncEventQueueAttributes",
+        new Object[] { "ln", 100, 150, AsyncEventQueueFactoryImpl.DEFAULT_BATCH_TIME_INTERVAL, true, "testDS", true, true, 5, OrderPolicy.THREAD });
+  }
+  
+ 
+  public void testConcurrentParallelAsyncEventQueueAttributesOrderPolicyKey() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        true, 100, 150, true, true, "testDS", true, 5, OrderPolicy.KEY });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateConcurrentAsyncEventQueueAttributes",
+        new Object[] { "ln", 100, 150, AsyncEventQueueFactoryImpl.DEFAULT_BATCH_TIME_INTERVAL, true, "testDS", true, true, 5, OrderPolicy.KEY });
+  }
+
+  public void testConcurrentParallelAsyncEventQueueAttributesOrderPolicyPartition() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        true, 100, 150, true, true, "testDS", true, 5, OrderPolicy.PARTITION });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateConcurrentAsyncEventQueueAttributes",
+        new Object[] { "ln", 100, 150, AsyncEventQueueFactoryImpl.DEFAULT_BATCH_TIME_INTERVAL, true, "testDS", true, true, 5, OrderPolicy.PARTITION });
+  }
+  
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated 
+   * WAN: Serial 
+   * Dispatcher threads: more than 1
+   * Order policy: key based ordering
+   */
+
+  public void testReplicatedSerialAsyncEventQueueWithMultipleDispatcherThreadsOrderPolicyKey() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 10, true, false, null, false, 3, OrderPolicy.KEY });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 10, true, false, null, false, 3, OrderPolicy.KEY });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 10, true, false, null, false, 3, OrderPolicy.KEY });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 10, true, false, null, false, 3, OrderPolicy.KEY });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        100 });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 100 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+  }
+  
+  /**
+   * Test configuration::
+   * 
+   * Region: Replicated 
+   * WAN: Serial 
+   * Dispatcher threads: more than 1
+   * Order policy: Thread ordering
+   */
+
+  public void testReplicatedSerialAsyncEventQueueWithMultipleDispatcherThreadsOrderPolicyThread() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 10, true, false, null, false, 3, OrderPolicy.THREAD });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 10, true, false, null, false, 3, OrderPolicy.THREAD });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 10, true, false, null, false, 3, OrderPolicy.THREAD });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        false, 100, 10, true, false, null, false, 3, OrderPolicy.THREAD });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createReplicatedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_RR", "ln", isOffHeap() });
+
+    AsyncInvocation inv1 = vm4.invokeAsync(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_RR",
+        50 });
+    AsyncInvocation inv2 = vm4.invokeAsync(AsyncEventQueueTestBase.class, "doNextPuts", new Object[] { testName + "_RR",
+      50, 100 });
+    AsyncInvocation inv3 = vm4.invokeAsync(AsyncEventQueueTestBase.class, "doNextPuts", new Object[] { testName + "_RR",
+      100, 150 });
+    
+    try {
+      inv1.join();
+      inv2.join();
+      inv3.join();
+    } catch (InterruptedException ie) {
+      fail(
+          "Cought interrupted exception while waiting for the task tgo complete.",
+          ie);
+    }
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 150 });// primary sender
+    vm5.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm6.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+    vm7.invoke(AsyncEventQueueTestBase.class, "validateAsyncEventListener",
+        new Object[] { "ln", 0 });// secondary
+  }
+  
+  /**
+   * Test configuration::
+   * 
+   * Region: PartitionedRegion 
+   * WAN: Parallel
+   * Dispatcher threads: more than 1
+   * Order policy: key based ordering
+   */
+  // Disabling test for bug #48323
+  public void testPartitionedParallelAsyncEventQueueWithMultipleDispatcherThreadsOrderPolicyKey() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        true, 100, 10, true, false, null, false, 3, OrderPolicy.KEY });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        true, 100, 10, true, false, null, false, 3, OrderPolicy.KEY });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        true, 100, 10, true, false, null, false, 3, OrderPolicy.KEY });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue", new Object[] { "ln",
+        true, 100, 10, true, false, null, false, 3, OrderPolicy.KEY });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_PR",
+        100 });
+    
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+      new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+      new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+      new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+      new Object[] { "ln" });
+  
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+      new Object[] { "ln"});
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+      new Object[] { "ln"});
+    int vm6size = (Integer)vm6.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+      new Object[] { "ln"});
+    int vm7size = (Integer)vm7.invoke(AsyncEventQueueTestBase.class, "getAsyncEventListenerMapSize",
+      new Object[] { "ln"});
+  
+    assertEquals(vm4size + vm5size + vm6size + vm7size, 100);
+  
+  }
+  
+  
+  /**
+   * Test configuration::
+   * 
+   * Region: PartitionedRegion 
+   * WAN: Parallel
+   * Dispatcher threads: more than 1
+   * Order policy: PARTITION based ordering
+   */
+  // Disabled test for bug #48323
+  public void testPartitionedParallelAsyncEventQueueWithMultipleDispatcherThreadsOrderPolicyPartition() {
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue",
+        new Object[] { "ln", true, 100, 10, true, false, null, false, 3,
+            OrderPolicy.PARTITION });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue",
+        new Object[] { "ln", true, 100, 10, true, false, null, false, 3,
+            OrderPolicy.PARTITION });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue",
+        new Object[] { "ln", true, 100, 10, true, false, null, false, 3,
+            OrderPolicy.PARTITION });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createConcurrentAsyncEventQueue",
+        new Object[] { "ln", true, 100, 10, true, false, null, false, 3,
+            OrderPolicy.PARTITION });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm5.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm6.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+    vm7.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR", "ln", isOffHeap() });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "doPuts", new Object[] { testName + "_PR",
+        100 });
+
+    vm4.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm5.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm6.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+    vm7.invoke(AsyncEventQueueTestBase.class, "waitForAsyncQueueToGetEmpty",
+        new Object[] { "ln" });
+
+    int vm4size = (Integer)vm4.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventListenerMapSize", new Object[] { "ln" });
+    int vm5size = (Integer)vm5.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventListenerMapSize", new Object[] { "ln" });
+    int vm6size = (Integer)vm6.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventListenerMapSize", new Object[] { "ln" });
+    int vm7size = (Integer)vm7.invoke(AsyncEventQueueTestBase.class,
+        "getAsyncEventListenerMapSize", new Object[] { "ln" });
+
+    assertEquals(100, vm4size + vm5size + vm6size + vm7size);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/476c6cd3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueOffHeapDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueOffHeapDUnitTest.java
new file mode 100644
index 0000000..41eb22d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueOffHeapDUnitTest.java
@@ -0,0 +1,16 @@
+package com.gemstone.gemfire.internal.cache.wan.concurrent;
+
+@SuppressWarnings("serial")
+public class ConcurrentAsyncEventQueueOffHeapDUnitTest extends
+    ConcurrentAsyncEventQueueDUnitTest {
+
+  public ConcurrentAsyncEventQueueOffHeapDUnitTest(String name) {
+    super(name);
+  }
+
+  @Override
+  public boolean isOffHeap() {
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/476c6cd3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueDUnitTest.java
new file mode 100644
index 0000000..425d1a6
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueDUnitTest.java
@@ -0,0 +1,53 @@
+/*=========================================================================
+ * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
+ * This product is protected by U.S. and international copyright
+ * and intellectual property laws. Pivotal products are covered by
+ * one or more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+/**
+ * 
+ */
+package com.gemstone.gemfire.internal.cache.wan.misc;
+
+import com.gemstone.gemfire.internal.cache.wan.AsyncEventQueueTestBase;
+
+/**
+ * @author skumar
+ *
+ */
+public class CommonParallelAsyncEventQueueDUnitTest extends AsyncEventQueueTestBase {
+  
+  private static final long serialVersionUID = 1L;
+
+  public CommonParallelAsyncEventQueueDUnitTest(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception  {
+    super.setUp();
+  }
+    
+  public void testSameSenderWithNonColocatedRegions() throws Exception {
+    addExpectedException("cannot have the same parallel async");
+    Integer lnPort = (Integer)vm0.invoke(AsyncEventQueueTestBase.class,
+        "createFirstLocatorWithDSId", new Object[] { 1 });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createCache", new Object[] { lnPort });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createAsyncEventQueue", new Object[] { "ln",
+      true, 100, 100, false, false, null, false });
+    vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+        new Object[] { testName + "_PR1", "ln", isOffHeap()  });
+    try {
+      vm4.invoke(AsyncEventQueueTestBase.class, "createPartitionedRegionWithAsyncEventQueue",
+          new Object[] { testName + "_PR2", "ln", isOffHeap()  });
+      fail("Expected IllegateStateException : cannot have the same parallel gateway sender");
+    }
+    catch (Exception e) {
+      if (!(e.getCause() instanceof IllegalStateException)
+          || !(e.getCause().getMessage()
+              .contains("cannot have the same parallel async event queue id"))) {
+        fail("Expected IllegalStateException", e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/476c6cd3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueOffHeapDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueOffHeapDUnitTest.java
new file mode 100644
index 0000000..8ab77b9
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueOffHeapDUnitTest.java
@@ -0,0 +1,16 @@
+package com.gemstone.gemfire.internal.cache.wan.misc;
+
+@SuppressWarnings("serial")
+public class CommonParallelAsyncEventQueueOffHeapDUnitTest extends
+    CommonParallelAsyncEventQueueDUnitTest {
+
+  public CommonParallelAsyncEventQueueOffHeapDUnitTest(String name) {
+    super(name);
+  }
+
+  @Override
+  public boolean isOffHeap() {
+    return true;
+  }
+
+}



[31/50] [abbrv] incubator-geode git commit: GEODE-563: Moving gfsh tests from closed

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.java
new file mode 100644
index 0000000..1e234c5
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.java
@@ -0,0 +1,371 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.CacheLoader;
+import com.gemstone.gemfire.cache.CacheLoaderException;
+import com.gemstone.gemfire.cache.LoaderHelper;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.management.DistributedRegionMXBean;
+import com.gemstone.gemfire.management.ManagementService;
+import com.gemstone.gemfire.management.ManagerMXBean;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData;
+import com.gemstone.gemfire.management.internal.cli.result.ResultData;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * The GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest class is test suite of test cases testing the Gfsh
+ * 'get' data command when a cache miss occurs on data in a Region with a CacheLoader defined.
+ * <p>
+ *
+ * @author John Blum
+ * @see com.gemstone.gemfire.management.internal.cli.commands.CliCommandTestBase
+ * @see com.gemstone.gemfire.management.internal.cli.commands.DataCommands
+ * @since 8.0
+ */
+@SuppressWarnings("unused")
+public class GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest extends CliCommandTestBase {
+
+  private static final String GEMFIRE_MANAGER_NAME = "GemManagerNode";
+  private static final String GEMFIRE_SERVER_NAME = "GemServerDataNode";
+  private static final String GEMFIRE_LOG_LEVEL = System.getProperty("logLevel", "config");
+  private static final String USERS_REGION_NAME = "Users";
+
+  protected static String getRegionPath(final String regionName) {
+    return (regionName.startsWith(Region.SEPARATOR) ? regionName : String.format("%1$s%2$s", Region.SEPARATOR,
+        regionName));
+  }
+
+  protected static String toString(final Result result) {
+    assert result != null : "The Result object from the command execution was null!";
+
+    StringBuilder buffer = new StringBuilder(System.getProperty("line.separator"));
+
+    while (result.hasNextLine()) {
+      buffer.append(result.nextLine());
+      buffer.append(System.getProperty("line.separator"));
+    }
+
+    return buffer.toString();
+  }
+
+  public GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest(final String testName) {
+    super(testName);
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+
+    Properties managerDistributedSystemProperties = createDistributedSystemProperties(GEMFIRE_MANAGER_NAME);
+    HeadlessGfsh gfsh = createDefaultSetup(managerDistributedSystemProperties);
+
+    assertNotNull(gfsh);
+    assertTrue(gfsh.isConnectedAndReady());
+
+    setupGemFire();
+    verifyGemFireSetup(createPeer(Host.getHost(0).getVM(0), managerDistributedSystemProperties));
+  }
+
+  private void setupGemFire() {
+    initializePeer(createPeer(Host.getHost(0).getVM(1), createDistributedSystemProperties(GEMFIRE_SERVER_NAME)));
+  }
+
+  protected Properties createDistributedSystemProperties(final String gemfireName) {
+    Properties distributedSystemProperties = new Properties();
+
+    distributedSystemProperties.setProperty(DistributionConfig.LOG_LEVEL_NAME, GEMFIRE_LOG_LEVEL);
+    distributedSystemProperties.setProperty(DistributionConfig.NAME_NAME, gemfireName);
+
+    return distributedSystemProperties;
+  }
+
+  protected Peer createPeer(final VM vm, final Properties distributedSystemProperties) {
+    return new Peer(vm, distributedSystemProperties);
+  }
+
+  protected void initializePeer(final Peer peer) {
+    peer.run(new SerializableRunnable(
+        String.format("Initializes the '%1$s' with the '%2$s' Region having a CacheLoader.", GEMFIRE_SERVER_NAME,
+            USERS_REGION_NAME)) {
+      @Override
+      public void run() {
+        // create the GemFire Distributed System with custom distribution configuration properties and settings
+        getSystem(peer.getConfiguration());
+
+        Cache cache = getCache();
+        RegionFactory<String, User> regionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+
+        regionFactory.setCacheLoader(new UserDataStoreCacheLoader());
+        regionFactory.setInitialCapacity(51);
+        regionFactory.setKeyConstraint(String.class);
+        regionFactory.setLoadFactor(0.75f);
+        regionFactory.setStatisticsEnabled(false);
+        regionFactory.setValueConstraint(User.class);
+
+        Region<String, User> users = regionFactory.create(USERS_REGION_NAME);
+
+        assertNotNull(users);
+        assertEquals("Users", users.getName());
+        assertEquals("/Users", users.getFullPath());
+        assertTrue(users.isEmpty());
+        assertNull(users.put("jonbloom", new User("jonbloom")));
+        assertFalse(users.isEmpty());
+        assertEquals(1, users.size());
+        assertEquals(new User("jonbloom"), users.get("jonbloom"));
+      }
+    });
+  }
+
+  private void verifyGemFireSetup(final Peer manager) {
+    manager.run(new SerializableRunnable("Verifies the GemFire Cluster was properly configured and initialized!") {
+      @Override
+      public void run() {
+        final ManagementService managementService = ManagementService.getExistingManagementService(getCache());
+
+        WaitCriterion waitOnManagerCriterion = new WaitCriterion() {
+          @Override
+          public boolean done() {
+            ManagerMXBean managerBean = managementService.getManagerMXBean();
+            DistributedRegionMXBean usersRegionBean = managementService.getDistributedRegionMXBean(
+                getRegionPath(USERS_REGION_NAME));
+
+            return !(managerBean == null || usersRegionBean == null);
+          }
+
+          @Override
+          public String description() {
+            return String.format("Probing for the GemFire Manager '%1$s' and '%2$s' Region MXBeans...",
+                manager.getName(), USERS_REGION_NAME);
+          }
+        };
+
+        DistributedTestCase.waitForCriterion(waitOnManagerCriterion, 30000, 2000, true);
+      }
+    });
+  }
+
+  protected void doHousekeeping() {
+    runCommand(CliStrings.LIST_MEMBER);
+
+    runCommand(new CommandStringBuilder(CliStrings.DESCRIBE_MEMBER).addOption(CliStrings.DESCRIBE_MEMBER__IDENTIFIER,
+        GEMFIRE_SERVER_NAME).toString());
+
+    runCommand(CliStrings.LIST_REGION);
+
+    runCommand(new CommandStringBuilder(CliStrings.DESCRIBE_REGION).addOption(CliStrings.DESCRIBE_REGION__NAME,
+        USERS_REGION_NAME).toString());
+  }
+
+  protected void log(final Result result) {
+    log("Result", toString(result));
+  }
+
+  protected void log(final String tag, final String message) {
+    //System.out.printf("%1$s (%2$s)%n", tag, message);
+    getLogWriter().info(String.format("%1$s (%2$s)%n", tag, message));
+  }
+
+  protected CommandResult runCommand(final String command) {
+    CommandResult result = executeCommand(command);
+
+    assertNotNull(result);
+    assertEquals(Result.Status.OK, result.getStatus());
+
+    log(result);
+
+    return result;
+  }
+
+  protected void assertResult(final boolean expectedResult, final CommandResult commandResult) {
+    if (ResultData.TYPE_COMPOSITE.equals(commandResult.getType())) {
+      boolean actualResult = (Boolean) ((CompositeResultData) commandResult.getResultData()).retrieveSectionByIndex(
+          0).retrieveObject("Result");
+      assertEquals(expectedResult, actualResult);
+    } else {
+      fail(String.format("Expected composite result data; but was '%1$s'!%n", commandResult.getType()));
+    }
+  }
+
+  public void testGetOnCacheMiss() {
+    doHousekeeping();
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.GET);
+    command.addOption(CliStrings.GET__REGIONNAME, USERS_REGION_NAME);
+    command.addOption(CliStrings.GET__KEY, "jonbloom");
+
+    assertResult(true, runCommand(command.toString()));
+
+    command = new CommandStringBuilder(CliStrings.GET);
+    command.addOption(CliStrings.GET__REGIONNAME, USERS_REGION_NAME);
+    command.addOption(CliStrings.GET__KEY, "jondoe");
+    command.addOption(CliStrings.GET__LOAD, "false");
+
+    assertResult(false, runCommand(command.toString()));
+
+    command = new CommandStringBuilder(CliStrings.GET);
+    command.addOption(CliStrings.GET__REGIONNAME, USERS_REGION_NAME);
+    command.addOption(CliStrings.GET__KEY, "jondoe");
+    command.addOption(CliStrings.GET__LOAD, "true");
+
+    assertResult(true, runCommand(command.toString()));
+
+    // NOTE test the unspecified default value for the --load-on-cache-miss
+    command = new CommandStringBuilder(CliStrings.GET);
+    command.addOption(CliStrings.GET__REGIONNAME, USERS_REGION_NAME);
+    command.addOption(CliStrings.GET__KEY, "janedoe");
+
+    assertResult(true, runCommand(command.toString()));
+
+    // NOTE now test an absolute cache miss both for in the Region as well as the CacheLoader
+    command = new CommandStringBuilder(CliStrings.GET);
+    command.addOption(CliStrings.GET__REGIONNAME, USERS_REGION_NAME);
+    command.addOption(CliStrings.GET__KEY, "nonexistinguser");
+    command.addOption(CliStrings.GET__LOAD, "true");
+
+    assertResult(false, runCommand(command.toString()));
+  }
+
+  protected static final class Peer implements Serializable {
+
+    private final Properties distributedSystemProperties;
+    private final VM vm;
+
+    public Peer(final VM vm, final Properties distributedSystemProperties) {
+      assert distributedSystemProperties != null : "The GemFire Distributed System configuration properties and settings cannot be null!";
+      this.vm = vm;
+      this.distributedSystemProperties = distributedSystemProperties;
+    }
+
+    public Properties getConfiguration() {
+      return this.distributedSystemProperties;
+    }
+
+    public String getName() {
+      return getConfiguration().getProperty(DistributionConfig.NAME_NAME);
+    }
+
+    public VM getVm() {
+      return vm;
+    }
+
+    public void run(final Runnable runnable) {
+      if (getVm() == null) {
+        runnable.run();
+      } else {
+        getVm().invoke(runnable);
+      }
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buffer = new StringBuilder(getClass().getSimpleName());
+
+      buffer.append(" {configuration = ").append(getConfiguration());
+      buffer.append(", name = ").append(getName());
+      buffer.append(", pid = ").append(getVm().getPid());
+      buffer.append("}");
+
+      return buffer.toString();
+    }
+  }
+
+  protected static class User implements Serializable {
+
+    private final String username;
+
+    public User(final String username) {
+      assert username != null : "The username cannot be null!";
+      this.username = username;
+    }
+
+    public String getUsername() {
+      return username;
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+      if (obj == this) {
+        return true;
+      }
+
+      if (!(obj instanceof User)) {
+        return false;
+      }
+
+      User that = (User) obj;
+
+      return this.getUsername().equals(that.getUsername());
+    }
+
+    @Override
+    public int hashCode() {
+      int hashValue = 17;
+      hashValue = 37 * hashValue + getUsername().hashCode();
+      return hashValue;
+    }
+
+    @Override
+    public String toString() {
+      return getUsername();
+    }
+  }
+
+  protected static class UserDataStoreCacheLoader implements CacheLoader<String, User>, Serializable {
+
+    private static final Map<String, User> userDataStore = new HashMap<String, User>(5);
+
+    static {
+      userDataStore.put("jackhandy", createUser("jackhandy"));
+      userDataStore.put("janedoe", createUser("janedoe"));
+      userDataStore.put("jondoe", createUser("jondoe"));
+      userDataStore.put("piedoe", createUser("piedoe"));
+      userDataStore.put("supertool", createUser("supertool"));
+    }
+
+    protected static User createUser(final String username) {
+      return new User(username);
+    }
+
+    @Override
+    public User load(final LoaderHelper<String, User> helper) throws CacheLoaderException {
+      return userDataStore.get(helper.getKey());
+    }
+
+    @Override
+    public void close() {
+      userDataStore.clear();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
new file mode 100644
index 0000000..bb99dc2
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
@@ -0,0 +1,817 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.DiskStoreFactory;
+import com.gemstone.gemfire.cache.EvictionAction;
+import com.gemstone.gemfire.cache.EvictionAttributes;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.query.Index;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.SharedConfiguration;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.cli.Result.Status;
+import com.gemstone.gemfire.management.internal.cli.domain.Stock;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+
+public class IndexCommandsDUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+  private static final String VM1Name = "VM1";
+  private static final String group1 = "G1";
+  private static final String indexName = "Id1";
+  private static final String parRegPersName = "ParRegPers";
+  private static final String repRegPersName = "RepRegPer";
+
+  public IndexCommandsDUnitTest(String name) {
+    super(name);
+    // TODO Auto-generated constructor stub
+  }
+
+  Region<?, ?> createParReg(String regionName, Cache cache, Class keyConstraint, Class valueConstraint) {
+    RegionFactory regionFactory = cache.createRegionFactory();
+    regionFactory.setDataPolicy(DataPolicy.PARTITION);
+    regionFactory.setKeyConstraint(keyConstraint);
+    regionFactory.setValueConstraint(valueConstraint);
+    return regionFactory.create(regionName);
+  }
+
+  private Region<?, ?> createParRegWithPersistence(String regionName, String diskStoreName, String diskDirName) {
+    Cache cache = getCache();
+    File diskStoreDirFile = new File(diskDirName);
+    diskStoreDirFile.deleteOnExit();
+
+    if (!diskStoreDirFile.exists()) {
+      diskStoreDirFile.mkdirs();
+    }
+
+    DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+    diskStoreFactory.setDiskDirs(new File[]{diskStoreDirFile});
+    diskStoreFactory.setMaxOplogSize(1);
+    diskStoreFactory.setAllowForceCompaction(true);
+    diskStoreFactory.setAutoCompact(false);
+    diskStoreFactory.create(diskStoreName);
+
+    /****
+     * Eviction Attributes
+     */
+    EvictionAttributes ea = EvictionAttributes.createLRUEntryAttributes(1, EvictionAction.OVERFLOW_TO_DISK);
+
+    RegionFactory regionFactory = cache.createRegionFactory();
+    regionFactory.setDiskStoreName(diskStoreName);
+    regionFactory.setDiskSynchronous(true);
+    regionFactory.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
+    regionFactory.setEvictionAttributes(ea);
+
+    return regionFactory.create(regionName);
+  }
+
+  private Region<?, ?> createRepRegWithPersistence(String regionName, String diskStoreName, String diskDirName) {
+    Cache cache = getCache();
+    File diskStoreDirFile = new File(diskDirName);
+    diskStoreDirFile.deleteOnExit();
+
+    if (!diskStoreDirFile.exists()) {
+      diskStoreDirFile.mkdirs();
+    }
+
+    DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+    diskStoreFactory.setDiskDirs(new File[]{diskStoreDirFile});
+    diskStoreFactory.setMaxOplogSize(1);
+    diskStoreFactory.setAllowForceCompaction(true);
+    diskStoreFactory.setAutoCompact(false);
+    diskStoreFactory.create(diskStoreName);
+
+    /****
+     * Eviction Attributes
+     */
+    EvictionAttributes ea = EvictionAttributes.createLRUEntryAttributes(1, EvictionAction.OVERFLOW_TO_DISK);
+
+    RegionFactory regionFactory = cache.createRegionFactory();
+    regionFactory.setDiskStoreName(diskStoreName);
+    regionFactory.setDiskSynchronous(true);
+    regionFactory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+    regionFactory.setEvictionAttributes(ea);
+
+    return regionFactory.create(regionName);
+  }
+
+  public void testCreateKeyIndexOnRegionWithPersistence() {
+    setupSystemPersist();
+
+    //Creating key indexes on Persistent Partitioned Region
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, "id1");
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "ty");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/" + parRegPersName);
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "key");
+    String commandString = csb.toString();
+    writeToLog("Command String :\n ", commandString);
+    CommandResult commandResult = executeCommand(commandString);
+    String resultAsString = commandResultToString(commandResult);
+    writeToLog("Command Result :\n", resultAsString);
+    assertTrue(Status.OK.equals(commandResult.getStatus()));
+    //Creating key indexes on Persistent Replicated Regions
+    csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, "id2");
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "ee");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/" + repRegPersName);
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "key");
+    commandString = csb.toString();
+    writeToLog("Command String :\n ", commandString);
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command Result :\n", resultAsString);
+    assertTrue(Status.OK.equals(commandResult.getStatus()));
+  }
+
+  public void testCreateAndDestroyIndex() {
+    setupSystem();
+    /***
+     * Basic Create and Destroy 
+     */
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+
+    String commandString = csb.toString();
+    writeToLog("Command String :\n ", commandString);
+    CommandResult commandResult = executeCommand(commandString);
+    String resultAsString = commandResultToString(commandResult);
+    writeToLog("testCreateAndDestroyIndex", resultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(resultAsString.contains(indexName));
+
+    csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    csb.addOption(CliStrings.DESTROY_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.DESTROY_INDEX__REGION, "/StocksParReg");
+    commandString = csb.toString();
+    writeToLog("Command String :\n ", commandString);
+
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("testCreateAndDestroyIndex", resultAsString);
+    assertEquals(commandResult.getStatus(), Status.OK);
+
+    commandResult = executeCommand(CliStrings.LIST_INDEX);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(commandResult.getStatus(), Status.OK);
+    assertFalse(resultAsString.contains(indexName));
+  }
+
+  public void testCreateIndexMultipleIterators() {
+    setupSystem();
+
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "\"h.low\"");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "\"/StocksParReg s, s.history h\"");
+
+    String commandString = csb.toString();
+    writeToLog("Command String :\n ", commandString);
+    CommandResult commandResult = executeCommand(commandString);
+    String resultAsString = commandResultToString(commandResult);
+    writeToLog("testCreateIndexMultipleIterators", resultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("testCreateIndexMultipleIterators", resultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(resultAsString.contains(indexName));
+  }
+
+  public void testCreateMultipleIndexes() {
+    setupSystem();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DEFINE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+
+    String commandString = csb.toString();
+    writeToLog("Command String :\n ", commandString);
+    CommandResult commandResult = executeCommand(commandString);
+    String resultAsString = commandResultToString(commandResult);
+    writeToLog("testCreateMultipleIndexes", resultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.DEFINE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName + "2");
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+
+    csb = new CommandStringBuilder(CliStrings.CREATE_DEFINED_INDEXES);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(resultAsString.contains(indexName));
+  }
+
+  public void testClearMultipleIndexes() {
+    setupSystem();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DEFINE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+
+    String commandString = csb.toString();
+    writeToLog("Command String :\n ", commandString);
+    CommandResult commandResult = executeCommand(commandString);
+    String resultAsString = commandResultToString(commandResult);
+    writeToLog("testClearMultipleIndexes", resultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.DEFINE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName + "2");
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+
+    csb = new CommandStringBuilder(CliStrings.CLEAR_DEFINED_INDEXES);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(!resultAsString.contains(indexName));
+  }
+
+  public void testCreateAndDestroyIndexOnMember() {
+    setupSystem();
+    /***
+     * Basic Create and Destroy 
+     */
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+    csb.addOption(CliStrings.CREATE_INDEX__MEMBER, VM1Name);
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "key");
+
+    String commandString = csb.toString();
+    CommandResult commandResult = executeCommand(commandString);
+    String resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexOnMember", resultAsString);
+
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexOnMember", resultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(resultAsString.contains(indexName));
+    assertTrue(resultAsString.contains(VM1Name));
+
+    csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    csb.addOption(CliStrings.DESTROY_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.DESTROY_INDEX__REGION, "/StocksParReg");
+    csb.addOption(CliStrings.DESTROY_INDEX__MEMBER, VM1Name);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexOnMember", resultAsString);
+    assertEquals(commandResult.getStatus(), Status.OK);
+
+    commandResult = executeCommand(CliStrings.LIST_INDEX);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexOnMember", resultAsString);
+    assertEquals(commandResult.getStatus(), Status.OK);
+    assertFalse(resultAsString.contains(VM1Name));
+  }
+
+  public void testCreateAndDestroyIndexOnGroup() {
+    setupSystem();
+    /***
+     * Basic Create and Destroy 
+     */
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "hash");
+    csb.addOption(CliStrings.CREATE_INDEX__GROUP, group1);
+
+    String commandString = csb.toString();
+    CommandResult commandResult = executeCommand(commandString);
+    String resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexOnGroup", resultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertEquals(true, resultAsString.contains(indexName));
+    assertEquals(true, resultAsString.contains(VM1Name));
+
+    csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    csb.addOption(CliStrings.DESTROY_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.DESTROY_INDEX__REGION, "/StocksParReg");
+    csb.addOption(CliStrings.DESTROY_INDEX__GROUP, group1);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexOnGroup", resultAsString);
+    assertEquals(commandResult.getStatus(), Status.OK);
+
+    commandResult = executeCommand(CliStrings.LIST_INDEX);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(commandResult.getStatus(), Status.OK);
+    assertFalse(resultAsString.contains(VM1Name));
+
+    /***
+     * In case of a partitioned region , the index might get created on a 
+     * member which hosts the region and is not the member of the group1
+     */
+    if (resultAsString.contains(indexName)) {
+      csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+      csb.addOption(CliStrings.DESTROY_INDEX__NAME, indexName);
+      csb.addOption(CliStrings.DESTROY_INDEX__REGION, "/StocksParReg");
+      commandString = csb.toString();
+      commandResult = executeCommand(commandString);
+      resultAsString = commandResultToString(commandResult);
+      assertEquals(commandResult.getStatus(), Status.OK);
+
+      commandResult = executeCommand(CliStrings.LIST_INDEX);
+      resultAsString = commandResultToString(commandResult);
+      writeToLog("Command String :\n ", commandString);
+      writeToLog("testCreateAndDestroyIndexOnGroup", resultAsString);
+
+      assertEquals(commandResult.getStatus(), Status.OK);
+      assertFalse(resultAsString.contains(indexName));
+      assertTrue(resultAsString.contains(CliStrings.LIST_INDEX__INDEXES_NOT_FOUND_MESSAGE));
+    }
+  }
+
+  public void testCreateAndDestroyIndexWithIncorrectInput() {
+    setupSystem();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "hash");
+    String commandString = csb.toString();
+    CommandResult commandResult = executeCommand(commandString);
+    String resultAsString = commandResultToString(commandResult);
+
+    assertEquals(commandResult.getStatus(), Status.OK);
+
+    //CREATE the same index 
+    csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "hash");
+
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertTrue(commandResult.getStatus().equals(Status.ERROR));
+    //assertTrue(resultAsString.contains(CliStrings.format(CliStrings.CREATE_INDEX__NAME__CONFLICT, indexName)));
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexWithIncorrectInput", resultAsString);
+
+
+    //Create index on a wrong regionPath
+    csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocsParReg");
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "hash");
+
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexWithIncorrectInput", resultAsString);
+    assertTrue(commandResult.getStatus().equals(Status.ERROR));
+    //assertTrue(resultAsString.contains(CliStrings.format(CliStrings.CREATE_INDEX__INVALID__REGIONPATH, "/StocsParReg")));
+
+    //Create index with wrong expression 
+    csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, "Id2");
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "rey");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "hash");
+
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexWithIncorrectInput", resultAsString);
+    assertTrue(commandResult.getStatus().equals(Status.ERROR));
+
+    //Create index with wrong type 
+    csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "bash");
+
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexWithIncorrectInput", resultAsString);
+    assertTrue(resultAsString.contains(CliStrings.CREATE_INDEX__INVALID__INDEX__TYPE__MESSAGE));
+    assertTrue(commandResult.getStatus().equals(Status.ERROR));
+
+    //Destroy index with incorrect indexName 
+    csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    csb.addOption(CliStrings.DESTROY_INDEX__NAME, "Id2");
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexWithIncorrectInput", resultAsString);
+    assertTrue(commandResult.getStatus().equals(Status.ERROR));
+    assertTrue(resultAsString.contains(CliStrings.format(CliStrings.DESTROY_INDEX__INDEX__NOT__FOUND, "Id2")));
+
+    //Destroy index with incorrect region 
+    csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    csb.addOption(CliStrings.DESTROY_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.DESTROY_INDEX__REGION, "Region");
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexWithIncorrectInput", resultAsString);
+    assertTrue(commandResult.getStatus().equals(Status.ERROR));
+    assertTrue(resultAsString.contains(CliStrings.format(CliStrings.DESTROY_INDEX__REGION__NOT__FOUND, "Region")));
+
+    //Destroy index with incorrect memberName
+    csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    csb.addOption(CliStrings.DESTROY_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.DESTROY_INDEX__REGION, "Region");
+    csb.addOption(CliStrings.DESTROY_INDEX__MEMBER, "wrongOne");
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexWithIncorrectInput", resultAsString);
+    assertTrue(commandResult.getStatus().equals(Status.ERROR));
+
+    //Destroy index with no option
+    csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    writeToLog("Command String :\n ", commandString);
+    writeToLog("testCreateAndDestroyIndexWithIncorrectInput", resultAsString);
+    assertTrue(commandResult.getStatus().equals(Status.ERROR));
+  }
+
+  public void testDestroyIndexWithoutIndexName() {
+    setupSystem();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "hash");
+    String commandString = csb.toString();
+    CommandResult commandResult = executeCommand(commandString);
+    String resultAsString = commandResultToString(commandResult);
+    assertEquals(commandResult.getStatus(), Status.OK);
+
+    csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertEquals(true, resultAsString.contains(indexName));
+    assertEquals(true, resultAsString.contains(VM1Name));
+
+    csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    csb.addOption(CliStrings.DESTROY_INDEX__GROUP, group1);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    csb.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    csb.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    csb.addOption(CliStrings.CREATE_INDEX__REGION, "/StocksParReg");
+    csb.addOption(CliStrings.CREATE_INDEX__TYPE, "hash");
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    csb.addOption(CliStrings.DESTROY_INDEX__REGION, "StocksParReg");
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+
+    csb = new CommandStringBuilder(CliStrings.LIST_INDEX);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    resultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(resultAsString.contains(CliStrings.LIST_INDEX__INDEXES_NOT_FOUND_MESSAGE));
+  }
+
+  /**
+   * Asserts that creating and destroying indexes correctly updates the shared configuration.
+   */
+  public void testCreateDestroyUpdatesSharedConfig() {
+    disconnectAllFromDS();
+
+    final String regionName = "testIndexSharedConfigRegion";
+    final String groupName = "testIndexSharedConfigGroup";
+
+    // Start the Locator and wait for shared configuration to be available
+    final int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+
+        final File locatorLogFile = new File("locator-" + locatorPort + ".log");
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, "Locator");
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "fine");
+        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
+              locatorProps);
+
+          DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+            @Override
+            public boolean done() {
+              return locator.isSharedConfigurationRunning();
+            }
+
+            @Override
+            public String description() {
+              return "Waiting for shared configuration to be started";
+            }
+          };
+          DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+        } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        }
+      }
+    });
+
+    // Start the default manager
+    Properties managerProps = new Properties();
+    managerProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    managerProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+    createDefaultSetup(managerProps);
+
+    // Create a cache in VM 1
+    VM vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        getSystem(localProps);
+        assertNotNull(getCache());
+
+        Region parReg = createParReg(regionName, getCache(), String.class, Stock.class);
+        parReg.put("VMW", new Stock("VMW", 98));
+      }
+    });
+
+    // Test creating the index
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_INDEX);
+    commandStringBuilder.addOption(CliStrings.CREATE_INDEX__EXPRESSION, "key");
+    commandStringBuilder.addOption(CliStrings.CREATE_INDEX__NAME, indexName);
+    commandStringBuilder.addOption(CliStrings.CREATE_INDEX__GROUP, groupName);
+    commandStringBuilder.addOption(CliStrings.CREATE_INDEX__REGION, "\"/" + regionName + " p\"");
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure the index exists in the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        String xmlFromConfig;
+        try {
+          xmlFromConfig = sharedConfig.getConfiguration(groupName).getCacheXmlContent();
+          assertTrue(xmlFromConfig.contains(indexName));
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service", e);
+        }
+      }
+    });
+
+    //Restart a member and make sure he gets the shared configuration
+    vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        getCache().close();
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        localProps.setProperty(DistributionConfig.USE_CLUSTER_CONFIGURATION_NAME, "true");
+        getSystem(localProps);
+        Cache cache = getCache();
+        assertNotNull(cache);
+        Region region = cache.getRegion(regionName);
+        assertNotNull(region);
+        Index index = cache.getQueryService().getIndex(region, indexName);
+        assertNotNull(index);
+      }
+    });
+
+    // Test destroying the index
+    commandStringBuilder = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
+    commandStringBuilder.addOption(CliStrings.DESTROY_INDEX__NAME, indexName);
+    commandStringBuilder.addOption(CliStrings.DESTROY_INDEX__GROUP, groupName);
+    commandStringBuilder.addOption(CliStrings.DESTROY_INDEX__REGION, "/" + regionName);
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure the index was removed from the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        String xmlFromConfig;
+        try {
+          xmlFromConfig = sharedConfig.getConfiguration(groupName).getCacheXmlContent();
+          assertFalse(xmlFromConfig.contains(indexName));
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service", e);
+        }
+      }
+    });
+
+    //Restart the data member cache to make sure that the index is destroyed.
+    vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        getCache().close();
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        localProps.setProperty(DistributionConfig.USE_CLUSTER_CONFIGURATION_NAME, "true");
+        getSystem(localProps);
+        Cache cache = getCache();
+        assertNotNull(cache);
+        Region region = cache.getRegion(regionName);
+        assertNotNull(region);
+        Index index = cache.getQueryService().getIndex(region, indexName);
+        assertNull(index);
+      }
+    });
+  }
+
+  private void writeToLog(String text, String resultAsString) {
+    getLogWriter().info(testName + "\n");
+    getLogWriter().info(resultAsString);
+  }
+
+  private void setupSystem() {
+    disconnectAllFromDS();
+    createDefaultSetup(null);
+    final String parRegName = "StocksParReg";
+
+    final VM manager = Host.getHost(0).getVM(0);
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+    manager.invoke(new SerializableCallable() {
+      public Object call() {
+        Region parReg = createParReg(parRegName, getCache(), String.class, Stock.class);
+        parReg.put("VMW", new Stock("VMW", 98));
+        return parReg.put("APPL", new Stock("APPL", 600));
+      }
+    });
+
+    vm1.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        Properties props = new Properties();
+        props.setProperty(DistributionConfig.NAME_NAME, VM1Name);
+        props.setProperty(DistributionConfig.GROUPS_NAME, group1);
+        getSystem(props);
+        Region parReg = createParReg(parRegName, getCache(), String.class, Stock.class);
+        parReg.put("MSFT", new Stock("MSFT", 27));
+        return parReg.put("GOOG", new Stock("GOOG", 540));
+      }
+    });
+  }
+
+  private void setupSystemPersist() {
+    disconnectAllFromDS();
+    createDefaultSetup(null);
+    final String parRegName = "StocksParReg";
+
+    final VM manager = Host.getHost(0).getVM(0);
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+    manager.invoke(new SerializableCallable() {
+      public Object call() {
+        Region parReg = createParReg(parRegName, getCache(), String.class, Stock.class);
+        parReg.put("VMW", new Stock("VMW", 98));
+        Region parRegPers = createParRegWithPersistence(parRegPersName, "testCreateIndexDiskstore1",
+            "testCreateIndexDiskDir1");
+        Region repRegPers = createRepRegWithPersistence(repRegPersName, "testCreateIndexDiskstore1",
+            "testCreateIndexDiskDir1");
+        return parReg.put("APPL", new Stock("APPL", 600));
+      }
+    });
+
+    vm1.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        Properties props = new Properties();
+        props.setProperty(DistributionConfig.NAME_NAME, VM1Name);
+        props.setProperty(DistributionConfig.GROUPS_NAME, group1);
+        getSystem(props);
+        Region parReg = createParReg(parRegName, getCache(), String.class, Stock.class);
+        parReg.put("MSFT", new Stock("MSFT", 27));
+        Region parRegPers = createParRegWithPersistence(parRegPersName, "testCreateIndexDiskstore2",
+            "testCreateIndexDiskDir2");
+        Region repRegPers = createRepRegWithPersistence(repRegPersName, "testCreateIndexDiskstore2",
+            "testCreateIndexDiskDir2");
+        return parReg.put("GOOG", new Stock("GOOG", 540));
+      }
+    });
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeDiskStoreCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeDiskStoreCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeDiskStoreCommandsDUnitTest.java
new file mode 100644
index 0000000..e7bc575
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeDiskStoreCommandsDUnitTest.java
@@ -0,0 +1,197 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.DiskStore;
+import com.gemstone.gemfire.cache.DiskStoreFactory;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.io.Serializable;
+import java.util.Properties;
+
+/**
+ * The ListAndDescribeDiskStoreCommandsDUnitTest class is a test suite of functional tests cases testing the proper
+ * functioning of the 'list disk-store' and 'describe disk-store' commands. </p>
+ *
+ * @author John Blum
+ * @see com.gemstone.gemfire.management.internal.cli.commands.CliCommandTestBase
+ * @see com.gemstone.gemfire.management.internal.cli.commands.DiskStoreCommands
+ * @since 7.0
+ */
+public class ListAndDescribeDiskStoreCommandsDUnitTest extends CliCommandTestBase {
+
+  protected static String toString(final Result result) {
+    assert result != null : "The Result object from the command execution cannot be null!";
+
+    final StringBuilder buffer = new StringBuilder(System.getProperty("line.separator"));
+
+    while (result.hasNextLine()) {
+      buffer.append(result.nextLine());
+      buffer.append(System.getProperty("line.separator"));
+    }
+
+    return buffer.toString().trim();
+  }
+
+  public ListAndDescribeDiskStoreCommandsDUnitTest(final String testName) {
+    super(testName);
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    createDefaultSetup(null);
+    setupGemFire();
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+  }
+
+  protected Peer createPeer(final Properties distributedSystemConfiguration, final VM vm) {
+    return new Peer(distributedSystemConfiguration, vm);
+  }
+
+  protected void setupGemFire() throws Exception {
+    final Host host = Host.getHost(0);
+
+    final VM vm1 = host.getVM(1);
+    final VM vm2 = host.getVM(2);
+
+    final Peer peer1 = createPeer(createDistributedSystemProperties("consumerServer"), vm1);
+    final Peer peer2 = createPeer(createDistributedSystemProperties("producerServer"), vm2);
+
+    createPersistentRegion(peer1, "consumers", "consumerData");
+    createPersistentRegion(peer1, "observers", "observerData");
+    createPersistentRegion(peer2, "producer", "producerData");
+    createPersistentRegion(peer2, "producer-factory", "producerData");
+  }
+
+  protected Properties createDistributedSystemProperties(final String gemfireName) {
+    final Properties distributedSystemProperties = new Properties();
+
+    distributedSystemProperties.setProperty(DistributionConfig.LOG_LEVEL_NAME, getDUnitLogLevel());
+    distributedSystemProperties.setProperty(DistributionConfig.NAME_NAME, gemfireName);
+
+    return distributedSystemProperties;
+  }
+
+  protected void createPersistentRegion(final Peer peer, final String regionName, final String diskStoreName) {
+    peer.run(new SerializableRunnable("Creating Persistent Region for Member " + peer.getName()) {
+      @Override
+      public void run() {
+        getSystem(peer.getDistributedSystemConfiguration());
+
+        final Cache cache = getCache();
+
+        DiskStore diskStore = cache.findDiskStore(diskStoreName);
+
+        if (diskStore == null) {
+          final DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+          diskStoreFactory.setDiskDirs(getDiskDirs());
+          diskStore = diskStoreFactory.create(diskStoreName);
+        }
+
+        final RegionFactory regionFactory = cache.createRegionFactory();
+
+        regionFactory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+        regionFactory.setDiskStoreName(diskStore.getName());
+        regionFactory.setScope(Scope.DISTRIBUTED_NO_ACK);
+        regionFactory.create(regionName);
+      }
+    });
+  }
+
+  public void testListDiskStore() throws Exception {
+    final Result result = executeCommand(CliStrings.LIST_DISK_STORE);
+
+    assertNotNull(result);
+    getLogWriter().info(toString(result));
+    assertEquals(Result.Status.OK, result.getStatus());
+  }
+
+  public void testDescribeDiskStore() throws Exception {
+    final Result result = executeCommand(
+        CliStrings.DESCRIBE_DISK_STORE + " --member=producerServer --name=producerData");
+
+    assertNotNull(result);
+    getLogWriter().info(toString(result));
+    assertEquals(Result.Status.OK, result.getStatus());
+  }
+
+  public void testDescribeDiskStoreWithInvalidMemberName() throws Exception {
+    final Result commandResult = executeCommand(
+        CliStrings.DESCRIBE_DISK_STORE + " --member=badMemberName --name=producerData");
+
+    assertNotNull(commandResult);
+    assertEquals(Result.Status.ERROR, commandResult.getStatus());
+    assertEquals(CliStrings.format(CliStrings.MEMBER_NOT_FOUND_ERROR_MESSAGE, "badMemberName"),
+        toString(commandResult));
+  }
+
+  public void testDescribeDiskStoreWithInvalidDiskStoreName() {
+    final Result commandResult = executeCommand(
+        CliStrings.DESCRIBE_DISK_STORE + " --member=producerServer --name=badDiskStoreName");
+
+    assertNotNull(commandResult);
+    assertEquals(Result.Status.ERROR, commandResult.getStatus());
+    assertEquals("A disk store with name (badDiskStoreName) was not found on member (producerServer).",
+        toString(commandResult));
+  }
+
+  protected static class Peer implements Serializable {
+
+    private final Properties distributedSystemConfiguration;
+    private final VM vm;
+
+    protected Peer(final Properties distributedSystemConfiguration, final VM vm) {
+      assert distributedSystemConfiguration != null : "The GemFire distributed system configuration properties cannot be null!";
+      this.distributedSystemConfiguration = distributedSystemConfiguration;
+      this.vm = vm;
+    }
+
+    public Properties getDistributedSystemConfiguration() {
+      return distributedSystemConfiguration;
+    }
+
+    public String getName() {
+      return getDistributedSystemConfiguration().getProperty(DistributionConfig.NAME_NAME);
+    }
+
+    public VM getVm() {
+      return vm;
+    }
+
+    public void run(final Runnable runnable) {
+      if (getVm() == null) {
+        runnable.run();
+      } else {
+        getVm().invoke(runnable);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java
new file mode 100644
index 0000000..5c172cc
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java
@@ -0,0 +1,320 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.EvictionAction;
+import com.gemstone.gemfire.cache.EvictionAttributes;
+import com.gemstone.gemfire.cache.FixedPartitionAttributes;
+import com.gemstone.gemfire.cache.PartitionAttributes;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
+import com.gemstone.gemfire.compression.SnappyCompressor;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.cache.RegionEntryContext;
+import com.gemstone.gemfire.management.cli.Result.Status;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import com.gemstone.gemfire.management.internal.cli.util.RegionAttributesNames;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.util.Properties;
+
+public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
+
+  public ListAndDescribeRegionDUnitTest(String name) {
+    super(name);
+    // TODO Auto-generated constructor stub
+  }
+
+  private static final String REGION1 = "region1";
+  private static final String REGION2 = "region2";
+  private static final String REGION3 = "region3";
+  private static final String SUBREGION1A = "subregion1A";
+  private static final String SUBREGION1B = "subregion1B";
+  private static final String SUBREGION1C = "subregion1C";
+  private static final String PR1 = "PR1";
+  private static final String LOCALREGIONONMANAGER = "LocalRegionOnManager";
+
+
+  static class CacheListener2 extends CacheListenerAdapter {
+  }
+
+  static class CacheListener1 extends CacheListenerAdapter {
+  }
+
+  private Properties createProperties(String name, String groups) {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOG_LEVEL_NAME, "info");
+    props.setProperty(DistributionConfig.STATISTIC_SAMPLING_ENABLED_NAME, "true");
+    props.setProperty(DistributionConfig.ENABLE_TIME_STATISTICS_NAME, "true");
+    props.setProperty(DistributionConfig.NAME_NAME, name);
+    props.setProperty(DistributionConfig.GROUPS_NAME, groups);
+    return props;
+  }
+
+  private void createPartitionedRegion1() {
+    final Cache cache = getCache();
+    // Create the data region
+    RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+    dataRegionFactory.create(PR1);
+  }
+
+  private void setupSystem() {
+    final Properties managerProps = createProperties("Manager", "G1");
+    createDefaultSetup(managerProps);
+
+    final Properties server1Props = createProperties("Server1", "G2");
+    final Host host = Host.getHost(0);
+    final VM[] servers = {host.getVM(0), host.getVM(1)};
+
+    //The mananger VM
+    servers[0].invoke(new SerializableRunnable() {
+      public void run() {
+        final Cache cache = getCache();
+        RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        dataRegionFactory.setConcurrencyLevel(4);
+        EvictionAttributes ea = EvictionAttributes.createLIFOEntryAttributes(100, EvictionAction.LOCAL_DESTROY);
+        dataRegionFactory.setEvictionAttributes(ea);
+        dataRegionFactory.setEnableAsyncConflation(true);
+
+        FixedPartitionAttributes fpa = FixedPartitionAttributes.createFixedPartition("Par1", true);
+        PartitionAttributes pa = new PartitionAttributesFactory().setLocalMaxMemory(100).setRecoveryDelay(
+            2).setTotalMaxMemory(200).setRedundantCopies(1).addFixedPartitionAttributes(fpa).create();
+        dataRegionFactory.setPartitionAttributes(pa);
+
+        dataRegionFactory.create(PR1);
+        createLocalRegion(LOCALREGIONONMANAGER);
+      }
+    });
+
+    servers[1].invoke(new SerializableRunnable() {
+      public void run() {
+        getSystem(server1Props);
+        final Cache cache = getCache();
+        RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        dataRegionFactory.setConcurrencyLevel(4);
+        EvictionAttributes ea = EvictionAttributes.createLIFOEntryAttributes(100, EvictionAction.LOCAL_DESTROY);
+        dataRegionFactory.setEvictionAttributes(ea);
+        dataRegionFactory.setEnableAsyncConflation(true);
+
+        FixedPartitionAttributes fpa = FixedPartitionAttributes.createFixedPartition("Par2", 4);
+        PartitionAttributes pa = new PartitionAttributesFactory().setLocalMaxMemory(150).setRecoveryDelay(
+            4).setTotalMaxMemory(200).setRedundantCopies(1).addFixedPartitionAttributes(fpa).create();
+        dataRegionFactory.setPartitionAttributes(pa);
+
+        dataRegionFactory.create(PR1);
+        createRegionsWithSubRegions();
+      }
+    });
+  }
+
+  private void createPartitionedRegion(String regionName) {
+
+    final Cache cache = getCache();
+    // Create the data region
+    RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+    dataRegionFactory.setConcurrencyLevel(4);
+    EvictionAttributes ea = EvictionAttributes.createLIFOEntryAttributes(100, EvictionAction.LOCAL_DESTROY);
+    dataRegionFactory.setEvictionAttributes(ea);
+    dataRegionFactory.setEnableAsyncConflation(true);
+
+    FixedPartitionAttributes fpa = FixedPartitionAttributes.createFixedPartition("Par1", true);
+    PartitionAttributes pa = new PartitionAttributesFactory().setLocalMaxMemory(100).setRecoveryDelay(
+        2).setTotalMaxMemory(200).setRedundantCopies(1).addFixedPartitionAttributes(fpa).create();
+    dataRegionFactory.setPartitionAttributes(pa);
+    dataRegionFactory.addCacheListener(new CacheListener1());
+    dataRegionFactory.addCacheListener(new CacheListener2());
+    dataRegionFactory.create(regionName);
+  }
+
+
+  private void createLocalRegion(final String regionName) {
+    final Cache cache = getCache();
+    // Create the data region
+    RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.LOCAL);
+    dataRegionFactory.create(regionName);
+  }
+
+  /**
+   * Creates a region that uses compression on region entry values.
+   *
+   * @param regionName a unique region name.
+   */
+  private void createCompressedRegion(final String regionName) {
+    final Cache cache = getCache();
+
+    RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+    dataRegionFactory.setCompressor(SnappyCompressor.getDefaultInstance());
+    dataRegionFactory.create(regionName);
+  }
+
+  @SuppressWarnings("deprecation")
+  private void createRegionsWithSubRegions() {
+    final Cache cache = getCache();
+
+    RegionFactory<String, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+    dataRegionFactory.setConcurrencyLevel(3);
+    Region<String, Integer> region1 = dataRegionFactory.create(REGION1);
+    region1.createSubregion(SUBREGION1C, region1.getAttributes());
+    Region<String, Integer> subregion2 = region1.createSubregion(SUBREGION1A, region1.getAttributes());
+
+    subregion2.createSubregion(SUBREGION1B, subregion2.getAttributes());
+    dataRegionFactory.create(REGION2);
+    dataRegionFactory.create(REGION3);
+  }
+
+
+  public void testListRegion() {
+    setupSystem();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.LIST_REGION);
+    String commandString = csb.toString();
+    CommandResult commandResult = executeCommand(commandString);
+    String commandResultAsString = commandResultToString(commandResult);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(commandResultAsString.contains(PR1));
+    assertTrue(commandResultAsString.contains(LOCALREGIONONMANAGER));
+    assertTrue(commandResultAsString.contains(REGION1));
+    assertTrue(commandResultAsString.contains(REGION2));
+    assertTrue(commandResultAsString.contains(REGION3));
+
+
+    csb = new CommandStringBuilder(CliStrings.LIST_REGION);
+    csb.addOption(CliStrings.LIST_REGION__MEMBER, "Manager");
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    commandResultAsString = commandResultToString(commandResult);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(commandResultAsString.contains(PR1));
+    assertTrue(commandResultAsString.contains(LOCALREGIONONMANAGER));
+
+    csb = new CommandStringBuilder(CliStrings.LIST_REGION);
+    csb.addOption(CliStrings.LIST_REGION__MEMBER, "Server1");
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    commandResultAsString = commandResultToString(commandResult);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(commandResultAsString.contains(PR1));
+    assertTrue(commandResultAsString.contains(REGION1));
+    assertTrue(commandResultAsString.contains(REGION2));
+    assertTrue(commandResultAsString.contains(REGION3));
+    assertTrue(commandResultAsString.contains(SUBREGION1A));
+
+    csb = new CommandStringBuilder(CliStrings.LIST_REGION);
+    csb.addOption(CliStrings.LIST_REGION__GROUP, "G1");
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    commandResultAsString = commandResultToString(commandResult);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(commandResultAsString.contains(PR1));
+    assertTrue(commandResultAsString.contains(LOCALREGIONONMANAGER));
+
+    csb = new CommandStringBuilder(CliStrings.LIST_REGION);
+    csb.addOption(CliStrings.LIST_REGION__GROUP, "G2");
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    commandResultAsString = commandResultToString(commandResult);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(commandResultAsString.contains(PR1));
+    assertTrue(commandResultAsString.contains(REGION1));
+    assertTrue(commandResultAsString.contains(REGION2));
+    assertTrue(commandResultAsString.contains(REGION3));
+    assertTrue(commandResultAsString.contains(SUBREGION1A));
+  }
+
+  public void testDescribeRegion() {
+    setupSystem();
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESCRIBE_REGION);
+    csb.addOption(CliStrings.DESCRIBE_REGION__NAME, PR1);
+    String commandString = csb.toString();
+    CommandResult commandResult = executeCommand(commandString);
+    String commandResultAsString = commandResultToString(commandResult);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(commandResultAsString.contains(PR1));
+    assertTrue(commandResultAsString.contains("Server1"));
+
+    csb = new CommandStringBuilder(CliStrings.DESCRIBE_REGION);
+    csb.addOption(CliStrings.DESCRIBE_REGION__NAME, LOCALREGIONONMANAGER);
+    commandString = csb.toString();
+    commandResult = executeCommand(commandString);
+    commandResultAsString = commandResultToString(commandResult);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(commandResultAsString.contains(LOCALREGIONONMANAGER));
+    assertTrue(commandResultAsString.contains("Manager"));
+  }
+
+  /**
+   * Asserts that a describe region command issued on a region with compression returns the correct non default region
+   * attribute for compression and the correct codec value.
+   */
+  public void testDescribeRegionWithCompressionCodec() {
+    final String regionName = "compressedRegion";
+    VM vm = Host.getHost(0).getVM(1);
+
+    setupSystem();
+
+    // Create compressed region
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        createCompressedRegion(regionName);
+      }
+    });
+
+    // Test the describe command; look for compression
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESCRIBE_REGION);
+    csb.addOption(CliStrings.DESCRIBE_REGION__NAME, regionName);
+    String commandString = csb.toString();
+    CommandResult commandResult = executeCommand(commandString);
+    String commandResultAsString = commandResultToString(commandResult);
+    assertEquals(Status.OK, commandResult.getStatus());
+    assertTrue(commandResultAsString.contains(regionName));
+    assertTrue(commandResultAsString.contains(RegionAttributesNames.COMPRESSOR));
+    assertTrue(commandResultAsString.contains(RegionEntryContext.DEFAULT_COMPRESSION_PROVIDER));
+
+    // Destroy compressed region
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Region region = getCache().getRegion(regionName);
+        assertNotNull(region);
+        region.destroyRegion();
+      }
+    });
+  }
+}


[13/50] [abbrv] incubator-geode git commit: GEODE-607: improve SimpleMemoryAllocatorImpl unit test coverage

Posted by kl...@apache.org.
GEODE-607: improve SimpleMemoryAllocatorImpl unit test coverage

Also changed ChunkType to an interface and removed dead code.
Refactored duplicate code into AbstractStoredObject.
Removed some more sqlf dead code.


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

Branch: refs/heads/feature/GEODE-217
Commit: f9744623ddb03b2920e85be5a68c4d4c2b4a3243
Parents: e19fa40
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Nov 25 11:43:15 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Dec 2 11:34:34 2015 -0800

----------------------------------------------------------------------
 .../internal/offheap/AbstractStoredObject.java  | 107 ++++++
 .../gemfire/internal/offheap/Chunk.java         |   1 -
 .../gemfire/internal/offheap/ChunkType.java     |  14 +-
 .../gemfire/internal/offheap/DataAsAddress.java |  87 +----
 .../gemfire/internal/offheap/Fragment.java      |  14 +
 .../internal/offheap/FreeListManager.java       |  14 +
 .../gemfire/internal/offheap/GemFireChunk.java  |   8 -
 .../internal/offheap/MemoryAllocator.java       |   2 -
 .../internal/offheap/MemoryBlockNode.java       |  14 +-
 .../internal/offheap/MemoryInspector.java       |   6 -
 .../offheap/OffHeapCachedDeserializable.java    |  85 +----
 .../internal/offheap/OffHeapStorage.java        |   1 -
 .../offheap/SimpleMemoryAllocatorImpl.java      | 284 +++++++-------
 .../internal/offheap/UnsafeMemoryChunk.java     |  10 +
 .../offheap/NullOffHeapMemoryStats.java         |   6 +
 .../offheap/NullOutOfOffHeapMemoryListener.java |   6 +
 .../offheap/SimpleMemoryAllocatorJUnitTest.java | 369 +++++++++++++++++--
 17 files changed, 643 insertions(+), 385 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObject.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObject.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObject.java
new file mode 100644
index 0000000..6dad277
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObject.java
@@ -0,0 +1,107 @@
+/*
+ * 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.offheap;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
+import com.gemstone.gemfire.internal.cache.RegionEntry;
+import com.gemstone.gemfire.internal.lang.StringUtils;
+
+public abstract class AbstractStoredObject implements StoredObject {
+  @Override
+  public Object getValueAsDeserializedHeapObject() {
+    return getDeserializedValue(null,null);
+  }
+  
+  @Override
+  public byte[] getValueAsHeapByteArray() {
+    if (isSerialized()) {
+      return getSerializedValue();
+    } else {
+      return (byte[])getDeserializedForReading();
+    }
+  }
+
+  @Override
+  public String getStringForm() {
+    try {
+      return StringUtils.forceToString(getDeserializedForReading());
+    } catch (RuntimeException ex) {
+      return "Could not convert object to string because " + ex;
+    }
+  }
+
+  @Override
+  public Object getDeserializedForReading() {
+    return getDeserializedValue(null,null);
+  }
+
+  @Override
+  public Object getDeserializedWritableCopy(Region r, RegionEntry re) {
+    return getDeserializedValue(null,null);
+  }
+
+  @Override
+  public Object getValue() {
+    if (isSerialized()) {
+      return getSerializedValue();
+    } else {
+      throw new IllegalStateException("Can not call getValue on StoredObject that is not serialized");
+    }
+  }
+
+  @Override
+  public void writeValueAsByteArray(DataOutput out) throws IOException {
+    DataSerializer.writeByteArray(getSerializedValue(), out);
+  }
+
+  @Override
+  public void sendTo(DataOutput out) throws IOException {
+    if (isSerialized()) {
+      out.write(getSerializedValue());
+    } else {
+      Object objToSend = (byte[]) getDeserializedForReading(); // deserialized as a byte[]
+      DataSerializer.writeObject(objToSend, out);
+    }
+  }
+
+  @Override
+  public void sendAsByteArray(DataOutput out) throws IOException {
+    byte[] bytes;
+    if (isSerialized()) {
+      bytes = getSerializedValue();
+    } else {
+      bytes = (byte[]) getDeserializedForReading();
+    }
+    DataSerializer.writeByteArray(bytes, out);
+    
+  }
+
+  @Override
+  public void sendAsCachedDeserializable(DataOutput out) throws IOException {
+    if (!isSerialized()) {
+      throw new IllegalStateException("sendAsCachedDeserializable can only be called on serialized StoredObjects");
+    }
+    InternalDataSerializer.writeDSFIDHeader(DataSerializableFixedID.VM_CACHED_DESERIALIZABLE, out);
+    sendAsByteArray(out);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
index e32a1c6..ed4bc43 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
@@ -389,7 +389,6 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
     @Override
     public void writeBytes(int offset, byte[] bytes, int bytesOffset, int size) {
       assert offset+size <= getDataSize();
-      SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(getBaseDataAddress() + offset, size);
       UnsafeMemoryChunk.writeAbsoluteBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkType.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkType.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkType.java
index 9841368..e48bb62 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkType.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkType.java
@@ -17,10 +17,14 @@
 package com.gemstone.gemfire.internal.offheap;
 
 /**
- * Used to create new chunks of a certain type.
+ * Describes the type of data stored in a chunk.
  */
-public abstract class ChunkType {
-  public abstract int getSrcType();
-  public abstract Chunk newChunk(long memoryAddress);
-  public abstract Chunk newChunk(long memoryAddress, int chunkSize);
+public interface ChunkType {
+  /**
+   * Returns an int that describes that type of
+   * data stored in the chunk.
+   * Currently the only supported type is
+   * Chunk.SRC_TYPE_GFE.
+   */
+  public int getSrcType();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
index 61204ba..5b14389 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
@@ -16,18 +16,11 @@
  */
 package com.gemstone.gemfire.internal.offheap;
 
-import java.io.DataOutput;
-import java.io.IOException;
-
-import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.internal.DataSerializableFixedID;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
 import com.gemstone.gemfire.internal.cache.EntryBits;
 import com.gemstone.gemfire.internal.cache.RegionEntry;
 import com.gemstone.gemfire.internal.cache.RegionEntryContext;
-import com.gemstone.gemfire.internal.lang.StringUtils;
 
 /**
  * Used to represent offheap addresses whose
@@ -35,7 +28,7 @@ import com.gemstone.gemfire.internal.lang.StringUtils;
  * location.
  * Instances of this class have a very short lifetime.
  */
-public class DataAsAddress implements StoredObject {
+public class DataAsAddress extends AbstractStoredObject {
   private final long address;
   
   public DataAsAddress(long addr) {
@@ -88,53 +81,6 @@ public class DataAsAddress implements StoredObject {
   }
 
   @Override
-  public Object getDeserializedForReading() {
-    return getDeserializedValue(null,null);
-  }
-  
-  @Override
-  public Object getValueAsDeserializedHeapObject() {
-    return getDeserializedValue(null,null);
-  }
-  
-  @Override
-  public byte[] getValueAsHeapByteArray() {
-    if (isSerialized()) {
-      return getSerializedValue();
-    } else {
-      return (byte[])getDeserializedForReading();
-    }
-  }
-
-  @Override
-  public String getStringForm() {
-    try {
-      return StringUtils.forceToString(getDeserializedForReading());
-    } catch (RuntimeException ex) {
-      return "Could not convert object to string because " + ex;
-    }
-  }
-
-  @Override
-  public Object getDeserializedWritableCopy(Region r, RegionEntry re) {
-    return getDeserializedValue(null,null);
-  }
-
-  @Override
-  public Object getValue() {
-    if (isSerialized()) {
-      return getSerializedValue();
-    } else {
-      throw new IllegalStateException("Can not call getValue on StoredObject that is not serialized");
-    }
-  }
-
-  @Override
-  public void writeValueAsByteArray(DataOutput out) throws IOException {
-    DataSerializer.writeByteArray(getSerializedValue(), out);
-  }
-
-  @Override
   public void fillSerializedValue(BytesAndBitsForCompactor wrapper,
       byte userBits) {
     byte[] value;
@@ -153,37 +99,6 @@ public class DataAsAddress implements StoredObject {
   }
   
   @Override
-  public void sendTo(DataOutput out) throws IOException {
-    if (isSerialized()) {
-      out.write(getSerializedValue());
-    } else {
-      Object objToSend = (byte[]) getDeserializedForReading(); // deserialized as a byte[]
-      DataSerializer.writeObject(objToSend, out);
-    }
-  }
-
-  @Override
-  public void sendAsByteArray(DataOutput out) throws IOException {
-    byte[] bytes;
-    if (isSerialized()) {
-      bytes = getSerializedValue();
-    } else {
-      bytes = (byte[]) getDeserializedForReading();
-    }
-    DataSerializer.writeByteArray(bytes, out);
-    
-  }
-  
-  @Override
-  public void sendAsCachedDeserializable(DataOutput out) throws IOException {
-    if (!isSerialized()) {
-      throw new IllegalStateException("sendAsCachedDeserializable can only be called on serialized StoredObjects");
-    }
-    InternalDataSerializer.writeDSFIDHeader(DataSerializableFixedID.VM_CACHED_DESERIALIZABLE, out);
-    sendAsByteArray(out);
-  }
-
-  @Override
   public boolean isSerialized() {
     return OffHeapRegionEntryHelper.isSerialized(this.address);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
index bd05ddb..ef56627 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
@@ -122,4 +122,18 @@ public class Fragment implements MemoryBlock {
   public ChunkType getChunkType() {
     return null;
   }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof Fragment) {
+      return getMemoryAddress() == ((Fragment) o).getMemoryAddress();
+    }
+    return false;
+  }
+  
+  @Override
+  public int hashCode() {
+    long value = this.getMemoryAddress();
+    return (int)(value ^ (value >>> 32));
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
index 48a0756..10e4148 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
@@ -803,5 +803,19 @@ public class FreeListManager {
     public ChunkType getChunkType() {
       return null;
     }
+    
+    @Override
+    public boolean equals(Object o) {
+      if (o instanceof TinyMemoryBlock) {
+        return getMemoryAddress() == ((TinyMemoryBlock) o).getMemoryAddress();
+      }
+      return false;
+    }
+    
+    @Override
+    public int hashCode() {
+      long value = this.getMemoryAddress();
+      return (int)(value ^ (value >>> 32));
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunk.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunk.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunk.java
index 3167613..20e4a2f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunk.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunk.java
@@ -28,14 +28,6 @@ public class GemFireChunk extends Chunk {
     public int getSrcType() {
       return Chunk.SRC_TYPE_GFE;
     }
-    @Override
-    public Chunk newChunk(long memoryAddress) {
-      return new GemFireChunk(memoryAddress);
-    }
-    @Override
-    public Chunk newChunk(long memoryAddress, int chunkSize) {
-      return new GemFireChunk(memoryAddress, chunkSize);
-    }
   };
   public GemFireChunk(long memoryAddress, int chunkSize) {
     super(memoryAddress, chunkSize, TYPE);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
index 231ff3a..0a014de 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
@@ -16,8 +16,6 @@
  */
 package com.gemstone.gemfire.internal.offheap;
 
-import com.gemstone.gemfire.compression.Compressor;
-
 /**
  * Basic contract for a heap that manages off heap memory. Any MemoryChunks allocated from a heap
  * are returned to that heap when freed.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
index 3f5f4dc..546feee 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
@@ -21,7 +21,6 @@ import java.util.Arrays;
 
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.CacheClosedException;
-import com.gemstone.gemfire.internal.offheap.MemoryBlock.State;
 
 /**
  * Basic implementation of MemoryBlock for test validation only.
@@ -155,4 +154,17 @@ public class MemoryBlockNode implements MemoryBlock {
   public ChunkType getChunkType() {
     return this.block.getChunkType();
   }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof MemoryBlockNode) {
+      o = ((MemoryBlockNode)o).block;
+    }
+    return this.block.equals(o);
+  }
+  
+  @Override
+  public int hashCode() {
+    return this.block.hashCode();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryInspector.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryInspector.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryInspector.java
index acf6d04..cde24bc 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryInspector.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryInspector.java
@@ -36,12 +36,6 @@ public interface MemoryInspector {
   
   public List<MemoryBlock> getAllocatedBlocks();
   
-  public List<MemoryBlock> getDeallocatedBlocks();
-  
-  public List<MemoryBlock> getUnusedBlocks();
-  
-  public MemoryBlock getBlockContaining(long memoryAddress);
-  
   public MemoryBlock getBlockAfter(MemoryBlock block);
   
   public List<MemoryBlock> getOrphans();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
index 143fb25..1ec722d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
@@ -16,20 +16,11 @@
  */
 package com.gemstone.gemfire.internal.offheap;
 
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-
-import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.internal.DSCODE;
-import com.gemstone.gemfire.internal.DataSerializableFixedID;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
-import com.gemstone.gemfire.internal.cache.CachedDeserializableFactory;
 import com.gemstone.gemfire.internal.cache.EntryBits;
 import com.gemstone.gemfire.internal.cache.RegionEntry;
-import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 
 /**
@@ -39,7 +30,7 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
  * @author darrel
  * @since 9.0
  */
-public abstract class OffHeapCachedDeserializable implements MemoryChunkWithRefCount {
+public abstract class OffHeapCachedDeserializable extends AbstractStoredObject implements MemoryChunkWithRefCount {
   public abstract void setSerializedValue(byte[] value);
   @Override
   public abstract byte[] getSerializedValue();
@@ -51,53 +42,6 @@ public abstract class OffHeapCachedDeserializable implements MemoryChunkWithRefC
   public abstract Object getDeserializedValue(Region r, RegionEntry re);
 
   @Override
-  public Object getValueAsDeserializedHeapObject() {
-    return getDeserializedValue(null, null);
-  }
-  
-  @Override
-  public byte[] getValueAsHeapByteArray() {
-    if (isSerialized()) {
-      return getSerializedValue();
-    } else {
-      return (byte[])getDeserializedForReading();
-    }
-  }
-  
-  @Override
-  public Object getDeserializedForReading() {
-    return getDeserializedValue(null, null);
-  }
-
-  @Override
-  public String getStringForm() {
-    try {
-      return StringUtils.forceToString(getDeserializedForReading());
-    } catch (RuntimeException ex) {
-      return "Could not convert object to string because " + ex;
-    }
-  }
-
-  @Override
-  public Object getDeserializedWritableCopy(Region r, RegionEntry re) {
-    return getDeserializedValue(null, null);
-  }
-
-  @Override
-  public Object getValue() {
-    if (isSerialized()) {
-      return getSerializedValue();
-    } else {
-      throw new IllegalStateException("Can not call getValue on StoredObject that is not serialized");
-    }
-  }
-
-  @Override
-  public void writeValueAsByteArray(DataOutput out) throws IOException {
-    DataSerializer.writeByteArray(getSerializedValue(), out);
-  }
-
-  @Override
   public void fillSerializedValue(BytesAndBitsForCompactor wrapper, byte userBits) {
     if (isSerialized()) {
       userBits = EntryBits.setSerialized(userBits, true);
@@ -114,33 +58,6 @@ public abstract class OffHeapCachedDeserializable implements MemoryChunkWithRefC
   public String toString() {
     return getShortClassName()+"@"+this.hashCode();
   }
-  @Override
-  public void sendTo(DataOutput out) throws IOException {
-    if (isSerialized()) {
-      out.write(getSerializedValue());
-    } else {
-      Object objToSend = (byte[]) getDeserializedForReading(); // deserialized as a byte[]
-      DataSerializer.writeObject(objToSend, out);
-    }
-  }
-  @Override
-  public void sendAsByteArray(DataOutput out) throws IOException {
-    byte[] bytes;
-    if (isSerialized()) {
-      bytes = getSerializedValue();
-    } else {
-      bytes = (byte[]) getDeserializedForReading();
-    }
-    DataSerializer.writeByteArray(bytes, out);
-  }
-  @Override
-  public void sendAsCachedDeserializable(DataOutput out) throws IOException {
-    if (!isSerialized()) {
-      throw new IllegalStateException("sendAsCachedDeserializable can only be called on serialized StoredObjects");
-    }
-    InternalDataSerializer.writeDSFIDHeader(DataSerializableFixedID.VM_CACHED_DESERIALIZABLE, out);
-    sendAsByteArray(out);
-  }
   public boolean checkDataEquals(@Unretained OffHeapCachedDeserializable other) {
     if (this == other) {
       return true;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
index 1a6cc8b..ef584f1 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
@@ -183,7 +183,6 @@ public class OffHeapStorage implements OffHeapMemoryStats {
       }
     }
     
-    //TODO:Asif: Fix it
     MemoryAllocator result;
     if (offHeapMemorySize == 0 || Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE)) {
       // Checking the FORCE_LOCATOR_DM_TYPE is a quick hack to keep our locator from allocating off heap memory.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
index f16253e..dfd05c6 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
@@ -70,6 +70,8 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
    * Sizes are always rounded up to the next multiple of this constant
    * so internal fragmentation will be limited to TINY_MULTIPLE-1 bytes per allocation
    * and on average will be TINY_MULTIPLE/2 given a random distribution of size requests.
+   * This does not account for the additional internal fragmentation caused by the off-heap header
+   * which currently is always 8 bytes.
    */
   public final static int TINY_MULTIPLE = Integer.getInteger("gemfire.OFF_HEAP_ALIGNMENT", 8);
   /**
@@ -77,6 +79,9 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
    */
   public final static int TINY_FREE_LIST_COUNT = Integer.getInteger("gemfire.OFF_HEAP_FREE_LIST_COUNT", 16384);
   public final static int MAX_TINY = TINY_MULTIPLE*TINY_FREE_LIST_COUNT;
+  /**
+   * How many unused bytes are allowed in a huge memory allocation.
+   */
   public final static int HUGE_MULTIPLE = 256;
   
   volatile OffHeapMemoryStats stats;
@@ -95,7 +100,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   private volatile MemoryUsageListener[] memoryUsageListeners = new MemoryUsageListener[0];
   
   private static SimpleMemoryAllocatorImpl singleton = null;
-  private static final AtomicReference<Thread> asyncCleanupThread = new AtomicReference<Thread>();
+  private static final AtomicReference<Thread> asyncCleanupThread = new AtomicReference<>();
   final ChunkFactory chunkFactory;
   
   public static SimpleMemoryAllocatorImpl getAllocator() {
@@ -106,69 +111,116 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     return result;
   }
 
-  private static final boolean PRETOUCH = Boolean.getBoolean("gemfire.OFF_HEAP_PRETOUCH_PAGES");
-  static final int OFF_HEAP_PAGE_SIZE = Integer.getInteger("gemfire.OFF_HEAP_PAGE_SIZE", UnsafeMemoryChunk.getPageSize());
-  private static final boolean DO_EXPENSIVE_VALIDATION = Boolean.getBoolean("gemfire.OFF_HEAP_DO_EXPENSIVE_VALIDATION");;
-  
-  public static MemoryAllocator create(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats stats, LogWriter lw, int slabCount, long offHeapMemorySize, long maxSlabSize) {
+  private static final boolean DO_EXPENSIVE_VALIDATION = Boolean.getBoolean("gemfire.OFF_HEAP_DO_EXPENSIVE_VALIDATION");
+  
+  public static MemoryAllocator create(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats stats, LogWriter lw, 
+      int slabCount, long offHeapMemorySize, long maxSlabSize) {
+    return create(ooohml, stats, lw, slabCount, offHeapMemorySize, maxSlabSize,
+        null, TINY_MULTIPLE, BATCH_SIZE, TINY_FREE_LIST_COUNT, HUGE_MULTIPLE, 
+        new UnsafeMemoryChunk.Factory() {
+      @Override
+      public UnsafeMemoryChunk create(int size) {
+        return new UnsafeMemoryChunk(size);
+      }
+    });
+  }
+
+  private static SimpleMemoryAllocatorImpl create(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats stats, LogWriter lw, 
+      int slabCount, long offHeapMemorySize, long maxSlabSize, 
+      UnsafeMemoryChunk[] slabs, int tinyMultiple, int batchSize, int tinyFreeListCount, int hugeMultiple,
+      UnsafeMemoryChunk.Factory memChunkFactory) {
     SimpleMemoryAllocatorImpl result = singleton;
     boolean created = false;
     try {
     if (result != null) {
-      result.reuse(ooohml, lw, stats, offHeapMemorySize);
-      lw.config("Reusing " + result.getTotalMemory() + " bytes of off-heap memory. The maximum size of a single off-heap object is " + result.largestSlab + " bytes.");
+      result.reuse(ooohml, lw, stats, offHeapMemorySize, slabs);
+      if (lw != null) {
+        lw.config("Reusing " + result.getTotalMemory() + " bytes of off-heap memory. The maximum size of a single off-heap object is " + result.largestSlab + " bytes.");
+      }
       created = true;
       LifecycleListener.invokeAfterReuse(result);
     } else {
-      // allocate memory chunks
-      //SimpleMemoryAllocatorImpl.cleanupPreviousAllocator();
-      lw.config("Allocating " + offHeapMemorySize + " bytes of off-heap memory. The maximum size of a single off-heap object is " + maxSlabSize + " bytes.");
-      UnsafeMemoryChunk[] slabs = new UnsafeMemoryChunk[slabCount];
-      long uncreatedMemory = offHeapMemorySize;
-      for (int i=0; i < slabCount; i++) {
-        try {
-        if (uncreatedMemory >= maxSlabSize) {
-          slabs[i] = new UnsafeMemoryChunk((int) maxSlabSize);
-          uncreatedMemory -= maxSlabSize;
-        } else {
-          // the last slab can be smaller then maxSlabSize
-          slabs[i] = new UnsafeMemoryChunk((int) uncreatedMemory);
+      if (slabs == null) {
+        // allocate memory chunks
+        //SimpleMemoryAllocatorImpl.cleanupPreviousAllocator();
+        if (lw != null) {
+          lw.config("Allocating " + offHeapMemorySize + " bytes of off-heap memory. The maximum size of a single off-heap object is " + maxSlabSize + " bytes.");
         }
-        } catch (OutOfMemoryError err) {
-          if (i > 0) {
-            lw.severe("Off-heap memory creation failed after successfully allocating " + (i*maxSlabSize) + " bytes of off-heap memory.");
-          }
-          for (int j=0; j < i; j++) {
-            if (slabs[j] != null) {
-              slabs[j].release();
+        slabs = new UnsafeMemoryChunk[slabCount];
+        long uncreatedMemory = offHeapMemorySize;
+        for (int i=0; i < slabCount; i++) {
+          try {
+            if (uncreatedMemory >= maxSlabSize) {
+              slabs[i] = memChunkFactory.create((int) maxSlabSize);
+              uncreatedMemory -= maxSlabSize;
+            } else {
+              // the last slab can be smaller then maxSlabSize
+              slabs[i] = memChunkFactory.create((int) uncreatedMemory);
+            }
+          } catch (OutOfMemoryError err) {
+            if (i > 0) {
+              if (lw != null) {
+                lw.severe("Off-heap memory creation failed after successfully allocating " + (i*maxSlabSize) + " bytes of off-heap memory.");
+              }
             }
+            for (int j=0; j < i; j++) {
+              if (slabs[j] != null) {
+                slabs[j].release();
+              }
+            }
+            throw err;
           }
-          throw err;
         }
       }
 
-      result = new SimpleMemoryAllocatorImpl(ooohml, stats, slabs);
-      created = true;
+      result = new SimpleMemoryAllocatorImpl(ooohml, stats, slabs, tinyMultiple, batchSize, tinyFreeListCount, hugeMultiple);
       singleton = result;
       LifecycleListener.invokeAfterCreate(result);
+      created = true;
     }
     } finally {
       if (!created) {
-        stats.close();
-        ooohml.close();
+        if (stats != null) {
+          stats.close();
+        }
+        if (ooohml != null) {
+          ooohml.close();
+        }
       }
     }
     return result;
   }
   // for unit tests
+  static SimpleMemoryAllocatorImpl create(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats stats, LogWriter lw, 
+      int slabCount, long offHeapMemorySize, long maxSlabSize, UnsafeMemoryChunk.Factory memChunkFactory) {
+    return create(ooohml, stats, lw, slabCount, offHeapMemorySize, maxSlabSize, 
+        null, TINY_MULTIPLE, BATCH_SIZE, TINY_FREE_LIST_COUNT, HUGE_MULTIPLE, memChunkFactory);
+  }
+  // for unit tests
   public static SimpleMemoryAllocatorImpl create(OutOfOffHeapMemoryListener oooml, OffHeapMemoryStats stats, UnsafeMemoryChunk[] slabs) {
-    SimpleMemoryAllocatorImpl result = new SimpleMemoryAllocatorImpl(oooml, stats, slabs);
-    singleton = result;
-    LifecycleListener.invokeAfterCreate(result);
-    return result;
+    return create(oooml, stats, slabs, TINY_MULTIPLE, BATCH_SIZE, TINY_FREE_LIST_COUNT, HUGE_MULTIPLE);
   }
+  // for unit tests
+  static SimpleMemoryAllocatorImpl create(OutOfOffHeapMemoryListener oooml, OffHeapMemoryStats stats, UnsafeMemoryChunk[] slabs,
+      int tinyMultiple, int batchSize, int tinyFreeListCount, int hugeMultiple) {
+    int slabCount = 0;
+    long offHeapMemorySize = 0;
+    long maxSlabSize = 0;
+    if (slabs != null) {
+      slabCount = slabs.length;
+      for (int i=0; i < slabCount; i++) {
+        int slabSize = slabs[i].getSize();
+        offHeapMemorySize += slabSize;
+        if (slabSize > maxSlabSize) {
+          maxSlabSize = slabSize;
+        }
+      }
+    }
+    return create(oooml, stats, null, slabCount, offHeapMemorySize, maxSlabSize, slabs, tinyMultiple, batchSize, tinyFreeListCount, hugeMultiple, null);
+  }
+  
   
-  private void reuse(OutOfOffHeapMemoryListener oooml, LogWriter lw, OffHeapMemoryStats newStats, long offHeapMemorySize) {
+  private void reuse(OutOfOffHeapMemoryListener oooml, LogWriter lw, OffHeapMemoryStats newStats, long offHeapMemorySize, UnsafeMemoryChunk[] slabs) {
     if (isClosed()) {
       throw new IllegalStateException("Can not reuse a closed off-heap memory manager.");
     }
@@ -176,100 +228,55 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
       throw new IllegalArgumentException("OutOfOffHeapMemoryListener is null");
     }
     if (getTotalMemory() != offHeapMemorySize) {
-      lw.warning("Using " + getTotalMemory() + " bytes of existing off-heap memory instead of the requested " + offHeapMemorySize);
+      if (lw != null) {
+        lw.warning("Using " + getTotalMemory() + " bytes of existing off-heap memory instead of the requested " + offHeapMemorySize);
+      }
+    }
+    if (slabs != null) {
+      // this will only happen in unit tests
+      if (slabs != this.slabs) {
+        // If the unit test gave us a different array
+        // of slabs then something is wrong because we
+        // are trying to reuse the old already allocated
+        // array which means that the new one will never
+        // be used. Note that this code does not bother
+        // comparing the contents of the arrays.
+        throw new IllegalStateException("attempted to reuse existing off-heap memory even though new off-heap memory was allocated");
+      }
     }
     this.ooohml = oooml;
     newStats.initialize(this.stats);
     this.stats = newStats;
   }
 
-  public static void cleanupPreviousAllocator() {
-    Thread t = asyncCleanupThread.getAndSet(null);
-    if (t != null) {
-//      try {
-//        // HACK to see if a delay fixes bug 47883
-//        Thread.sleep(3000);
-//      } catch (InterruptedException ignore) {
-//      }
-      t.interrupt();
-      try {
-        t.join(FREE_PAUSE_MILLIS);
-      } catch (InterruptedException ignore) {
-        Thread.currentThread().interrupt();
-      }
-    }
-  }
-  
-  private SimpleMemoryAllocatorImpl(final OutOfOffHeapMemoryListener oooml, final OffHeapMemoryStats stats, final UnsafeMemoryChunk[] slabs) {
+  private SimpleMemoryAllocatorImpl(final OutOfOffHeapMemoryListener oooml, final OffHeapMemoryStats stats, final UnsafeMemoryChunk[] slabs,
+      int tinyMultiple, int batchSize, int tinyFreeListCount, int hugeMultiple) {
     if (oooml == null) {
       throw new IllegalArgumentException("OutOfOffHeapMemoryListener is null");
     }
-    if (TINY_MULTIPLE <= 0 || (TINY_MULTIPLE & 3) != 0) {
+    if (tinyMultiple <= 0 || (tinyMultiple & 3) != 0) {
       throw new IllegalStateException("gemfire.OFF_HEAP_ALIGNMENT must be a multiple of 8.");
     }
-    if (TINY_MULTIPLE > 256) {
+    if (tinyMultiple > 256) {
       // this restriction exists because of the dataSize field in the object header.
       throw new IllegalStateException("gemfire.OFF_HEAP_ALIGNMENT must be <= 256 and a multiple of 8.");
     }
-    if (BATCH_SIZE <= 0) {
+    if (batchSize <= 0) {
       throw new IllegalStateException("gemfire.OFF_HEAP_BATCH_ALLOCATION_SIZE must be >= 1.");
     }
-    if (TINY_FREE_LIST_COUNT <= 0) {
+    if (tinyFreeListCount <= 0) {
       throw new IllegalStateException("gemfire.OFF_HEAP_FREE_LIST_COUNT must be >= 1.");
     }
-    assert HUGE_MULTIPLE <= 256;
+    if (hugeMultiple > 256 || hugeMultiple < 0) {
+      // this restriction exists because of the dataSize field in the object header.
+      throw new IllegalStateException("HUGE_MULTIPLE must be >= 0 and <= 256 but it was " + hugeMultiple);
+    }
     
     this.ooohml = oooml;
     this.stats = stats;
     this.slabs = slabs;
-    if(GemFireCacheImpl.sqlfSystem()) {
-      throw new IllegalStateException("offheap sqlf not supported");
-//       String provider = GemFireCacheImpl.SQLF_FACTORY_PROVIDER;
-//       try {
-//         Class<?> factoryProvider = Class.forName(provider);
-//         Method method = factoryProvider.getDeclaredMethod("getChunkFactory");        
-//         this.chunkFactory  = (ChunkFactory)method.invoke(null, (Object [])null);
-//       }catch (Exception e) {
-//         throw new IllegalStateException("Exception in obtaining ChunkFactory class",  e);
-//       }
-
-    }else {
-      
-      this.chunkFactory = new GemFireChunkFactory();
-    }
+    this.chunkFactory = new GemFireChunkFactory();
     
-    if (PRETOUCH) {
-      final int tc;
-      if (Runtime.getRuntime().availableProcessors() > 1) {
-        tc = Runtime.getRuntime().availableProcessors() / 2;
-      } else {
-        tc = 1;
-      }
-      Thread[] threads = new Thread[tc];
-      for (int i=0; i < tc; i++) {
-        final int threadId = i;
-        threads[i] = new Thread(new Runnable() {
-          @Override
-          public void run() {
-            for (int slabId=threadId; slabId < slabs.length; slabId+=tc) {
-              final int slabSize = slabs[slabId].getSize();
-              for (int pageId=0; pageId < slabSize; pageId+=OFF_HEAP_PAGE_SIZE) {
-                slabs[slabId].writeByte(pageId, (byte) 0);
-              }
-            }
-          }
-        });
-        threads[i].start();
-      }
-      for (int i=0; i < tc; i++) {
-        try {
-          threads[i].join();
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          break;
-        }
-      }
-    }
     //OSProcess.printStacks(0, InternalDistributedSystem.getAnyInstance().getLogWriter(), false);
     this.stats.setFragments(slabs.length);
     largestSlab = slabs[0].getSize();
@@ -437,7 +444,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   private void realClose() {
     // Removing this memory immediately can lead to a SEGV. See 47885.
     if (setClosed()) {
-      freeSlabsAsync(this.slabs);
+      freeSlabs(this.slabs);
       this.stats.close();
       singleton = null;
     }
@@ -456,41 +463,11 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   }
   
 
-  private static final int FREE_PAUSE_MILLIS = Integer.getInteger("gemfire.OFF_HEAP_FREE_PAUSE_MILLIS", 90000);
-
-  
-  
-  private static void freeSlabsAsync(final UnsafeMemoryChunk[] slabs) {
-    //debugLog("called freeSlabsAsync", false);
-    // since we no longer free off-heap memory on every cache close
-    // and production code does not free it but instead reuses it
-    // we should be able to free it sync.
-    // If it turns out that it does need to be async then we need
-    // to make sure we call cleanupPreviousAllocator.
+  private static void freeSlabs(final UnsafeMemoryChunk[] slabs) {
+    //debugLog("called freeSlabs", false);
     for (int i=0; i < slabs.length; i++) {
       slabs[i].release();
     }
-//    Thread t = new Thread(new Runnable() {
-//      @Override
-//      public void run() {
-//        // pause this many millis before freeing the slabs.
-//        try {
-//          Thread.sleep(FREE_PAUSE_MILLIS);
-//        } catch (InterruptedException ignore) {
-//          // If we are interrupted we should wakeup
-//          // and free our slabs.
-//        }
-//        //debugLog("returning offheap memory to OS", false);
-//        for (int i=0; i < slabs.length; i++) {
-//          slabs[i].free();
-//        }
-//        //debugLog("returned offheap memory to OS", false);
-//        asyncCleanupThread.compareAndSet(Thread.currentThread(), null);
-//      }
-//    }, "asyncSlabDeallocator");
-//    t.setDaemon(true);
-//    t.start();
-//    asyncCleanupThread.set(t);    
   }
   
   void freeChunk(long addr) {
@@ -504,7 +481,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   /**
    * Return the slabId of the slab that contains the given addr.
    */
-  protected int findSlab(long addr) {
+  int findSlab(long addr) {
     for (int i=0; i < this.slabs.length; i++) {
       UnsafeMemoryChunk slab = this.slabs[i];
       long slabAddr = slab.getMemoryAddress();
@@ -591,11 +568,11 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     if (addr >= 0 && addr < 1024) {
       throw new IllegalStateException("addr was smaller than expected 0x" + addr);
     }
-    validateAddressAndSizeWithinSlab(addr, size);
+    validateAddressAndSizeWithinSlab(addr, size, DO_EXPENSIVE_VALIDATION);
   }
 
-  static void validateAddressAndSizeWithinSlab(long addr, int size) {
-    if (DO_EXPENSIVE_VALIDATION) {
+  static void validateAddressAndSizeWithinSlab(long addr, int size, boolean doExpensiveValidation) {
+    if (doExpensiveValidation) {
       SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.singleton;
       if (ma != null) {
         for (int i=0; i < ma.slabs.length; i++) {
@@ -686,21 +663,6 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   }
 
   @Override
-  public List<MemoryBlock> getDeallocatedBlocks() {
-    return null;
-  }
-
-  @Override
-  public List<MemoryBlock> getUnusedBlocks() {
-    return null;
-  }
-  
-  @Override
-  public MemoryBlock getBlockContaining(long memoryAddress) {
-    return null;
-  }
-  
-  @Override
   public MemoryBlock getBlockAfter(MemoryBlock block) {
     if (block == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunk.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunk.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunk.java
index 06fee7b..4f0e86d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunk.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunk.java
@@ -265,4 +265,14 @@ public class UnsafeMemoryChunk implements MemoryChunk {
     sb.append("}");
     return sb.toString();
   }
+  
+  /**
+   * Used to create UnsafeMemoryChunk instances.
+   */
+  public interface Factory {
+    /** Create and return an UnsafeMemoryChunk.
+     * @throws OutOfMemoryError if the create fails
+     */
+    public UnsafeMemoryChunk create(int size);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOffHeapMemoryStats.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOffHeapMemoryStats.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOffHeapMemoryStats.java
index 7c668f1..88bab77 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOffHeapMemoryStats.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOffHeapMemoryStats.java
@@ -24,6 +24,7 @@ import com.gemstone.gemfire.Statistics;
  * @author Kirk Lund
  */
 public class NullOffHeapMemoryStats implements OffHeapMemoryStats {
+  private boolean isClosed;
 
   public void incFreeMemory(long value) {
   }
@@ -100,9 +101,14 @@ public class NullOffHeapMemoryStats implements OffHeapMemoryStats {
   }
   @Override
   public void close() {
+    this.isClosed = true;
   }
   @Override
   public void initialize(OffHeapMemoryStats stats) {
     stats.close();
   }     
+  
+  public boolean isClosed() {
+    return this.isClosed;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOutOfOffHeapMemoryListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOutOfOffHeapMemoryListener.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOutOfOffHeapMemoryListener.java
index caa913a..7d02c9f 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOutOfOffHeapMemoryListener.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOutOfOffHeapMemoryListener.java
@@ -24,10 +24,16 @@ import com.gemstone.gemfire.OutOfOffHeapMemoryException;
  * @author Kirk Lund
  */
 public class NullOutOfOffHeapMemoryListener implements OutOfOffHeapMemoryListener {
+  private boolean isClosed;
   @Override
   public void outOfOffHeapMemory(OutOfOffHeapMemoryException cause) {
   }
   @Override
   public void close() {
+    this.isClosed = true;
+  }
+  
+  public boolean isClosed() {
+    return this.isClosed;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f9744623/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
index 19dfebb..1477764 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
@@ -16,21 +16,21 @@
  */
 package com.gemstone.gemfire.internal.offheap;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.OutOfOffHeapMemoryException;
+import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.internal.logging.NullLogWriter;
+import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk.Factory;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -40,15 +40,198 @@ public class SimpleMemoryAllocatorJUnitTest {
     return ((v+multiple-1)/multiple)*multiple;
   }
   @Test
+  public void testNullGetAllocator() {
+    try {
+      SimpleMemoryAllocatorImpl.getAllocator();
+      fail("expected CacheClosedException");
+    } catch (CacheClosedException expected) {
+    }
+  }
+  @Test
+  public void testConstructor() {
+    try {
+      SimpleMemoryAllocatorImpl.create(null, null, null);
+      fail("expected IllegalArgumentException");
+    } catch (IllegalArgumentException expected) {
+    }
+    try {
+      SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), null, null, -1, 0, 0, 0);
+      fail("expected IllegalStateException");
+    } catch (IllegalStateException expected) {
+      assertEquals(true, expected.getMessage().contains("gemfire.OFF_HEAP_ALIGNMENT must be a multiple of 8"));
+    }
+    try {
+      SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), null, null, 9, 0, 0, 0);
+      fail("expected IllegalStateException");
+    } catch (IllegalStateException expected) {
+      assertEquals(true, expected.getMessage().contains("gemfire.OFF_HEAP_ALIGNMENT must be a multiple of 8"));
+    }
+    try {
+      SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), null, null, 256+8, 0, 0, 0);
+      fail("expected IllegalStateException");
+    } catch (IllegalStateException expected) {
+      assertEquals(true, expected.getMessage().contains("gemfire.OFF_HEAP_ALIGNMENT must be <= 256"));
+    }
+    try {
+      SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), null, null, 8, 0, 0, 0);
+      fail("expected IllegalStateException");
+    } catch (IllegalStateException expected) {
+      assertEquals(true, expected.getMessage().contains("gemfire.OFF_HEAP_BATCH_ALLOCATION_SIZE must be >= 1."));
+    }
+    try {
+      SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), null, null, 8, 1, 0, 0);
+      fail("expected IllegalStateException");
+    } catch (IllegalStateException expected) {
+      assertEquals(true, expected.getMessage().contains("gemfire.OFF_HEAP_FREE_LIST_COUNT must be >= 1."));
+    }
+    try {
+      SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), null, null, 8, 1, 1, -1);
+      fail("expected IllegalStateException");
+    } catch (IllegalStateException expected) {
+      assertEquals(true, expected.getMessage().contains("HUGE_MULTIPLE must be >= 0 and <= 256 but it was -1"));
+    }
+    try {
+      SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), null, null, 8, 1, 1, 257);
+      fail("expected IllegalStateException");
+    } catch (IllegalStateException expected) {
+      assertEquals(true, expected.getMessage().contains("HUGE_MULTIPLE must be >= 0 and <= 256 but it was 257"));
+    }
+     
+  }
+  /**
+   * Logger that remembers the last severe message
+   */
+  private static class LastSevereLogger extends NullLogWriter {
+    private String lastSevereMessage;
+    private Throwable lastSevereThrowable;
+    
+    private void setLastSevere(String msg, Throwable ex) {
+      this.lastSevereMessage = msg;
+      this.lastSevereThrowable = ex;
+    }
+    public String getLastSevereMessage() {
+      return this.lastSevereMessage;
+    }
+    public Throwable getLastSevereThrowable() {
+      return this.lastSevereThrowable;
+    }
+    @Override
+    public void severe(String msg, Throwable ex) {
+      setLastSevere(msg, ex);
+    }
+    @Override
+    public void severe(String msg) {
+      setLastSevere(msg, null);
+    }
+    @Override
+    public void severe(Throwable ex) {
+      setLastSevere(null, ex);
+    }
+  }
+  @Test
+  public void testCreate() {
+    {
+      NullOutOfOffHeapMemoryListener listener = new NullOutOfOffHeapMemoryListener();
+      NullOffHeapMemoryStats stats = new NullOffHeapMemoryStats();
+      LastSevereLogger logger = new LastSevereLogger();
+      try {
+        SimpleMemoryAllocatorImpl.create(listener, stats, logger, 10, 950, 100,
+            new UnsafeMemoryChunk.Factory() {
+          @Override
+          public UnsafeMemoryChunk create(int size) {
+            throw new OutOfMemoryError("expected");
+          }
+        });
+      } catch (OutOfMemoryError expected) {
+      }
+      assertTrue(listener.isClosed());
+      assertTrue(stats.isClosed());
+      assertEquals(null, logger.getLastSevereThrowable());
+      assertEquals(null, logger.getLastSevereMessage());
+     }
+    {
+      NullOutOfOffHeapMemoryListener listener = new NullOutOfOffHeapMemoryListener();
+      NullOffHeapMemoryStats stats = new NullOffHeapMemoryStats();
+      LastSevereLogger logger = new LastSevereLogger();
+      int MAX_SLAB_SIZE = 100;
+      try {
+        Factory factory = new UnsafeMemoryChunk.Factory() {
+          private int createCount = 0;
+          @Override
+          public UnsafeMemoryChunk create(int size) {
+            createCount++;
+            if (createCount == 1) {
+              return new UnsafeMemoryChunk(size);
+            } else {
+              throw new OutOfMemoryError("expected");
+            }
+          }
+        };
+        SimpleMemoryAllocatorImpl.create(listener, stats, logger, 10, 950, MAX_SLAB_SIZE, factory);
+      } catch (OutOfMemoryError expected) {
+      }
+      assertTrue(listener.isClosed());
+      assertTrue(stats.isClosed());
+      assertEquals(null, logger.getLastSevereThrowable());
+      assertEquals("Off-heap memory creation failed after successfully allocating " + MAX_SLAB_SIZE + " bytes of off-heap memory.", logger.getLastSevereMessage());
+    }
+    {
+      NullOutOfOffHeapMemoryListener listener = new NullOutOfOffHeapMemoryListener();
+      NullOffHeapMemoryStats stats = new NullOffHeapMemoryStats();
+      Factory factory = new UnsafeMemoryChunk.Factory() {
+        @Override
+        public UnsafeMemoryChunk create(int size) {
+          return new UnsafeMemoryChunk(size);
+        }
+      };
+      MemoryAllocator ma = 
+        SimpleMemoryAllocatorImpl.create(listener, stats, new NullLogWriter(), 10, 950, 100, factory);
+      try {
+        assertFalse(listener.isClosed());
+        assertFalse(stats.isClosed());
+        ma.close();
+        assertTrue(listener.isClosed());
+        assertFalse(stats.isClosed());
+        listener = new NullOutOfOffHeapMemoryListener();
+        NullOffHeapMemoryStats stats2 = new NullOffHeapMemoryStats();
+        {
+          UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+          try {
+            SimpleMemoryAllocatorImpl.create(listener, stats2, new UnsafeMemoryChunk[]{slab});
+          } catch (IllegalStateException expected) {
+            assertTrue("unexpected message: " + expected.getMessage(), 
+                expected.getMessage().equals("attempted to reuse existing off-heap memory even though new off-heap memory was allocated"));
+          } finally {
+            slab.release();
+          }
+          assertFalse(stats.isClosed());
+          assertTrue(listener.isClosed());
+          assertTrue(stats2.isClosed());
+        }
+        listener = new NullOutOfOffHeapMemoryListener();
+        stats2 = new NullOffHeapMemoryStats();
+        MemoryAllocator ma2 = SimpleMemoryAllocatorImpl.create(listener, stats2, new NullLogWriter(), 10, 950, 100, factory);
+        assertSame(ma, ma2);
+        assertTrue(stats.isClosed());
+        assertFalse(listener.isClosed());
+        assertFalse(stats2.isClosed());
+        stats = stats2;
+      } finally {
+        ma.close();
+        assertTrue(listener.isClosed());
+        assertFalse(stats.isClosed());
+        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+        assertTrue(stats.isClosed());
+      }
+    }
+  }
+  @Test
   public void testBasics() {
     int BATCH_SIZE = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.BATCH_SIZE;
     int TINY_MULTIPLE = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.TINY_MULTIPLE;
-//    int BIG_MULTIPLE = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.FreeListManager.BIG_MULTIPLE;
     int HUGE_MULTIPLE = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.HUGE_MULTIPLE;
     int perObjectOverhead = com.gemstone.gemfire.internal.offheap.Chunk.OFF_HEAP_HEADER_SIZE;
     int maxTiny = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.MAX_TINY-perObjectOverhead;
-//    int MIN_BIG_SIZE = round(BIG_MULTIPLE, maxTiny+perObjectOverhead+1)-perObjectOverhead;
-//    int maxBig = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.FreeListManager.MAX_BIG-perObjectOverhead;
     int minHuge = maxTiny+1;
     int TOTAL_MEM = (maxTiny+perObjectOverhead)*BATCH_SIZE /*+ (maxBig+perObjectOverhead)*BATCH_SIZE*/ + round(TINY_MULTIPLE, minHuge+1+perObjectOverhead)*BATCH_SIZE + (TINY_MULTIPLE+perObjectOverhead)*BATCH_SIZE /*+ (MIN_BIG_SIZE+perObjectOverhead)*BATCH_SIZE*/ + round(TINY_MULTIPLE, minHuge+perObjectOverhead+1);
     UnsafeMemoryChunk slab = new UnsafeMemoryChunk(TOTAL_MEM);
@@ -57,14 +240,10 @@ public class SimpleMemoryAllocatorJUnitTest {
       assertEquals(TOTAL_MEM, ma.getFreeMemory());
       assertEquals(TOTAL_MEM, ma.freeList.getFreeFragmentMemory());
       assertEquals(0, ma.freeList.getFreeTinyMemory());
-//      assertEquals(0, ma.freeList.getFreeBigMemory());
       assertEquals(0, ma.freeList.getFreeHugeMemory());
       MemoryChunk tinymc = ma.allocate(maxTiny, null);
       assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
       assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeTinyMemory());
-//      MemoryChunk bigmc = ma.allocate(maxBig);
-//      assertEquals(TOTAL_MEM-round(BIG_MULTIPLE, maxBig+perObjectOverhead)-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-//      assertEquals(round(BIG_MULTIPLE, maxBig+perObjectOverhead)*(BATCH_SIZE-1), ma.getFreeList().getFreeBigMemory());
       MemoryChunk hugemc = ma.allocate(minHuge, null);
       assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, minHuge+perObjectOverhead)/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
       long freeSlab = ma.freeList.getFreeFragmentMemory();
@@ -73,9 +252,6 @@ public class SimpleMemoryAllocatorJUnitTest {
       hugemc.release();
       assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead), ma.freeList.getFreeHugeMemory()-oldFreeHugeMemory);
       assertEquals(TOTAL_MEM/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-//      long oldFreeBigMemory = ma.freeList.getFreeBigMemory();
-//      bigmc.free();
-//      assertEquals(round(BIG_MULTIPLE, maxBig+perObjectOverhead), ma.freeList.getFreeBigMemory()-oldFreeBigMemory);
       assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
       long oldFreeTinyMemory = ma.freeList.getFreeTinyMemory();
       tinymc.release();
@@ -85,17 +261,12 @@ public class SimpleMemoryAllocatorJUnitTest {
       tinymc = ma.allocate(maxTiny, null);
       assertEquals(oldFreeTinyMemory, ma.freeList.getFreeTinyMemory());
       assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-//      bigmc = ma.allocate(maxBig);
-//      assertEquals(oldFreeBigMemory, ma.freeList.getFreeBigMemory());
-//      assertEquals(TOTAL_MEM-round(BIG_MULTIPLE, maxBig+perObjectOverhead)-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
       hugemc = ma.allocate(minHuge, null);
       assertEquals(oldFreeHugeMemory, ma.freeList.getFreeHugeMemory());
       assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, minHuge+perObjectOverhead)/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
       hugemc.release();
       assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead), ma.freeList.getFreeHugeMemory()-oldFreeHugeMemory);
       assertEquals(TOTAL_MEM/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-//      bigmc.free();
-//      assertEquals(round(BIG_MULTIPLE, maxBig+perObjectOverhead), ma.freeList.getFreeBigMemory()-oldFreeBigMemory);
       assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
       tinymc.release();
       assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.freeList.getFreeTinyMemory()-oldFreeTinyMemory);
@@ -108,9 +279,6 @@ public class SimpleMemoryAllocatorJUnitTest {
       freeSlab = ma.freeList.getFreeFragmentMemory();
       tinymc.release();
       assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead)+(round(TINY_MULTIPLE, 1+perObjectOverhead)*BATCH_SIZE), ma.freeList.getFreeTinyMemory()-oldFreeTinyMemory);
-//      bigmc = ma.allocate(MIN_BIG_SIZE);
-//      assertEquals(MIN_BIG_SIZE+perObjectOverhead, bigmc.getSize());
-//      assertEquals(freeSlab-((MIN_BIG_SIZE+perObjectOverhead)*BATCH_SIZE), ma.freeList.getFreeFragmentMemory());
       
       hugemc = ma.allocate(minHuge+1, null);
       assertEquals(round(TINY_MULTIPLE, minHuge+1+perObjectOverhead), hugemc.getSize());
@@ -129,13 +297,6 @@ public class SimpleMemoryAllocatorJUnitTest {
       assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*BATCH_SIZE, ma.freeList.getFreeHugeMemory());
       // now that we do compaction the following allocate works.
       hugemc = ma.allocate(minHuge + HUGE_MULTIPLE + HUGE_MULTIPLE-1, null);
-
-      //      assertEquals(minHuge+minHuge+1, ma.freeList.getFreeHugeMemory());
-//      hugemc.free();
-//      assertEquals(minHuge+minHuge+1+minHuge + HUGE_MULTIPLE + HUGE_MULTIPLE-1, ma.freeList.getFreeHugeMemory());
-//      hugemc = ma.allocate(minHuge + HUGE_MULTIPLE);
-//      assertEquals(minHuge + HUGE_MULTIPLE + HUGE_MULTIPLE-1, hugemc.getSize());
-//      assertEquals(minHuge+minHuge+1, ma.freeList.getFreeHugeMemory());
     } finally {
       SimpleMemoryAllocatorImpl.freeOffHeapMemory();
     }
@@ -165,6 +326,144 @@ public class SimpleMemoryAllocatorJUnitTest {
   }
   
   @Test
+  public void testDebugLog() {
+    SimpleMemoryAllocatorImpl.debugLog("test debug log", false);
+    SimpleMemoryAllocatorImpl.debugLog("test debug log", true);
+  }
+  @Test
+  public void testGetLostChunks() {
+    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024*1024);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      assertEquals(Collections.emptyList(), ma.getLostChunks());
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  @Test
+  public void testFindSlab() {
+    final int SLAB_SIZE = 1024*1024;
+    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(SLAB_SIZE);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      assertEquals(0, ma.findSlab(slab.getMemoryAddress()));
+      assertEquals(0, ma.findSlab(slab.getMemoryAddress()+SLAB_SIZE-1));
+      try {
+        ma.findSlab(slab.getMemoryAddress()-1);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+      }
+      try {
+        ma.findSlab(slab.getMemoryAddress()+SLAB_SIZE);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+      }
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  @Test
+  public void testValidateAddressAndSize() {
+    final int SLAB_SIZE = 1024*1024;
+    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(SLAB_SIZE);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      try {
+        SimpleMemoryAllocatorImpl.validateAddress(0L);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().contains("addr was smaller than expected"));
+      }
+      try {
+        SimpleMemoryAllocatorImpl.validateAddress(1L);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().contains("Valid addresses must be in one of the following ranges:"));
+      }
+      SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), SLAB_SIZE, false);
+      SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), SLAB_SIZE, true);
+      SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), -1, true);
+      try {
+        SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress()-1, SLAB_SIZE, true);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().equals(" address 0x" + Long.toString(slab.getMemoryAddress()-1, 16) + " does not address the original slab memory"));
+      }
+      try {
+        SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), SLAB_SIZE+1, true);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().equals(" address 0x" + Long.toString(slab.getMemoryAddress()+SLAB_SIZE, 16) + " does not address the original slab memory"));
+      }
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  @Test
+  public void testMemoryInspection() {
+    final int SLAB_SIZE = 1024*1024;
+    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(SLAB_SIZE);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      MemoryInspector inspector = ma.getMemoryInspector();
+      assertNotNull(inspector);
+      assertEquals(null, inspector.getFirstBlock());
+      assertEquals(Collections.emptyList(), ma.getInspectionSnapshot());
+      assertEquals(Collections.emptyList(), ma.getAllocatedBlocks());
+      assertEquals(null, ma.getBlockAfter(null));
+      inspector.createInspectionSnapshot();
+      // call this twice for code coverage
+      inspector.createInspectionSnapshot();
+      try {
+        assertEquals(ma.getAllBlocks(), ma.getInspectionSnapshot());
+        MemoryBlock firstBlock = inspector.getFirstBlock();
+        assertNotNull(firstBlock);
+        assertEquals(1024*1024, firstBlock.getBlockSize());
+        assertEquals("N/A", firstBlock.getDataType());
+        assertEquals(-1, firstBlock.getFreeListId());
+        assertTrue(firstBlock.getMemoryAddress() > 0);
+        assertNull(firstBlock.getNextBlock());
+        assertEquals(0, firstBlock.getRefCount());
+        assertEquals(0, firstBlock.getSlabId());
+        assertEquals(MemoryBlock.State.UNUSED, firstBlock.getState());
+        assertFalse(firstBlock.isCompressed());
+        assertFalse(firstBlock.isSerialized());
+        assertEquals(null, ma.getBlockAfter(firstBlock));
+      } finally {
+        inspector.clearInspectionSnapshot();
+      }
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+
+  @Test
+  public void testClose() {
+    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024*1024);
+    boolean freeSlab = true;
+    UnsafeMemoryChunk[] slabs = new UnsafeMemoryChunk[]{slab};
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), slabs);
+      ma.close();
+      ma.close();
+      System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
+      try {
+        ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), slabs);
+        ma.close();
+        freeSlab = false;
+        ma.close();
+      } finally {
+        System.clearProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY);
+      }
+    } finally {
+      if (freeSlab) {
+        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      }
+    }
+    
+  }
+  
+  @Test
   public void testCompaction() {
     final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.Chunk.OFF_HEAP_HEADER_SIZE;
     final int BIG_ALLOC_SIZE = 150000;
@@ -306,8 +605,18 @@ public class SimpleMemoryAllocatorJUnitTest {
       smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead, null);
       assertEquals(true, this.memoryUsageEventReceived);
       
+      MemoryUsageListener unaddedListener = new MemoryUsageListener() {
+        @Override
+        public void updateMemoryUsed(final long bytesUsed) {
+          throw new IllegalStateException("Should never be called");
+        }
+      };
+      ma.removeMemoryUsageListener(unaddedListener);
+      
       ma.removeMemoryUsageListener(listener);
       
+      ma.removeMemoryUsageListener(unaddedListener);
+
       this.expectedMemoryUsage = SMALL_ALLOC_SIZE * 2;
       this.memoryUsageEventReceived = false;
       smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead, null);


[15/50] [abbrv] incubator-geode git commit: With disable tcp(udp) now we don't throttle serial executor queue

Posted by kl...@apache.org.
With disable tcp(udp) now we don't throttle serial executor 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/fba68678
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/fba68678
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/fba68678

Branch: refs/heads/feature/GEODE-217
Commit: fba68678d268cf6c142d2f9dc275a54c0d6974af
Parents: 74e1364
Author: Hitesh Khamesra <hi...@yahoo.com>
Authored: Fri Dec 4 09:02:17 2015 -0800
Committer: Hitesh Khamesra <hi...@yahoo.com>
Committed: Fri Dec 4 09:05:40 2015 -0800

----------------------------------------------------------------------
 .../distributed/internal/DistributionManager.java       | 12 +++++++++---
 1 file changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fba68678/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
index 5d3bdce..7a9f7c0 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
@@ -842,7 +842,10 @@ public class DistributionManager
             " SERIAL_QUEUE_SIZE_THROTTLE :" + SERIAL_QUEUE_SIZE_THROTTLE
         ); 
       }
-      this.serialQueuedExecutorPool = new SerialQueuedExecutorPool(this.threadGroup, this.stats);
+      //  when TCP/IP is disabled we can't throttle the serial queue or we run the risk of 
+      // distributed deadlock when we block the UDP reader thread
+      boolean throttlingDisabled = system.getConfig().getDisableTcp();
+      this.serialQueuedExecutorPool = new SerialQueuedExecutorPool(this.threadGroup, this.stats, throttlingDisabled);
     }
       
     {
@@ -4119,14 +4122,17 @@ public class DistributionManager
     DistributionStats stats;
     ThreadGroup threadGroup;
     
+    final boolean throttlingDisabled;
+    
     /**
      * Constructor.
      * @param group thread group to which the threads will belog to.
      * @param stats 
      */
-    SerialQueuedExecutorPool(ThreadGroup group, DistributionStats stats) {
+    SerialQueuedExecutorPool(ThreadGroup group, DistributionStats stats, boolean throttlingDisabled) {
       this.threadGroup = group;
       this.stats = stats;
+      this.throttlingDisabled = throttlingDisabled;
     }
 
     /*
@@ -4250,7 +4256,7 @@ public class DistributionManager
       
       BlockingQueue poolQueue;
       
-      if (SERIAL_QUEUE_BYTE_LIMIT == 0) {
+      if (SERIAL_QUEUE_BYTE_LIMIT == 0 || this.throttlingDisabled) {
         poolQueue = new OverflowQueueWithDMStats(stats.getSerialQueueHelper());
       } else {
         poolQueue = new ThrottlingMemLinkedQueueWithDMStats(SERIAL_QUEUE_BYTE_LIMIT, SERIAL_QUEUE_THROTTLE, SERIAL_QUEUE_SIZE_LIMIT, SERIAL_QUEUE_SIZE_THROTTLE, this.stats.getSerialQueueHelper());


[21/50] [abbrv] incubator-geode git commit: GEODE-622: add OffHeapStorage unit test coverage

Posted by kl...@apache.org.
GEODE-622: add OffHeapStorage unit test coverage

Removed unused OFF_HEAP_TOTAL_SIZE system property.


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

Branch: refs/heads/feature/GEODE-217
Commit: 812d51c4d71b3163e9c26cf2d3046bd8267ebe69
Parents: e414a49
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Dec 2 15:06:34 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Mon Dec 7 11:09:30 2015 -0800

----------------------------------------------------------------------
 .../internal/offheap/OffHeapStorage.java        |  29 +--
 .../offheap/SimpleMemoryAllocatorImpl.java      |  22 +-
 .../offheap/OffHeapStorageJUnitTest.java        | 202 +++++++++++++++++++
 3 files changed, 219 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/812d51c4/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
index ef584f1..82cbfeb 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
@@ -147,6 +147,7 @@ public class OffHeapStorage implements OffHeapMemoryStats {
         result = MAX_SLAB_SIZE;
       }
     }
+    assert result > 0 && result <= MAX_SLAB_SIZE && result <= offHeapMemorySize;
     return result;
   }
   
@@ -175,14 +176,6 @@ public class OffHeapStorage implements OffHeapMemoryStats {
    * @return MemoryAllocator for off-heap storage
    */
   public static MemoryAllocator createOffHeapStorage(LogWriter lw, StatisticsFactory sf, long offHeapMemorySize, DistributedSystem system) {
-    // TODO: delete this block of code after tests are changed to use new config
-    if (offHeapMemorySize == 0 && !Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE)) {
-      String offHeapConfig = System.getProperty("gemfire.OFF_HEAP_TOTAL_SIZE");
-      if (offHeapConfig != null && !offHeapConfig.equals("")) {
-        offHeapMemorySize = parseLongWithUnits(offHeapConfig, 0L, 1024*1024);
-      }
-    }
-    
     MemoryAllocator result;
     if (offHeapMemorySize == 0 || Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE)) {
       // Checking the FORCE_LOCATOR_DM_TYPE is a quick hack to keep our locator from allocating off heap memory.
@@ -199,15 +192,6 @@ public class OffHeapStorage implements OffHeapMemoryStats {
       
       // determine off-heap and slab sizes
       final long maxSlabSize = calcMaxSlabSize(offHeapMemorySize);
-      assert maxSlabSize > 0;
-      
-      // validate sizes
-      if (maxSlabSize > MAX_SLAB_SIZE) {
-        throw new IllegalArgumentException("gemfire.OFF_HEAP_SLAB_SIZE of value " + offHeapMemorySize + " exceeds maximum value of " + MAX_SLAB_SIZE);
-      }
-      if (maxSlabSize > offHeapMemorySize) {
-        throw new IllegalArgumentException("The off heap slab size (which is " + maxSlabSize + "; set it with gemfire.OFF_HEAP_SLAB_SIZE) must be less than or equal to the total size (which is " + offHeapMemorySize + "; set it with gemfire.OFF_HEAP_SLAB_SIZE).");
-      }
       
       final int slabCount = calcSlabCount(maxSlabSize, offHeapMemorySize);
 
@@ -222,9 +206,10 @@ public class OffHeapStorage implements OffHeapMemoryStats {
   }
   
   private static final long MAX_SLAB_SIZE = Integer.MAX_VALUE;
-  private static final long MIN_SLAB_SIZE = 1024;
+  static final long MIN_SLAB_SIZE = 1024;
 
-  private static int calcSlabCount(long maxSlabSize, long offHeapMemorySize) {
+  // non-private for unit test access
+  static int calcSlabCount(long maxSlabSize, long offHeapMemorySize) {
     long result = offHeapMemorySize / maxSlabSize;
     if ((offHeapMemorySize % maxSlabSize) >= MIN_SLAB_SIZE) {
       result++;
@@ -430,13 +415,13 @@ public class OffHeapStorage implements OffHeapMemoryStats {
         if (this.ids == null) {
           return;
         }
-        final InternalDistributedSystem dsToDisconnect = this.ids;
-        this.ids = null; // set null to prevent memory leak after closure!
-        
         if (stayConnectedOnOutOfOffHeapMemory) {
           return;
         }
         
+        final InternalDistributedSystem dsToDisconnect = this.ids;
+        this.ids = null; // set null to prevent memory leak after closure!
+        
         if (dsToDisconnect.getDistributionManager().getRootCause() == null) {
           dsToDisconnect.getDistributionManager().setRootCause(cause);
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/812d51c4/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
index dfd05c6..d053797 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
@@ -25,12 +25,13 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionService;
 import com.gemstone.gemfire.internal.cache.BucketRegion;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
@@ -100,7 +101,6 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   private volatile MemoryUsageListener[] memoryUsageListeners = new MemoryUsageListener[0];
   
   private static SimpleMemoryAllocatorImpl singleton = null;
-  private static final AtomicReference<Thread> asyncCleanupThread = new AtomicReference<>();
   final ChunkFactory chunkFactory;
   
   public static SimpleMemoryAllocatorImpl getAllocator() {
@@ -297,8 +297,8 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   public List<Chunk> getLostChunks() {
     List<Chunk> liveChunks = this.freeList.getLiveChunks();
     List<Chunk> regionChunks = getRegionLiveChunks();
-    Set liveChunksSet = new HashSet(liveChunks);
-    Set regionChunksSet = new HashSet(regionChunks);
+    Set<Chunk> liveChunksSet = new HashSet<>(liveChunks);
+    Set<Chunk> regionChunksSet = new HashSet<>(regionChunks);
     liveChunksSet.removeAll(regionChunksSet);
     return new ArrayList<Chunk>(liveChunksSet);
   }
@@ -308,23 +308,22 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
    */
   private List<Chunk> getRegionLiveChunks() {
     ArrayList<Chunk> result = new ArrayList<Chunk>();
-    GemFireCacheImpl gfc = GemFireCacheImpl.getInstance();
+    RegionService gfc = GemFireCacheImpl.getInstance();
     if (gfc != null) {
-      Iterator rootIt = gfc.rootRegions().iterator();
+      Iterator<Region<?,?>> rootIt = gfc.rootRegions().iterator();
       while (rootIt.hasNext()) {
-        Region rr = (Region) rootIt.next();
+        Region<?,?> rr = rootIt.next();
         getRegionLiveChunks(rr, result);
-        Iterator srIt = rr.subregions(true).iterator();
+        Iterator<Region<?,?>> srIt = rr.subregions(true).iterator();
         while (srIt.hasNext()) {
-          Region sr = (Region)srIt.next();
-          getRegionLiveChunks(sr, result);
+          getRegionLiveChunks(srIt.next(), result);
         }
       }
     }
     return result;
   }
 
-  private void getRegionLiveChunks(Region r, List<Chunk> result) {
+  private void getRegionLiveChunks(Region<?,?> r, List<Chunk> result) {
     if (r.getAttributes().getOffHeap()) {
 
       if (r instanceof PartitionedRegion) {
@@ -375,7 +374,6 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     return result;
   }
   
-  @SuppressWarnings("unused")
   public static void debugLog(String msg, boolean logStack) {
     if (logStack) {
       logger.info(msg, new RuntimeException(msg));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/812d51c4/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
index 8b61ab0..de21487 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
@@ -17,14 +17,29 @@
 package com.gemstone.gemfire.internal.offheap;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.OutOfOffHeapMemoryException;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.jayway.awaitility.Awaitility;
 
 @Category(UnitTest.class)
 public class OffHeapStorageJUnitTest {
@@ -76,4 +91,191 @@ public class OffHeapStorageJUnitTest {
     assertEquals(GIGABYTE, OffHeapStorage.parseOffHeapMemorySize("1g"));
     assertEquals(Integer.MAX_VALUE * GIGABYTE, OffHeapStorage.parseOffHeapMemorySize("" + Integer.MAX_VALUE + "g"));
   }
+  @Test
+  public void testCalcMaxSlabSize() {
+    assertEquals(100, OffHeapStorage.calcMaxSlabSize(100L));
+    assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcMaxSlabSize(Long.MAX_VALUE));
+    try {
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "99");
+      assertEquals(99*1024*1024, OffHeapStorage.calcMaxSlabSize(100L*1024*1024));
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "88m");
+      assertEquals(88*1024*1024, OffHeapStorage.calcMaxSlabSize(100L*1024*1024));
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "77M");
+      assertEquals(77*1024*1024, OffHeapStorage.calcMaxSlabSize(100L*1024*1024));
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "1g");
+      assertEquals(1*1024*1024*1024, OffHeapStorage.calcMaxSlabSize(2L*1024*1024*1024));
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "1G");
+      assertEquals(1L*1024*1024*1024, OffHeapStorage.calcMaxSlabSize(2L*1024*1024*1024+1));
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "foobarG");
+      try {
+        OffHeapStorage.calcMaxSlabSize(100);
+        fail("expected IllegalArgumentException");
+      } catch (IllegalArgumentException expected) {
+      }
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "");
+      assertEquals(100, OffHeapStorage.calcMaxSlabSize(100L));
+      assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcMaxSlabSize(Long.MAX_VALUE));
+    } finally {
+      System.clearProperty("gemfire.OFF_HEAP_SLAB_SIZE");
+    }
+  }
+  @Test
+  public void testCreateOffHeapStorage() {
+    System.setProperty(InternalLocator.FORCE_LOCATOR_DM_TYPE, "true");
+    try {
+      assertEquals(null, OffHeapStorage.createOffHeapStorage(null, null, 0, null));
+    } finally {
+      System.clearProperty(InternalLocator.FORCE_LOCATOR_DM_TYPE);
+    }
+    // TODO: mock the StatiticsFactory and InternalDistributedSystem that createOffHeapStorage require
+    Cache c = new CacheFactory().set("mcast-port", "0").create();
+    try {
+      try {
+        OffHeapStorage.createOffHeapStorage(null, c.getDistributedSystem(), OffHeapStorage.MIN_SLAB_SIZE-1, c.getDistributedSystem());
+      } catch (IllegalArgumentException expected) {
+        expected.getMessage().equals("The amount of off heap memory must be at least " + OffHeapStorage.MIN_SLAB_SIZE + " but it was set to " + (OffHeapStorage.MIN_SLAB_SIZE-1));
+      }
+      try {
+        OffHeapStorage.createOffHeapStorage(null, c.getDistributedSystem(), OffHeapStorage.MIN_SLAB_SIZE, null);
+      } catch (IllegalArgumentException expected) {
+        expected.getMessage().equals("InternalDistributedSystem is null");
+      }
+      MemoryAllocator ma = OffHeapStorage.createOffHeapStorage(null, c.getDistributedSystem(), 1024*1024, c.getDistributedSystem());
+      try {
+        OffHeapMemoryStats stats = ma.getStats();
+        assertEquals(1024*1024, stats.getFreeMemory());
+        assertEquals(1024*1024, stats.getMaxMemory());
+        assertEquals(0, stats.getUsedMemory());
+        assertEquals(0, stats.getCompactions());
+        assertEquals(0, stats.getCompactionTime());
+        assertEquals(0, stats.getFragmentation());
+        assertEquals(1, stats.getFragments());
+        assertEquals(1024*1024, stats.getLargestFragment());
+        assertEquals(0, stats.getObjects());
+        assertEquals(0, stats.getReads());
+
+        stats.incFreeMemory(100);
+        assertEquals(1024*1024+100, stats.getFreeMemory());
+        stats.incFreeMemory(-100);
+        assertEquals(1024*1024, stats.getFreeMemory());
+
+        stats.incMaxMemory(100);
+        assertEquals(1024*1024+100, stats.getMaxMemory());
+        stats.incMaxMemory(-100);
+        assertEquals(1024*1024, stats.getMaxMemory());
+
+        stats.incUsedMemory(100);
+        assertEquals(100, stats.getUsedMemory());
+        stats.incUsedMemory(-100);
+        assertEquals(0, stats.getUsedMemory());
+
+        stats.incObjects(100);
+        assertEquals(100, stats.getObjects());
+        stats.incObjects(-100);
+        assertEquals(0, stats.getObjects());
+
+        stats.incReads();
+        assertEquals(1, stats.getReads());
+
+        stats.setFragmentation(100);
+        assertEquals(100, stats.getFragmentation());
+        stats.setFragmentation(0);
+        assertEquals(0, stats.getFragmentation());
+
+        stats.setFragments(2);
+        assertEquals(2, stats.getFragments());
+        stats.setFragments(1);
+        assertEquals(1, stats.getFragments());
+
+        stats.setLargestFragment(100);
+        assertEquals(100, stats.getLargestFragment());
+        stats.setLargestFragment(1024*1024);
+        assertEquals(1024*1024, stats.getLargestFragment());
+
+        boolean originalEnableClockStats = DistributionStats.enableClockStats;
+        DistributionStats.enableClockStats = true;
+        try {
+          long start = stats.startCompaction();
+          while (stats.startCompaction() == start) {
+            Thread.yield();
+          }
+          stats.endCompaction(start);
+          assertEquals(1, stats.getCompactions());
+          assertTrue(stats.getCompactionTime() > 0);
+        } finally {
+          DistributionStats.enableClockStats = originalEnableClockStats;
+        }
+
+        stats.incObjects(100);
+        stats.incUsedMemory(100);
+        stats.setFragmentation(100);
+        OffHeapStorage ohs = (OffHeapStorage) stats;
+        ohs.initialize(new NullOffHeapMemoryStats());
+        assertEquals(0, stats.getFreeMemory());
+        assertEquals(0, stats.getMaxMemory());
+        assertEquals(0, stats.getUsedMemory());
+        assertEquals(0, stats.getCompactions());
+        assertEquals(0, stats.getCompactionTime());
+        assertEquals(0, stats.getFragmentation());
+        assertEquals(0, stats.getFragments());
+        assertEquals(0, stats.getLargestFragment());
+        assertEquals(0, stats.getObjects());
+        assertEquals(0, stats.getReads());
+        System.setProperty(OffHeapStorage.STAY_CONNECTED_ON_OUTOFOFFHEAPMEMORY_PROPERTY, "true");
+        try {
+          try {
+            ma.allocate(1024*1024+1, null);
+            fail("expected OutOfOffHeapMemoryException");
+          } catch (OutOfOffHeapMemoryException expected) {
+          }
+          assertTrue(c.getDistributedSystem().isConnected());
+          try {
+            ma.allocate(1024*1024+1, null);
+            fail("expected OutOfOffHeapMemoryException");
+          } catch (OutOfOffHeapMemoryException expected) {
+          }
+          assertTrue(c.getDistributedSystem().isConnected());
+        } finally {
+          System.clearProperty(OffHeapStorage.STAY_CONNECTED_ON_OUTOFOFFHEAPMEMORY_PROPERTY);
+        }
+        try {
+          ma.allocate(1024*1024+1, null);
+          fail("expected OutOfOffHeapMemoryException");
+        } catch (OutOfOffHeapMemoryException expected) {
+        }
+        try {
+          ma.allocate(1024*1024+1, null);
+          fail("expected OutOfOffHeapMemoryException");
+        } catch (OutOfOffHeapMemoryException expected) {
+        }
+        Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> {
+          return !c.getDistributedSystem().isConnected();
+        });
+
+      } finally {
+        System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
+        try {
+          ma.close();
+        } finally {
+          System.clearProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY);
+        }
+      }
+   } finally {
+      c.close();
+    }
+  }
+  @Test
+  public void testCalcSlabCount() {
+    final long MSS = OffHeapStorage.MIN_SLAB_SIZE;
+    assertEquals(100, OffHeapStorage.calcSlabCount(MSS*4, MSS*4*100));
+    assertEquals(100, OffHeapStorage.calcSlabCount(MSS*4, (MSS*4*100) + (MSS-1)));
+    assertEquals(101, OffHeapStorage.calcSlabCount(MSS*4, (MSS*4*100) + MSS));
+    assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcSlabCount(MSS, MSS * Integer.MAX_VALUE));
+    assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcSlabCount(MSS, (MSS * Integer.MAX_VALUE) + MSS-1));
+    try {
+      OffHeapStorage.calcSlabCount(MSS, (((long)MSS) * Integer.MAX_VALUE) + MSS);
+      fail("Expected IllegalArgumentException");
+    } catch (IllegalArgumentException expected) {
+    }
+  }
 }


[12/50] [abbrv] incubator-geode git commit: Fixing a find-bugs issue

Posted by kl...@apache.org.
Fixing a find-bugs issue

GMSJoinLeave had a static variable that should have been final but
was not.


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

Branch: refs/heads/feature/GEODE-217
Commit: e19fa40ca5079e0ce6930d60b04245dbfa07d6dd
Parents: ae8c475
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Wed Dec 2 10:12:04 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Wed Dec 2 10:13:08 2015 -0800

----------------------------------------------------------------------
 .../internal/membership/gms/membership/GMSJoinLeave.java         | 4 ++--
 .../distributed/internal/membership/MembershipJUnitTest.java     | 4 ++--
 gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java   | 4 ++--
 3 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e19fa40c/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
index 2f8d734..2f9c514 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
@@ -88,7 +88,7 @@ import com.gemstone.gemfire.security.AuthenticationFailedException;
  */
 public class GMSJoinLeave implements JoinLeave, MessageHandler {
   
-  public static String BYPASS_DISCOVERY = "gemfire.bypass-discovery";
+  public static final String BYPASS_DISCOVERY_PROPERTY = "gemfire.bypass-discovery";
 
   /** amount of time to wait for responses to FindCoordinatorRequests */
   private static final int DISCOVERY_TIMEOUT = Integer.getInteger("gemfire.discovery-timeout", 3000);
@@ -221,7 +221,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
   public boolean join() {
 
     try {
-      if (Boolean.getBoolean(BYPASS_DISCOVERY)) {
+      if (Boolean.getBoolean(BYPASS_DISCOVERY_PROPERTY)) {
         synchronized(viewInstallationLock) {
           becomeCoordinator();
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e19fa40c/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
index 91889df..2ce1ca7 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
@@ -185,14 +185,14 @@ public class MembershipJUnitTest {
 
       // start the first membership manager
       try {
-        System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY, "true");
+        System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
         DistributedMembershipListener listener1 = mock(DistributedMembershipListener.class);
         DMStats stats1 = mock(DMStats.class);
         System.out.println("creating 1st membership manager");
         m1 = MemberFactory.newMembershipManager(listener1, config, transport, stats1);
         m1.startEventProcessing();
       } finally {
-        System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY);
+        System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
       }
       
       // start the second membership manager

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e19fa40c/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java b/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
index ad437c9..f3109f3 100644
--- a/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
+++ b/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
@@ -245,11 +245,11 @@ public class DUnitLauncher {
         //Tell the locator it's the first in the system for
         //faster boot-up
         
-        System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY, "true");
+        System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
         try {
           Locator.startLocatorAndDS(locatorPort, locatorLogFile, p);
         } finally {
-          System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY);
+          System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
         }
         
         return null;


[32/50] [abbrv] incubator-geode git commit: GEODE-563: Moving gfsh tests from closed

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
new file mode 100644
index 0000000..1a64faf
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
@@ -0,0 +1,2087 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.PartitionAttributes;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.query.QueryInvalidException;
+import com.gemstone.gemfire.cache.query.data.Portfolio;
+import com.gemstone.gemfire.cache.query.internal.CompiledValue;
+import com.gemstone.gemfire.cache.query.internal.QCompiler;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.management.DistributedRegionMXBean;
+import com.gemstone.gemfire.management.ManagementService;
+import com.gemstone.gemfire.management.ManagerMXBean;
+import com.gemstone.gemfire.management.MemberMXBean;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.CliUtil;
+import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
+import com.gemstone.gemfire.management.internal.cli.domain.DataCommandRequest;
+import com.gemstone.gemfire.management.internal.cli.dto.Car;
+import com.gemstone.gemfire.management.internal.cli.dto.Key1;
+import com.gemstone.gemfire.management.internal.cli.dto.ObjectWithCharAttr;
+import com.gemstone.gemfire.management.internal.cli.dto.Value1;
+import com.gemstone.gemfire.management.internal.cli.dto.Value2;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.json.GfJsonArray;
+import com.gemstone.gemfire.management.internal.cli.json.GfJsonException;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData;
+import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData.SectionResultData;
+import com.gemstone.gemfire.management.internal.cli.result.ResultData;
+import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+import hydra.GsRandom;
+
+import java.io.File;
+import java.io.IOException;
+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.Properties;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * Dunit class for testing gemfire data commands : get, put, remove, select, rebalance
+ *
+ * @author ajayp
+ * @author tushark
+ */
+
+@SuppressWarnings("serial")
+public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+  private static final String REGION_NAME = "FunctionCommandsReplicatedRegion";
+  private static final String REBALANCE_REGION_NAME = "GemfireDataCommandsDUnitTestRegion";
+  private static final String REBALANCE_REGION2_NAME = "GemfireDataCommandsDUnitTestRegion2";
+  private static final String DATA_REGION_NAME = "GemfireDataCommandsTestRegion";
+  private static final String DATA_REGION_NAME_VM1 = "GemfireDataCommandsTestRegion_Vm1";
+  private static final String DATA_REGION_NAME_VM2 = "GemfireDataCommandsTestRegion_Vm2";
+  private static final String DATA_REGION_NAME_PATH = "/GemfireDataCommandsTestRegion";
+  private static final String DATA_REGION_NAME_VM1_PATH = "/GemfireDataCommandsTestRegion_Vm1";
+  private static final String DATA_REGION_NAME_VM2_PATH = "/GemfireDataCommandsTestRegion_Vm2";
+
+  private static final String DATA_PAR_REGION_NAME = "GemfireDataCommandsTestParRegion";
+  private static final String DATA_PAR_REGION_NAME_VM1 = "GemfireDataCommandsTestParRegion_Vm1";
+  private static final String DATA_PAR_REGION_NAME_VM2 = "GemfireDataCommandsTestParRegion_Vm2";
+  private static final String DATA_PAR_REGION_NAME_PATH = "/GemfireDataCommandsTestParRegion";
+  private static final String DATA_PAR_REGION_NAME_VM1_PATH = "/GemfireDataCommandsTestParRegion_Vm1";
+  private static final String DATA_PAR_REGION_NAME_VM2_PATH = "/GemfireDataCommandsTestParRegion_Vm2";
+
+  private static final String DATA_REGION_NAME_CHILD_1 = "ChildRegionRegion1";
+  private static final String DATA_REGION_NAME_CHILD_1_PATH = "/GemfireDataCommandsTestRegion/ChildRegionRegion1";
+  private static final String DATA_REGION_NAME_CHILD_1_2 = "ChildRegionRegion12";
+  private static final String DATA_REGION_NAME_CHILD_1_2_PATH = "/GemfireDataCommandsTestRegion/ChildRegionRegion1/ChildRegionRegion12";
+
+
+  private static final String keyTemplate = "('id':'?','name':'name?')";
+  private static final String valueTemplate = "('stateName':'State?','population':?1,'capitalCity':'capital?','areaInSqKm':?2)";
+  private static final String carTemplate = "\"('attributes':?map,'make':'?make','model':'?model','colors':?list,'attributeSet':?set)\"";
+
+  final static int COUNT = 5;
+
+  public GemfireDataCommandsDUnitTest(String name) {
+    super(name);
+  }
+
+  public static String getMemberId() {
+    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    return cache.getDistributedSystem().getDistributedMember().getId();
+  }
+
+
+  void setupForGetPutRemoveLocateEntry(String testName) {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.NAME_NAME, testName + "Manager");
+    HeadlessGfsh gfsh = createDefaultSetup(props);
+    assertNotNull(gfsh);
+    assertEquals(true, gfsh.isConnectedAndReady());
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+        RegionFactory regionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+        Region dataRegion = regionFactory.create(DATA_REGION_NAME);
+        assertNotNull(dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
+
+        dataRegion = dataRegion.createSubregion(DATA_REGION_NAME_CHILD_1, dataRegion.getAttributes());
+        assertNotNull(dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
+
+        dataRegion = dataRegion.createSubregion(DATA_REGION_NAME_CHILD_1_2, dataRegion.getAttributes());
+        assertNotNull(dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
+
+        dataRegion = regionFactory.create(DATA_REGION_NAME_VM1);
+        assertNotNull(dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
+
+        PartitionAttributes partitionAttrs = new PartitionAttributesFactory().setRedundantCopies(2).create();
+        RegionFactory<Object, Object> partitionRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        partitionRegionFactory.setPartitionAttributes(partitionAttrs);
+        Region dataParRegion = partitionRegionFactory.create(DATA_PAR_REGION_NAME);
+        assertNotNull(dataParRegion);
+        getLogWriter().info("Created Region " + dataParRegion);
+        dataParRegion = partitionRegionFactory.create(DATA_PAR_REGION_NAME_VM1);
+        assertNotNull(dataParRegion);
+        getLogWriter().info("Created Region " + dataParRegion);
+
+      }
+    });
+
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+        RegionFactory regionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+        Region dataRegion = regionFactory.create(DATA_REGION_NAME);
+        assertNotNull(dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
+
+        dataRegion = dataRegion.createSubregion(DATA_REGION_NAME_CHILD_1, dataRegion.getAttributes());
+        assertNotNull(dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
+
+        dataRegion = dataRegion.createSubregion(DATA_REGION_NAME_CHILD_1_2, dataRegion.getAttributes());
+        assertNotNull(dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
+
+        dataRegion = regionFactory.create(DATA_REGION_NAME_VM2);
+        assertNotNull(dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
+
+
+        PartitionAttributes partitionAttrs = new PartitionAttributesFactory().setRedundantCopies(2).create();
+        RegionFactory<Object, Object> partitionRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        partitionRegionFactory.setPartitionAttributes(partitionAttrs);
+        Region dataParRegion = partitionRegionFactory.create(DATA_PAR_REGION_NAME);
+        assertNotNull(dataParRegion);
+        getLogWriter().info("Created Region " + dataParRegion);
+        dataParRegion = partitionRegionFactory.create(DATA_PAR_REGION_NAME_VM2);
+        assertNotNull(dataParRegion);
+        getLogWriter().info("Created Region " + dataParRegion);
+
+      }
+    });
+
+    final String vm1MemberId = (String) vm1.invoke(GemfireDataCommandsDUnitTest.class, "getMemberId");
+    final String vm2MemberId = (String) vm2.invoke(GemfireDataCommandsDUnitTest.class, "getMemberId");
+    getLogWriter().info("Vm1 ID : " + vm1MemberId);
+    getLogWriter().info("Vm2 ID : " + vm2MemberId);
+
+    final VM manager = Host.getHost(0).getVM(0);
+
+    SerializableRunnable checkRegionMBeans = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        final ManagementService service = ManagementService.getManagementService(cache);
+
+        final WaitCriterion waitForMaangerMBean = new WaitCriterion() {
+          @Override
+          public boolean done() {
+            ManagerMXBean bean1 = service.getManagerMXBean();
+            DistributedRegionMXBean bean2 = service.getDistributedRegionMXBean(DATA_REGION_NAME_PATH);
+            if (bean1 == null) {
+              getLogWriter().info("Still probing for ManagerMBean");
+              return false;
+            } else {
+              getLogWriter().info("Still probing for DistributedRegionMXBean=" + bean2);
+              return (bean2 != null);
+            }
+          }
+
+          @Override
+          public String description() {
+            return "Probing for ManagerMBean";
+          }
+        };
+
+        DistributedTestCase.waitForCriterion(waitForMaangerMBean, 30000, 2000, true);
+
+        assertNotNull(service.getMemberMXBean());
+        assertNotNull(service.getManagerMXBean());
+        DistributedRegionMXBean bean = service.getDistributedRegionMXBean(DATA_REGION_NAME_PATH);
+        assertNotNull(bean);
+
+        WaitCriterion waitForRegionMBeans = new WaitCriterion() {
+          @Override
+          public boolean done() {
+
+            DistributedRegionMXBean beans[] = new DistributedRegionMXBean[6];
+            beans[0] = service.getDistributedRegionMXBean(DATA_REGION_NAME_PATH);
+            beans[1] = service.getDistributedRegionMXBean(DATA_REGION_NAME_VM1_PATH);
+            beans[2] = service.getDistributedRegionMXBean(DATA_REGION_NAME_VM2_PATH);
+            beans[3] = service.getDistributedRegionMXBean(DATA_PAR_REGION_NAME_PATH);
+            beans[4] = service.getDistributedRegionMXBean(DATA_PAR_REGION_NAME_VM1_PATH);
+            beans[5] = service.getDistributedRegionMXBean(DATA_PAR_REGION_NAME_VM2_PATH);
+            //SubRegion Bug : Proxy creation has some issues.
+            //beans[6] = service.getDistributedRegionMXBean(DATA_REGION_NAME_CHILD_1_PATH);
+            //beans[7] = service.getDistributedRegionMXBean(DATA_REGION_NAME_CHILD_1_2_PATH);
+            boolean flag = true;
+            for (DistributedRegionMXBean b : beans) {
+              if (b == null) {
+                flag = false;
+                break;
+              }
+            }
+
+            if (!flag) {
+              getLogWriter().info(
+                  "Still probing for regionMbeans " + DATA_REGION_NAME_PATH + "=" + beans[0] + " " + DATA_REGION_NAME_VM1_PATH + "=" + beans[1] + " " + DATA_REGION_NAME_VM2_PATH + "=" + beans[2] + " " + DATA_PAR_REGION_NAME_PATH + "=" + beans[3] + " " + DATA_PAR_REGION_NAME_VM1_PATH + "=" + beans[4] + " " + DATA_PAR_REGION_NAME_VM2_PATH + "=" + beans[5] + " "
+                  //+ DATA_REGION_NAME_CHILD_1_PATH
+                  // +"="+ beans[6]  + " " + DATA_REGION_NAME_CHILD_1_2_PATH
+                  // +"=" + beans[7]
+              );
+              return false;
+            } else {
+              getLogWriter().info(
+                  "Probing complete for regionMbeans " + DATA_REGION_NAME_PATH + "=" + beans[0] + " " + DATA_REGION_NAME_VM1_PATH + "=" + beans[1] + " " + DATA_REGION_NAME_VM2_PATH + "=" + beans[2] + " " + DATA_PAR_REGION_NAME_PATH + "=" + beans[3] + " " + DATA_PAR_REGION_NAME_VM1_PATH + "=" + beans[4] + " " + DATA_PAR_REGION_NAME_VM2_PATH + "=" + beans[5] + " "
+                  //+ DATA_REGION_NAME_CHILD_1_PATH
+                  // +"="+ beans[6]  + " " + DATA_REGION_NAME_CHILD_1_2_PATH
+                  // +"=" + beans[7]
+              );
+              //Fails here Rishi Need Fix here
+              //if(bean1.getMemberCount()==2 && bean1.getMemberCount()==1 && bean1.getMemberCount()==1)              
+              return true;
+              //else{
+              //   getLogWriter().info("Still probing for regionMbeans for aggregation  bean1=" + bean1.getMemberCount() + " bean2="+ bean2.getMemberCount() + " bean3" + bean3.getMemberCount());
+              //   return false;
+              // }
+            }
+          }
+
+          @Override
+          public String description() {
+            return "Probing for regionMbeans";
+          }
+        };
+
+        DistributedTestCase.waitForCriterion(waitForRegionMBeans, 30000, 2000, true);
+
+        String regions[] = {DATA_REGION_NAME_PATH, DATA_REGION_NAME_VM1_PATH, DATA_REGION_NAME_VM2_PATH, DATA_PAR_REGION_NAME_PATH, DATA_PAR_REGION_NAME_VM1_PATH, DATA_PAR_REGION_NAME_VM2_PATH, /*DATA_REGION_NAME_CHILD_1_PATH, DATA_REGION_NAME_CHILD_1_2_PATH*/};
+
+        for (String region : regions) {
+          bean = service.getDistributedRegionMXBean(region);
+          assertNotNull(bean);
+          String[] membersName = bean.getMembers();
+          getLogWriter().info(
+              "Members Array for region " + region + " : " + StringUtils.objectToString(membersName, true, 10));
+          if (bean.getMemberCount() < 1) fail(
+              "Even after waiting mbean reports number of member hosting region " + DATA_REGION_NAME_VM1_PATH + " is less than one");
+          //assertEquals(1, membersName.length); //exists in one members vm1
+        }
+      }
+    };
+    manager.invoke(checkRegionMBeans);
+  }
+
+  void setupForSelect() {
+    setupForGetPutRemoveLocateEntry("setupForSelect");
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+    //To avoid pagination issues and Gfsh waiting for user input
+    executeCommand("set variable --name=APP_FETCH_SIZE --value=" + COUNT);
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = CacheFactory.getAnyInstance();
+        String regions[] = {DATA_PAR_REGION_NAME_PATH, DATA_PAR_REGION_NAME_VM1_PATH, DATA_REGION_NAME_CHILD_1_PATH, DATA_REGION_NAME_CHILD_1_2_PATH};
+        for (String r : regions) {
+          Region dataRegion = cache.getRegion(r);
+          for (int j = 0; j < 10; j++) {
+            dataRegion.put(new Integer(j), new Portfolio(j));
+          }
+        }
+        Region dataRegion = cache.getRegion(DATA_REGION_NAME_PATH);
+        for (int j = 0; j < 10; j++) {
+          dataRegion.put(new Integer(j), new Value1(j));
+        }
+
+        dataRegion = cache.getRegion(DATA_REGION_NAME_VM1_PATH);
+        for (int j = 0; j < 10; j++) {
+          dataRegion.put(new Integer(j), new Value1WithValue2(j));
+        }
+      }
+    });
+
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = CacheFactory.getAnyInstance();
+        String regions[] = {DATA_REGION_NAME_VM2_PATH, DATA_PAR_REGION_NAME_VM2_PATH};
+        for (String r : regions) {
+          Region dataRegion = cache.getRegion(r);
+          for (int j = 0; j < 10; j++) {
+            dataRegion.put(new Integer(j), new Portfolio(j));
+          }
+        }
+      }
+    });
+  }
+
+  private void doQueryRegionsAssociatedMembers(String queryTemplate, int expectedMembers, boolean returnAll,
+      String... regions) {
+    Cache cache = CacheFactory.getAnyInstance();
+
+    String query = queryTemplate;
+    int i = 1;
+    for (String r : regions) {
+      query = query.replace("?" + i, r);
+      i++;
+    }
+    getLogWriter().info("Checking members for query : " + query);
+    QCompiler compiler = new QCompiler();
+    Set<String> regionsInQuery = null;
+    try {
+      CompiledValue compiledQuery = compiler.compileQuery(query);
+      Set regionSet = new HashSet();
+      compiledQuery.getRegionsInQuery(regionSet, null);//GFSH ENV VARIBLES
+      regionsInQuery = Collections.unmodifiableSet(regionSet);
+      getLogWriter().info("Region in query : " + regionsInQuery);
+      if (regionsInQuery.size() > 0) {
+        Set<DistributedMember> members = DataCommands.getQueryRegionsAssociatedMembers(regionsInQuery, cache,
+            returnAll);
+        getLogWriter().info("Members for Region in query : " + members);
+        if (expectedMembers != -1) {
+          assertNotNull(members);
+          assertEquals(expectedMembers, members.size());
+        } else assertEquals(0, members.size());
+      } else {
+        assertEquals(-1, expectedMembers);//Regions do not exist at all
+      }
+    } catch (QueryInvalidException qe) {
+      fail("Invalid Query", qe);
+    }
+  }
+
+  public void doTestGetRegionAssociatedMembersForSelect() {
+    final VM manager = Host.getHost(0).getVM(0);
+    final String queryTemplate1 = "select * from ?1, ?2 ";
+    //final String queryTemplate2 = "select * from ?1, ?2, ?3";    
+    manager.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        doQueryRegionsAssociatedMembers(queryTemplate1, 0, true,
+            new String[]{DATA_REGION_NAME_VM1_PATH, DATA_REGION_NAME_VM2_PATH});
+        doQueryRegionsAssociatedMembers(queryTemplate1, 2, true,
+            new String[]{DATA_REGION_NAME_PATH, DATA_REGION_NAME_CHILD_1_PATH});
+        doQueryRegionsAssociatedMembers(queryTemplate1, 1, false,
+            new String[]{DATA_REGION_NAME_PATH, DATA_REGION_NAME_CHILD_1_PATH});
+        doQueryRegionsAssociatedMembers(queryTemplate1, 1, true,
+            new String[]{DATA_REGION_NAME_VM1_PATH, DATA_REGION_NAME_PATH});
+        doQueryRegionsAssociatedMembers(queryTemplate1, 1, false,
+            new String[]{DATA_REGION_NAME_VM1_PATH, DATA_REGION_NAME_PATH});
+        doQueryRegionsAssociatedMembers(queryTemplate1, 1, true,
+            new String[]{DATA_REGION_NAME_VM2_PATH, DATA_REGION_NAME_PATH});
+        doQueryRegionsAssociatedMembers(queryTemplate1, 1, false,
+            new String[]{DATA_REGION_NAME_VM2_PATH, DATA_REGION_NAME_PATH});
+        doQueryRegionsAssociatedMembers(queryTemplate1, 0, true,
+            new String[]{DATA_PAR_REGION_NAME_VM2_PATH, DATA_PAR_REGION_NAME_VM1_PATH});
+        doQueryRegionsAssociatedMembers(queryTemplate1, 0, false,
+            new String[]{DATA_PAR_REGION_NAME_VM2_PATH, DATA_PAR_REGION_NAME_VM1_PATH});
+        doQueryRegionsAssociatedMembers(queryTemplate1, -1, true,
+            new String[]{DATA_PAR_REGION_NAME_VM2_PATH, "/jfgkdfjgkd"}); //one wrong region
+        doQueryRegionsAssociatedMembers(queryTemplate1, -1, false,
+            new String[]{DATA_PAR_REGION_NAME_VM2_PATH, "/jfgkdfjgkd"}); //one wrong region
+        doQueryRegionsAssociatedMembers(queryTemplate1, -1, true,
+            new String[]{"/dhgfdhgf", "/dhgddhd"}); // both regions wrong
+        doQueryRegionsAssociatedMembers(queryTemplate1, -1, false,
+            new String[]{"/dhgfdhgf", "/dhgddhd"}); // both regions wrong
+      }
+    });
+  }
+
+  public void doTestSelectProjection() {
+    Random random = new Random(System.nanoTime());
+    int randomInteger = random.nextInt(COUNT);
+    String query = "query --query=\"select ID , status , createTime , pk, floatMinValue from " + DATA_PAR_REGION_NAME_PATH + " where ID <= " + randomInteger + "\" --interactive=false";
+    CommandResult cmdResult = executeCommand(query);
+    printCommandOutput(cmdResult);
+    validateSelectResult(cmdResult, true, (randomInteger + 1),
+        new String[]{"ID", "status", "createTime", "pk", "floatMinValue"});
+  }
+
+  public void doTestSelectProjectionProcessCommand() {
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Random random = new Random(System.nanoTime());
+        int randomInteger = random.nextInt(COUNT);
+        String query = "query --query=\"select ID , status , createTime , pk, floatMinValue from " + DATA_PAR_REGION_NAME_PATH + " where ID <= " + randomInteger + "\" --interactive=false";
+        ManagementService service = ManagementService.getExistingManagementService(getCache());
+        MemberMXBean member = service.getMemberMXBean();
+        String cmdResult = member.processCommand(query);
+        assertNotNull(cmdResult);
+        getLogWriter().info("Text Command Output : " + cmdResult);
+      }
+    });
+  }
+
+  public void doTestSelectProjectionWithNestedField() {
+    Random random = new Random(System.nanoTime());
+    int randomInteger = random.nextInt(COUNT);
+    String query = "query --query=\"select employeeId, name, department, value2 from " + DATA_REGION_NAME_VM1_PATH + " where employeeId <= " + randomInteger + "\" --interactive=false";
+    CommandResult cmdResult = executeCommand(query);
+    printCommandOutput(cmdResult);
+    String expectedCols[] = {"employeeId", "name", "department", "value2"};
+    validateSelectResult(cmdResult, true, (randomInteger + 1), expectedCols);
+
+    // Test with collections
+    query = "query --query=\"select ID , status , createTime , pk, floatMinValue, collectionHolderMap from " + DATA_PAR_REGION_NAME_PATH + " where ID <= " + randomInteger + "\" --interactive=false";
+    cmdResult = executeCommand(query);
+    printCommandOutput(cmdResult);
+    expectedCols = new String[]{"ID", "status", "createTime", "pk", "floatMinValue", "collectionHolderMap"};
+    validateSelectResult(cmdResult, true, (randomInteger + 1), expectedCols);
+  }
+
+  public void doTestSelectBeansAsResult() {
+    Random random = new Random(System.nanoTime());
+    int randomInteger = random.nextInt(COUNT);
+    String query = "query --query=\"select * from " + DATA_REGION_NAME_PATH + " where employeeId <= " + randomInteger + "\" --interactive=false";
+    CommandResult cmdResult = executeCommand(query);
+    printCommandOutput(cmdResult);
+    String expectedCols[] = {"name", "lastName", "department", "age", "employeeId"};
+    validateSelectResult(cmdResult, true, (randomInteger + 1), expectedCols);
+  }
+
+  public void doTestSelectBeansWithNestedFieldAsResult() {
+    Random random = new Random(System.nanoTime());
+    int randomInteger = random.nextInt(COUNT);
+    String query = "query --query=\"select employeeId, name, department, value2 from " + DATA_REGION_NAME_VM1_PATH + " where employeeId <= " + randomInteger + "\" --interactive=false";
+    CommandResult cmdResult = executeCommand(query);
+    printCommandOutput(cmdResult);
+    String expectedCols[] = {"employeeId", "name", "department", "value2"};
+    validateSelectResult(cmdResult, true, (randomInteger + 1), expectedCols);
+  }
+
+  public void doTestSelectWithGfshEnvVariables() {
+    Random random = new Random(System.nanoTime());
+    int randomInteger = random.nextInt(COUNT);
+    String query = "query --query=\"select ID , status , createTime , pk, floatMinValue from ${DATA_REGION} where ID <= ${PORTFOLIO_ID}" + " and status='${STATUS}'" + "\" --interactive=false";
+    executeCommand("set variable --name=DATA_REGION --value=" + DATA_REGION_NAME_PATH);
+    executeCommand("set variable --name=PORTFOLIO_ID --value=" + randomInteger);
+    executeCommand("set variable --name=STATUS --value=" + (new GsRandom().nextBoolean() ? "active" : "inactive"));
+    CommandResult cmdResult = executeCommand(query);
+    printCommandOutput(cmdResult);
+    validateSelectResult(cmdResult, true, -1, null);
+    ExpectedException ex = addExpectedException(QueryInvalidException.class.getSimpleName(), Host.getHost(0).getVM(0));
+    try {
+      query = "query --query=\"select ID , status , createTime , pk, floatMinValue from ${DATA_REGION2} where ID <= ${PORTFOLIO_ID2}" + " and status='${STATUS2}'" + "\" --interactive=false";
+      cmdResult = executeCommand(query);
+      printCommandOutput(cmdResult);
+      validateSelectResult(cmdResult, false, 0, null);
+    } finally {
+      ex.remove();
+    }
+  }
+
+  public void doTestBug48013() {
+    String query = "query --query=\"SELECT e FROM " + DATA_REGION_NAME_PATH + ".entries e\" --interactive=false";
+    CommandResult cmdResult = executeCommand(query);
+    printCommandOutput(cmdResult);
+    validateResult(cmdResult, true);
+  }
+
+
+  public void testSelectCommand() {
+    setupForSelect();
+    doTestGetRegionAssociatedMembersForSelect();
+    doTestSelectWithGfshEnvVariables();
+    doTestSelectProjection();
+    doTestBug48013();
+    doTestSelectProjectionProcessCommand();
+    doTestSelectProjectionWithNestedField();
+    doTestSelectBeansAsResult();
+    doTestSelectBeansWithNestedFieldAsResult();
+  }
+
+
+  public void testPrimitivesWithDataCommands() {
+    setupForGetPutRemoveLocateEntry("testPrimitives");
+    Byte byteKey = Byte.parseByte("41");
+    Byte byteValue = Byte.parseByte("31");
+    Short shortKey = Short.parseShort("123");
+    Short shortValue = Short.parseShort("121");
+    Integer integerKey = Integer.parseInt("123456");
+    Integer integerValue = Integer.parseInt("12345678");
+    Float floatKey = Float.valueOf("12432.2325");
+    Float flaotValue = Float.valueOf("111111.1111");
+    Double doubleKey = Double.valueOf("12432.235425");
+    Double doubleValue = Double.valueOf("111111.111111");
+
+    getLogWriter().info("Testing Byte Wrappers");
+    testGetPutLocateEntryFromShellAndGemfire(byteKey, byteValue, Byte.class, true, true);
+    getLogWriter().info("Testing Short Wrappers");
+    testGetPutLocateEntryFromShellAndGemfire(shortKey, shortValue, Short.class, true, true);
+    getLogWriter().info("Testing Integer Wrappers");
+    testGetPutLocateEntryFromShellAndGemfire(integerKey, integerValue, Integer.class, true, true);
+    getLogWriter().info("Testing Float Wrappers");
+    testGetPutLocateEntryFromShellAndGemfire(floatKey, flaotValue, Float.class, true, true);
+    getLogWriter().info("Testing Double Wrappers");
+    testGetPutLocateEntryFromShellAndGemfire(doubleKey, doubleValue, Double.class, true, true);
+  }
+
+  private void testGetPutLocateEntryFromShellAndGemfire(final Object key, final Object value, Class klass,
+      boolean addRegionPath, boolean expResult) {
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+    SerializableRunnable putTask = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        region.clear();
+        region.put(key, value);
+      }
+    };
+
+    SerializableRunnable getTask = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        assertEquals(true, region.containsKey(key));
+        assertEquals(value, region.get(key));
+      }
+    };
+
+    SerializableRunnable removeTask = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        assertEquals(true, region.containsKey(key));
+        region.remove(key);
+      }
+    };
+
+
+    SerializableRunnable clearTask = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        region.clear();
+      }
+    };
+
+    String getCommand = "get --key=" + key + " --key-class=" + klass.getCanonicalName() + " --value-class=" + klass.getCanonicalName();
+    if (addRegionPath) getCommand += " --region=" + DATA_REGION_NAME_PATH;
+
+    String locateEntryCommand = "locate entry --key=" + key + " --key-class=" + klass.getCanonicalName() + " --value-class=" + klass.getCanonicalName();
+    if (addRegionPath) locateEntryCommand += " --region=" + DATA_REGION_NAME_PATH;
+
+    String removeCommand = "remove --key=" + key + " --key-class=" + klass.getCanonicalName();
+    if (addRegionPath) removeCommand += " --region=" + DATA_REGION_NAME_PATH;
+    String putCommand = "put --key=" + key + " --key-class=" + klass.getCanonicalName() + " --value=" + value + " --value-class=" + klass.getCanonicalName();
+    if (addRegionPath) putCommand += " --region=" + DATA_REGION_NAME_PATH;
+
+    if (expResult) {
+      //Do put from shell check gemfire get do gemfire remove
+      CommandResult cmdResult = executeCommand(putCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, true);
+      vm1.invoke(getTask);
+      vm1.invoke(removeTask);
+
+      vm1.invoke(clearTask);
+
+      //Do put from gemfire check from shell do gemfire remove
+      vm1.invoke(putTask);
+      cmdResult = executeCommand(getCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, true);
+      cmdResult = executeCommand(locateEntryCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, true);
+      vm1.invoke(removeTask);
+
+      vm1.invoke(clearTask);
+
+      //Do put from shell check from gemfire do remove from shell get from shell exepct false    
+      cmdResult = executeCommand(putCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, true);
+      vm1.invoke(getTask);
+      cmdResult = executeCommand(removeCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, true);
+      cmdResult = executeCommand(getCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, false);
+      cmdResult = executeCommand(locateEntryCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, false);
+    } else {
+      //Do put from shell check gemfire get do gemfire remove
+      CommandResult cmdResult = executeCommand(putCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, false);
+      vm1.invoke(clearTask);
+
+      //Do put from gemfire check from shell do gemfire remove
+      vm1.invoke(putTask);
+      cmdResult = executeCommand(getCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, false);
+      cmdResult = executeCommand(locateEntryCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, false);
+      vm1.invoke(removeTask);
+      vm1.invoke(clearTask);
+
+      //Do put from shell check from gemfire do remove from shell get from shell exepct false    
+      cmdResult = executeCommand(putCommand);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, false);
+    }
+  }
+
+  public void testSimplePutCommand() {
+    final String keyPrefix = "testKey";
+    final String valuePrefix = "testValue";
+
+    setupForGetPutRemoveLocateEntry("tesSimplePut");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+    for (int i = 0; i < COUNT; i++) {
+      String command = "put";
+      String key = keyPrefix + i;
+      String value = valuePrefix + i;
+      command = command + " " + "--key=" + key + " --value=" + value + " --region=" + DATA_REGION_NAME_PATH;
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      validateResult(cmdResult, true);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    }
+
+    //Bug : 51587 : GFSH command failing when ; is present in either key or value in put operation
+    String command = "put";
+    String key = keyPrefix + "\\;" + COUNT;
+    String value = valuePrefix + "\\;" + COUNT;
+    command = command + " " + "--key=" + key + " --value=" + value + " --region=" + DATA_REGION_NAME_PATH;
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+    validateResult(cmdResult, true);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    SerializableRunnable checkPutKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        for (int i = 0; i < COUNT; i++) {
+          String key = keyPrefix + i;
+          assertEquals(true, region.containsKey(key));
+        }
+        //Validation for Bug 51587
+        String key = keyPrefix + "\\;" + COUNT;
+        assertEquals(true, region.containsKey(key));
+      }
+    };
+
+    vm1.invoke(checkPutKeys);
+    vm2.invoke(checkPutKeys);
+  }
+
+  private void validateResult(CommandResult cmdResult, boolean expected) {
+    if (ResultData.TYPE_COMPOSITE.equals(cmdResult.getType())) {
+      CompositeResultData rd = (CompositeResultData) cmdResult.getResultData();
+      SectionResultData section = rd.retrieveSectionByIndex(0);
+      boolean result = (Boolean) section.retrieveObject("Result");
+      assertEquals(expected, result);
+    } else fail("Expected CompositeResult Returned Result Type " + cmdResult.getType());
+  }
+
+  private void validateLocationsResult(CommandResult cmdResult, int expected) {
+    if (ResultData.TYPE_COMPOSITE.equals(cmdResult.getType())) {
+      CompositeResultData rd = (CompositeResultData) cmdResult.getResultData();
+      SectionResultData section = rd.retrieveSectionByIndex(0);
+      int result = (Integer) section.retrieveObject("Locations Found");
+      assertEquals(expected, result);
+    } else fail("Expected CompositeResult Returned Result Type " + cmdResult.getType());
+  }
+
+  private void validateJSONGetResult(CommandResult cmdResult, String[] expectedCols) {
+    CompositeResultData rd = (CompositeResultData) cmdResult.getResultData();
+    SectionResultData section = rd.retrieveSectionByIndex(0);
+    TabularResultData table = section.retrieveTableByIndex(0);
+    GfJsonArray array = table.getHeaders();
+    assertEquals(expectedCols.length, array.size());
+    try {
+      for (String col : expectedCols) {
+        boolean found = false;
+        getLogWriter().info("Validating column " + col);
+        for (int i = 0; i < array.size(); i++) {
+          String header = (String) array.get(i);
+          if (col.equals(header)) found = true;
+        }
+        assertEquals(true, found);
+      }
+    } catch (GfJsonException e) {
+      fail("Error accessing table data", e);
+    }
+  }
+
+  private void validateJSONGetResultValues(CommandResult cmdResult, String[] expectedCols) {
+    CompositeResultData rd = (CompositeResultData) cmdResult.getResultData();
+    SectionResultData section = rd.retrieveSectionByIndex(0);
+    TabularResultData table = section.retrieveTableByIndex(0);
+    GfJsonArray array = table.getHeaders();
+    assertEquals(expectedCols.length, array.size());
+    try {
+      for (String col : expectedCols) {
+        boolean found = false;
+        getLogWriter().info("Validating column " + col);
+        for (int i = 0; i < array.size(); i++) {
+          String header = (String) array.get(i);
+          if (col.equals(header)) found = true;
+        }
+        assertEquals(true, found);
+
+        List<String> values = table.retrieveAllValues(col);
+        for (String value : values) {
+          assertNotSame("null", value);
+        }
+
+      }
+    } catch (GfJsonException e) {
+      fail("Error accessing table data", e);
+    }
+  }
+
+  private void validateSelectResult(CommandResult cmdResult, boolean expectedFlag, int expectedRows, String[] cols) {
+    if (ResultData.TYPE_COMPOSITE.equals(cmdResult.getType())) {
+      CompositeResultData rd = (CompositeResultData) cmdResult.getResultData();
+      SectionResultData section = rd.retrieveSectionByIndex(0);
+      boolean result = (Boolean) section.retrieveObject("Result");
+      assertEquals(expectedFlag, result);
+      if (expectedFlag && expectedRows != -1) {
+        int rowsReturned = (Integer) section.retrieveObject("Rows");
+        assertEquals(expectedRows, rowsReturned);
+        if (rowsReturned > 0 && cols != null) {
+          try {
+            TabularResultData table = section.retrieveTableByIndex(0);
+            GfJsonArray array = table.getHeaders();
+            assertEquals(cols.length, array.size());
+            for (String col : cols) {
+              boolean found = false;
+              getLogWriter().info("Validating column " + col);
+              for (int i = 0; i < array.size(); i++) {
+                String header = (String) array.get(i);
+                if (col.equals(header)) found = true;
+              }
+              assertEquals(true, found);
+            }
+          } catch (GfJsonException e) {
+            fail("Error accessing table data", e);
+          }
+        }
+      }
+    } else fail("Expected CompositeResult Returned Result Type " + cmdResult.getType());
+  }
+
+
+  public void testSimplePutIfAbsentCommand() {
+    final String keyPrefix = "testKey";
+    final String valuePrefix = "testValue";
+
+    setupForGetPutRemoveLocateEntry("testSimplePutIfAbsent");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+
+    SerializableRunnable putKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        region.clear();
+        for (int i = 0; i < COUNT; i++) {
+          String key = keyPrefix + i;
+          String value = valuePrefix + i;
+          region.put(key, value);
+        }
+        assertEquals(COUNT, region.size());
+      }
+    };
+    vm1.invoke(putKeys);
+
+    for (int i = 0; i < COUNT; i++) {
+      String command = "put";
+      String key = keyPrefix + i;
+      String value = valuePrefix + i + i;
+      command = command + " " + "--key=" + key + " --value=" + value + " --region=" + DATA_REGION_NAME_PATH + " --skip-if-exists=true";
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+    }
+
+    SerializableRunnable checkPutIfAbsentKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        for (int i = COUNT + 1; i < COUNT; i++) {
+          String key = keyPrefix + i;
+          String notExpectedvalue = valuePrefix + i + i;
+          String expectedvalue = valuePrefix + i;
+          String value = (String) region.get(key);
+          assertNotNull(value);
+          assertEquals(value, expectedvalue);
+          if (value.equals(notExpectedvalue)) fail("Value is overriden even if put-If-absent was true");
+        }
+      }
+    };
+
+    vm1.invoke(checkPutIfAbsentKeys);
+    vm2.invoke(checkPutIfAbsentKeys);
+
+  }
+
+  public void testSimpleRemoveCommand() {
+    final String keyPrefix = "testKey";
+    final String valuePrefix = "testValue";
+
+    setupForGetPutRemoveLocateEntry("testSimpleRemove");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+
+    SerializableRunnable putKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        region.clear();
+        for (int i = 0; i < COUNT; i++) {
+          String key = keyPrefix + i;
+          String value = valuePrefix + i;
+          region.put(key, value);
+        }
+        assertEquals(COUNT, region.size());
+      }
+    };
+    vm1.invoke(putKeys);
+
+    for (int i = 0; i < COUNT; i++) {
+      String command = "remove";
+      String key = keyPrefix + i;
+      String value = valuePrefix + i;
+      command = command + " " + "--key=" + key + " --region=" + DATA_REGION_NAME_PATH;
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+    }
+
+    SerializableRunnable checkRemoveKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        for (int i = 0; i < COUNT; i++) {
+          String key = keyPrefix + i;
+          assertEquals(false, region.containsKey(key));
+        }
+        assertEquals(0, region.size());
+      }
+    };
+
+    vm1.invoke(checkRemoveKeys);
+    vm2.invoke(checkRemoveKeys);
+  }
+
+  public void testSimpleGetLocateEntryCommand() {
+    final String keyPrefix = "testKey";
+    final String valuePrefix = "testValue";
+
+    setupForGetPutRemoveLocateEntry("testSimpleGetLocateEntry");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+    SerializableRunnable putKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        region.clear();
+        for (int i = 0; i < COUNT; i++) {
+          String key = keyPrefix + i;
+          String value = valuePrefix + i;
+          region.put(key, value);
+        }
+      }
+    };
+
+    vm1.invoke(putKeys);
+    for (int i = 0; i < COUNT; i++) {
+      String command = "get";
+      String key = keyPrefix + i;
+      String value = valuePrefix + i;
+      command = command + " " + "--key=" + key + " --region=" + DATA_REGION_NAME_PATH;
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+
+      command = "locate entry";
+      command = command + " " + "--key=" + key + " --region=" + DATA_REGION_NAME_PATH;
+      cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+
+    }
+
+  }
+
+  public void testRecursiveLocateEntryCommand() {
+    final String keyPrefix = "testKey";
+    final String valuePrefix = "testValue";
+
+    setupForGetPutRemoveLocateEntry("testRecursiveLocateEntry");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+    SerializableRunnable putKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        Region region2 = cache.getRegion(DATA_REGION_NAME_CHILD_1_PATH);
+        Region region3 = cache.getRegion(DATA_REGION_NAME_CHILD_1_2_PATH);
+        assertNotNull(region);
+        region.clear();
+        for (int i = 0; i < COUNT; i++) {
+          String key = keyPrefix + i;
+          String value = valuePrefix + i;
+          region.put(key, value);
+          region2.put(key, value);
+          region3.put(key, value);
+        }
+      }
+    };
+
+    vm1.invoke(putKeys);
+    for (int i = 0; i < COUNT; i++) {
+      String key = keyPrefix + i;
+      String command = "locate entry";
+      command = command + " " + "--key=" + key + " --region=" + DATA_REGION_NAME_PATH + " --recursive=true";
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+      validateLocationsResult(cmdResult, 6); //3 Regions X 2 members = 6
+    }
+
+  }
+
+  public void testGetLocateEntryFromRegionOnDifferentVM() {
+    final String keyPrefix = "testKey";
+    final String valuePrefix = "testValue";
+
+    setupForGetPutRemoveLocateEntry("testGetLocateEntryFromRegionOnDifferentVM");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+    SerializableRunnable putKeys1 = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_VM1_PATH);
+        Region parRegion = cache.getRegion(DATA_PAR_REGION_NAME_VM1_PATH);
+        assertNotNull(region);
+        region.clear();
+        for (int i = 0; i < COUNT; i++) {
+          if (i % 2 == 0) {
+            String key = keyPrefix + i;
+            String value = valuePrefix + i;
+            region.put(key, value);
+            parRegion.put(key, value);
+          }
+        }
+      }
+    };
+
+    SerializableRunnable putKeys2 = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_VM2_PATH);
+        Region parRegion = cache.getRegion(DATA_PAR_REGION_NAME_VM2_PATH);
+        assertNotNull(region);
+        region.clear();
+        for (int i = 0; i < COUNT; i++) {
+          if (i % 2 != 0) {
+            String key = keyPrefix + i;
+            String value = valuePrefix + i;
+            region.put(key, value);
+            parRegion.put(key, value);
+          }
+        }
+      }
+    };
+
+    vm1.invoke(putKeys1);
+    vm2.invoke(putKeys2);
+    for (int i = 0; i < COUNT; i++) {
+      String command = "get";
+      String key = keyPrefix + i;
+      String value = valuePrefix + i;
+      if (i % 2 == 0) command = command + " " + "--key=" + key + " --region=" + DATA_REGION_NAME_VM1_PATH;
+      else if (i % 2 == 1) command = command + " " + "--key=" + key + " --region=" + DATA_REGION_NAME_VM2_PATH;
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+
+      command = "locate entry";
+      if (i % 2 == 0) command = command + " " + "--key=" + key + " --region=" + DATA_REGION_NAME_VM1_PATH;
+      else if (i % 2 == 1) command = command + " " + "--key=" + key + " --region=" + DATA_REGION_NAME_VM2_PATH;
+      cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+
+
+      command = "locate entry";
+      if (i % 2 == 0) command = command + " " + "--key=" + key + " --region=" + DATA_PAR_REGION_NAME_VM1_PATH;
+      else if (i % 2 == 1) command = command + " " + "--key=" + key + " --region=" + DATA_PAR_REGION_NAME_VM2_PATH;
+      cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+      validateLocationsResult(cmdResult, 1); //1 Regions X (2-1) 2 Copies but redundancy not satisfied = 1
+    }
+  }
+
+  public void testGetLocateEntryLocationsForPR() {
+    final String keyPrefix = "testKey";
+    final String valuePrefix = "testValue";
+
+    setupForGetPutRemoveLocateEntry("testGetLocateEntryLocationsForPR");
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+    SerializableRunnable putKeys1 = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_PAR_REGION_NAME_PATH);
+        assertNotNull(region);
+        for (int i = 0; i < COUNT; i++) {
+          String key = keyPrefix + i;
+          String value = valuePrefix + i;
+          region.put(key, value);
+        }
+      }
+    };
+
+    vm1.invoke(putKeys1);
+
+    for (int i = 0; i < COUNT; i++) {
+      String key = keyPrefix + i;
+      String command = "locate entry";
+      command = command + " " + "--key=" + key + " --region=" + DATA_PAR_REGION_NAME_PATH;
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+      validateLocationsResult(cmdResult, 2); //2 Members
+    }
+  }
+
+  public void testPutFromRegionOnDifferentVM() {
+    final String keyPrefix = "testKey";
+    final String valuePrefix = "testValue";
+
+    setupForGetPutRemoveLocateEntry("testPutFromRegionOnDifferentVM");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+    for (int i = 0; i < COUNT; i++) {
+      String command = "put";
+      String key = keyPrefix + i;
+      String value = valuePrefix + i;
+      if (i % 2 == 0)
+        command = command + " " + "--key=" + key + " --value=" + value + " --region=" + DATA_REGION_NAME_VM1_PATH;
+      else command = command + " " + "--key=" + key + " --value=" + value + " --region=" + DATA_REGION_NAME_VM2_PATH;
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+    }
+
+    SerializableRunnable checkPutKeysInVM1 = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_VM1_PATH);
+        assertNotNull(region);
+        for (int i = 0; i < COUNT; i++) {
+          if (i % 2 == 0) {
+            String key = keyPrefix + i;
+            assertEquals(true, region.containsKey(key));
+          }
+        }
+      }
+    };
+
+    SerializableRunnable checkPutKeysInVM2 = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_VM2_PATH);
+        assertNotNull(region);
+        for (int i = 0; i < COUNT; i++) {
+          if (i % 2 != 0) {
+            String key = keyPrefix + i;
+            assertEquals(true, region.containsKey(key));
+          }
+        }
+      }
+    };
+
+    vm1.invoke(checkPutKeysInVM1);
+    vm2.invoke(checkPutKeysInVM2);
+  }
+
+  public void testGetLocateEntryJsonKeys() {
+    final String keyPrefix = "testKey";
+
+    setupForGetPutRemoveLocateEntry("testGetLocateEntryJsonKeys");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+    SerializableRunnable putKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        region.clear();
+        for (int i = 0; i < COUNT; i++) {
+          String keyString = keyPrefix + i;
+          Key1 key = new Key1();
+          key.setId(keyString);
+          key.setName("name" + keyString);
+          Value2 value2 = new Value2();
+          value2.setStateName("State" + keyString);
+          value2.setCapitalCity("capital" + keyString);
+          value2.setPopulation(i * 100);
+          value2.setAreaInSqKm(i * 100.4365);
+          region.put(key, value2);
+        }
+
+        //Added for Bug #51175
+        List<String> colors = new ArrayList<String>();
+        colors.add("White");
+        colors.add("Red");
+        colors.add("Blue");
+        Map<String, String> attrMap = new HashMap<String, String>();
+        attrMap.put("power", "90hp");
+        attrMap.put("turningRadius", "4mtr");
+        attrMap.put("engineCapacity", "1248cc");
+        attrMap.put("turboGeometry", "VGT");
+
+        Set<String> attrSet = new HashSet<String>();
+        attrSet.add("power");
+        attrSet.add("turningRadius");
+
+        for (int i = COUNT; i < COUNT + 5; i++) {
+          String keyString = keyPrefix + i;
+          Key1 key = new Key1();
+          key.setId(keyString);
+          key.setName("name" + keyString);
+          Car car = new Car();
+          car.setMake("Make" + keyString);
+          car.setModel("Model" + keyString);
+          car.setColors(colors);
+          car.setAttributes(attrMap);
+          car.setAttributeSet(attrSet);
+          region.put(key, car);
+        }
+      }
+    };
+
+    String expectedCols[] = {"id", "name", "stateName", "capitalCity", "population", "areaInSqKm"};
+    vm1.invoke(putKeys);
+    for (int i = 0; i < COUNT; i++) {
+      String command = "get";
+      String keyString = keyPrefix + i;
+      String population = "" + i * 100;
+      String area = "" + i * (100.4365);
+      String keyJson = keyTemplate.replaceAll("\\?", keyString);
+      String valueJson = valueTemplate.replaceAll("\\?1", population);
+      valueJson = valueJson.replaceAll("\\?2", area);
+      valueJson = valueJson.replaceAll("\\?", keyString);
+      getLogWriter().info("Getting key with json key : " + keyJson);
+      command = command + " " + "--key=" + keyJson + " --region=" + DATA_REGION_NAME_PATH + " --key-class=" + Key1.class.getCanonicalName();
+      command = command + " --value-class=" + Value2.class.getCanonicalName();
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+      validateJSONGetResult(cmdResult, expectedCols);
+
+      command = "locate entry";
+      command = command + " " + "--key=" + keyJson + " --region=" + DATA_REGION_NAME_PATH + " --key-class=" + Key1.class.getCanonicalName();
+      command = command + " --value-class=" + Value2.class.getCanonicalName();
+      cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+    }
+
+    //Added for Bug #51175
+    expectedCols = new String[]{"id", "name", "attributes", "make", "model", "colors", "attributeSet"};
+    for (int i = COUNT; i < COUNT + 5; i++) {
+      String command = "get";
+      String keyString = keyPrefix + i;
+      String population = "" + i * 100;
+      String area = "" + i * (100.4365);
+      String keyJson = keyTemplate.replaceAll("\\?", keyString);
+      String valueJson = valueTemplate.replaceAll("\\?1", population);
+      valueJson = valueJson.replaceAll("\\?2", area);
+      valueJson = valueJson.replaceAll("\\?", keyString);
+      getLogWriter().info("Getting key with json key : " + keyJson);
+      command = command + " " + "--key=" + keyJson + " --region=" + DATA_REGION_NAME_PATH + " --key-class=" + Key1.class.getCanonicalName();
+      command = command + " --value-class=" + Value2.class.getCanonicalName();
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+      //validateJSONGetResult(cmdResult, expectedCols);
+      validateJSONGetResultValues(cmdResult, expectedCols);
+
+      command = "locate entry";
+      command = command + " " + "--key=" + keyJson + " --region=" + DATA_REGION_NAME_PATH + " --key-class=" + Key1.class.getCanonicalName();
+      command = command + " --value-class=" + Value2.class.getCanonicalName();
+      cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+    }
+  }
+
+  public void testPutJsonKeys() {
+    final String keyPrefix = "testKey";
+
+    setupForGetPutRemoveLocateEntry("testPutJsonKeys");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+    for (int i = 0; i < COUNT; i++) {
+      String command = "put";
+      String keyString = keyPrefix + i;
+      String population = "" + i * 100;
+      String area = "" + i * (100.4365);
+      String keyJson = keyTemplate.replaceAll("\\?", keyString);
+      String valueJson = valueTemplate.replaceAll("\\?1", population);
+      valueJson = valueJson.replaceAll("\\?2", area);
+      valueJson = valueJson.replaceAll("\\?", keyString);
+      getLogWriter().info("Putting key with json key : " + keyJson);
+      getLogWriter().info("Putting key with json valye : " + valueJson);
+      command = command + " " + "--key=" + keyJson + " --value=" + valueJson + " --region=" + DATA_REGION_NAME_PATH;
+      command = command + " --key-class=" + Key1.class.getCanonicalName() + " --value-class=" + Value2.class.getCanonicalName();
+      ;
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+    }
+
+    //Bug #51175
+    for (int i = COUNT; i < COUNT + 5; i++) {
+      String command = "put";
+      String keyString = keyPrefix + i;
+      String id = "" + i * 100;
+      String make = "" + i * (100.4365);
+      String model = "" + i * (100.4365);
+      String list = "['red','white','blue']";
+      String set = "['red','white','blue']";
+      String map = "{'power':'90hp'}";
+      String keyJson = keyTemplate.replaceAll("\\?", keyString);
+
+      String valueJson = carTemplate.replaceAll("\\?make", make);
+      valueJson = valueJson.replaceAll("\\?model", model);
+      valueJson = valueJson.replaceAll("\\?list", list);
+      valueJson = valueJson.replaceAll("\\?set", set);
+      valueJson = valueJson.replaceAll("\\?map", map);
+
+      getLogWriter().info("Putting key with json key : " + keyJson);
+      getLogWriter().info("Putting key with json valye : " + valueJson);
+      command = command + " " + "--key=" + keyJson + " --value=" + valueJson + " --region=" + DATA_REGION_NAME_PATH;
+      command = command + " --key-class=" + Key1.class.getCanonicalName() + " --value-class=" + Car.class.getCanonicalName();
+      ;
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+    }
+
+    SerializableRunnable checkPutKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        for (int i = 0; i < COUNT + 5; i++) {
+          String keyString = keyPrefix + i;
+          Key1 key = new Key1();
+          key.setId(keyString);
+          key.setName("name" + keyString);
+          assertEquals(true, region.containsKey(key));
+
+          //Bug #51175
+          if (i >= COUNT) {
+            Car car = (Car) region.get(key);
+            assertNotNull(car.getAttributes());
+            assertNotNull(car.getAttributeSet());
+            assertNotNull(car.getColors());
+          }
+
+        }
+      }
+    };
+
+    vm1.invoke(checkPutKeys);
+    vm2.invoke(checkPutKeys);
+
+    doBugCheck50449();
+  }
+
+  public void doBugCheck50449() {
+    String command = "put --key-class=" + ObjectWithCharAttr.class.getCanonicalName() + " --value=456 --key=\"('name':'hesdfdsfy2','t':456, 'c':'d')\"" + " --region=" + DATA_REGION_NAME_PATH;
+    CommandResult cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    validateResult(cmdResult, true);
+
+    command = "put --key-class=" + ObjectWithCharAttr.class.getCanonicalName() + " --value=123 --key=\"('name':'hesdfdsfy2','t':123, 'c':'d')\"" + " --region=" + DATA_REGION_NAME_PATH;
+    cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    validateResult(cmdResult, true);
+
+    command = "get --key-class=" + ObjectWithCharAttr.class.getCanonicalName() + " --key=\"('name':'','t':123, 'c':'d')\"" + " --region=" + DATA_REGION_NAME_PATH;
+    cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    validateResult(cmdResult, true);
+
+    command = "get --key-class=" + ObjectWithCharAttr.class.getCanonicalName() + " --key=\"('name':'','t':456, 'c':'d')\"" + " --region=" + DATA_REGION_NAME_PATH;
+    cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    validateResult(cmdResult, true);
+
+    // check wrong key
+    command = "get --key-class=" + ObjectWithCharAttr.class.getCanonicalName() + " --key=\"('name':'','t':999, 'c':'d')\"" + " --region=" + DATA_REGION_NAME_PATH;
+    cmdResult = executeCommand(command);
+    printCommandOutput(cmdResult);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    validateResult(cmdResult, false);
+  }
+
+  public void testRemoveJsonCommand() {
+    final String keyPrefix = "testKey";
+
+    setupForGetPutRemoveLocateEntry("testRemoveJsonCommand");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+
+    SerializableRunnable putKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        region.clear();
+        for (int i = 0; i < COUNT; i++) {
+          String keyString = keyPrefix + i;
+          Key1 key = new Key1();
+          key.setId(keyString);
+          key.setName("name" + keyString);
+          Value2 value2 = new Value2();
+          value2.setStateName("State" + keyString);
+          value2.setCapitalCity("capital" + keyString);
+          value2.setPopulation(i * 100);
+          value2.setAreaInSqKm(i * 100.4365);
+          region.put(key, value2);
+        }
+      }
+    };
+
+    vm1.invoke(putKeys);
+
+    for (int i = 0; i < COUNT; i++) {
+      String command = "remove";
+      String keyString = keyPrefix + i;
+      String keyJson = keyTemplate.replaceAll("\\?", keyString);
+      getLogWriter().info("Removing key with json key : " + keyJson);
+      command = command + " " + "--key=" + keyJson + " --region=" + DATA_REGION_NAME_PATH + " --key-class=" + Key1.class.getCanonicalName();
+      CommandResult cmdResult = executeCommand(command);
+      printCommandOutput(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      validateResult(cmdResult, true);
+    }
+
+    SerializableRunnable checkRemoveKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(DATA_REGION_NAME_PATH);
+        assertNotNull(region);
+        for (int i = 0; i < COUNT; i++) {
+          String keyString = keyPrefix + i;
+          Key1 key = new Key1();
+          key.setId(keyString);
+          key.setName("name" + keyString);
+          assertEquals(false, region.containsKey(key));
+        }
+        assertEquals(0, region.size());
+      }
+    };
+
+    vm1.invoke(checkRemoveKeys);
+    vm2.invoke(checkRemoveKeys);
+  }
+
+  private Region<?, ?> createParReg(String regionName, Cache cache) {
+    RegionFactory regionFactory = cache.createRegionFactory();
+    regionFactory.setDataPolicy(DataPolicy.PARTITION);
+    return regionFactory.create(regionName);
+  }
+
+  private Region<?, ?> createReplicatedRegion(String regionName, Cache cache) {
+    RegionFactory regionFactory = cache.createRegionFactory();
+    regionFactory.setDataPolicy(DataPolicy.REPLICATE);
+    return regionFactory.create(regionName);
+  }
+
+  public void testImportExportData() throws InterruptedException, IOException {
+    final String regionName = "Region1";
+    final String exportFileName = "export.gfd";
+    final VM manager = Host.getHost(0).getVM(0);
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final File exportFile = new File(exportFileName);
+    final String filePath = exportFile.getCanonicalPath();
+
+    try {
+      if (!exportFile.exists()) {
+        exportFile.createNewFile();
+      }
+      exportFile.deleteOnExit();
+
+      createDefaultSetup(null);
+
+      manager.invoke(new SerializableCallable() {
+        public Object call() {
+          Region region = createParReg(regionName, getCache());
+          return region.put("Manager", "ASD");
+        }
+      });
+
+      vm1.invoke(new SerializableCallable() {
+        @Override
+        public Object call() throws Exception {
+          Region region = createParReg(regionName, getCache());
+          return region.put("VM1", "QWE");
+        }
+      });
+
+      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.EXPORT_DATA);
+      csb.addOption(CliStrings.EXPORT_DATA__REGION, regionName);
+      csb.addOption(CliStrings.EXPORT_DATA__MEMBER, "Manager");
+      csb.addOption(CliStrings.EXPORT_DATA__FILE, filePath);
+      String commandString = csb.toString();
+
+      CommandResult cmdResult = executeCommand(commandString);
+      String resultAsString = commandResultToString(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      getLogWriter().info("Command Output");
+      getLogWriter().info(resultAsString);
+
+      vm1.invoke(new SerializableRunnable() {
+        public void run() {
+          Region region = getCache().getRegion(regionName);
+          region.destroy("Manager");
+          region.destroy("VM1");
+        }
+      });
+
+      /**
+       * Import the data 
+       */
+
+      csb = new CommandStringBuilder(CliStrings.IMPORT_DATA);
+      csb.addOption(CliStrings.IMPORT_DATA__REGION, regionName);
+      csb.addOption(CliStrings.IMPORT_DATA__FILE, filePath);
+      csb.addOption(CliStrings.IMPORT_DATA__MEMBER, "Manager");
+
+      commandString = csb.toString();
+      cmdResult = executeCommand(commandString);
+      resultAsString = commandResultToString(cmdResult);
+
+      getLogWriter().info("Result of import data");
+      getLogWriter().info(resultAsString);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      /**
+       *  Validate the region entries after import
+       *  They must match the entries before export
+       */
+
+      manager.invoke(new SerializableRunnable() {
+        public void run() {
+          Region region = getCache().getRegion(regionName);
+          assertEquals(region.get("Manager"), "ASD");
+          assertEquals(region.get("VM1"), "QWE");
+        }
+      });
+
+      //Test for bad input
+      csb = new CommandStringBuilder(CliStrings.EXPORT_DATA);
+      csb.addOption(CliStrings.EXPORT_DATA__REGION, "FDSERW");
+      csb.addOption(CliStrings.EXPORT_DATA__FILE, filePath);
+      csb.addOption(CliStrings.EXPORT_DATA__MEMBER, "Manager");
+      commandString = csb.getCommandString();
+
+      cmdResult = executeCommand(commandString);
+      resultAsString = commandResultToString(cmdResult);
+      getLogWriter().info("Result of import data with wrong region name");
+      getLogWriter().info(resultAsString);
+      assertEquals(Result.Status.ERROR, cmdResult.getStatus());
+
+      csb = new CommandStringBuilder(CliStrings.IMPORT_DATA);
+      csb.addOption(CliStrings.IMPORT_DATA__REGION, regionName);
+      csb.addOption(CliStrings.IMPORT_DATA__FILE, "#WEQW");
+      csb.addOption(CliStrings.IMPORT_DATA__MEMBER, "Manager");
+      commandString = csb.getCommandString();
+
+      cmdResult = executeCommand(commandString);
+      resultAsString = commandResultToString(cmdResult);
+      getLogWriter().info("Result of import data with wrong file");
+      getLogWriter().info(resultAsString);
+      assertEquals(Result.Status.ERROR, cmdResult.getStatus());
+
+    } finally {
+      exportFile.delete();
+    }
+  }
+
+  void setupWith2Regions() {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+    createDefaultSetup(null);
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create(REBALANCE_REGION_NAME);
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+        region = dataRegionFactory.create(REBALANCE_REGION2_NAME);
+        for (int i = 0; i < 50; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create(REBALANCE_REGION_NAME);
+        for (int i = 0; i < 150; i++) {
+          region.put("key" + (i + 400), "value" + (i + 400));
+        }
+        region = dataRegionFactory.create(REBALANCE_REGION2_NAME);
+        for (int i = 0; i < 100; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+  SerializableRunnable checkRegionMBeans = new SerializableRunnable() {
+    @Override
+    public void run() {
+      final WaitCriterion waitForMaangerMBean = new WaitCriterion() {
+        @Override
+        public boolean done() {
+          final ManagementService service = ManagementService.getManagementService(getCache());
+          final DistributedRegionMXBean bean = service.getDistributedRegionMXBean(
+              Region.SEPARATOR + REBALANCE_REGION_NAME);
+          if (bean == null) {
+            getLogWriter().info("Still probing for checkRegionMBeans ManagerMBean");
+            return false;
+          } else {
+            // verify that bean is proper before executing tests
+            if (bean.getMembers() != null && bean.getMembers().length > 1 && bean.getMemberCount() > 0 && service.getDistributedSystemMXBean().listRegions().length >= 2) {
+              return true;
+            } else {
+              return false;
+            }
+          }
+        }
+
+        @Override
+        public String description() {
+          return "Probing for testRebalanceCommandForSimulateWithNoMember ManagerMBean";
+        }
+      };
+      DistributedTestCase.waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
+      DistributedRegionMXBean bean = ManagementService.getManagementService(getCache()).getDistributedRegionMXBean(
+          "/" + REBALANCE_REGION_NAME);
+      assertNotNull(bean);
+    }
+  };
+
+  public void testRebalanceCommandForTimeOut() {
+    setupTestRebalanceForEntireDS();
+
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+    getLogWriter().info("testRebalanceCommandForTimeOut verified Mbean and executin command");
+    String command = "rebalance --time-out=1";
+    CommandResult cmdResult = executeCommand(command);
+    getLogWriter().info("testRebalanceCommandForTimeOut just after executing " + cmdResult);
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testRebalanceCommandForTimeOut stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceCommandForTimeOut failed as did not get CommandResult");
+    }
+  }
+
+  public void testRebalanceCommandForTimeOutForRegion() {
+    setupTestRebalanceForEntireDS();
+
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+
+    getLogWriter().info("testRebalanceCommandForTimeOutForRegion verified Mbean and executin command");
+
+    String command = "rebalance --time-out=1 --include-region=" + "/" + REBALANCE_REGION_NAME;
+    CommandResult cmdResult = executeCommand(command);
+
+    getLogWriter().info("testRebalanceCommandForTimeOutForRegion just after executing " + cmdResult);
+
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testRebalanceCommandForTimeOutForRegion stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceCommandForTimeOut failed as did not get CommandResult");
+    }
+  }
+
+  public void testRebalanceCommandForSimulate() {
+    setupTestRebalanceForEntireDS();
+
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+
+    getLogWriter().info("testRebalanceCommandForSimulate verified Mbean and executin command");
+    String command = "rebalance --simulate=true --include-region=" + "/" + REBALANCE_REGION_NAME;
+    CommandResult cmdResult = executeCommand(command);
+    getLogWriter().info("testRebalanceCommandForSimulate just after executing " + cmdResult);
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testRebalanceCommandForSimulate stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceCommandForSimulate failed as did not get CommandResult");
+    }
+  }
+
+  public void testRebalanceCommandForSimulateWithNoMember() {
+    setupTestRebalanceForEntireDS();
+
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+
+    getLogWriter().info("testRebalanceCommandForSimulateWithNoMember verified Mbean and executin command");
+
+    String command = "rebalance --simulate=true";
+    CommandResult cmdResult = executeCommand(command);
+
+    getLogWriter().info("testRebalanceCommandForSimulateWithNoMember just after executing " + cmdResult);
+
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testRebalanceCommandForSimulateWithNoMember stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceCommandForSimulateWithNoMember failed as did not get CommandResult");
+    }
+  }
+
+  public void testRebalanceForIncludeRegionFunction() {
+    // setup();
+    setupWith2Regions();
+
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+    getLogWriter().info("testRebalanceForIncludeRegionFunction verified Mbean and executin command");
+    String command = "rebalance --include-region=" + "/" + REBALANCE_REGION_NAME + ",/" + REBALANCE_REGION2_NAME;
+    CommandResult cmdResult = executeCommand(command);
+    getLogWriter().info("testRebalanceForIncludeRegionFunction just after executing " + cmdResult);
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testRebalanceForIncludeRegionFunction stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
+    }
+  }
+
+  public void testSimulateForEntireDS() {
+    setupTestRebalanceForEntireDS();
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+
+    getLogWriter().info("testSimulateForEntireDS verified Mbean and executin command");
+
+    String command = "rebalance --simulate=true";
+
+    CommandResult cmdResult = executeCommand(command);
+
+    getLogWriter().info("testSimulateForEntireDS just after executing " + cmdResult);
+
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testSimulateForEntireDS stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
+    }
+  }
+
+  public void testRebalanceForEntireDS() {
+    setupTestRebalanceForEntireDS();
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+    getLogWriter().info("testRebalanceForEntireDS verified Mbean and executin command");
+    String command = "rebalance";
+    CommandResult cmdResult = executeCommand(command);
+    getLogWriter().info("testRebalanceForEntireDS just after executing " + cmdResult);
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testRebalanceForEntireDS stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
+    }
+  }
+
+  void setupTestRebalanceForEntireDS() {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+    createDefaultSetup(null);
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create(REBALANCE_REGION_NAME);
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+        region = dataRegionFactory.create(REBALANCE_REGION_NAME + "Another1");
+        for (int i = 0; i < 100; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create(REBALANCE_REGION_NAME);
+        for (int i = 0; i < 100; i++) {
+          region.put("key" + (i + 400), "value" + (i + 400));
+        }
+        region = dataRegionFactory.create(REBALANCE_REGION_NAME + "Another2");
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+  private static void printCommandOutput(CommandResult cmdResult) {
+    assertNotNull(cmdResult);
+    getLogWriter().info("Command Output : ");
+    StringBuilder sb = new StringBuilder();
+    cmdResult.resetToFirstLine();
+    while (cmdResult.hasNextLine()) {
+      sb.append(cmdResult.nextLine()).append(DataCommandRequest.NEW_LINE);
+    }
+    getLogWriter().info(sb.toString());
+    getLogWriter().info("");
+  }
+
+  public static class Value1WithValue2 extends Value1 {
+    private Value2 value2 = null;
+
+    public Value1WithValue2(int i) {
+      super(i);
+      value2 = new Value2(i);
+    }
+
+    public Value2 getValue2() {
+      return value2;
+    }
+
+    public void setValue2(Value2 value2) {
+      this.value2 = value2;
+    }
+  }
+
+  public void testRebalanceForExcludeRegionFunction() {
+    setupWith2Regions();
+
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+
+    getLogWriter().info("testRebalanceForExcludeRegionFunction verified Mbean and executing command");
+
+    String command = "rebalance --exclude-region=" + "/" + REBALANCE_REGION2_NAME;
+    getLogWriter().info("testRebalanceForExcludeRegionFunction command : " + command);
+    CommandResult cmdResult = executeCommand(command);
+    getLogWriter().info("testRebalanceForExcludeRegionFunction just after executing " + cmdResult);
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testRebalanceForExcludeRegionFunction stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
+    }
+  }
+
+  public void waitForListClientMbean(final String regionName) {
+
+    final VM manager = Host.getHost(0).getVM(0);
+
+    manager.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        final ManagementService service = ManagementService.getManagementService(cache);
+
+        final WaitCriterion waitForMaangerMBean = new WaitCriterion() {
+          @Override
+          public boolean done() {
+            ManagerMXBean bean1 = service.getManagerMXBean();
+            DistributedRegionMXBean bean2 = service.getDistributedRegionMXBean(regionName);
+            if (bean1 == null) {
+              getLogWriter().info("waitForListClientMbean Still probing for ManagerMBean");
+              return false;
+            } else {
+              getLogWriter().info("waitForListClientMbean Still probing for DistributedRegionMXBean=" + bean2);
+              if (bean2 == null) {
+                bean2 = service.getDistributedRegionMXBean(Region.SEPARATOR + regionName);
+              }
+              if (bean2 == null) {
+                getLogWriter().info(
+                    "waitForListClientMbean Still probing for DistributedRegionMXBean with separator = " + bean2);
+                return false;
+              } else {
+                getLogWriter().info(
+                    "waitForListClientMbean Still probing for DistributedRegionMXBean with separator Not null  " + bean2.getMembers().length);
+                if (bean2.getMembers().length > 1) {
+                  return true;
+                } else {
+                  return false;
+                }
+              }
+            }
+          }
+
+          @Override
+          public String description() {
+            return "waitForListClientMbean Probing for ManagerMBean";
+          }
+        };
+
+        DistributedTestCase.waitForCriterion(waitForMaangerMBean, 30000, 2000, true);
+        DistributedRegionMXBean bean = service.getDistributedRegionMXBean(regionName);
+        if (bean == null) {
+          bean = service.getDistributedRegionMXBean(Region.SEPARATOR + regionName);
+        }
+        assertNotNull(bean);
+      }
+    });
+
+  }
+
+  public void testRegionsViaMbeanAndFunctions() {
+
+    setupForGetPutRemoveLocateEntry("tesSimplePut");
+    waitForListClientMbean(DATA_REGION_NAME_PATH);
+    final VM manager = Host.getHost(0).getVM(0);
+
+    String memSizeFromMbean = (String) manager.invoke(new SerializableCallable() {
+      public Object call() {
+        Cache cache = GemFireCacheImpl.getInstance();
+        DistributedRegionMXBean bean = ManagementService.getManagementService(cache).getDistributedRegionMXBean(
+            DATA_REGION_NAME_PATH);
+
+        if (bean == null)// try with slash ahead
+          bean = ManagementService.getManagementService(cache).getDistributedRegionMXBean(
+              Region.SEPARATOR + DATA_REGION_NAME_PATH);
+
+        if (bean == null) {
+          return null;
+        }
+
+        String[] membersName = bean.getMembers();
+        return "" + membersName.length;
+      }
+    });
+
+    getLogWriter().info("testRegionsViaMbeanAndFunctions memSizeFromMbean= " + memSizeFromMbean);
+
+    String memSizeFromFunctionCall = (String) manager.invoke(new SerializableCallable() {
+      public Object call() {
+        Cache cache = GemFireCacheImpl.getInstance();
+        CliUtil.getMembersForeRegionViaFunction(cache, DATA_REGION_NAME_PATH);
+        return "" + CliUtil.getMembersForeRegionViaFunction(cache, DATA_REGION_NAME_PATH).size();
+      }
+    });
+
+    getLogWriter().info("testRegionsViaMbeanAndFunctions memSizeFromFunctionCall= " + memSizeFromFunctionCall);
+    assertTrue(memSizeFromFunctionCall.equals(memSizeFromMbean));
+  }
+
+  public void testRegionsViaMbeanAndFunctionsForPartRgn() {
+    setupWith2Regions();
+    waitForListClientMbean(REBALANCE_REGION_NAME);
+    final VM manager = Host.getHost(0).getVM(0);
+
+    String memSizeFromMbean = (String) manager.invoke(new SerializableCallable() {
+      public Object call() {
+        Cache cache = GemFireCacheImpl.getInstance();
+        DistributedRegionMXBean bean = ManagementService.getManagementService(cache).getDistributedRegionMXBean(
+            REBALANCE_REGION_NAME);
+
+        if (bean == null) {
+          bean = ManagementService.getManagementService(cache).getDistributedRegionMXBean(
+              Region.SEPARATOR + REBALANCE_REGION_NAME);
+        }
+
+        if (bean == null) {
+          return null;
+        }
+
+        String[] membersName = bean.getMembers();
+        return "" + membersName.length;
+      }
+    });
+
+    getLogWriter().info("testRegionsViaMbeanAndFunctionsForPartRgn memSizeFromMbean= " + memSizeFromMbean);
+
+    String memSizeFromFunctionCall = (String) manager.invoke(new SerializableCallable() {
+      public Object call() {
+        Cache cache = GemFireCacheImpl.getInstance();
+        return "" + CliUtil.getMembersForeRegionViaFunction(cache, REBALANCE_REGION_NAME).size();
+      }
+    });
+
+    getLogWriter().info(
+        "testRegionsViaMbeanAndFunctionsForPartRgn memSizeFromFunctionCall= " + memSizeFromFunctionCall);
+    assertTrue(memSizeFromFunctionCall.equals(memSizeFromMbean));
+  }
+
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+  }
+}



[44/50] [abbrv] incubator-geode git commit: GEODE-623: add unit test for RefCountChangeInfo

Posted by kl...@apache.org.
GEODE-623: add unit test for RefCountChangeInfo


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

Branch: refs/heads/feature/GEODE-217
Commit: 68dfcab10d68b1babb2035bc4c87c93acf52077c
Parents: 476c6cd
Author: Scott Jewell <sj...@pivotal.io>
Authored: Fri Dec 4 13:06:58 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Dec 8 15:45:57 2015 -0800

----------------------------------------------------------------------
 .../internal/offheap/RefCountChangeInfo.java    |  43 +++--
 .../offheap/RefCountChangeInfoJUnitTest.java    | 159 +++++++++++++++++++
 2 files changed, 185 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/68dfcab1/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
index 56cab97..67688ed 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
@@ -68,20 +68,11 @@ public class RefCountChangeInfo extends Throwable {
       ps.print("@");
       ps.print(System.identityHashCode(this.owner));
     }
+    
     ps.println(": ");
-    StackTraceElement[] trace = getStackTrace();
-    // skip the initial elements from SimpleMemoryAllocatorImpl
-    int skip=0;
-    for (int i=0; i < trace.length; i++) {
-      if (!trace[i].getClassName().contains("SimpleMemoryAllocatorImpl")) {
-        skip = i;
-        break;
-      }
-    }
-    for (int i=skip; i < trace.length; i++) {
-      ps.println("\tat " + trace[i]);
-    }
+    cleanStackTrace(ps); 
     ps.flush();
+    
     return baos.toString();
   }
   
@@ -99,14 +90,32 @@ public class RefCountChangeInfo extends Throwable {
   }
 
   private String stackTraceString;
-  private String getStackTraceString() {
+  String getStackTraceString() {
     String result = this.stackTraceString;
     if (result == null) {
-      StringPrintWriter spr = new StringPrintWriter();
-      printStackTrace(spr);
-      result = spr.getBuilder().toString();
-      this.stackTraceString = result;
+	ByteArrayOutputStream baos = new ByteArrayOutputStream(64*1024);
+	PrintStream spr = new PrintStream(baos);
+
+	cleanStackTrace(spr);
+	result = baos.toString();
+	this.stackTraceString = result;
     }
     return result;
   }
+  
+  private void cleanStackTrace(PrintStream ps) {
+      StackTraceElement[] trace = getStackTrace();
+      // skip the initial elements from the offheap package
+      int skip=0;
+      for (int i=0; i < trace.length; i++) {
+	if (!trace[i].getClassName().contains("com.gemstone.gemfire.internal.offheap")) {
+          skip = i;
+          break;
+        }
+      }
+      for (int i=skip; i < trace.length; i++) {
+        ps.println("\tat " + trace[i]);
+      }   
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/68dfcab1/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfoJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfoJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfoJUnitTest.java
new file mode 100644
index 0000000..fc726ce
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfoJUnitTest.java
@@ -0,0 +1,159 @@
+/*
+ * 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.offheap;
+
+import static org.junit.Assert.*;
+
+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;
+
+@Category(UnitTest.class)
+public class RefCountChangeInfoJUnitTest {
+
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public void testGetOwner() {
+
+    String owner1 = new String("Info1");
+    String notOwner1 = new String("notInfo1");
+
+    RefCountChangeInfo refInfo1 = new RefCountChangeInfo(true, 1, owner1);
+
+    assertEquals(owner1, refInfo1.getOwner());
+
+    try {
+      assertEquals(owner1, notOwner1);
+      fail("Expected owner1 != notOwner1");
+    } catch (AssertionError e) {
+      // Ignore expected error
+    }
+
+  }
+
+  @Test
+  public void testGetDupCount() {
+
+    String owner1 = new String("Info1");
+    String owner2 = new String("Info2");
+
+    RefCountChangeInfo refInfo1 = new RefCountChangeInfo(true, 1, owner1);
+    assertEquals(0, refInfo1.getDupCount());
+
+    RefCountChangeInfo refInfo2 = new RefCountChangeInfo(true, 1, owner1);
+    assertTrue(refInfo1.isDuplicate(refInfo2));
+    assertEquals(1, refInfo1.getDupCount());
+
+    // owner not used in isDup
+    RefCountChangeInfo refInfo3 = new RefCountChangeInfo(true, 1, owner2);
+    assertTrue(refInfo1.isDuplicate(refInfo3));
+    assertEquals(2, refInfo1.getDupCount());
+
+    RefCountChangeInfo refInfo4 = new RefCountChangeInfo(false, 1, owner2);
+    assertFalse(refInfo1.isDuplicate(refInfo4));
+    assertEquals(2, refInfo1.getDupCount());
+
+  }
+
+  @Test
+  public void testDecDupCount() {
+
+    String owner1 = new String("Info1");
+    String owner2 = new String("Info2");
+
+    RefCountChangeInfo refInfo1 = new RefCountChangeInfo(true, 1, owner1);
+    assertEquals(0, refInfo1.getDupCount());
+
+    RefCountChangeInfo refInfo2 = new RefCountChangeInfo(true, 1, owner1);
+    assertTrue(refInfo1.isDuplicate(refInfo2));
+    assertEquals(1, refInfo1.getDupCount());
+
+    // owner not used in isDuplicate check
+    RefCountChangeInfo refInfo3 = new RefCountChangeInfo(true, 1, owner2);
+    assertTrue(refInfo1.isDuplicate(refInfo3));
+    assertEquals(2, refInfo1.getDupCount());
+
+    refInfo1.decDupCount();
+    assertEquals(1, refInfo1.getDupCount());
+
+    refInfo1.decDupCount();
+    assertEquals(0, refInfo1.getDupCount());
+
+  }
+
+  @Test
+  public void testToString() {
+
+    String owner1 = new String("Info1");
+
+    RefCountChangeInfo refInfo1 = new RefCountChangeInfo(true, 1, owner1);
+
+    RefCountChangeInfo refInfo2 = new RefCountChangeInfo(true, 1, owner1);
+    assertEquals(refInfo1.toString(), refInfo2.toString());
+
+    RefCountChangeInfo refInfo3 = new RefCountChangeInfo(false, 1, owner1);
+    try {
+      assertEquals(refInfo1.toString(), refInfo3.toString());
+      fail("expected refInfo1.toString() != refInfo3.toString()");
+    } catch (AssertionError e) {
+      // ignore expected IllegalArgumentException
+    }
+
+    RefCountChangeInfo refInfo4 = new RefCountChangeInfo(true, 2, owner1);
+    try {
+      assertEquals(refInfo1.toString(), refInfo4.toString());
+      fail("expected refInfo1.toString() != refInfo4.toString()");
+    } catch (AssertionError e) {
+      // ignore expected IllegalArgumentException
+    }
+
+  }
+
+  @Test
+  public void testIsDuplicate() {
+
+    String owner1 = new String("Info1");
+    String owner2 = new String("Info2");
+
+    RefCountChangeInfo refInfo1 = new RefCountChangeInfo(true, 1, owner1);
+    assertEquals(0, refInfo1.getDupCount());
+
+    RefCountChangeInfo refInfo2 = new RefCountChangeInfo(true, 1, owner1);
+    assertTrue(refInfo1.isDuplicate(refInfo2));
+    assertEquals(1, refInfo1.getDupCount());
+
+    RefCountChangeInfo refInfo3 = new RefCountChangeInfo(false, 1, owner1);
+    assertFalse(refInfo1.isDuplicate(refInfo3));
+    assertEquals(1, refInfo1.getDupCount());
+
+    RefCountChangeInfo refInfo4 = new RefCountChangeInfo(true, 1, owner2);
+    assertTrue(refInfo1.isDuplicate(refInfo4));
+    assertEquals(2, refInfo1.getDupCount());
+
+  }
+
+}


[28/50] [abbrv] incubator-geode git commit: GEODE-563: Moving gfsh tests from closed

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java
new file mode 100644
index 0000000..3a8811d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java
@@ -0,0 +1,164 @@
+/*
+ * 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.ClassBuilder;
+import com.gemstone.gemfire.internal.ClassPathLoader;
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.CommandManager;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Unit tests for configuring user commands.
+ *
+ * @author David Hoots
+ * @since 8.0
+ */
+public class UserCommandsDUnitTest extends CliCommandTestBase {
+  private static final long serialVersionUID = 1L;
+  final File jarDirectory = new File(
+      (new File(ClassPathLoader.class.getProtectionDomain().getCodeSource().getLocation().getPath())).getParent(),
+      "ext");
+  final File jarFile = new File(this.jarDirectory, "UserCommandsDUnit.jar");
+  boolean deleteJarDirectory = false;
+
+  public UserCommandsDUnitTest(String name) throws Exception {
+    super(name);
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    createUserCommandJarFile();
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+    if (this.deleteJarDirectory) {
+      FileUtil.delete(this.jarDirectory);
+    } else {
+      FileUtil.delete(this.jarFile);
+    }
+
+    System.clearProperty(CommandManager.USER_CMD_PACKAGES_PROPERTY);
+    ClassPathLoader.setLatestToDefault();
+    CommandManager.clearInstance();
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      private static final long serialVersionUID = 1L;
+
+      @Override
+      public void run() {
+        System.clearProperty(CommandManager.USER_CMD_PACKAGES_PROPERTY);
+        ClassPathLoader.setLatestToDefault();
+        CommandManager.clearInstance();
+      }
+    });
+  }
+
+  public void createUserCommandJarFile() throws IOException {
+    this.deleteJarDirectory = this.jarDirectory.mkdir();
+
+    StringBuffer stringBuffer = new StringBuffer();
+
+    stringBuffer.append("package junit.ucdunit;");
+    stringBuffer.append("import org.springframework.shell.core.CommandMarker;");
+    stringBuffer.append("import org.springframework.shell.core.annotation.CliAvailabilityIndicator;");
+    stringBuffer.append("import org.springframework.shell.core.annotation.CliCommand;");
+    stringBuffer.append("import org.springframework.shell.core.annotation.CliOption;");
+    stringBuffer.append("import com.gemstone.gemfire.management.cli.Result;");
+    stringBuffer.append("import com.gemstone.gemfire.management.internal.cli.CliUtil;");
+    stringBuffer.append("import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;");
+    stringBuffer.append("import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;");
+
+    stringBuffer.append("public final class UCDunitClass implements CommandMarker { public UCDunitClass() {}");
+    stringBuffer.append("@CliCommand(value = { \"ucdunitcmd\" }, help = \"ucdunitcmd help\")");
+    stringBuffer.append(
+        "public final Result ucdunitcmd(@CliOption(key = { \"name\" }, help = \"ucdunitcmd name help\") String name) {");
+    stringBuffer.append("return ResultBuilder.createInfoResult(\"ucdunitcmd \" + name); }");
+    stringBuffer.append("@CliAvailabilityIndicator({ \"ucdunitcmd\" })");
+    stringBuffer.append("public final boolean isAvailable() { return true; } }");
+
+    ClassBuilder classBuilder = new ClassBuilder();
+    final byte[] jarBytes = classBuilder.createJarFromClassContent("junit/ucdunit/UCDunitClass",
+        stringBuffer.toString());
+
+    final FileOutputStream outStream = new FileOutputStream(this.jarFile);
+    outStream.write(jarBytes);
+    outStream.close();
+  }
+
+  @Test
+  public void testCommandLineProperty() {
+    System.setProperty(CommandManager.USER_CMD_PACKAGES_PROPERTY, "junit.ucdunit");
+
+    ClassPathLoader.setLatestToDefault();
+    CommandManager.clearInstance();
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      private static final long serialVersionUID = 1L;
+
+      @Override
+      public void run() {
+        System.setProperty(CommandManager.USER_CMD_PACKAGES_PROPERTY, "junit.ucdunit");
+        ClassPathLoader.setLatestToDefault();
+        CommandManager.clearInstance();
+      }
+    });
+
+    createDefaultSetup(null);
+
+    CommandResult cmdResult = executeCommand("ucdunitcmd");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+  }
+
+  @Test
+  public void testGemFireProperty() {
+    System.setProperty(CommandManager.USER_CMD_PACKAGES_PROPERTY, "junit.ucdunit");
+
+    ClassPathLoader.setLatestToDefault();
+    CommandManager.clearInstance();
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      private static final long serialVersionUID = 1L;
+
+      @Override
+      public void run() {
+        ClassPathLoader.setLatestToDefault();
+        CommandManager.clearInstance();
+      }
+    });
+
+    Properties properties = new Properties();
+    properties.setProperty(DistributionConfig.USER_COMMAND_PACKAGES, "junit.ucdunit");
+    createDefaultSetup(properties);
+
+    CommandResult cmdResult = executeCommand("ucdunitcmd");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+  }
+}


[06/50] [abbrv] incubator-geode git commit: GEODE-611: Change findbugs annotations to use ASL library

Posted by kl...@apache.org.
GEODE-611: Change findbugs annotations to use ASL library

Swtich the findbugs annotation dependency to use an ASL version
from https://github.com/stephenc/findbugs-annotations.  This
library only supports the SuppressWarning annotation, not
SuppressFBWarning so some source files were updated.


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

Branch: refs/heads/feature/GEODE-217
Commit: 4d6df30af1d4d6be997eb4b46fb57790c197c6de
Parents: 5a9fdb8
Author: Anthony Baker <ab...@pivotal.io>
Authored: Sun Nov 29 08:17:42 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Tue Dec 1 11:38:16 2015 -0800

----------------------------------------------------------------------
 build.gradle                                    |  1 +
 gemfire-core/build.gradle                       |  1 -
 .../com/gemstone/gemfire/SystemFailure.java     |  4 +--
 .../gemfire/admin/jmx/internal/AgentImpl.java   | 15 +++++------
 .../internal/GemFireHealthConfigJmxImpl.java    | 14 +++++-----
 .../cache/query/internal/CompiledOperation.java |  4 +--
 .../gemfire/internal/StatArchiveWriter.java     | 28 +++++++++-----------
 .../internal/cache/AbstractUpdateOperation.java |  4 +--
 .../internal/cache/TXStateProxyImpl.java        |  4 +--
 .../cache/tier/sockets/AcceptorImpl.java        |  4 +--
 .../cache/tier/sockets/BaseCommand.java         | 18 +------------
 .../tier/sockets/ClientProxyMembershipID.java   |  5 +---
 .../cache/versions/RegionVersionVector.java     |  6 ++---
 .../internal/datasource/AbstractPoolCache.java  |  4 +--
 .../internal/memcached/commands/GetCommand.java |  7 +----
 .../internal/statistics/SampleCollector.java    |  4 +--
 .../gemfire/internal/util/DebuggerSupport.java  |  4 +--
 .../gemfire/internal/util/SingletonValue.java   |  6 ++---
 .../CustomEntryConcurrentHashMap.java           |  8 ++----
 gradle/dependency-versions.properties           |  1 +
 20 files changed, 46 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 991488b..b5465b8 100755
--- a/build.gradle
+++ b/build.gradle
@@ -304,6 +304,7 @@ subprojects {
     compile 'org.springframework:spring-expression:' + project.'springframework.version'
     compile 'org.springframework:spring-web:' + project.'springframework.version'
     compile 'org.springframework:spring-webmvc:' + project.'springframework.version'
+    compile 'com.github.stephenc.findbugs:findbugs-annotations:' + project.'stephenc-findbugs.version'
 
     testCompile 'com.jayway.awaitility:awaitility:' + project.'awaitility.version'
     testCompile 'com.github.stefanbirkner:system-rules:' + project.'system-rules.version'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-core/build.gradle b/gemfire-core/build.gradle
index 8e45a97..dd3b765 100755
--- a/gemfire-core/build.gradle
+++ b/gemfire-core/build.gradle
@@ -19,7 +19,6 @@ dependencies {
   compile 'com.fasterxml.jackson.core:jackson-annotations:' + project.'jackson.version'
   compile 'com.fasterxml.jackson.core:jackson-core:' + project.'jackson.version'
   compile 'com.fasterxml.jackson.core:jackson-databind:' + project.'jackson.version'
-  compile 'com.google.code.findbugs:annotations:' + project.'annotations.version'
   provided 'com.google.guava:guava:' + project.'guava.version'
   compile 'commons-io:commons-io:' + project.'commons-io.version'
   compile 'commons-lang:commons-lang:' + project.'commons-lang.version'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/SystemFailure.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/SystemFailure.java b/gemfire-core/src/main/java/com/gemstone/gemfire/SystemFailure.java
index b37f257..494f5f7 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/SystemFailure.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/SystemFailure.java
@@ -24,8 +24,6 @@ import com.gemstone.gemfire.internal.admin.remote.RemoteGfManagerAgent;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * Catches and responds to JVM failure
  * <p>
@@ -190,7 +188,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
  * @author jpenney
  * @since 5.1
  */
-@SuppressFBWarnings(value="DM_GC", justification="This class performs System.gc as last ditch effort during out-of-memory condition.") 
+@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DM_GC", justification="This class performs System.gc as last ditch effort during out-of-memory condition.") 
 public final class SystemFailure {
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/AgentImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/AgentImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/AgentImpl.java
index 2acea99..4d1ad41 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/AgentImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/AgentImpl.java
@@ -44,6 +44,8 @@ import javax.management.remote.JMXServiceURL;
 import javax.management.remote.rmi.RMIConnectorServer;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
+import mx4j.tools.adaptor.http.HttpAdaptor;
+
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.GemFireException;
@@ -56,6 +58,7 @@ import com.gemstone.gemfire.admin.jmx.Agent;
 import com.gemstone.gemfire.admin.jmx.AgentConfig;
 import com.gemstone.gemfire.admin.jmx.AgentFactory;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.Banner;
 import com.gemstone.gemfire.internal.GemFireVersion;
 import com.gemstone.gemfire.internal.admin.remote.TailLogResponse;
@@ -70,10 +73,6 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.logging.log4j.LogWriterAppender;
 import com.gemstone.gemfire.internal.logging.log4j.LogWriterAppenders;
-import com.gemstone.gemfire.i18n.StringId;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import mx4j.tools.adaptor.http.HttpAdaptor;
 
 /**
  * The GemFire JMX Agent provides the ability to administrate one GemFire
@@ -406,7 +405,7 @@ implements com.gemstone.gemfire.admin.jmx.Agent,
    *
    * @return the object name of the system that the Agent is now connected to
    */
-  @SuppressFBWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD", justification="This is only a style warning.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD", justification="This is only a style warning.") 
   public ObjectName connectToSystem()
   throws AdminException, MalformedObjectNameException {
     synchronized(CONN_SYNC) {
@@ -454,7 +453,7 @@ implements com.gemstone.gemfire.admin.jmx.Agent,
   /**
    * Disconnects from the current DistributedSystem (if connected to one).
    */
-  @SuppressFBWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD", justification="This is only a style warning.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD", justification="This is only a style warning.") 
   public void disconnectFromSystem() {
     synchronized(CONN_SYNC) {
       try {
@@ -878,7 +877,7 @@ implements com.gemstone.gemfire.admin.jmx.Agent,
   /**
    * Creates a LogWriterI18n for this Agent to use in logging.
    */
-  @SuppressFBWarnings(value="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE", justification="Return value for file delete is not important here.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE", justification="Return value for file delete is not important here.") 
   private void initLogWriter() throws com.gemstone.gemfire.admin.AdminException {
     final LogConfig logConfig = this.agentConfig.createLogConfig();
     
@@ -1570,7 +1569,7 @@ class ConnectionNotificationAdapter implements NotificationListener {
    *          the listener. The MBean object should not use or modify the
    *          object. (NOTE: copied from javax.management.NotificationListener)
    */
-  @SuppressFBWarnings(value="BC_UNCONFIRMED_CAST", justification="Only JMXConnectionNotification instances are used.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="BC_UNCONFIRMED_CAST", justification="Only JMXConnectionNotification instances are used.") 
   public void handleNotification(Notification notification, Object handback) {
     if (handback instanceof AgentImpl) {
       AgentImpl agent = (AgentImpl) handback;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/GemFireHealthConfigJmxImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/GemFireHealthConfigJmxImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/GemFireHealthConfigJmxImpl.java
index a55a616..c05d4b8 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/GemFireHealthConfigJmxImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/GemFireHealthConfigJmxImpl.java
@@ -16,13 +16,13 @@
  */
 package com.gemstone.gemfire.admin.jmx.internal;
 
-import com.gemstone.gemfire.admin.*;
-import com.gemstone.gemfire.admin.internal.*;
+import javax.management.ObjectName;
+import javax.management.modelmbean.ModelMBean;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-import javax.management.*;
-import javax.management.modelmbean.*;
+import com.gemstone.gemfire.admin.AdminException;
+import com.gemstone.gemfire.admin.GemFireHealth;
+import com.gemstone.gemfire.admin.GemFireHealthConfig;
+import com.gemstone.gemfire.admin.internal.GemFireHealthConfigImpl;
 
 /**
  * The JMX "managed resource" that represents the configuration for
@@ -46,7 +46,7 @@ import javax.management.modelmbean.*;
  *
  * @since 3.5
  */
-@SuppressFBWarnings(justification="This class is deprecated. Also, any further changes so close to the release is inadvisable.") 
+@edu.umd.cs.findbugs.annotations.SuppressWarnings(justification="This class is deprecated. Also, any further changes so close to the release is inadvisable.") 
 public class GemFireHealthConfigJmxImpl
   implements GemFireHealthConfig, ManagedResource, java.io.Serializable {
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledOperation.java
index 957f0fe..d9f0d6c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledOperation.java
@@ -41,8 +41,6 @@ import com.gemstone.gemfire.pdx.PdxSerializationException;
 import com.gemstone.gemfire.pdx.internal.PdxInstanceImpl;
 import com.gemstone.gemfire.pdx.internal.PdxString;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * Class Description
  *
@@ -223,7 +221,7 @@ public class CompiledOperation extends AbstractCompiledValue {
     return context.addDependencies(this, rcvr.computeDependencies(context));
   }
  
-  @SuppressFBWarnings(value="RV_RETURN_VALUE_OF_PUTIFABSENT_IGNORED", justification="Does not matter if the methodDispatch that isn't stored in the map is used") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RV_RETURN_VALUE_OF_PUTIFABSENT_IGNORED", justification="Does not matter if the methodDispatch that isn't stored in the map is used") 
   private Object eval0(Object receiver, Class resolutionType, ExecutionContext context)
   throws TypeMismatchException, FunctionDomainException, NameResolutionException,
           QueryInvocationTargetException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/StatArchiveWriter.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/StatArchiveWriter.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/StatArchiveWriter.java
index c2bf3bd..562cce3 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/StatArchiveWriter.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/StatArchiveWriter.java
@@ -16,19 +16,6 @@
  */
 package com.gemstone.gemfire.internal;
 
-import com.gemstone.gemfire.GemFireIOException;
-import com.gemstone.gemfire.InternalGemFireException;
-import com.gemstone.gemfire.StatisticDescriptor;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.statistics.ResourceInstance;
-import com.gemstone.gemfire.internal.statistics.ResourceType;
-import com.gemstone.gemfire.internal.statistics.SampleHandler;
-import com.gemstone.gemfire.internal.statistics.StatArchiveDescriptor;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 import java.io.BufferedOutputStream;
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -49,6 +36,17 @@ import java.util.zip.GZIPOutputStream;
 
 import org.apache.logging.log4j.Logger;
 
+import com.gemstone.gemfire.GemFireIOException;
+import com.gemstone.gemfire.InternalGemFireException;
+import com.gemstone.gemfire.StatisticDescriptor;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.statistics.ResourceInstance;
+import com.gemstone.gemfire.internal.statistics.ResourceType;
+import com.gemstone.gemfire.internal.statistics.SampleHandler;
+import com.gemstone.gemfire.internal.statistics.StatArchiveDescriptor;
+
 /**
  * StatArchiveWriter provides APIs to write statistic snapshots to an archive
  * file.
@@ -168,7 +166,7 @@ public class StatArchiveWriter implements StatArchiveFormat, SampleHandler {
     }
   }
   
-  @SuppressFBWarnings(value="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE", justification="Best effort attempt to delete a GFS file without any samples.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE", justification="Best effort attempt to delete a GFS file without any samples.") 
   private static void deleteFileIfPossible(File file) {
     file.delete();
   }
@@ -288,7 +286,7 @@ public class StatArchiveWriter implements StatArchiveFormat, SampleHandler {
     }
   }
   
-  @SuppressFBWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD", justification="This is only for debugging and there is never more than one instance being traced because there is only one stat sampler.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD", justification="This is only for debugging and there is never more than one instance being traced because there is only one stat sampler.") 
   public void allocatedResourceInstance(ResourceInstance statResource) {
     if (logger.isTraceEnabled(LogMarker.STATISTICS)) {
       logger.trace(LogMarker.STATISTICS, "StatArchiveWriter#allocatedResourceInstance statResource={}", statResource);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractUpdateOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractUpdateOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractUpdateOperation.java
index b94ce8b..3ac006b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractUpdateOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractUpdateOperation.java
@@ -45,8 +45,6 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.util.DelayedAction;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * Common code for both UpdateOperation and DistributedPutAllOperation.
  *
@@ -57,7 +55,7 @@ public abstract class AbstractUpdateOperation extends DistributedCacheOperation
   
   public static volatile boolean test_InvalidVersion;
   
-  @SuppressFBWarnings(value="UWF_UNWRITTEN_PUBLIC_OR_PROTECTED_FIELD",
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UWF_UNWRITTEN_PUBLIC_OR_PROTECTED_FIELD",
       justification="test hook that is unset normally")
   public static volatile DelayedAction test_InvalidVersionAction;
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxyImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxyImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxyImpl.java
index 533a1f0..fda1a3a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxyImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxyImpl.java
@@ -50,8 +50,6 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * @author mthomas
  *
@@ -570,7 +568,7 @@ public class TXStateProxyImpl implements TXStateProxy {
   /* (non-Javadoc)
    * @see com.gemstone.gemfire.internal.cache.InternalDataView#entryCount(com.gemstone.gemfire.internal.cache.LocalRegion)
    */
-  @SuppressFBWarnings(value="UL_UNRELEASED_LOCK", justification="This method unlocks and then conditionally undoes the unlock in the finally-block. Review again at later time.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK", justification="This method unlocks and then conditionally undoes the unlock in the finally-block. Review again at later time.") 
   public int entryCount(LocalRegion localRegion) {
     // if size is the first operation in the transaction, then reset the txState
     boolean resetTXState = this.realDeal == null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
index c30102f..89c073f 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
@@ -92,8 +92,6 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.tcp.ConnectionTable;
 import com.gemstone.gemfire.internal.util.ArrayUtils;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * Implements the acceptor thread on the bridge server. Accepts connections from
  * the edge and starts up threads to process requests from these.
@@ -1610,7 +1608,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
   }
 
   @Override
-  @SuppressFBWarnings(value="REC_CATCH_EXCEPTION", justification="Allow this thread to die")
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION", justification="Allow this thread to die")
   public void close() {
     if (!isRunning()) {
       return;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
index 1e6b8d8..52ea6f5 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
@@ -47,25 +47,12 @@ import com.gemstone.gemfire.cache.CacheWriterException;
 import com.gemstone.gemfire.cache.InterestResultPolicy;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
-import com.gemstone.gemfire.cache.TransactionDataNodeHasDepartedException;
-import com.gemstone.gemfire.cache.TransactionDataRebalancedException;
 import com.gemstone.gemfire.cache.TransactionException;
-import com.gemstone.gemfire.cache.operations.QueryOperationContext;
 import com.gemstone.gemfire.cache.persistence.PartitionOfflineException;
-import com.gemstone.gemfire.cache.query.Query;
-import com.gemstone.gemfire.cache.query.QueryException;
-import com.gemstone.gemfire.cache.query.QueryInvalidException;
-import com.gemstone.gemfire.cache.query.SelectResults;
-import com.gemstone.gemfire.cache.query.Struct;
-import com.gemstone.gemfire.cache.query.internal.CqEntry;
-import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
-import com.gemstone.gemfire.cache.query.internal.types.CollectionTypeImpl;
-import com.gemstone.gemfire.cache.query.internal.types.StructTypeImpl;
 import com.gemstone.gemfire.cache.query.types.CollectionType;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DistributionStats;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
-import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
@@ -92,12 +79,9 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
-import com.gemstone.gemfire.internal.security.AuthorizeRequestPP;
 import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
 import com.gemstone.gemfire.security.GemFireSecurityException;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * @author ashahid
  *
@@ -1162,7 +1146,7 @@ public abstract class BaseCommand implements Command {
    * Handles both RR and PR cases
    */
   @SuppressWarnings("rawtypes")
-  @SuppressFBWarnings(value="NP_NULL_PARAM_DEREF", justification="Null value handled in sendNewRegisterInterestResponseChunk()")
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_PARAM_DEREF", justification="Null value handled in sendNewRegisterInterestResponseChunk()")
   private static void handleKVSingleton(LocalRegion region, Object entryKey,
       boolean serializeValues, ServerConnection servConn)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
index 4e7a179..daa07f4 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
@@ -39,15 +39,12 @@ import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedM
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.VersionedDataInputStream;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * This class represents a ConnectionProxy of the CacheClient
  * 
@@ -484,7 +481,7 @@ public final class ClientProxyMembershipID
   /**
    * call this when the distributed system ID has been modified
    */
-  @SuppressFBWarnings(value = "ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD", justification = "Only applicable in client DS and in that case too multiple instances do not modify it at the same time.")
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD", justification = "Only applicable in client DS and in that case too multiple instances do not modify it at the same time.")
   public void updateID(DistributedMember idm) {
 //    this.transientPort = ((InternalDistributedMember)this.memberId).getPort();
 //    if (this.transientPort == 0) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVector.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVector.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVector.java
index 1416b62..61423d1 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVector.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVector.java
@@ -52,8 +52,6 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * RegionVersionVector tracks the highest region-level version number of
  * operations applied to a region for each member that has the region.<p>
@@ -296,7 +294,7 @@ public abstract class RegionVersionVector<T extends VersionSource<?>> implements
     // this could block for a while if a limit has been set on the waiting-thread-pool
     dm.getWaitingThreadPool().execute(
       new Runnable() {
-      @SuppressFBWarnings(value={"UL_UNRELEASED_LOCK","IMSE_DONT_CATCH_IMSE"})
+      @edu.umd.cs.findbugs.annotations.SuppressWarnings(value={"UL_UNRELEASED_LOCK","IMSE_DONT_CATCH_IMSE"})
       public void run() {
         boolean haveLock = false;
         synchronized(clearLockSync) {
@@ -667,7 +665,7 @@ public abstract class RegionVersionVector<T extends VersionSource<?>> implements
    * each oplog we recover.
    * @param latestOplog 
    */
-  @SuppressFBWarnings(value="ML_SYNC_ON_FIELD_TO_GUARD_CHANGING_THAT_FIELD",
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ML_SYNC_ON_FIELD_TO_GUARD_CHANGING_THAT_FIELD",
       justification="sync on localExceptions guards concurrent modification but this is a replacement")
   public void initRecoveredVersion(T member, RegionVersionHolder<T> v, boolean latestOplog) {
     RegionVersionHolder<T> recovered = v.clone();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/datasource/AbstractPoolCache.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/datasource/AbstractPoolCache.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/datasource/AbstractPoolCache.java
index a42cd6f..b615327 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/datasource/AbstractPoolCache.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/datasource/AbstractPoolCache.java
@@ -36,8 +36,6 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * AbstractPoolCache implements the ConnectionPoolCache interface. This is base
  * class for the all connection pools. The class also implements the
@@ -88,7 +86,7 @@ public abstract class AbstractPoolCache implements ConnectionPoolCache,
    *          configuration for the pool.
    * @throws PoolException
    */
-  @SuppressFBWarnings(value="SC_START_IN_CTOR",
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SC_START_IN_CTOR",
       justification="the thread started is a cleanup thread and is not active until there is a timeout tx")
   public AbstractPoolCache(EventListener eventListner,
       ConfiguredDataSourceProperties configs) throws PoolException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/memcached/commands/GetCommand.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/memcached/commands/GetCommand.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/memcached/commands/GetCommand.java
index da1ef53..8fd0847 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/memcached/commands/GetCommand.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/memcached/commands/GetCommand.java
@@ -18,8 +18,6 @@ package com.gemstone.gemfire.internal.memcached.commands;
 
 import java.nio.ByteBuffer;
 import java.nio.CharBuffer;
-import java.nio.charset.CharacterCodingException;
-import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
@@ -28,7 +26,6 @@ import java.util.Set;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.internal.memcached.Command;
 import com.gemstone.gemfire.internal.memcached.KeyWrapper;
 import com.gemstone.gemfire.internal.memcached.Reply;
 import com.gemstone.gemfire.internal.memcached.RequestReader;
@@ -36,8 +33,6 @@ import com.gemstone.gemfire.internal.memcached.ResponseStatus;
 import com.gemstone.gemfire.internal.memcached.ValueWrapper;
 import com.gemstone.gemfire.memcached.GemFireMemcachedServer.Protocol;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * 
  * The retrieval commands "get" and "gets" operates like this:<br/>
@@ -167,7 +162,7 @@ public class GetCommand extends AbstractCommand {
     return composeReply(results, isGets);
   }
 
-  @SuppressFBWarnings(value = "NP_NULL_PARAM_DEREF",
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NULL_PARAM_DEREF",
       justification = "findbugs complains that v is null while putting into buffer, but it is not")
   private ByteBuffer composeReply(Map<Object, ValueWrapper> results, boolean isGets) {
     Iterator<Entry<Object, ValueWrapper>> it = results.entrySet().iterator();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/statistics/SampleCollector.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/statistics/SampleCollector.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/statistics/SampleCollector.java
index ee00ff2..9b40e95 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/statistics/SampleCollector.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/statistics/SampleCollector.java
@@ -37,8 +37,6 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * Captures sample of statistics. The SampleCollector contains maps of 
  * StatisticsTypes to ResourceTypes and Statistics instances to 
@@ -148,7 +146,7 @@ public class SampleCollector {
    * @param config defines the configuration for the StatArchiveHandler
    * @param nanosTimeStamp the nanos time stamp to initialize stat archiver with
    */
-  @SuppressFBWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD", justification="There is never more than one SampleCollector instance.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD", justification="There is never more than one SampleCollector instance.") 
   public void initialize(StatArchiveHandlerConfig config, long nanosTimeStamp) {
     synchronized (SampleCollector.class) {
       instance = this;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/DebuggerSupport.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/DebuggerSupport.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/DebuggerSupport.java
index a5f8343..49ce32e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/DebuggerSupport.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/DebuggerSupport.java
@@ -20,8 +20,6 @@ package com.gemstone.gemfire.internal.util;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  *
  * @author Eric Zoerner
@@ -38,7 +36,7 @@ public abstract class DebuggerSupport  {
     waitForJavaDebugger(logger, null);
   }
   
-  @SuppressFBWarnings(value="IL_INFINITE_LOOP", justification="Endless loop is for debugging purposes.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IL_INFINITE_LOOP", justification="Endless loop is for debugging purposes.") 
   public static void waitForJavaDebugger(LogWriterI18n logger, String extraLogMsg) {
     boolean cont = false;
     String msg = ":";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/SingletonValue.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/SingletonValue.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/SingletonValue.java
index c04e467..5f902b6 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/SingletonValue.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/SingletonValue.java
@@ -22,8 +22,6 @@ import java.io.InterruptedIOException;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * A builder that caches the singleton value. 
  * 
@@ -205,7 +203,7 @@ public class SingletonValue<T extends Closeable> {
     }
   }
   
-  @SuppressFBWarnings(
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(
       value="UL_UNRELEASED_LOCK", 
       justification="findbugs is wrong and Darrel agrees")
   public T get() throws IOException {
@@ -263,7 +261,7 @@ public class SingletonValue<T extends Closeable> {
     }
   }
   
-  @SuppressFBWarnings(
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(
       value="UL_UNRELEASED_LOCK", 
       justification="findbugs is wrong and Darrel agrees")
   private T acquireValue() throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/concurrent/CustomEntryConcurrentHashMap.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/concurrent/CustomEntryConcurrentHashMap.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/concurrent/CustomEntryConcurrentHashMap.java
index ad38724..75dc330 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/concurrent/CustomEntryConcurrentHashMap.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/concurrent/CustomEntryConcurrentHashMap.java
@@ -51,7 +51,6 @@ package com.gemstone.gemfire.internal.util.concurrent;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.lang.reflect.Array;
 import java.util.AbstractCollection;
 import java.util.AbstractMap;
 import java.util.AbstractSet;
@@ -59,7 +58,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Enumeration;
-import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -78,8 +76,6 @@ import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
 import com.gemstone.gemfire.internal.size.SingleObjectSizer;
 import com.gemstone.gemfire.internal.util.ArrayUtils;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 /**
  * A hash table supporting full concurrency of retrievals and adjustable
  * expected concurrency for updates. This class obeys the same functional
@@ -1436,7 +1432,7 @@ RETRYLOOP:
    * @return the number of key-value mappings in this map
    */
   @Override
-  @SuppressFBWarnings(value="UL_UNRELEASED_LOCK", justification="The lock() calls are followed by unlock() calls without finally-block. Leaving this as is because it's lifted from JDK code and we want to minimize changes.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK", justification="The lock() calls are followed by unlock() calls without finally-block. Leaving this as is because it's lifted from JDK code and we want to minimize changes.") 
   public final int size() {
     final Segment<K, V>[] segments = this.segments;
     long sum = 0;
@@ -1536,7 +1532,7 @@ RETRYLOOP:
    *           if the specified value is null
    */
   @Override
-  @SuppressFBWarnings(value="UL_UNRELEASED_LOCK", justification="Leaving this as is because it's lifted from JDK code and we want to minimize changes.") 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK", justification="Leaving this as is because it's lifted from JDK code and we want to minimize changes.") 
   public final boolean containsValue(final Object value) {
     if (value == null) {
       throw new NullPointerException();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4d6df30a/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 4c3c16e..3e6b6a5 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -58,6 +58,7 @@ spring-data-gemfire.version = 1.5.1.RELEASE
 spring-hateos.version = 0.16.0.RELEASE
 spring-shell.version = 1.0.0.RELEASE
 springframework.version = 3.2.12.RELEASE
+stephenc-findbugs.version = 1.3.9-1
 spymemcached.version = 2.9.0
 swagger.version = 1.3.2
 swagger-springmvc.version = 0.8.2



[09/50] [abbrv] incubator-geode git commit: added null check

Posted by kl...@apache.org.
added null check


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

Branch: refs/heads/feature/GEODE-217
Commit: dce479e4c54ff3f355158ee911ae4dae90857827
Parents: 2258d74
Author: Hitesh Khamesra <hk...@pivotal.io>
Authored: Tue Dec 1 10:30:25 2015 -0800
Committer: Hitesh Khamesra <hk...@pivotal.io>
Committed: Tue Dec 1 16:50:35 2015 -0800

----------------------------------------------------------------------
 .../internal/membership/gms/messenger/JGroupsMessenger.java        | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dce479e4/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
index 3a00f62..4e68b63 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
@@ -404,7 +404,7 @@ public class JGroupsMessenger implements Messenger {
     logger.info("processing JGroups IOException: " + e.getMessage());
     NetView v = this.view;
     JGAddress jgMbr = (JGAddress)dest;
-    if (v != null) {
+    if (jgMbr != null && v != null) {
       List<InternalDistributedMember> members = v.getMembers();
       InternalDistributedMember recipient = null;
       for (InternalDistributedMember mbr: members) {


[02/50] [abbrv] incubator-geode git commit: corrections for GMSHealthMonitor from an impromptu code review

Posted by kl...@apache.org.
corrections for GMSHealthMonitor from an impromptu code review

Engineers working on the GEODE-77 effort reviewed some of the final-check
code in GMSHealthMonitor and came up with these corrections.

This commit also includes some test clean-up that I hope will help with
GEODE-589.


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

Branch: refs/heads/feature/GEODE-217
Commit: 253549d555abc5cb8bb12595a9c4ac09052ad94d
Parents: e4448ad
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Dec 1 08:35:15 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 1 08:35:15 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/fd/GMSHealthMonitor.java       | 13 ++-----------
 .../gms/messenger/JGroupsMessenger.java           |  8 +++++++-
 .../internal/cache/xmlcache/CacheCreation.java    |  7 ++++++-
 .../cache30/ClientMembershipDUnitTest.java        | 12 +++++++++---
 .../gemfire/cache30/ClientServerCCEDUnitTest.java |  2 +-
 ...istributedAckPersistentRegionCCEDUnitTest.java | 18 +++++++++++++++---
 .../cache30/DistributedAckRegionDUnitTest.java    |  2 +-
 .../DistributedNoAckRegionCCEDUnitTest.java       | 11 +++++++++--
 .../gemfire/cache30/MultiVMRegionTestCase.java    |  4 ++--
 .../gemfire/cache30/TXDistributedDUnitTest.java   |  2 +-
 .../cache/locks/TXLockServiceDUnitTest.java       |  2 +-
 .../src/test/java/dunit/DistributedTestCase.java  |  9 +++++++++
 .../java/dunit/standalone/ProcessManager.java     |  8 --------
 13 files changed, 63 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
index 1c005cc..bd60236 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
@@ -441,13 +441,9 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
           }
           TimeStamp ts = memberTimeStamps.get(member);
           if (pingResp.getResponseMsg() == null) {
-            // double check the activity map
             if (isStopping) {
               return true;
             }
-            if (checkRecentActivity(member)) {
-              return true;
-            }
             logger.trace("no heartbeat response received from {} and no recent activity", member);
             return false;
           } else {
@@ -486,10 +482,6 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
    */
   private boolean doTCPCheckMember(InternalDistributedMember suspectMember, int port) {
     Socket clientSocket = null;
-    // first check for a recent timestamp
-    if (checkRecentActivity(suspectMember)) {
-      return true;
-    }
     try {
       logger.debug("Checking member {} with TCP socket connection {}:{}.", suspectMember, suspectMember.getInetAddress(), port);
       clientSocket = SocketCreator.getDefaultInstance().connect(suspectMember.getInetAddress(), port,
@@ -516,10 +508,10 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
           return true;
         } else {
           //received ERROR
-          return checkRecentActivity(suspectMember);
+          return false;
         }
       } else {// cannot establish TCP connection with suspect member
-        return checkRecentActivity(suspectMember);
+        return false;
       }
     } catch (SocketTimeoutException e) {
       logger.debug("tcp/ip connection timed out");
@@ -1192,7 +1184,6 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
               }
               if (!failed) {
                 logger.info("Final check passed");
-                contactedBy(mbr);
               }
               // whether it's alive or not, at this point we allow it to
               // be watched again

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
index b37c930..91c32ae 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
@@ -551,7 +551,13 @@ public class JGroupsMessenger implements Messenger {
     }
     
     filterOutgoingMessage(msg);
-    
+
+    // JGroupsMessenger does not support direct-replies, so register
+    // the message's processor if necessary
+    if ((msg instanceof DirectReplyMessage) && msg.isDirectAck() && msg.getProcessorId() <= 0) {
+      ((DirectReplyMessage)msg).registerProcessor();
+    }
+
     InternalDistributedMember[] destinations = msg.getRecipients();
     boolean allDestinations = msg.forAll();
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
index ed67ca4..f7063bc 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
@@ -226,6 +226,11 @@ public class CacheCreation implements InternalCache {
   public CacheCreation() {
     this(false);
   }
+  
+  /** clear thread locals that may have been set by previous uses of CacheCreation */
+  public static void clearThreadLocals() {
+    createInProgress = new ThreadLocal<>();
+  }
 
   /**
    * @param forParsing if true then this creation is used for parsing xml;
@@ -357,7 +362,7 @@ public class CacheCreation implements InternalCache {
    * Used from PoolManager to defer to CacheCreation as a manager of pools.
    * @since 5.7
    */
-  private static final ThreadLocal createInProgress = new ThreadLocal();
+  private static ThreadLocal createInProgress = new ThreadLocal();
 
   /**
    * Returns null if the current thread is not doing a CacheCreation create.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
index d9e9f4c..76b5b75 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
@@ -1328,12 +1328,16 @@ public class ClientMembershipDUnitTest extends ClientServerTestCase {
         Properties config = new Properties();
         config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
         config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+        // 11/30/2015 this test is periodically failing during distributedTest runs
+        // so we are setting the log-level to fine to figure out what's going on
+        config.setProperty(DistributionConfig.LOG_LEVEL_NAME, "fine");
         getSystem(config);
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, -1, null);
+        Pool p = ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, -1, null);
         createRegion(name, factory.create());
         assertNotNull(getRootRegion().getSubregion(name));
+        assertTrue(p.getServers().size() > 0);
       }
     };
 
@@ -1376,8 +1380,10 @@ public class ClientMembershipDUnitTest extends ClientServerTestCase {
       String connectedClient = (String)iter.next();
       getLogWriter().info("[testGetConnectedClients] checking for client " + connectedClient);
       assertTrue(clientMemberIds.contains(connectedClient));
-      getLogWriter().info("[testGetConnectedClients] count for connectedClient: " + 
-                          connectedClients.get(connectedClient));
+      Object[] result = (Object[])connectedClients.get(connectedClient);
+      getLogWriter().info("[testGetConnectedClients] result: " + 
+                          (result==null? "none"
+                              : String.valueOf(result[0])+"; connections="+result[1]));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerCCEDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerCCEDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerCCEDUnitTest.java
index 9847d7b..784d0c8 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerCCEDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerCCEDUnitTest.java
@@ -613,7 +613,7 @@ public class ClientServerCCEDUnitTest extends CacheTestCase {
         // bug #50683 - secondary durable queue retains all GC messages
         cf.set("durable-client-id", ""+vm.getPid());
         cf.set("durable-client-timeout", "" + 200);
-        cf.set("log-level", "fine");
+        cf.set("log-level", getDUnitLogLevel());
         ClientCache cache = getClientCache(cf);
         ClientRegionFactory crf = cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY);
         crf.setConcurrencyChecksEnabled(ccEnabled);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
index 3994b49..b17c3c3 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
@@ -36,17 +36,17 @@ import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-
+
 import dunit.AsyncInvocation;
 import dunit.Host;
 import dunit.SerializableCallable;
 import dunit.SerializableRunnable;
 import dunit.VM;
 import dunit.DistributedTestCase.WaitCriterion;
-
+
 import java.io.IOException;
 import java.util.Map;
-
+
 import junit.framework.Assert;
 
 /**
@@ -73,6 +73,18 @@ public class DistributedAckPersistentRegionCCEDUnitTest extends DistributedAckRe
     return factory.create();
   }
 
+//  public void testClearWithConcurrentEventsAsync() throws Exception {
+//    int end = 100;
+//    for (int i=0; i<end; i++) {
+//      System.out.println("Starting run #" + i);
+//      super.testClearWithConcurrentEventsAsync();
+//      if (i<(end-1)) {
+//        tearDown();
+//        setUp();
+//      }
+//    }
+//  }
+  
   public void testClearOnNonReplicateWithConcurrentEvents() {}
   
   public void testConcurrentEventsOnNonReplicatedRegion() {}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionDUnitTest.java
index d3ef4ac..cf258b4 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionDUnitTest.java
@@ -74,7 +74,7 @@ public class DistributedAckRegionDUnitTest extends MultiVMRegionTestCase {
   public Properties getDistributedSystemProperties() {
     Properties p = new Properties();
     p.put(DistributionConfig.STATISTIC_SAMPLING_ENABLED_NAME, "true");
-    p.put(DistributionConfig.LOG_LEVEL_NAME, "config");
+    p.put(DistributionConfig.LOG_LEVEL_NAME, getDUnitLogLevel());
     return p;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEDUnitTest.java
index 791341e..4c77562 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEDUnitTest.java
@@ -266,8 +266,15 @@ public class DistributedNoAckRegionCCEDUnitTest extends
   }
   
   
-  public void testTombstones() {
-    versionTestTombstones();
+  public void testTombstones() throws Exception {
+//    for (int i=0; i<1000; i++) {
+//      System.out.println("starting run #"+i);
+      versionTestTombstones();
+//      if (i < 999) {
+//        tearDown();
+//        setUp();
+//      }
+//    }
   }
   
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
index b67b1f9..b995494 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
@@ -8229,12 +8229,12 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
     return !o1.equals(o2);
   }
   
-  protected AsyncInvocation performOps4ClearWithConcurrentEvents(VM vm, final int opNum) {
+  protected AsyncInvocation performOps4ClearWithConcurrentEvents(VM vm, final int msToRun) {
     SerializableRunnable performOps = new SerializableRunnable("perform concurrent ops") {
       public void run() {
         try {
           boolean includeClear = true;
-          doOpsLoop(opNum, includeClear);
+          doOpsLoop(msToRun, includeClear);
         } catch (CacheException e) {
           fail("while performing concurrent operations", e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/TXDistributedDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/TXDistributedDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/TXDistributedDUnitTest.java
index b318841..c0034cd 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/TXDistributedDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/TXDistributedDUnitTest.java
@@ -522,7 +522,7 @@ public class TXDistributedDUnitTest extends CacheTestCase {
   @Override
   public Properties getDistributedSystemProperties() {
     Properties p = super.getDistributedSystemProperties();
-    p.put("log-level", "fine");
+    p.put("log-level", getDUnitLogLevel());
     return p;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/locks/TXLockServiceDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/locks/TXLockServiceDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/locks/TXLockServiceDUnitTest.java
index 002854e..bc6fad1 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/locks/TXLockServiceDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/locks/TXLockServiceDUnitTest.java
@@ -661,7 +661,7 @@ public class TXLockServiceDUnitTest extends DistributedTestCase {
   
   public Properties getDistributedSystemProperties() {
     Properties props = super.getDistributedSystemProperties();
-    props.setProperty("log-level", "fine");
+    props.setProperty("log-level", getDUnitLogLevel());
     return props;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/test/java/dunit/DistributedTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/DistributedTestCase.java b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
index 7db4033..a3d4785 100755
--- a/gemfire-core/src/test/java/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
@@ -49,6 +49,7 @@ import com.gemstone.gemfire.cache.hdfs.internal.HDFSStoreImpl;
 import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HoplogConfig;
 import com.gemstone.gemfire.cache.query.QueryTestUtils;
 import com.gemstone.gemfire.cache.query.internal.QueryObserverHolder;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.GlobalLockingDUnitTest;
 import com.gemstone.gemfire.cache30.MultiVMRegionTestCase;
 import com.gemstone.gemfire.cache30.RegionTestCase;
@@ -76,6 +77,7 @@ import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.tier.sockets.DataSerializerPropogationDUnitTest;
+import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogService;
@@ -756,6 +758,9 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
     closeCache();
     
     SocketCreator.resolve_dns = true;
+    CacheCreation.clearThreadLocals();
+    System.getProperties().remove("gemfire.log-level");
+    System.getProperties().remove("jgroups.resolve_dns");
     InitialImageOperation.slowImageProcessing = 0;
     DistributionMessageObserver.setInstance(null);
     QueryTestUtils.setCache(null);
@@ -767,10 +772,14 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
     MultiVMRegionTestCase.CCRegion = null;
     InternalClientMembership.unregisterAllListeners();
     ClientStatsManager.cleanupForTests();
+    ClientServerTestCase.AUTO_LOAD_BALANCE = false;
     unregisterInstantiatorsInThisVM();
     DistributionMessageObserver.setInstance(null);
     QueryObserverHolder.reset();
     DiskStoreObserver.setInstance(null);
+    System.getProperties().remove("gemfire.log-level");
+    System.getProperties().remove("jgroups.resolve_dns");
+    
     if (InternalDistributedSystem.systemAttemptingReconnect != null) {
       InternalDistributedSystem.systemAttemptingReconnect.stopReconnecting();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/253549d5/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java b/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
index b6a4493..60ac04d 100644
--- a/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
+++ b/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
@@ -34,11 +34,8 @@ import java.util.Map;
 
 import org.apache.commons.io.FileUtils;
 
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
-import com.gemstone.gemfire.internal.sequencelog.SequenceLoggerImpl;
 
 import dunit.RemoteDUnitVMIF;
 
@@ -47,8 +44,6 @@ import dunit.RemoteDUnitVMIF;
  *
  */
 public class ProcessManager {
-  public static final boolean IS_WINDOWS = System.getProperty("os.name").contains("Windows");
-  
   private int namingPort;
   private Map<Integer, ProcessHolder> processes = new HashMap<Integer, ProcessHolder>();
   private File log4jConfig;
@@ -179,9 +174,6 @@ public class ProcessManager {
       "-Dgemfire.DEFAULT_MAX_OPLOG_SIZE=10",
       "-Dgemfire.disallowMcastDefaults=true",
       "-ea",
-      // use IPv4 on Windows
-      // see https://github.com/belaban/JGroups/wiki/FAQ
-//      (IS_WINDOWS?"-Djava.net.preferIPv4Stack=true":""),
       agent,
       "dunit.standalone.ChildVM"
     };


[25/50] [abbrv] incubator-geode git commit: GEODE-53 - Applying latest feedback from mailing list:

Posted by kl...@apache.org.
GEODE-53 - Applying latest feedback from mailing list:

 - Moving mailing lists to top-level menu and up in the community page
 - Removed logos and Tools section
 - Changed mailing list archive link from ASF to markmail (better UI)
 - Merged fixes from PR #47


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

Branch: refs/heads/feature/GEODE-217
Commit: d16e78d1db74af2de0e03502078650850c5b48aa
Parents: 31e85e1
Author: William Markito <wm...@pivotal.io>
Authored: Mon Dec 7 16:05:52 2015 -0800
Committer: William Markito <wm...@pivotal.io>
Committed: Mon Dec 7 16:05:52 2015 -0800

----------------------------------------------------------------------
 gemfire-site/content/community/index.html       | 68 ++++++++++----------
 gemfire-site/content/index.html                 |  9 +--
 gemfire-site/content/releases/index.html        |  3 +-
 .../website/content/community/index.html        | 65 +++++++++----------
 gemfire-site/website/content/index.html         |  6 +-
 gemfire-site/website/layouts/header.html        |  3 +-
 6 files changed, 78 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d16e78d1/gemfire-site/content/community/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/community/index.html b/gemfire-site/content/community/index.html
index caf98e2..b1d0783 100644
--- a/gemfire-site/content/community/index.html
+++ b/gemfire-site/content/community/index.html
@@ -223,7 +223,8 @@
                 <li><a href="https://issues.apache.org/jira/browse/GEODE/"
                        target="_blank"><span class="icns icon-bug"></span></a></li>
                 <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank"><span class="icns icon-stackexchange"></span></a></li>
-                <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li>
+                <li><a href="/community/#mailing-lists"><span class="icns icon-envelope"></span></a></li>
+                <!-- <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li> -->
                 <li><a href="https://twitter.com/apachegeode" target="_blank"><span class="icns icon-twitter"></span></a></li>
                 <li><a href="https://cwiki.apache.org/confluence/display/geode/" target="_blank"><span class="icns icon-edit"></span></a></li>
                 <li><a href="/releases/"><span class="icns icon-releases"></span></a></li>
@@ -250,6 +251,35 @@
 	</div>
 </section>
 
+<section class="bf-community">
+    <div class="container">
+    	<div class="row">
+    	    <div class="col-md-12">
+    	    	<h2 class="icns-envelope" id="mailing-lists"><span>Mailing-lists</span></h2>
+			</div>
+		</div>
+		<div class="row">
+	    	<div class="col-md-4">
+	    		<h3>Users</h3>
+	    		<p><em>Perfect if you build apps against Apache Geode or deploy Apache Geode.</em></p>
+	    		<p>To subscribe, send a blank email to<br/><a href="mailto:user-subscribe@geode.incubator.apache.org">user-subscribe@geode.incubator.apache.org</a>.</p>
+	    		<p>You can also <a href="http://markmail.org/search/?q=list%3Aorg.apache.geode.user+order%3Adate-backward">read the archives</a>.</p>
+			</div>
+	    	<div class="col-md-4">
+	    		<h3>Dev</h3>
+	    		<p><em>If you are building contributions & modifications to Apache Geode this is the list for you.</em><p>
+	    		<p>To subscribe, send a blank email to<br/><a href="mailto:dev-subscribe@geode.incubator.apache.org">dev-subscribe@geode.incubator.apache.org</a>.</p>
+	    		<p>You can also <a href="http://markmail.org/search/?q=list%3Aorg.apache.geode.dev+order%3Adate-backward">read the archives</a>.</p>
+			</div>
+	    	<div class="col-md-4">
+	    		<h3>Commits</h3>
+	    		<p><em>This list receives an email whenever new code is contributed to Apache Geode.</em><p>
+	    		<p>To subscribe, send a blank email to<br/><a href="mailto:commits-subscribe@geode.incubator.apache.org">commits-subscribe@geode.incubator.apache.org</a>.</p>
+	    		<p>You can also <a href="http://markmail.org/search/?q=list%3Aorg.apache.geode.commits+order%3Adate-backward">read the archives</a>.</p>
+			</div>
+		</div>
+	</div>
+</section>
 
 <section class="bf-community">
     <div class="container">
@@ -341,36 +371,6 @@
     <div class="container">
     	<div class="row">
     	    <div class="col-md-12">
-    	    	<h2 class="icns-envelope" id="mailing-lists"><span>Mailing-lists</span></h2>
-			</div>
-		</div>
-		<div class="row">
-	    	<div class="col-md-4">
-	    		<h3>Users</h3>
-	    		<p><em>Perfect if you build apps against Apache Geode or deploy Apache Geode.</em></p>
-	    		<p>To subscribe, send a blank email to<br/><a href="mailto:user-subscribe@geode.incubator.apache.org">user-subscribe@geode.incubator.apache.org</a>.</p>
-	    		<p>You can also <a href="https://mail-archives.apache.org/mod_mbox/incubator-geode-user/">read the archives</a>.</p>
-			</div>
-	    	<div class="col-md-4">
-	    		<h3>Dev</h3>
-	    		<p><em>If you are building contributions & modifications to Apache Geode this is the list for you.</em><p>
-	    		<p>To subscribe, send a blank email to<br/><a href="mailto:dev-subscribe@geode.incubator.apache.org">dev-subscribe@geode.incubator.apache.org</a>.</p>
-	    		<p>You can also <a href="https://mail-archives.apache.org/mod_mbox/incubator-geode-dev/">read the archives</a>.</p>
-			</div>
-	    	<div class="col-md-4">
-	    		<h3>Commits</h3>
-	    		<p><em>This list receives an email whenever new code is contributed to Apache Geode.</em><p>
-	    		<p>To subscribe, send a blank email to<br/><a href="mailto:commits-subscribe@geode.incubator.apache.org">commits-subscribe@geode.incubator.apache.org</a>.</p>
-	    		<p>You can also <a href="https://mail-archives.apache.org/mod_mbox/incubator-geode-commits/">read the archives</a>.</p>
-			</div>
-		</div>
-	</div>
-</section>
-
-<section class="bf-community">
-    <div class="container">
-    	<div class="row">
-    	    <div class="col-md-12">
     	    	<h2 class="icns-briefcase" id="deployments"><span>Deployments</span></h2>
 			</div>
 		</div>
@@ -502,14 +502,14 @@
 	</div>
 </section>
 
-<section class="bf-community tools">
+<!-- <section class="bf-community tools">
     <div class="container">
     	<div class="row">
     	    <div class="col-md-12">
     	    	<h2 class="icns-tools" id="tools"><span>Tools</span></h2>
 			</div>
 		</div>
-		<div class="row">
+		 <div class="row">
 			<div class="col-md-2 text-center">
 				<a href="http://www.jetbrains.com/idea/" target="_blank"><img src="/img/intellij.png" /></a>
 				<p>IntelliJ IDEA</p>
@@ -521,7 +521,7 @@
 		</div>
 		These great companies have provided free product licenses to the Apache Geode team. We use these tools and love them.  Thank you!
 	</div>
-</section>
+</section> -->
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d16e78d1/gemfire-site/content/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/index.html b/gemfire-site/content/index.html
index de8327d..a450eea 100644
--- a/gemfire-site/content/index.html
+++ b/gemfire-site/content/index.html
@@ -54,7 +54,8 @@
                 <li><a href="https://issues.apache.org/jira/browse/GEODE/"
                        target="_blank"><span class="icns icon-bug"></span></a></li>
                 <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank"><span class="icns icon-stackexchange"></span></a></li>
-                <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li>
+                <li><a href="/community/#mailing-lists"><span class="icns icon-envelope"></span></a></li>
+                <!-- <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li> -->
                 <li><a href="https://twitter.com/apachegeode" target="_blank"><span class="icns icon-twitter"></span></a></li>
                 <li><a href="https://cwiki.apache.org/confluence/display/geode/" target="_blank"><span class="icns icon-edit"></span></a></li>
                 <li><a href="/releases/"><span class="icns icon-releases"></span></a></li>
@@ -154,7 +155,7 @@
 <section class="bf-questions">
     <div class="container">
             <div class="col-md-12 text-center cta">
-                And much more... Interested ? You can check our <a href="https://cwiki.apache.org/confluence/display/GEODE/Index#Index-Geodein5minutesGeodein5minutes" target="_blank" class="btn btn-inverse btn-lg">Geode in 5 minutes tutorial</a> <span class="avoidwrap">, ask a question on the <a href="/community/" class="btn btn-inverse btn-lg">Mailing lists</a> or <a href="http://stackoverflow.com/search?q=Apache%20Geode" class="btn btn-inverse btn-lg">StackOverflow</a></span>
+                And much more... Interested? You can check our <a href="https://cwiki.apache.org/confluence/display/GEODE/Index#Index-Geodein5minutesGeodein5minutes" target="_blank" class="btn btn-inverse btn-lg">Geode in 5 minutes tutorial</a> <span class="avoidwrap">, ask a question on the <a href="/community/" class="btn btn-inverse btn-lg">Mailing lists</a> or <a href="http://stackoverflow.com/search?q=Apache%20Geode" class="btn btn-inverse btn-lg">StackOverflow</a></span>
             </div>
     </div>
 </section
@@ -167,14 +168,14 @@
                 <h2>About the Project</h2>
                 <p>Apache Geode is a data management platform that provides real-time, consistent access to data-intensive applications throughout widely distributed cloud architectures.</p>
 
-                <p>By pooling memory, CPU, network resources, and optionally local disk across multiple processes to manage application objects and behavior, it uses dynamic replication and data partitioning techniques to implement high availability, improved performance, scalability, and fault tolerance. In addition to being a distributed data container, Apache Geode is an in-memory data management system that provides reliable asynchronous event notifications and guaranteed message delivery.</p>
+                <p>By pooling memory, CPU, network resources, and (optionally) local disk across multiple processes to manage application objects and behavior, it uses dynamic replication and data partitioning techniques to implement high availability, improved performance, scalability, and fault tolerance. In addition to being a distributed data container, Apache Geode is an in-memory data management system that provides reliable asynchronous event notifications and guaranteed message delivery.</p>
 
                 <p>Apache Geode is a mature, robust technology originally developed by GemStone Systems in Beaverton, Oregon.
 Commercially available as GemFireâ„¢, the technology was first widely deployed in the financial sector as the transactional, low-latency data engine used
 in Wall Street trading platforms.
 Today Apache Geode is used by over 600 enterprise customers for high-scale business applications that must meet low latency and 24x7 availability requirements.</p>
 
-                <p>This project is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.</p>
+                <p>This project is undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.</p>
             </div>
             <!--
             <div class="col-md-4 text-left">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d16e78d1/gemfire-site/content/releases/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/releases/index.html b/gemfire-site/content/releases/index.html
index 513fb25..25bd47c 100644
--- a/gemfire-site/content/releases/index.html
+++ b/gemfire-site/content/releases/index.html
@@ -54,7 +54,8 @@
                 <li><a href="https://issues.apache.org/jira/browse/GEODE/"
                        target="_blank"><span class="icns icon-bug"></span></a></li>
                 <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank"><span class="icns icon-stackexchange"></span></a></li>
-                <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li>
+                <li><a href="/community/#mailing-lists"><span class="icns icon-envelope"></span></a></li>
+                <!-- <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li> -->
                 <li><a href="https://twitter.com/apachegeode" target="_blank"><span class="icns icon-twitter"></span></a></li>
                 <li><a href="https://cwiki.apache.org/confluence/display/geode/" target="_blank"><span class="icns icon-edit"></span></a></li>
                 <li><a href="/releases/"><span class="icns icon-releases"></span></a></li>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d16e78d1/gemfire-site/website/content/community/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/community/index.html b/gemfire-site/website/content/community/index.html
index 43c9cba..7abb0a9 100644
--- a/gemfire-site/website/content/community/index.html
+++ b/gemfire-site/website/content/community/index.html
@@ -13,6 +13,35 @@
 	</div>
 </section>
 
+<section class="bf-community">
+    <div class="container">
+    	<div class="row">
+    	    <div class="col-md-12">
+    	    	<h2 class="icns-envelope" id="mailing-lists"><span>Mailing-lists</span></h2>
+			</div>
+		</div>
+		<div class="row">
+	    	<div class="col-md-4">
+	    		<h3>Users</h3>
+	    		<p><em>Perfect if you build apps against Apache Geode or deploy Apache Geode.</em></p>
+	    		<p>To subscribe, send a blank email to<br/><a href="mailto:user-subscribe@geode.incubator.apache.org">user-subscribe@geode.incubator.apache.org</a>.</p>
+	    		<p>You can also <a href="http://markmail.org/search/?q=list%3Aorg.apache.geode.user+order%3Adate-backward">read the archives</a>.</p>
+			</div>
+	    	<div class="col-md-4">
+	    		<h3>Dev</h3>
+	    		<p><em>If you are building contributions & modifications to Apache Geode this is the list for you.</em><p>
+	    		<p>To subscribe, send a blank email to<br/><a href="mailto:dev-subscribe@geode.incubator.apache.org">dev-subscribe@geode.incubator.apache.org</a>.</p>
+	    		<p>You can also <a href="http://markmail.org/search/?q=list%3Aorg.apache.geode.dev+order%3Adate-backward">read the archives</a>.</p>
+			</div>
+	    	<div class="col-md-4">
+	    		<h3>Commits</h3>
+	    		<p><em>This list receives an email whenever new code is contributed to Apache Geode.</em><p>
+	    		<p>To subscribe, send a blank email to<br/><a href="mailto:commits-subscribe@geode.incubator.apache.org">commits-subscribe@geode.incubator.apache.org</a>.</p>
+	    		<p>You can also <a href="http://markmail.org/search/?q=list%3Aorg.apache.geode.commits+order%3Adate-backward">read the archives</a>.</p>
+			</div>
+		</div>
+	</div>
+</section>
 
 <section class="bf-community">
     <div class="container">
@@ -104,36 +133,6 @@
     <div class="container">
     	<div class="row">
     	    <div class="col-md-12">
-    	    	<h2 class="icns-envelope" id="mailing-lists"><span>Mailing-lists</span></h2>
-			</div>
-		</div>
-		<div class="row">
-	    	<div class="col-md-4">
-	    		<h3>Users</h3>
-	    		<p><em>Perfect if you build apps against Apache Geode or deploy Apache Geode.</em></p>
-	    		<p>To subscribe, send a blank email to<br/><a href="mailto:user-subscribe@geode.incubator.apache.org">user-subscribe@geode.incubator.apache.org</a>.</p>
-	    		<p>You can also <a href="https://mail-archives.apache.org/mod_mbox/incubator-geode-user/">read the archives</a>.</p>
-			</div>
-	    	<div class="col-md-4">
-	    		<h3>Dev</h3>
-	    		<p><em>If you are building contributions & modifications to Apache Geode this is the list for you.</em><p>
-	    		<p>To subscribe, send a blank email to<br/><a href="mailto:dev-subscribe@geode.incubator.apache.org">dev-subscribe@geode.incubator.apache.org</a>.</p>
-	    		<p>You can also <a href="https://mail-archives.apache.org/mod_mbox/incubator-geode-dev/">read the archives</a>.</p>
-			</div>
-	    	<div class="col-md-4">
-	    		<h3>Commits</h3>
-	    		<p><em>This list receives an email whenever new code is contributed to Apache Geode.</em><p>
-	    		<p>To subscribe, send a blank email to<br/><a href="mailto:commits-subscribe@geode.incubator.apache.org">commits-subscribe@geode.incubator.apache.org</a>.</p>
-	    		<p>You can also <a href="https://mail-archives.apache.org/mod_mbox/incubator-geode-commits/">read the archives</a>.</p>
-			</div>
-		</div>
-	</div>
-</section>
-
-<section class="bf-community">
-    <div class="container">
-    	<div class="row">
-    	    <div class="col-md-12">
     	    	<h2 class="icns-briefcase" id="deployments"><span>Deployments</span></h2>
 			</div>
 		</div>
@@ -265,14 +264,14 @@
 	</div>
 </section>
 
-<section class="bf-community tools">
+<!-- <section class="bf-community tools">
     <div class="container">
     	<div class="row">
     	    <div class="col-md-12">
     	    	<h2 class="icns-tools" id="tools"><span>Tools</span></h2>
 			</div>
 		</div>
-		<div class="row">
+		 <div class="row">
 			<div class="col-md-2 text-center">
 				<a href="http://www.jetbrains.com/idea/" target="_blank"><img src="/img/intellij.png" /></a>
 				<p>IntelliJ IDEA</p>
@@ -284,4 +283,4 @@
 		</div>
 		These great companies have provided free product licenses to the Apache Geode team. We use these tools and love them.  Thank you!
 	</div>
-</section>
+</section> -->

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d16e78d1/gemfire-site/website/content/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/index.html b/gemfire-site/website/content/index.html
index 1925150..17058e7 100644
--- a/gemfire-site/website/content/index.html
+++ b/gemfire-site/website/content/index.html
@@ -89,7 +89,7 @@ title: Performance is key. Consistency is a must.
 <section class="bf-questions">
     <div class="container">
             <div class="col-md-12 text-center cta">
-                And much more... Interested ? You can check our <a href="https://cwiki.apache.org/confluence/display/GEODE/Index#Index-Geodein5minutesGeodein5minutes" target="_blank" class="btn btn-inverse btn-lg">Geode in 5 minutes tutorial</a> <span class="avoidwrap">, ask a question on the <a href="/community/" class="btn btn-inverse btn-lg">Mailing lists</a> or <a href="http://stackoverflow.com/search?q=Apache%20Geode" class="btn btn-inverse btn-lg">StackOverflow</a></span>
+                And much more... Interested? You can check our <a href="https://cwiki.apache.org/confluence/display/GEODE/Index#Index-Geodein5minutesGeodein5minutes" target="_blank" class="btn btn-inverse btn-lg">Geode in 5 minutes tutorial</a> <span class="avoidwrap">, ask a question on the <a href="/community/" class="btn btn-inverse btn-lg">Mailing lists</a> or <a href="http://stackoverflow.com/search?q=Apache%20Geode" class="btn btn-inverse btn-lg">StackOverflow</a></span>
             </div>
     </div>
 </section
@@ -102,14 +102,14 @@ title: Performance is key. Consistency is a must.
                 <h2>About the Project</h2>
                 <p>Apache Geode is a data management platform that provides real-time, consistent access to data-intensive applications throughout widely distributed cloud architectures.</p>
 
-                <p>By pooling memory, CPU, network resources, and optionally local disk across multiple processes to manage application objects and behavior, it uses dynamic replication and data partitioning techniques to implement high availability, improved performance, scalability, and fault tolerance. In addition to being a distributed data container, Apache Geode is an in-memory data management system that provides reliable asynchronous event notifications and guaranteed message delivery.</p>
+                <p>By pooling memory, CPU, network resources, and (optionally) local disk across multiple processes to manage application objects and behavior, it uses dynamic replication and data partitioning techniques to implement high availability, improved performance, scalability, and fault tolerance. In addition to being a distributed data container, Apache Geode is an in-memory data management system that provides reliable asynchronous event notifications and guaranteed message delivery.</p>
 
                 <p>Apache Geode is a mature, robust technology originally developed by GemStone Systems in Beaverton, Oregon.
 Commercially available as GemFireâ„¢, the technology was first widely deployed in the financial sector as the transactional, low-latency data engine used
 in Wall Street trading platforms.
 Today Apache Geode is used by over 600 enterprise customers for high-scale business applications that must meet low latency and 24x7 availability requirements.</p>
 
-                <p>This project is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.</p>
+                <p>This project is undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.</p>
             </div>
             <!--
             <div class="col-md-4 text-left">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d16e78d1/gemfire-site/website/layouts/header.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/layouts/header.html b/gemfire-site/website/layouts/header.html
index 1e7f18a..8ea572a 100644
--- a/gemfire-site/website/layouts/header.html
+++ b/gemfire-site/website/layouts/header.html
@@ -220,7 +220,8 @@
                 <li><a href="https://issues.apache.org/jira/browse/GEODE/"
                        target="_blank"><span class="icns icon-bug"></span></a></li>
                 <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank"><span class="icns icon-stackexchange"></span></a></li>
-                <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li>
+                <li><a href="/community/#mailing-lists"><span class="icns icon-envelope"></span></a></li>
+                <!-- <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li> -->
                 <li><a href="https://twitter.com/apachegeode" target="_blank"><span class="icns icon-twitter"></span></a></li>
                 <li><a href="https://cwiki.apache.org/confluence/display/geode/" target="_blank"><span class="icns icon-edit"></span></a></li>
                 <li><a href="/releases/"><span class="icns icon-releases"></span></a></li>


[40/50] [abbrv] incubator-geode git commit: Merge branch 'feature/GEODE-390' into develop

Posted by kl...@apache.org.
Merge branch 'feature/GEODE-390' into develop

Closes #37


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

Branch: refs/heads/feature/GEODE-217
Commit: 386d1ac8c9ed486d7c9415ac3d14bcc78dc15c97
Parents: 80b59bf 4a07f45
Author: Jens Deppe <jd...@pivotal.io>
Authored: Tue Dec 8 14:22:29 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Tue Dec 8 14:22:29 2015 -0800

----------------------------------------------------------------------
 .../cache/partition/PartitionManager.java       | 377 ----------------
 .../partition/PartitionManagerDUnitTest.java    | 443 -------------------
 .../fixed/FixedPartitioningTestBase.java        |  83 ----
 ...ngWithColocationAndPersistenceDUnitTest.java | 106 -----
 4 files changed, 1009 deletions(-)
----------------------------------------------------------------------



[17/50] [abbrv] incubator-geode git commit: Adding additional unit tests for GMSHealthMonitor and GMSJoinLeave Moved GMSHealthMonitorJUnitTest to package level of GMSHealthMonitor Minor refactoring to allow testing

Posted by kl...@apache.org.
Adding additional unit tests for GMSHealthMonitor and GMSJoinLeave
Moved GMSHealthMonitorJUnitTest to package level of GMSHealthMonitor
Minor refactoring to allow testing


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

Branch: refs/heads/feature/GEODE-217
Commit: 3259c0238a97413268297b663219d72bf8b130c9
Parents: e0bf685
Author: Jason Huynh <hu...@gmail.com>
Authored: Wed Dec 2 15:11:40 2015 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Fri Dec 4 10:40:52 2015 -0800

----------------------------------------------------------------------
 .../membership/InternalDistributedMember.java   |  16 +-
 .../membership/gms/fd/GMSHealthMonitor.java     |  60 +-
 .../membership/gms/membership/GMSJoinLeave.java |   8 +
 .../gms/fd/GMSHealthMonitorJUnitTest.java       | 576 +++++++++++++++++++
 .../gms/membership/GMSJoinLeaveJUnitTest.java   |  31 +-
 .../fd/GMSHealthMonitorJUnitTest.java           | 461 ---------------
 6 files changed, 666 insertions(+), 486 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3259c023/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
index 10478b7..b112b92 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
@@ -292,7 +292,21 @@ public final class InternalDistributedMember
    * @throws UnknownHostException if the given hostname cannot be resolved
    */
   public InternalDistributedMember(String i, int p, Version version) throws UnknownHostException {
-    netMbr = MemberFactory.newNetMember(i, p);
+    this (i, p, version, MemberFactory.newNetMember(i, p));
+  }
+  
+  /**
+   * Create a InternalDistributedMember referring to the current host (as defined by the given
+   * string).<p>
+   *
+   * <b>
+   * THIS METHOD IS FOR TESTING ONLY.  DO NOT USE IT TO CREATE IDs FOR
+   * USE IN THE PRODUCT.  IT DOES NOT PROPERLY INITIALIZE ATTRIBUTES NEEDED
+   * FOR P2P FUNCTIONALITY.
+   * </b>
+   **/
+  public InternalDistributedMember(String i, int p, Version version, NetMember netMember) throws UnknownHostException {
+    netMbr = netMember;
     defaultToCurrentHost();
     this.vmKind = DistributionManager.NORMAL_DM_TYPE;
     this.versionObj = version;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3259c023/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
index fcda1a0..cc64f9b 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
@@ -56,6 +56,7 @@ import org.jgroups.util.UUID;
 
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.GemFireConfigException;
+import com.gemstone.gemfire.SystemConnectException;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
@@ -173,9 +174,8 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
   
   // For TCP check
   private ExecutorService serverSocketExecutor;
-  private static final int OK = 0x7B;
-  private static final int ERROR = 0x00;  
-  private InetAddress socketAddress;
+  static final int OK = 0x7B;
+  static final int ERROR = 0x00;  
   private volatile int socketPort;
   private volatile ServerSocket serverSocket;
 
@@ -491,11 +491,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
         InputStream in = clientSocket.getInputStream();
         DataOutputStream out = new DataOutputStream(clientSocket.getOutputStream());
         GMSMember gmbr = (GMSMember) suspectMember.getNetMember();
-        out.writeShort(Version.CURRENT_ORDINAL);
-        out.writeInt(gmbr.getVmViewId());
-        out.writeLong(gmbr.getUuidLSBs());
-        out.writeLong(gmbr.getUuidMSBs());
-        out.flush();
+        writeMemberToStream(gmbr, out);
         clientSocket.shutdownOutput();
         logger.debug("Connected - reading response", suspectMember);
         int b = in.read();
@@ -531,6 +527,14 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
     return false;
   }
   
+  void writeMemberToStream(GMSMember gmbr, DataOutputStream out) throws IOException {
+    out.writeShort(Version.CURRENT_ORDINAL);
+    out.writeInt(gmbr.getVmViewId());
+    out.writeLong(gmbr.getUuidLSBs());
+    out.writeLong(gmbr.getUuidMSBs());
+    out.flush();
+  }
+  
   /*
    * (non-Javadoc)
    * 
@@ -610,32 +614,37 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
 
   }
 
-  /**
-   * start the thread that listens for tcp/ip connections and responds
-   * to connection attempts
-   */
-  private void startTcpServer() {
-    // allocate a socket here so there are no race conditions between knowing the FD
-    // socket port and joining the system
-    socketAddress = localAddress.getInetAddress();
-    int[] portRange = services.getConfig().getMembershipPortRange();            
+  ServerSocket createServerSocket(InetAddress socketAddress, int[] portRange) {
+    ServerSocket serverSocket = null;
     try {
       serverSocket = SocketCreator.getDefaultInstance().createServerSocketUsingPortRange(socketAddress, 50/*backlog*/, true/*isBindAddress*/, false/*useNIO*/, 65536/*tcpBufferSize*/, portRange);
       socketPort = serverSocket.getLocalPort();
     } catch (IOException e) {
       throw new GemFireConfigException("Unable to allocate a failure detection port in the membership-port range", e);
+    } catch (SystemConnectException e) {
+      throw new GemFireConfigException("Unable to allocate a failure detection port in the membership-port range", e);
     }
+    return serverSocket;
+  }
+  
+  /**
+   * start the thread that listens for tcp/ip connections and responds
+   * to connection attempts
+   */
+  private void startTcpServer(ServerSocket ssocket) {
+    // allocate a socket here so there are no race conditions between knowing the FD
+    // socket port and joining the system
 
     serverSocketExecutor.execute(new Runnable() {
       @Override
       public void run() {
-        logger.info("Started failure detection server thread on {}:{}.", socketAddress, socketPort);
+        logger.info("Started failure detection server thread on {}:{}.", ssocket.getInetAddress(), socketPort);
         Socket socket = null;
         try {
           while (!services.getCancelCriterion().isCancelInProgress() 
               && !GMSHealthMonitor.this.isStopping) {
             try {
-              socket = serverSocket.accept();
+              socket = ssocket.accept();
               if (GMSHealthMonitor.this.playingDead) {
                 continue;
               }
@@ -658,9 +667,9 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
           logger.info("GMSHealthMonitor server thread exiting");
         } finally {
           // close the server socket
-          if (serverSocket != null && !serverSocket.isClosed()) {
+          if (ssocket != null && !ssocket.isClosed()) {
             try {
-              serverSocket.close();
+              ssocket.close();
               serverSocket = null;
               logger.info("GMSHealthMonitor server socket closed.");
             } catch (IOException e) {
@@ -841,8 +850,9 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
 
   @Override
   public void started() {
-    this.localAddress = services.getMessenger().getMemberID();
-    startTcpServer();
+    setLocalAddress( services.getMessenger().getMemberID());
+    serverSocket = createServerSocket(localAddress.getInetAddress(), services.getConfig().getMembershipPortRange());
+    startTcpServer(serverSocket);
     startHeartbeatThread();
   }
 
@@ -941,6 +951,10 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
   public void emergencyClose() {
     stopServices();
   }
+  
+  void setLocalAddress(InternalDistributedMember idm) {
+    this.localAddress = idm;
+  }
 
   @Override
   public void processMessage(DistributionMessage m) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3259c023/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
index 2986238..84a0bd7 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
@@ -1058,6 +1058,14 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
       joinResponse.notify();
     }
   }
+  
+  /**
+   * for testing, do not use in any other case as it is not thread safe
+   * @param req
+   */
+  JoinResponseMessage[] getJoinResponseMessage() {
+    return joinResponse;
+  }
 
   private void processFindCoordinatorRequest(FindCoordinatorRequest req) {
     FindCoordinatorResponse resp;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3259c023/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
new file mode 100644
index 0000000..86205b9
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
@@ -0,0 +1,576 @@
+/*
+ * 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.distributed.internal.membership.gms.fd;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.jgroups.util.UUID;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.distributed.internal.membership.NetView;
+import com.gemstone.gemfire.distributed.internal.membership.gms.GMSMember;
+import com.gemstone.gemfire.distributed.internal.membership.gms.ServiceConfig;
+import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
+import com.gemstone.gemfire.distributed.internal.membership.gms.Services.Stopper;
+import com.gemstone.gemfire.distributed.internal.membership.gms.fd.GMSHealthMonitor.ClientSocketHandler;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.JoinLeave;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Messenger;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.HeartbeatMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.HeartbeatRequestMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.SuspectMembersMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.SuspectRequest;
+import com.gemstone.gemfire.internal.SocketCreator;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class GMSHealthMonitorJUnitTest {
+
+  private Services services;
+  private ServiceConfig mockConfig;
+  private DistributionConfig mockDistConfig;
+  private List<InternalDistributedMember> mockMembers;
+  private Messenger messenger;
+  private JoinLeave joinLeave;
+  private GMSHealthMonitor gmsHealthMonitor;
+  private Manager manager;
+  final long memberTimeout = 1000l;
+  private int[] portRange= new int[]{0, 65535};
+
+  @Before
+  public void initMocks() throws UnknownHostException {
+    System.setProperty("gemfire.bind-address", "localhost");
+    mockDistConfig = mock(DistributionConfig.class);
+    mockConfig = mock(ServiceConfig.class);
+    messenger = mock(Messenger.class);
+    joinLeave = mock(JoinLeave.class);
+    manager = mock(Manager.class);
+    services = mock(Services.class);
+    Stopper stopper = mock(Stopper.class);
+
+    when(mockConfig.getDistributionConfig()).thenReturn(mockDistConfig);
+    when(mockConfig.getMemberTimeout()).thenReturn(memberTimeout);
+    when(mockConfig.getMembershipPortRange()).thenReturn(portRange);
+    when(services.getConfig()).thenReturn(mockConfig);
+    when(services.getMessenger()).thenReturn(messenger);
+    when(services.getJoinLeave()).thenReturn(joinLeave);
+    when(services.getCancelCriterion()).thenReturn(stopper);
+    when(services.getManager()).thenReturn(manager);
+    when(stopper.isCancelInProgress()).thenReturn(false);
+    
+
+    if (mockMembers == null) {
+      mockMembers = new ArrayList<InternalDistributedMember>();
+      for (int i = 0; i < 7; i++) {
+        InternalDistributedMember mbr = new InternalDistributedMember("localhost", 8888 + i);
+  
+        if (i == 0 || i == 1) {
+          mbr.setVmKind(DistributionManager.LOCATOR_DM_TYPE);
+          mbr.getNetMember().setPreferredForCoordinator(true);
+        }
+        mockMembers.add(mbr);
+      }
+    }
+    when(joinLeave.getMemberID()).thenReturn(mockMembers.get(3));
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
+    gmsHealthMonitor = new GMSHealthMonitor();
+    gmsHealthMonitor.init(services);
+    gmsHealthMonitor.start();
+  }
+
+  @After
+  public void tearDown() {
+    gmsHealthMonitor.stop();
+  }
+
+  @Test
+  public void testHMServiceStarted() throws IOException {
+
+    InternalDistributedMember mbr = new InternalDistributedMember(SocketCreator.getLocalHost(), 12345);
+    mbr.setVmViewId(1);
+    when(messenger.getMemberID()).thenReturn(mbr);
+    gmsHealthMonitor.started();
+    
+    NetView v = new NetView(mbr, 1, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    gmsHealthMonitor.processMessage(new HeartbeatRequestMessage(mbr, 1));
+    verify(messenger, atLeastOnce()).send(any(HeartbeatMessage.class));
+  }
+
+  /**
+   * checks who is next neighbor
+   */
+  @Test
+  public void testHMNextNeighborVerify() throws IOException {
+
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
+    gmsHealthMonitor.started();
+
+    gmsHealthMonitor.installView(v);
+
+    Assert.assertEquals(mockMembers.get(4), gmsHealthMonitor.getNextNeighbor());
+
+  }
+
+  @Test
+  public void testHMNextNeighborAfterTimeout() throws Exception {
+    System.out.println("testHMNextNeighborAfterTimeout starting");
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+//    System.out.printf("memberID is %s view is %s\n", mockMembers.get(3), v);
+    
+    // 3rd is current member
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
+    gmsHealthMonitor.started();
+
+    gmsHealthMonitor.installView(v);
+
+    // allow the monitor to give up on the initial "next neighbor" and
+    // move on to the one after it
+    long giveup = System.currentTimeMillis() + memberTimeout + 500;
+    InternalDistributedMember expected = mockMembers.get(5);
+    InternalDistributedMember neighbor = gmsHealthMonitor.getNextNeighbor();
+    while (System.currentTimeMillis() < giveup && neighbor != expected) {
+      Thread.sleep(5);
+      neighbor = gmsHealthMonitor.getNextNeighbor();
+    }
+
+    // neighbor should change to 5th
+    System.out.println("testHMNextNeighborAfterTimeout ending");
+    Assert.assertEquals("expected " + expected + " but found " + neighbor
+        + ".  view="+v, expected, neighbor);
+  }
+
+  /**
+   * it checks neighbor before member-timeout, it should be same
+   */
+
+  @Test
+  public void testHMNextNeighborBeforeTimeout() throws IOException {
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    // 3rd is current member
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
+    gmsHealthMonitor.started();
+
+    gmsHealthMonitor.installView(v);
+
+    //Should we remove these sleeps and force the checkmember directly instead of waiting?
+    try {
+      // member-timeout is 1000 ms.  We initiate a check and choose
+      // a new neighbor at 500 ms
+      Thread.sleep(memberTimeout/GMSHealthMonitor.LOGICAL_INTERVAL - 100);
+    } catch (InterruptedException e) {
+    }
+    // neighbor should be same
+    System.out.println("next neighbor is " + gmsHealthMonitor.getNextNeighbor() +
+        "\nmy address is " + mockMembers.get(3) +
+        "\nview is " + v);
+
+    Assert.assertEquals(mockMembers.get(4), gmsHealthMonitor.getNextNeighbor());
+  }
+  
+  /***
+   * checks whether member-check thread sends suspectMembers message
+   */
+  @Test
+  public void testSuspectMembersCalledThroughMemberCheckThread() throws Exception {
+    System.out.println("testSuspectMembersCalledThroughMemberCheckThread starting");
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    // 3rd is current member
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
+    gmsHealthMonitor.started();
+
+    gmsHealthMonitor.installView(v);
+
+    // when the view is installed we start a heartbeat timeout.  After
+    // that expires we request a heartbeat
+    Thread.sleep(3*memberTimeout + 100);
+
+    System.out.println("testSuspectMembersCalledThroughMemberCheckThread ending");
+    assertTrue(gmsHealthMonitor.isSuspectMember(mockMembers.get(4)));
+  }
+
+  /***
+   * checks ping thread didn't sends suspectMembers message before timeout
+   */
+  @Test
+  public void testSuspectMembersNotCalledThroughPingThreadBeforeTimeout() {
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    // 3rd is current member
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
+    gmsHealthMonitor.started();
+    
+    gmsHealthMonitor.installView(v);
+    InternalDistributedMember neighbor = gmsHealthMonitor.getNextNeighbor();
+
+    try {
+      // member-timeout is 1000 ms
+      // plus 100 ms for ack
+      Thread.sleep(memberTimeout - 200);
+    } catch (InterruptedException e) {
+    }
+
+    assertFalse(gmsHealthMonitor.isSuspectMember(neighbor));
+  }
+
+  /***
+   * Checks whether suspect thread sends suspectMembers message
+   */
+  @Test
+  public void testSuspectMembersCalledThroughSuspectThread() throws Exception {
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+    
+    // 3rd is current member
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
+
+    gmsHealthMonitor.installView(v);
+
+    gmsHealthMonitor.suspect(mockMembers.get(1), "Not responding");
+
+    Thread.sleep(GMSHealthMonitor.MEMBER_SUSPECT_COLLECTION_INTERVAL + 1000);
+
+    verify(messenger, atLeastOnce()).send(any(SuspectMembersMessage.class));
+  }
+
+  /***
+   * Checks suspect thread doesn't sends suspectMembers message before timeout
+   */
+  @Test
+  public void testSuspectMembersNotCalledThroughSuspectThreadBeforeTimeout() {
+
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    MethodExecuted messageSent = new MethodExecuted();
+    // 3rd is current member
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
+
+    gmsHealthMonitor.installView(v);
+
+    gmsHealthMonitor.suspect(mockMembers.get(1), "Not responding");
+
+    when(messenger.send(isA(SuspectMembersMessage.class))).thenAnswer(messageSent);
+
+    try {
+      // suspect thread timeout is 200 ms
+      Thread.sleep(100l);
+    } catch (InterruptedException e) {
+    }
+
+    assertTrue("SuspectMembersMessage shouldn't have sent", !messageSent.isMethodExecuted());
+  }
+
+  /***
+   * Send remove member message after doing final check, ping Timeout
+   */
+  @Test
+  public void testRemoveMemberCalled() throws Exception {
+    System.out.println("testRemoveMemberCalled starting");
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    // 3rd is current member
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(0)); // coordinator and local member
+    gmsHealthMonitor.started();
+
+    gmsHealthMonitor.installView(v);
+    
+    Thread.sleep(memberTimeout/GMSHealthMonitor.LOGICAL_INTERVAL);
+
+    ArrayList<InternalDistributedMember> recipient = new ArrayList<InternalDistributedMember>();
+    recipient.add(mockMembers.get(0));
+    ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
+    SuspectRequest sr = new SuspectRequest(mockMembers.get(1), "Not Responding");// removing member 1
+    as.add(sr);
+    SuspectMembersMessage sm = new SuspectMembersMessage(recipient, as);
+    sm.setSender(mockMembers.get(0));
+
+    gmsHealthMonitor.processMessage(sm);
+
+    Thread.sleep(2*memberTimeout + 200);
+
+    System.out.println("testRemoveMemberCalled ending");
+    verify(joinLeave, atLeastOnce()).remove(any(InternalDistributedMember.class), any(String.class));
+  }
+
+  /***
+   * Shouldn't send remove member message before doing final check, or before ping Timeout
+   */
+  @Test
+  public void testRemoveMemberNotCalledBeforeTimeout() {
+    System.out.println("testRemoveMemberNotCalledBeforeTimeout starting");
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    // 3rd is current member
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(0)); // coordinator and local member
+    when(joinLeave.getMemberID()).thenReturn(mockMembers.get(0)); // coordinator and local member
+    gmsHealthMonitor.started();
+
+    gmsHealthMonitor.installView(v);
+
+    ArrayList<InternalDistributedMember> recipient = new ArrayList<InternalDistributedMember>();
+    recipient.add(mockMembers.get(0));
+    ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
+    SuspectRequest sr = new SuspectRequest(mockMembers.get(1), "Not Responding");// removing member 1
+    as.add(sr);
+    SuspectMembersMessage sm = new SuspectMembersMessage(recipient, as);
+    sm.setSender(mockMembers.get(0));
+
+    gmsHealthMonitor.processMessage(sm);
+
+    try {
+      // this happens after final check, ping timeout
+      Thread.sleep(memberTimeout-100);
+    } catch (InterruptedException e) {
+    }
+
+    System.out.println("testRemoveMemberNotCalledBeforeTimeout ending");
+    verify(joinLeave, never()).remove(any(InternalDistributedMember.class), any(String.class));
+  }
+
+  /***
+   * Send remove member message after doing final check for coordinator, ping timeout
+   * This test trying to remove coordinator
+   */
+  @Test
+  public void testRemoveMemberCalledAfterDoingFinalCheckOnCoordinator() throws Exception {
+
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    // preferred coordinators are 0 and 1
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(1));// next preferred coordinator
+    gmsHealthMonitor.started();
+
+    gmsHealthMonitor.installView(v);
+    
+    Thread.sleep(memberTimeout/GMSHealthMonitor.LOGICAL_INTERVAL);
+
+    ArrayList<InternalDistributedMember> recipient = new ArrayList<InternalDistributedMember>();
+    recipient.add(mockMembers.get(0));
+    recipient.add(mockMembers.get(1));
+    ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
+    SuspectRequest sr = new SuspectRequest(mockMembers.get(0), "Not Responding");// removing coordinator
+    as.add(sr);
+    SuspectMembersMessage sm = new SuspectMembersMessage(recipient, as);
+    sm.setSender(mockMembers.get(4));// member 4 sends suspect message
+
+    gmsHealthMonitor.processMessage(sm);
+
+    // this happens after final check, ping timeout = 1000 ms
+    Thread.sleep(memberTimeout + 200);
+
+    verify(joinLeave, atLeastOnce()).remove(any(InternalDistributedMember.class), any(String.class));
+  }
+
+  /***
+   * validates HealthMonitor.CheckIfAvailable api
+   */
+  @Test
+  public void testCheckIfAvailable() {
+
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    // 3rd is current member
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
+
+    gmsHealthMonitor.installView(v);
+
+    long startTime = System.currentTimeMillis();
+
+    boolean retVal = gmsHealthMonitor.checkIfAvailable(mockMembers.get(1), "Not responding", false);
+
+    long timeTaken = System.currentTimeMillis() - startTime;
+
+    assertTrue("This should have taken member ping timeout 100ms ", timeTaken > 90);
+    assertTrue("CheckIfAvailable should have return false", !retVal);
+  }
+
+  @Test
+  public void testShutdown() {
+
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+
+    // 3rd is current member
+    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
+
+    gmsHealthMonitor.installView(v);
+
+    gmsHealthMonitor.stop();
+
+    try {
+      // this happens after final check, membertimeout = 1000
+      Thread.sleep(100l);
+    } catch (InterruptedException e) {
+    }
+
+    assertTrue("HeathMonitor should have shutdown", gmsHealthMonitor.isShutdown());
+
+  }
+  
+  @Test
+  public void testCreateServerSocket() throws Exception {
+    try (ServerSocket socket = gmsHealthMonitor.createServerSocket(InetAddress.getLocalHost(), portRange)) {
+      Assert.assertTrue( portRange[0] <= socket.getLocalPort() && socket.getLocalPort() <= portRange[1]);
+    }
+  }
+
+  @Test
+  public void testCreateServerSocketPortRangeInvalid() throws Exception {
+    try (ServerSocket socket = gmsHealthMonitor.createServerSocket(InetAddress.getLocalHost(), new int[]{-1, -1})) {
+      Assert.fail("socket was created with invalid port range");
+    }
+    catch (IllegalArgumentException e) {
+      
+    }
+  }
+  
+  @Test
+  public void testClientSocketHandler() throws Exception {
+    int viewId = 2;
+    long msb = 3;
+    long lsb = 4;
+    GMSMember otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
+    GMSMember gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
+    executeTestClientSocketHandler(gmsMember, otherMember, GMSHealthMonitor.OK);
+  }
+
+  @Test
+  public void testClientSocketHandlerWhenMsbDoNotMatch() throws Exception {
+    int viewId = 2;
+    long msb = 3;
+    long lsb = 4;
+    GMSMember otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb + 1, lsb);
+    GMSMember gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
+    executeTestClientSocketHandler(gmsMember, otherMember, GMSHealthMonitor.ERROR);
+  }
+  
+  @Test
+  public void testClientSocketHandlerWhenLsbDoNotMatch() throws Exception {
+    int viewId = 2;
+    long msb = 3;
+    long lsb = 4;
+    GMSMember otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb + 1);
+    GMSMember gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
+    executeTestClientSocketHandler(gmsMember, otherMember, GMSHealthMonitor.ERROR);
+  }
+  
+  @Test
+  public void testClientSocketHandlerWhenViewIdDoNotMatch() throws Exception {
+    int viewId = 2;
+    long msb = 3;
+    long lsb = 4;
+    GMSMember otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId + 1, msb, lsb);
+    GMSMember gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
+    executeTestClientSocketHandler(gmsMember, otherMember, GMSHealthMonitor.ERROR);
+  }
+  
+  public void executeTestClientSocketHandler(GMSMember gmsMember, GMSMember otherMember, int expectedResult) throws Exception {
+    //We have already set the view id in the member but when creating the IDM it resets it to -1 for some reason
+    int viewId = gmsMember.getVmViewId();
+    
+    InternalDistributedMember testMember = new InternalDistributedMember("localhost", 9000, Version.CURRENT, gmsMember);
+    //We set to our expected test viewId in the IDM as well as reseting the gms member
+    testMember.setVmViewId(viewId);
+    gmsMember.setBirthViewId(viewId);
+    
+
+    //Set up the incoming/received bytes.  We just wrap output streams and write out the gms member information
+    byte[] receivedBytes = writeMemberToBytes(otherMember);
+    InputStream mockInputStream = new ByteArrayInputStream(receivedBytes);
+    
+    //configure the mock to return the mocked incoming bytes and provide an outputstream that we will check
+    Socket fakeSocket = mock(Socket.class);
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    when(fakeSocket.getInputStream()).thenReturn(mockInputStream);
+    when(fakeSocket.getOutputStream()).thenReturn(outputStream);
+
+    //run the socket handler
+    gmsHealthMonitor.setLocalAddress(testMember);
+    ClientSocketHandler handler = gmsHealthMonitor.new ClientSocketHandler(fakeSocket);
+    handler.run();
+    
+    //verify the written bytes are as expected
+    DataInputStream dis = new DataInputStream(new ByteArrayInputStream(outputStream.toByteArray()));
+    int byteReply = dis.read();
+    Assert.assertEquals(expectedResult, byteReply);
+  }
+
+  private GMSMember createGMSMember(short version, int viewId, long msb, long lsb) {
+    GMSMember gmsMember = new GMSMember();
+    gmsMember.setVersionOrdinal(version);
+    gmsMember.setBirthViewId(viewId);
+    gmsMember.setUUID(new UUID(msb, lsb));
+    return gmsMember;
+  }
+  
+  private byte[] writeMemberToBytes(GMSMember gmsMember) throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dataReceive = new DataOutputStream(baos);
+    gmsHealthMonitor.writeMemberToStream(gmsMember, dataReceive);
+    return baos.toByteArray();
+  }
+
+
+  private class MethodExecuted implements Answer {
+    private boolean methodExecuted = false;
+
+    public boolean isMethodExecuted() {
+      return methodExecuted;
+    }
+
+    @Override
+    public Object answer(InvocationOnMock invocation) throws Throwable {
+      methodExecuted = true;
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3259c023/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
index abc7a2f..9895f68 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
@@ -19,7 +19,7 @@ package com.gemstone.gemfire.distributed.internal.membership.gms.membership;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.isA;
+import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
@@ -244,6 +244,21 @@ public class GMSJoinLeaveJUnitTest {
     verify(messenger).send(any(JoinResponseMessage.class));
   }
   
+  //This test does not test the actual join process but rather that the join response gets loggedß
+  @Test
+  public void testProcessJoinResponseIsRecorded() throws IOException {
+    initMocks();
+    when(services.getAuthenticator()).thenReturn(authenticator);
+    when(authenticator.authenticate(mockMembers[0], null)).thenThrow(new AuthenticationFailedException("we want to fail auth here"));
+    when(services.getMessenger()).thenReturn(messenger);
+      
+    JoinResponseMessage[] joinResponse = gmsJoinLeave.getJoinResponseMessage();
+    
+    JoinResponseMessage jrm = new JoinResponseMessage();
+    gmsJoinLeave.processMessage(jrm);
+    Assert.assertEquals(jrm, joinResponse[0]);
+  }
+  
   /**
    * prepares and install a view
    * @throws IOException
@@ -635,6 +650,20 @@ public class GMSJoinLeaveJUnitTest {
     verify(manager).quorumLost(crashes, newView);
   }
   
+  //Possibly modify test to check for network partition message in the force disconnect
+  @Test
+  public void testNetworkPartitionMessageReceived() throws Exception {
+    initMocks();
+    gmsJoinLeave.becomeCoordinatorForTest();
+    List<InternalDistributedMember> members = Arrays.asList(mockMembers);
+    Set<InternalDistributedMember> empty = Collections.<InternalDistributedMember>emptySet();
+    NetView v = new NetView(mockMembers[0], 2, members, empty, empty);
+    NetworkPartitionMessage message = new NetworkPartitionMessage();
+    gmsJoinLeave.processMessage(message);
+    verify(manager).forceDisconnect(any(String.class));
+  }
+
+  
   @Test 
   public void testQuorumLossNotificationWithNetworkPartitionDetectionDisabled() throws IOException {
     initMocks(false);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3259c023/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
deleted file mode 100644
index 41a99a6..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
+++ /dev/null
@@ -1,461 +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.distributed.internal.membership.gms.membership.fd;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
-import com.gemstone.gemfire.distributed.internal.membership.NetView;
-import com.gemstone.gemfire.distributed.internal.membership.gms.ServiceConfig;
-import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
-import com.gemstone.gemfire.distributed.internal.membership.gms.Services.Stopper;
-import com.gemstone.gemfire.distributed.internal.membership.gms.fd.GMSHealthMonitor;
-import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.JoinLeave;
-import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
-import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Messenger;
-import com.gemstone.gemfire.distributed.internal.membership.gms.messages.HeartbeatMessage;
-import com.gemstone.gemfire.distributed.internal.membership.gms.messages.HeartbeatRequestMessage;
-import com.gemstone.gemfire.distributed.internal.membership.gms.messages.SuspectMembersMessage;
-import com.gemstone.gemfire.distributed.internal.membership.gms.messages.SuspectRequest;
-import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class GMSHealthMonitorJUnitTest {
-
-  private Services services;
-  private ServiceConfig mockConfig;
-  private DistributionConfig mockDistConfig;
-  private List<InternalDistributedMember> mockMembers;
-  private Messenger messenger;
-  private JoinLeave joinLeave;
-  private GMSHealthMonitor gmsHealthMonitor;
-  private Manager manager;
-  final long memberTimeout = 1000l;
-  private int[] portRange= new int[]{0, 65535};
-
-  @Before
-  public void initMocks() throws UnknownHostException {
-    System.setProperty("gemfire.bind-address", "localhost");
-    mockDistConfig = mock(DistributionConfig.class);
-    mockConfig = mock(ServiceConfig.class);
-    messenger = mock(Messenger.class);
-    joinLeave = mock(JoinLeave.class);
-    manager = mock(Manager.class);
-    services = mock(Services.class);
-    Stopper stopper = mock(Stopper.class);
-
-    when(mockConfig.getDistributionConfig()).thenReturn(mockDistConfig);
-    when(mockConfig.getMemberTimeout()).thenReturn(memberTimeout);
-    when(mockConfig.getMembershipPortRange()).thenReturn(portRange);
-    when(services.getConfig()).thenReturn(mockConfig);
-    when(services.getMessenger()).thenReturn(messenger);
-    when(services.getJoinLeave()).thenReturn(joinLeave);
-    when(services.getCancelCriterion()).thenReturn(stopper);
-    when(services.getManager()).thenReturn(manager);
-    when(stopper.isCancelInProgress()).thenReturn(false);
-    
-
-    if (mockMembers == null) {
-      mockMembers = new ArrayList<InternalDistributedMember>();
-      for (int i = 0; i < 7; i++) {
-        InternalDistributedMember mbr = new InternalDistributedMember("localhost", 8888 + i);
-  
-        if (i == 0 || i == 1) {
-          mbr.setVmKind(DistributionManager.LOCATOR_DM_TYPE);
-          mbr.getNetMember().setPreferredForCoordinator(true);
-        }
-        mockMembers.add(mbr);
-      }
-    }
-    when(joinLeave.getMemberID()).thenReturn(mockMembers.get(3));
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-    gmsHealthMonitor = new GMSHealthMonitor();
-    gmsHealthMonitor.init(services);
-    gmsHealthMonitor.start();
-  }
-
-  @After
-  public void tearDown() {
-    gmsHealthMonitor.stop();
-    System.getProperties().remove("gemfire.bind-address");
-  }
-
-  @Test
-  public void testHMServiceStarted() throws IOException {
-
-    InternalDistributedMember mbr = new InternalDistributedMember(SocketCreator.getLocalHost(), 12345);
-    mbr.setVmViewId(1);
-    when(messenger.getMemberID()).thenReturn(mbr);
-    gmsHealthMonitor.started();
-    
-    NetView v = new NetView(mbr, 1, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    gmsHealthMonitor.processMessage(new HeartbeatRequestMessage(mbr, 1));
-    verify(messenger, atLeastOnce()).send(any(HeartbeatMessage.class));
-  }
-
-  /**
-   * checks who is next neighbor
-   */
-  @Test
-  public void testHMNextNeighborVerify() throws IOException {
-
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-    gmsHealthMonitor.started();
-
-    gmsHealthMonitor.installView(v);
-
-    Assert.assertEquals(mockMembers.get(4), gmsHealthMonitor.getNextNeighbor());
-
-  }
-
-  @Test
-  public void testHMNextNeighborAfterTimeout() throws Exception {
-    System.out.println("testHMNextNeighborAfterTimeout starting");
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-//    System.out.printf("memberID is %s view is %s\n", mockMembers.get(3), v);
-    
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-    gmsHealthMonitor.started();
-
-    gmsHealthMonitor.installView(v);
-
-    // allow the monitor to give up on the initial "next neighbor" and
-    // move on to the one after it
-    long giveup = System.currentTimeMillis() + memberTimeout + 600;
-    InternalDistributedMember expected = mockMembers.get(5);
-    InternalDistributedMember neighbor = gmsHealthMonitor.getNextNeighbor();
-    while (System.currentTimeMillis() < giveup && neighbor != expected) {
-      Thread.sleep(5);
-      neighbor = gmsHealthMonitor.getNextNeighbor();
-    }
-
-    // neighbor should change to 5th
-    System.out.println("testHMNextNeighborAfterTimeout ending");
-    Assert.assertEquals("expected " + expected + " but found " + neighbor
-        + ".  view="+v, expected, neighbor);
-  }
-
-  /**
-   * it checks neighbor before member-timeout, it should be same
-   */
-
-  @Test
-  public void testHMNextNeighborBeforeTimeout() throws IOException {
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-    gmsHealthMonitor.started();
-
-    gmsHealthMonitor.installView(v);
-
-    try {
-      // member-timeout is 1000 ms.  We initiate a check and choose
-      // a new neighbor at 500 ms
-      Thread.sleep(memberTimeout/GMSHealthMonitor.LOGICAL_INTERVAL - 100);
-    } catch (InterruptedException e) {
-    }
-    // neighbor should be same
-    System.out.println("next neighbor is " + gmsHealthMonitor.getNextNeighbor() +
-        "\nmy address is " + mockMembers.get(3) +
-        "\nview is " + v);
-
-    Assert.assertEquals(mockMembers.get(4), gmsHealthMonitor.getNextNeighbor());
-  }
-  
-  /***
-   * checks whether member-check thread sends suspectMembers message
-   */
-  @Test
-  public void testSuspectMembersCalledThroughMemberCheckThread() throws Exception {
-    System.out.println("testSuspectMembersCalledThroughMemberCheckThread starting");
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-    gmsHealthMonitor.started();
-
-    gmsHealthMonitor.installView(v);
-
-    // when the view is installed we start a heartbeat timeout.  After
-    // that expires we request a heartbeat
-    Thread.sleep(3*memberTimeout + 100);
-
-    System.out.println("testSuspectMembersCalledThroughMemberCheckThread ending");
-    assertTrue(gmsHealthMonitor.isSuspectMember(mockMembers.get(4)));
-  }
-
-  /***
-   * checks ping thread didn't sends suspectMembers message before timeout
-   */
-  @Test
-  public void testSuspectMembersNotCalledThroughPingThreadBeforeTimeout() {
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-    gmsHealthMonitor.started();
-    
-    gmsHealthMonitor.installView(v);
-    InternalDistributedMember neighbor = gmsHealthMonitor.getNextNeighbor();
-
-    try {
-      // member-timeout is 1000 ms
-      // plus 100 ms for ack
-      Thread.sleep(memberTimeout - 200);
-    } catch (InterruptedException e) {
-    }
-
-    assertFalse(gmsHealthMonitor.isSuspectMember(neighbor));
-  }
-
-  /***
-   * Checks whether suspect thread sends suspectMembers message
-   */
-  @Test
-  public void testSuspectMembersCalledThroughSuspectThread() throws Exception {
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-    
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-
-    gmsHealthMonitor.installView(v);
-
-    gmsHealthMonitor.suspect(mockMembers.get(1), "Not responding");
-
-    Thread.sleep(GMSHealthMonitor.MEMBER_SUSPECT_COLLECTION_INTERVAL + 1000);
-
-    verify(messenger, atLeastOnce()).send(any(SuspectMembersMessage.class));
-  }
-
-  /***
-   * Checks suspect thread doesn't sends suspectMembers message before timeout
-   */
-  @Test
-  public void testSuspectMembersNotCalledThroughSuspectThreadBeforeTimeout() {
-
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    MethodExecuted messageSent = new MethodExecuted();
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-
-    gmsHealthMonitor.installView(v);
-
-    gmsHealthMonitor.suspect(mockMembers.get(1), "Not responding");
-
-    when(messenger.send(isA(SuspectMembersMessage.class))).thenAnswer(messageSent);
-
-    try {
-      // suspect thread timeout is 200 ms
-      Thread.sleep(100l);
-    } catch (InterruptedException e) {
-    }
-
-    assertTrue("SuspectMembersMessage shouldn't have sent", !messageSent.isMethodExecuted());
-  }
-
-  /***
-   * Send remove member message after doing final check, ping Timeout
-   */
-  @Test
-  public void testRemoveMemberCalled() throws Exception {
-    System.out.println("testRemoveMemberCalled starting");
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(0)); // coordinator and local member
-    gmsHealthMonitor.started();
-
-    gmsHealthMonitor.installView(v);
-    
-    Thread.sleep(memberTimeout/GMSHealthMonitor.LOGICAL_INTERVAL);
-
-    ArrayList<InternalDistributedMember> recipient = new ArrayList<InternalDistributedMember>();
-    recipient.add(mockMembers.get(0));
-    ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
-    SuspectRequest sr = new SuspectRequest(mockMembers.get(1), "Not Responding");// removing member 1
-    as.add(sr);
-    SuspectMembersMessage sm = new SuspectMembersMessage(recipient, as);
-    sm.setSender(mockMembers.get(0));
-
-    gmsHealthMonitor.processMessage(sm);
-
-    Thread.sleep(2*memberTimeout + 200);
-
-    System.out.println("testRemoveMemberCalled ending");
-    verify(joinLeave, atLeastOnce()).remove(any(InternalDistributedMember.class), any(String.class));
-  }
-
-  /***
-   * Shouldn't send remove member message before doing final check, or before ping Timeout
-   */
-  @Test
-  public void testRemoveMemberNotCalledBeforeTimeout() {
-    System.out.println("testRemoveMemberNotCalledBeforeTimeout starting");
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(0)); // coordinator and local member
-    when(joinLeave.getMemberID()).thenReturn(mockMembers.get(0)); // coordinator and local member
-    gmsHealthMonitor.started();
-
-    gmsHealthMonitor.installView(v);
-
-    ArrayList<InternalDistributedMember> recipient = new ArrayList<InternalDistributedMember>();
-    recipient.add(mockMembers.get(0));
-    ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
-    SuspectRequest sr = new SuspectRequest(mockMembers.get(1), "Not Responding");// removing member 1
-    as.add(sr);
-    SuspectMembersMessage sm = new SuspectMembersMessage(recipient, as);
-    sm.setSender(mockMembers.get(0));
-
-    gmsHealthMonitor.processMessage(sm);
-
-    try {
-      // this happens after final check, ping timeout
-      Thread.sleep(memberTimeout-100);
-    } catch (InterruptedException e) {
-    }
-
-    System.out.println("testRemoveMemberNotCalledBeforeTimeout ending");
-    verify(joinLeave, never()).remove(any(InternalDistributedMember.class), any(String.class));
-  }
-
-  /***
-   * Send remove member message after doing final check for coordinator, ping timeout
-   * This test trying to remove coordinator
-   */
-  @Test
-  public void testRemoveMemberCalledAfterDoingFinalCheckOnCoordinator() throws Exception {
-
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    // preferred coordinators are 0 and 1
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(1));// next preferred coordinator
-    gmsHealthMonitor.started();
-
-    gmsHealthMonitor.installView(v);
-    
-    Thread.sleep(memberTimeout/GMSHealthMonitor.LOGICAL_INTERVAL);
-
-    ArrayList<InternalDistributedMember> recipient = new ArrayList<InternalDistributedMember>();
-    recipient.add(mockMembers.get(0));
-    recipient.add(mockMembers.get(1));
-    ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
-    SuspectRequest sr = new SuspectRequest(mockMembers.get(0), "Not Responding");// removing coordinator
-    as.add(sr);
-    SuspectMembersMessage sm = new SuspectMembersMessage(recipient, as);
-    sm.setSender(mockMembers.get(4));// member 4 sends suspect message
-
-    gmsHealthMonitor.processMessage(sm);
-
-    // this happens after final check, ping timeout = 1000 ms
-    Thread.sleep(memberTimeout + 200);
-
-    verify(joinLeave, atLeastOnce()).remove(any(InternalDistributedMember.class), any(String.class));
-  }
-
-  /***
-   * validates HealthMonitor.CheckIfAvailable api
-   */
-  @Test
-  public void testCheckIfAvailable() {
-
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-
-    gmsHealthMonitor.installView(v);
-
-    long startTime = System.currentTimeMillis();
-
-    boolean retVal = gmsHealthMonitor.checkIfAvailable(mockMembers.get(1), "Not responding", false);
-
-    long timeTaken = System.currentTimeMillis() - startTime;
-
-    assertTrue("This should have taken member ping timeout 100ms ", timeTaken > 90);
-    assertTrue("CheckIfAvailable should have return false", !retVal);
-  }
-
-  @Test
-  public void testShutdown() {
-
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-
-    gmsHealthMonitor.installView(v);
-
-    gmsHealthMonitor.stop();
-
-    try {
-      // this happens after final check, membertimeout = 1000
-      Thread.sleep(100l);
-    } catch (InterruptedException e) {
-    }
-
-    assertTrue("HeathMonitor should have shutdown", gmsHealthMonitor.isShutdown());
-
-  }
-
-  private class MethodExecuted implements Answer {
-    private boolean methodExecuted = false;
-
-    public boolean isMethodExecuted() {
-      return methodExecuted;
-    }
-
-    @Override
-    public Object answer(InvocationOnMock invocation) throws Throwable {
-      methodExecuted = true;
-      return null;
-    }
-  }
-}


[22/50] [abbrv] incubator-geode git commit: "new" unit tests. These were unnecessarily dependent on some classes remaining in the closed-source GemFire repo.

Posted by kl...@apache.org.
"new" unit tests.  These were unnecessarily dependent on some classes remaining
in the closed-source GemFire repo.


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

Branch: refs/heads/feature/GEODE-217
Commit: 8f9b32161815fe5c007616adb4a3db5df78cd3b4
Parents: 812d51c
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Dec 7 13:06:50 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Dec 7 13:10:41 2015 -0800

----------------------------------------------------------------------
 .../InstantiatorPropagationDUnitTest.java       | 1761 ++++++++++++++++++
 1 file changed, 1761 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8f9b3216/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InstantiatorPropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InstantiatorPropagationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InstantiatorPropagationDUnitTest.java
new file mode 100644
index 0000000..063112a
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InstantiatorPropagationDUnitTest.java
@@ -0,0 +1,1761 @@
+/*=========================================================================
+ * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
+ * This product is protected by U.S. and international copyright
+ * and intellectual property laws. Pivotal products are covered by
+ * one or more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+package com.gemstone.gemfire.internal.cache.tier.sockets;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Properties;
+import java.util.Random;
+
+import com.gemstone.gemfire.DataSerializable;
+import com.gemstone.gemfire.Instantiator;
+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.MirrorType;
+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.PoolManager;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.InternalInstantiator;
+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.EventID;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.VM;
+
+public class InstantiatorPropagationDUnitTest extends DistributedTestCase {
+  private static Cache cache = null;
+
+  private static VM client1 = null;
+
+  private static VM client2 = null;
+
+  private static VM server1 = null;
+
+  private static VM server2 = null;
+
+  private static int PORT1 = -1;
+
+  private static int PORT2 = -1;
+
+  private static int instanceCountWithAllPuts = 3;
+
+  private static int instanceCountWithOnePut = 1;
+
+  private static final String REGION_NAME = "ClientServerInstantiatorRegistrationDUnitTest";
+  
+  protected static EventID eventId;
+
+  static boolean testEventIDResult = false;
+
+  public static boolean testObject20Loaded = false;
+
+
+
+
+  public InstantiatorPropagationDUnitTest(String name) {
+    super(name);
+    // TODO Auto-generated constructor stub
+  }
+
+  public void setUp() throws Exception {
+    super.setUp();
+    final Host host = Host.getHost(0);
+    client1 = host.getVM(0);
+    client2 = host.getVM(1);
+    server1 = host.getVM(2);
+    server2 = host.getVM(3);
+  }
+
+  private void createCache(Properties props) throws Exception {
+    DistributedSystem ds = getSystem(props);
+    ds.disconnect();
+    ds = getSystem(props);
+    assertNotNull(ds);
+    cache = CacheFactory.create(ds);
+    assertNotNull(cache);
+  }
+
+  public static void createClientCache(String host, Integer port1)
+      throws Exception {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    new InstantiatorPropagationDUnitTest("temp").createCache(props);
+    Pool p = PoolManager.createFactory().addServer(host, port1.intValue())
+        .setMinConnections(1).setSubscriptionEnabled(true).setPingInterval(200)
+        .create("ClientServerInstantiatorRegistrationDUnitTestPool");
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setPoolName(p.getName());
+    Region r = cache.createRegion(REGION_NAME, factory.create());
+    r.registerInterest("ALL_KEYS");
+  }
+
+  protected int getMaxThreads() {
+    return 0;
+  }
+
+  private int initServerCache(VM server) {
+    Object[] args = new Object[] { new Integer(getMaxThreads()) };
+    return ((Integer)server.invoke(InstantiatorPropagationDUnitTest.class,
+        "createServerCache", args)).intValue();
+  }
+
+  public static Integer createServerCache(Integer maxThreads) throws Exception {
+    new InstantiatorPropagationDUnitTest("temp").createCache(new Properties());
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setMirrorType(MirrorType.KEYS_VALUES);
+    RegionAttributes attrs = factory.create();
+    cache.createRegion(REGION_NAME, attrs);
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    CacheServer server1 = cache.addCacheServer();
+    server1.setPort(port);
+    server1.setMaxThreads(maxThreads.intValue());
+    server1.start();
+    return new Integer(port);
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+    // close the clients first
+    closeCache();
+    client1.invoke(InstantiatorPropagationDUnitTest.class, "closeCache");
+    client2.invoke(InstantiatorPropagationDUnitTest.class, "closeCache");
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class, "closeCache");
+    server1.invoke(InstantiatorPropagationDUnitTest.class, "closeCache");
+  }
+
+  public static void closeCache() {
+    if (cache != null && !cache.isClosed()) {
+      cache.close();
+      cache.getDistributedSystem().disconnect();
+    }
+  }
+  
+  public static void unregisterInstantiatorsInAllVMs() {
+    invokeInEveryVM(DistributedTestCase.class, "unregisterInstantiatorsInThisVM");
+  }
+
+  public static void verifyInstantiators(final int numOfInstantiators) {
+    WaitCriterion wc = new WaitCriterion() {
+      String excuse;
+
+      public boolean done() {
+        return InternalInstantiator.getInstantiators().length == numOfInstantiators;
+      }
+
+      public String description() {
+        return "expected " + numOfInstantiators + " but got this "
+            + InternalInstantiator.getInstantiators().length
+          + " instantiators=" + java.util.Arrays.toString(InternalInstantiator.getInstantiators());
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+  }
+
+  public static void registerTestObject1() throws Exception {
+
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject1");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject1", e);
+    }
+  }
+
+  public static void registerTestObject2() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject2");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject2", e);
+    }
+  }
+
+  public static void registerTestObject3() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject3");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject3", e);
+    }
+  }
+
+  public static void registerTestObject4() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject4");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject4", e);
+    }
+  }
+
+  public static void registerTestObject5() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject5");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject5", e);
+    }
+  }
+
+  public static void registerTestObject6() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject6");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject6", e);
+    }
+  }
+
+  public static void registerTestObject7() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject7");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject7", e);
+    }
+  }
+
+  public static void registerTestObject8() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject8");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject8", e);
+    }
+  }
+
+  public static void registerTestObject9() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject9");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject9", e);
+    }
+  }
+
+  public static void registerTestObject10() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject10");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject10", e);
+    }
+  }
+
+  public static void registerTestObject11() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject11");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject11", e);
+    }
+  }
+
+  public static void registerTestObject12() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject12");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject11", e);
+    }
+  }
+
+  public static void registerTestObject13() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject13");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject13", e);
+    }
+  }
+
+  public static void registerTestObject14() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject14");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject14", e);
+    }
+  }
+
+  public static void registerTestObject15() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject15");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject15", e);
+    }
+  }
+
+  public static void registerTestObject16() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject16");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject16", e);
+    }
+  }
+
+  public static void registerTestObject17() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject17");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject17", e);
+    }
+  }
+
+  public static void registerTestObject18() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject18");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject18", e);
+    }
+  }
+  
+  public static void registerTestObject19() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject19");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject19", e);
+    }
+  }
+
+  public static void registerTestObject20() throws Exception {
+    try {
+      Class cls = Class
+          .forName("com.gemstone.gemfire.internal.cache.tier.sockets.TestObject20");
+      ConfigurableObject obj = (ConfigurableObject)cls.newInstance();
+      obj.init(0);
+    }
+    catch (Exception e) {
+      fail("Test failed due to exception in TestObject20", e);
+    }
+  }
+
+  public static void stopServer() {
+    try {
+      assertEquals("Expected exactly one BridgeServer", 1, cache
+          .getCacheServers().size());
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
+          .iterator().next();
+      assertNotNull(bs);
+      bs.stop();
+    }
+    catch (Exception ex) {
+      fail("while setting stopServer  " + ex);
+    }
+  }
+
+  public static void startServer() {
+    try {
+      Cache c = CacheFactory.getAnyInstance();
+      assertEquals("Expected exactly one BridgeServer", 1, c.getCacheServers()
+          .size());
+      CacheServerImpl bs = (CacheServerImpl)c.getCacheServers().iterator()
+          .next();
+      assertNotNull(bs);
+      bs.start();
+    }
+    catch (Exception ex) {
+      fail("while startServer()  " + ex);
+    }
+  }
+
+  /**
+   * In this test the server is up first.2 Instantiators are registered on it.
+   * Verified if the 2 instantiators get propagated to client when client gets
+   * connected.
+   */
+  public void testServerUpFirstClientLater() throws Exception {
+    PORT1 = initServerCache(server1);
+
+    unregisterInstantiatorsInAllVMs();
+    
+    pause(3000);
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "registerTestObject1");
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "registerTestObject2");
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(2) });
+
+    client1
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT1) });
+
+    // // wait for client2 to come online
+    pause(3000);
+    //
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(2) });
+    //
+    // // Put some entries from the client
+    client1.invoke(new CacheSerializableRunnable("Put entries from client") {
+      public void run2() throws CacheException {
+        Region region = cache.getRegion(REGION_NAME);
+        for (int i = 1; i <= 10; i++) {
+          region.put(i, i);
+        }
+      }
+    });
+
+    // Run getAll
+    client1
+        .invoke(new CacheSerializableRunnable("Get all entries from server") {
+          public void run2() throws CacheException {
+            // Invoke getAll
+            Region region = cache.getRegion(REGION_NAME);
+            // Verify result size is correct
+            assertEquals(1, region.get(1));
+          }
+        });
+
+    server1.invoke(new CacheSerializableRunnable("Put entry from client") {
+      public void run2() throws CacheException {
+        Region region = cache.getRegion(REGION_NAME);
+        region.put(1, 20);
+      }
+    });
+    //
+    pause(3000);
+    // Run getAll
+    client1.invoke(new CacheSerializableRunnable("Get entry from client") {
+      public void run2() throws CacheException {
+        // Invoke getAll
+        Region region = cache.getRegion(REGION_NAME);
+        // Verify result size is correct
+        assertEquals(20, region.get(1));
+      }
+    });
+
+    unregisterInstantiatorsInAllVMs();
+  }
+
+  /**
+   * In this test there are 2 clients and 2 servers.Registered one instantiator
+   * on one client. Verified, if that instantiator gets propagated to the server
+   * the client is connected to(server1), to the other server(server2) in the DS
+   * and the client(client2) that is connected to server2.
+   */
+  public void testInstantiatorsWith2ClientsN2Servers() throws Exception {
+    PORT1 = initServerCache(server1);
+    PORT2 = initServerCache(server2);
+
+    pause(3000);
+
+    client1
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT1) });
+    client2
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT2) });
+
+    unregisterInstantiatorsInAllVMs();
+
+    // wait for client2 to come online
+    pause(2000);
+
+
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "registerTestObject3");
+    pause(4000);
+
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(1) });
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(1) });
+
+    server2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(1) });
+
+    client2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(1) });
+
+    unregisterInstantiatorsInAllVMs();
+  }
+
+  /**
+   * First register an instantiator on client1. Stop the server1. Now register 2
+   * instantiators on server1. Now check that server1,server2,client2 has all 3
+   * instantiators. Client1 should have only 1 instantiator since the server1
+   * was stopped when 2 instantiators were added on it.
+   */
+  public void testInstantiatorsWithServerKill() throws Exception {
+    PORT1 = initServerCache(server1);
+    PORT2 = initServerCache(server2);
+
+    client1
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT1) });
+    client2
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT2) });
+
+    unregisterInstantiatorsInAllVMs();
+
+    // wait for client2 to come online
+    pause(2000);
+
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "registerTestObject4");
+    pause(4000);
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class, "stopServer");
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "registerTestObject5");
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "registerTestObject6");
+
+    server2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithAllPuts) });
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithAllPuts) });
+
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithOnePut) });
+
+    client2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithAllPuts) });
+
+    unregisterInstantiatorsInAllVMs();
+  }
+
+  /**
+   * 2 clients n 2 servers.Registered instantiators on both client n server to
+   * check if propagation of instantiators to n fro (from client n server) is
+   * taking place.Diff from the previous test in the case that server is not
+   * stopped.So registering an instantiator on server should propagate that to
+   * client as well.
+   */
+  public void testInstantiators() throws Exception {
+    PORT1 = initServerCache(server1);
+    PORT2 = initServerCache(server2);
+
+    client1
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT1) });
+    client2
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT2) });
+
+    unregisterInstantiatorsInAllVMs();
+
+    // wait for client2 to come online
+    pause(2000);
+
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "registerTestObject10");
+    pause(4000);
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "registerTestObject11");
+    pause(4000);
+
+    server2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(2) });
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(2) });
+
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(2) });
+
+    client2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(2) });
+
+    unregisterInstantiatorsInAllVMs();
+  }
+
+  /**
+   * Test's Number of Instantiators at all clients & servers with one Server
+   * being stopped and then restarted
+   */
+  public void _testInstantiatorsWithServerKillAndReInvoked() throws Exception {
+    PORT1 = initServerCache(server1);
+    PORT2 = initServerCache(server2);
+    client1
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT1) });
+    client2
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT2) });
+
+    unregisterInstantiatorsInAllVMs();
+
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "registerTestObject7");
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithOnePut) });
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithOnePut) });
+
+    server2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithOnePut) });
+
+    client2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithOnePut) });
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class, "stopServer");
+
+    try {
+      client1.invoke(InstantiatorPropagationDUnitTest.class,
+          "registerTestObject8");
+    }
+    catch (Exception expected) {// we are putting in a client whose server is
+      // dead
+    }
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class, "startServer");
+
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithAllPuts) });
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithAllPuts) });
+
+    server2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(
+            instanceCountWithAllPuts) });
+
+    unregisterInstantiatorsInAllVMs();
+  }
+
+  /**
+   * In this test there are 2 clients connected to 1 server and 1 client
+   * connected to the other server.Registered one instantiator on one
+   * client(client1). Verified, if that instantiator gets propagated to the
+   * server the client is connected to(server1), to client2, to the other
+   * server(server2) in the DS and the client that is connected to server2.
+   * 
+   */
+  public void testInstantiatorCount() throws Exception {
+    PORT1 = initServerCache(server1);
+    PORT2 = initServerCache(server2);
+
+    client1
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT1) });
+    client2
+        .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+            new Object[] { getServerHostName(server1.getHost()),
+                new Integer(PORT1) });
+    createClientCache(getServerHostName(server2.getHost()), new Integer(PORT2));
+    unregisterInstantiatorsInAllVMs();
+
+    // wait for client2 to come online
+    pause(2000);
+
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "registerTestObject12");
+    pause(4000);
+
+    client1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(1) });
+
+    server1.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(1) });
+
+    server2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(1) });
+
+    client2.invoke(InstantiatorPropagationDUnitTest.class,
+        "verifyInstantiators", new Object[] { new Integer(1) });
+
+    verifyInstantiators(1);
+
+    unregisterInstantiatorsInAllVMs();
+  }
+
+  public static void createClientCache_EventId(String host, Integer port1) throws Exception
+  {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    new InstantiatorPropagationDUnitTest("temp").createCache(props);
+    Pool p = PoolManager.createFactory()
+      .addServer(host, port1.intValue())
+      .setSubscriptionEnabled(true)
+      .create("RegisterInstantiatorEventIdDUnitTestPool");
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    factory.setPoolName(p.getName());
+    cache.createRegion(REGION_NAME, factory.create());
+  }
+  /**
+   * Test's same eventId being same for the Instantiators at all clients &
+   * servers
+   * 
+   */
+  // disabled - the eventID received does not match the sender's eventID.  Why is this a requirement anyway?
+  public void _testInstantiatorsEventIdVerificationClientsAndServers()
+      throws Exception {
+    PORT1 = initServerCache(server1, 1);
+    PORT2 = initServerCache(server2, 2);
+
+    createClientCache_EventId(getServerHostName(server1.getHost()), new Integer(PORT1));
+
+    unregisterInstantiatorsInAllVMs();
+    
+    client2.invoke(InstantiatorPropagationDUnitTest.class,
+        "createClientCache_EventId", new Object[] {
+            getServerHostName(server1.getHost()), new Integer(PORT2) });
+    setClientServerObserver1();
+    client2.invoke(InstantiatorPropagationDUnitTest.class,
+        "setClientServerObserver2");
+
+    registerTestObject19();
+
+    pause(10000);
+
+    Boolean pass = (Boolean)client2.invoke(
+        InstantiatorPropagationDUnitTest.class, "verifyResult");
+    assertTrue("EventId found Different", pass.booleanValue());
+
+    PoolImpl.IS_INSTANTIATOR_CALLBACK = false;
+
+  }
+  
+  public void testLazyRegistrationOfInstantiators()
+      throws Exception {
+    try {
+      PORT1 = initServerCache(server1);
+      PORT2 = initServerCache(server2);
+  
+      unregisterInstantiatorsInAllVMs();
+
+      pause(3000);
+  
+      createClientCache(getServerHostName(server1.getHost()),
+          new Integer(PORT1));
+  
+      client2
+          .invoke(InstantiatorPropagationDUnitTest.class, "createClientCache",
+              new Object[] {getServerHostName(server2.getHost()),
+                  new Integer(PORT2)});
+  
+      pause(3000);
+      unregisterInstantiatorsInAllVMs();
+  
+      assertTestObject20NotLoaded();
+      server1.invoke(InstantiatorPropagationDUnitTest.class, "assertTestObject20NotLoaded");
+      server2.invoke(InstantiatorPropagationDUnitTest.class, "assertTestObject20NotLoaded");
+      client2.invoke(InstantiatorPropagationDUnitTest.class, "assertTestObject20NotLoaded");
+  
+      registerTestObject20();
+      pause(5000);
+      assertTestObject20Loaded();
+      server1.invoke(InstantiatorPropagationDUnitTest.class, "assertTestObject20Loaded");
+      //server2.invoke(InstantiatorPropagationDUnitTest.class, "assertTestObject20Loaded"); // classes are not initialized after loading in p2p path
+      client2.invoke(InstantiatorPropagationDUnitTest.class, "assertTestObject20NotLoaded");
+    } finally {
+      unregisterInstantiatorsInAllVMs();
+      disconnectAllFromDS();
+    }
+  }
+
+  public static void assertTestObject20Loaded() {
+    assertTrue("TestObject20 is expected to be loaded into VM.", testObject20Loaded);
+  }
+
+  public static void assertTestObject20NotLoaded() {
+    assertFalse("TestObject20 is not expected to be loaded into VM.", testObject20Loaded);
+  }
+
+  public static Boolean verifyResult() {
+    boolean temp = testEventIDResult;
+    testEventIDResult = false;
+    return new Boolean(temp);
+  }
+  
+  /**
+   * this method initializes the appropriate server cache
+   * 
+   * @param server
+   * @param serverNo
+   * @return portNo.
+   */
+
+  private int initServerCache(VM server, int serverNo)
+  {
+    Object[] args = new Object[] { new Integer(getMaxThreads()) };
+    if (serverNo == 1) {
+      return ((Integer)server.invoke(
+          InstantiatorPropagationDUnitTest.class,
+          "createServerCacheOne", args)).intValue();
+    }
+    else {
+      return ((Integer)server.invoke(
+          InstantiatorPropagationDUnitTest.class,
+          "createServerCacheTwo", args)).intValue();
+    }
+  }
+
+  /**
+   * This method creates the server cache
+   * 
+   * @param maxThreads
+   * @return
+   * @throws Exception
+   */
+  public static Integer createServerCacheTwo(Integer maxThreads)
+      throws Exception
+  {
+    new InstantiatorPropagationDUnitTest("temp")
+        .createCache(new Properties());
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setMirrorType(MirrorType.KEYS_VALUES);
+
+    RegionAttributes attrs = factory.create();
+    cache.createRegion(REGION_NAME, attrs);
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    CacheServer server1 = cache.addCacheServer();
+    server1.setPort(port);
+    server1.setMaxThreads(maxThreads.intValue());
+    server1.setNotifyBySubscription(true);
+    server1.start();
+    return new Integer(port);
+  }
+
+  /**
+   * This method creates the server cache
+   * 
+   * @param maxThreads
+   * @return
+   * @throws Exception
+   */
+  public static Integer createServerCacheOne(Integer maxThreads)
+      throws Exception
+  {
+    new InstantiatorPropagationDUnitTest("temp")
+        .createCache(new Properties());
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setMirrorType(MirrorType.KEYS_VALUES);
+    RegionAttributes attrs = factory.create();
+    cache.createRegion(REGION_NAME, attrs);
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    CacheServer server1 = cache.addCacheServer();
+    server1.setPort(port);
+    server1.setMaxThreads(maxThreads.intValue());
+    server1.setNotifyBySubscription(true);
+    server1.start();
+    return new Integer(port);
+  }
+
+  public static void setClientServerObserver1()
+  {
+    PoolImpl.IS_INSTANTIATOR_CALLBACK = true;
+    ClientServerObserverHolder
+        .setInstance(new ClientServerObserverAdapter() {
+          public void beforeSendingToServer(EventID eventID)
+          {
+            eventId = eventID;
+            System.out.println("client2= "+client2 + " eventid= "+eventID);
+            client2.invoke(InstantiatorPropagationDUnitTest.class,
+                "setEventId", new Object[] { eventId });
+
+          }
+
+        });
+  }
+
+  /**
+   * sets the EventId value in the VM
+   * 
+   * @param eventID
+   */
+  public static void setEventId(EventID eventID)
+  {
+    eventId = eventID;
+  }
+  
+  public static void setClientServerObserver2()
+  {
+    PoolImpl.IS_INSTANTIATOR_CALLBACK = true;
+    ClientServerObserverHolder
+        .setInstance(new ClientServerObserverAdapter() {
+          public void afterReceivingFromServer(EventID eventID)
+          {
+            System.out.println("Observer2 received " + eventID + "; my eventID is " + eventId);
+            testEventIDResult = eventID.equals(eventId);
+          }
+
+        });
+  }
+}
+
+
+abstract class ConfigurableObject {
+  public abstract void init(int index);
+  public abstract int getIndex();
+  public abstract void validate(int index);
+}
+
+
+
+
+class TestObject1 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  public TestObject1() {
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  static {
+    Instantiator.register(new Instantiator(TestObject1.class, -100123) {
+      public DataSerializable newInstance() {
+        return new TestObject1();
+      }
+    });
+  }
+
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+
+}
+
+class TestObject2 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  public TestObject2() {
+  }
+
+  static {
+    Instantiator.register(new Instantiator(TestObject2.class, -100122) {
+      public DataSerializable newInstance() {
+        return new TestObject2();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+
+}
+
+class TestObject3 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  public TestObject3() {
+  }
+
+  static {
+    Instantiator.register(new Instantiator(TestObject3.class, -121) {
+      public DataSerializable newInstance() {
+        return new TestObject3();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject4 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  public TestObject4() {
+  }
+
+  static {
+    Instantiator.register(new Instantiator(TestObject4.class, -122) {
+      public DataSerializable newInstance() {
+        return new TestObject4();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject5 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  public TestObject5() {
+  }
+
+  static {
+    Instantiator.register(new Instantiator(TestObject5.class, -123) {
+      public DataSerializable newInstance() {
+        return new TestObject5();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject6 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  public TestObject6() {
+  }
+
+  static {
+    Instantiator.register(new Instantiator(TestObject6.class, -124) {
+      public DataSerializable newInstance() {
+        return new TestObject6();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject7 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  public TestObject7() {
+  }
+
+  static {
+    Instantiator.register(new Instantiator(TestObject7.class, -125) {
+      public DataSerializable newInstance() {
+        return new TestObject7();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject8 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  public TestObject8() {
+  }
+
+  static {
+    Instantiator.register(new Instantiator(TestObject8.class, -126) {
+      public DataSerializable newInstance() {
+        return new TestObject8();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject9 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  public TestObject9() {
+  }
+
+  static {
+    Instantiator.register(new Instantiator(TestObject9.class, -127) {
+      public DataSerializable newInstance() {
+        return new TestObject9();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject10 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  public TestObject10() {
+  }
+
+  static {
+    Instantiator.register(new Instantiator(TestObject10.class, -128) {
+      public DataSerializable newInstance() {
+        return new TestObject10();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject11 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  static {
+    Instantiator.register(new Instantiator(TestObject11.class, -129) {
+      public DataSerializable newInstance() {
+        return new TestObject11();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject12 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  static {
+    Instantiator.register(new Instantiator(TestObject12.class, -130) {
+      public DataSerializable newInstance() {
+        return new TestObject12();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject13 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  static {
+    Instantiator.register(new Instantiator(TestObject13.class, -131) {
+      public DataSerializable newInstance() {
+        return new TestObject13();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject14 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  static {
+    Instantiator.register(new Instantiator(TestObject14.class, -132) {
+      public DataSerializable newInstance() {
+        return new TestObject14();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject15 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  static {
+    Instantiator.register(new Instantiator(TestObject15.class, -133) {
+      public DataSerializable newInstance() {
+        return new TestObject15();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject16 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  static {
+    Instantiator.register(new Instantiator(TestObject16.class, -134) {
+      public DataSerializable newInstance() {
+        return new TestObject16();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject17 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  static {
+    Instantiator.register(new Instantiator(TestObject17.class, -135) {
+      public DataSerializable newInstance() {
+        return new TestObject17();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject18 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  static {
+    Instantiator.register(new Instantiator(TestObject18.class, -1136) {
+      public DataSerializable newInstance() {
+        return new TestObject18();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject19 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  static {
+    Instantiator.register(new Instantiator(TestObject19.class, -136) {
+      public DataSerializable newInstance() {
+        return new TestObject19();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+}
+
+class TestObject20 extends ConfigurableObject implements DataSerializable {
+
+  private int field1;
+
+  static {
+    InstantiatorPropagationDUnitTest.testObject20Loaded = true;
+    Instantiator.register(new Instantiator(TestObject20.class, -138) {
+      public DataSerializable newInstance() {
+        return new TestObject20();
+      }
+    });
+  }
+
+  /**
+   * Initializes a Instantiator1DUnitTestObject1.
+   */
+  public void init(int index) {
+    Random random = new Random();
+    this.field1 = random.nextInt();
+  }
+
+  public int getIndex() {
+    return 1;
+  }
+
+  public void validate(int index) {
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    this.field1 = in.readInt();
+  }
+
+  public void toData(DataOutput out) throws IOException {
+    out.writeInt(this.field1);
+  }
+  
+}


[24/50] [abbrv] incubator-geode git commit: new unit tests and code clean-up

Posted by kl...@apache.org.
new unit tests and code clean-up

New unit tests to increase code coverage.  I also tinkered with
trying to get EclEmma output from DUnit ChildVMs but it seems to
be run in a shutdownHook in parallel with the DUnitLauncher
shutdownHook.


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

Branch: refs/heads/feature/GEODE-217
Commit: bd43c341e8483df7fff1caabee666791e75f0dd4
Parents: 8f9b321
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Dec 7 13:30:00 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Dec 7 13:30:00 2015 -0800

----------------------------------------------------------------------
 .../internal/DistributionMessage.java           |   2 +-
 .../internal/membership/MemberAttributes.java   | 131 +----
 .../membership/gms/membership/GMSJoinLeave.java |  44 +-
 .../gms/messages/HeartbeatMessage.java          |   2 +-
 .../gms/messages/HeartbeatRequestMessage.java   |   2 +-
 .../gms/messages/InstallViewMessage.java        |   2 +-
 .../gms/messages/JoinResponseMessage.java       |  10 +-
 .../membership/gms/messages/ViewAckMessage.java |   2 +-
 .../gms/messenger/AddressManager.java           |  21 +-
 .../membership/gms/messenger/GMSPingPonger.java |  22 +-
 .../membership/gms/messenger/JGAddress.java     |  23 +-
 .../gms/messenger/JGroupsMessenger.java         | 385 +++++++--------
 .../membership/gms/messenger/Transport.java     |   2 +-
 .../internal/tcpserver/TcpServer.java           |   2 +-
 .../internal/i18n/ParentLocalizedStrings.java   |   4 +-
 .../gemfire/cache30/ReconnectDUnitTest.java     |   2 +-
 .../internal/DistributionManagerDUnitTest.java  |   1 +
 .../membership/MembershipJUnitTest.java         | 116 ++++-
 .../membership/gms/MembershipManagerHelper.java |   1 +
 .../messenger/JGroupsMessengerJUnitTest.java    | 481 ++++++++++++++++---
 .../src/test/java/dunit/RemoteDUnitVMIF.java    |   2 +
 .../src/test/java/dunit/standalone/ChildVM.java |  11 +-
 .../java/dunit/standalone/DUnitLauncher.java    |  24 +
 .../java/dunit/standalone/ProcessManager.java   |  14 +-
 .../java/dunit/standalone/RemoteDUnitVM.java    |   7 +-
 25 files changed, 821 insertions(+), 492 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionMessage.java
index 23f9dee..80ae4c0 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionMessage.java
@@ -174,7 +174,7 @@ public abstract class DistributionMessage
     }
   } 
   
-  public final boolean isDirectAck() {
+  public boolean isDirectAck() {
     return acker != null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberAttributes.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberAttributes.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberAttributes.java
index 7cd89d7..2d4d980 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberAttributes.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberAttributes.java
@@ -16,26 +16,20 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership;
 
-import com.gemstone.gemfire.DataSerializable;
-import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.InternalGemFireError;
-import com.gemstone.gemfire.distributed.DurableClientAttributes;
-import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.StringTokenizer;
 
-import java.io.*;
-import java.util.*;
+import com.gemstone.gemfire.distributed.DurableClientAttributes;
 
 /**
- * The attributes of a distributed member. These attributes are stored as
- * the AdditionalBytes in JGroups' IpAddress.
+ * The attributes of a distributed member.  This is largely deprecated as
+ * GMSMember holds all of this information.
  *
  * @author Kirk Lund
  * @since 5.0
  */
-public class MemberAttributes implements DataSerializable {
-  private static final long serialVersionUID = -3257772958884802693L;
+public class MemberAttributes {
   
   public static final MemberAttributes INVALID = new MemberAttributes(-1, -1, -1, -1, null, null, null);
   
@@ -68,23 +62,6 @@ public class MemberAttributes implements DataSerializable {
     this.durableClientAttributes = durableClientAttributes;
   }
   
-  /** Constructs new MemberAttributes from DataInput.  */
-  public MemberAttributes(byte[] b) throws IOException, ClassNotFoundException {
-    this.byteInfo = b;
-    DataInputStream in = 
-      new DataInputStream(new ByteArrayInputStream(b));
-    fromData(in);
-  }
-  
-  public MemberAttributes(MemberAttributes other) {
-    this.dcPort = other.dcPort;
-    this.vmPid = other.vmPid;
-    this.vmKind = other.vmKind;
-    this.name = other.name;
-    this.groups = other.groups;
-    this.durableClientAttributes = other.durableClientAttributes;
-  }
-  
   /** Returns direct channel port. */
   public int getPort() {
     return this.dcPort;
@@ -115,22 +92,6 @@ public class MemberAttributes implements DataSerializable {
     return this.durableClientAttributes;
   }
 
-  /** Parses comma-separated-values into array of groups (strings). */
-  public static String[] parseGroups(String csv) {
-    if (csv == null || csv.length() == 0) {
-      return new String[0];
-    }
-    List groups = new ArrayList();
-    StringTokenizer st = new StringTokenizer(csv, ",");
-    while (st.hasMoreTokens()) {
-      String groupName = st.nextToken().trim();
-      // TODO make case insensitive
-      if (!groups.contains(groupName)) { // only add each group once
-        groups.add(groupName);
-      }
-    }
-    return (String[]) groups.toArray(new String[groups.size()]);
-  }
   /** Parses comma-separated-roles/groups into array of groups (strings). */
   public static String[] parseGroups(String csvRoles, String csvGroups) {
     List<String> groups = new ArrayList<String>();
@@ -138,6 +99,8 @@ public class MemberAttributes implements DataSerializable {
     parseCsv(groups, csvGroups);
     return (String[]) groups.toArray(new String[groups.size()]);
   }
+  
+  
   private static void parseCsv(List<String> groups, String csv) {
     if (csv == null || csv.length() == 0) {
       return;
@@ -151,82 +114,6 @@ public class MemberAttributes implements DataSerializable {
     }
   }
   
-  /** Writes the contents of this object to the given output. */
-  public void toData(DataOutput out) throws IOException {
-    out.writeInt(this.dcPort);
-    out.writeInt(this.vmPid);
-    out.writeInt(this.vmKind);
-    DataSerializer.writeString(this.name, out);
-    DataSerializer.writeStringArray(this.groups, out);
-    DataSerializer.writeString(this.durableClientAttributes==null ? "" : this.durableClientAttributes.getId(), out);
-    DataSerializer.writeInteger(Integer.valueOf(this.durableClientAttributes==null ? 300 : this.durableClientAttributes.getTimeout()), out);
-  }
-
-  /** Reads the contents of this object from the given input. */
-  public void fromData(DataInput in)
-  throws IOException, ClassNotFoundException {
-    this.dcPort = in.readInt();
-    this.vmPid = in.readInt();
-    this.vmKind = in.readInt();
-    this.name = DataSerializer.readString(in);
-    this.groups = DataSerializer.readStringArray(in);
-    String durableId = DataSerializer.readString(in);
-    int durableTimeout = DataSerializer.readInteger(in).intValue();
-    this.durableClientAttributes = new DurableClientAttributes(durableId, durableTimeout);
-  }
-  
-  private byte[] byteInfo;
-  
-  /** Returns the contents of this objects serialized as a byte array. */
-  public byte[] toByteArray() {
-    if (byteInfo != null) {
-      return byteInfo;
-    }
-    try {
-      HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
-      toData(hdos);
-      byteInfo = hdos.toByteArray();
-      return byteInfo;
-    }
-    catch (IOException e) {
-      throw new InternalGemFireError(LocalizedStrings.MemberAttributes_IOEXCEPTION_ON_A_BYTE_ARRAY_0.toLocalizedString(e));
-    }
-  }
-  
-  public static MemberAttributes fromByteArray(byte[] bytes) {
-    try {
-      return new MemberAttributes(bytes);
-    }
-    catch (IOException e) {
-      throw new InternalGemFireError(LocalizedStrings.MemberAttributes_IOEXCEPTION_ON_A_BYTE_ARRAY_0.toLocalizedString(e));
-    }
-    catch (ClassNotFoundException e) {
-      throw new InternalGemFireError(LocalizedStrings.MemberAttributes_CLASSNOTFOUNDEXCEPTION_IN_DESERIALIZATION_0.toLocalizedString(e));
-    }
-  }
-
-	/**
-	 * Returns a string representation of the object.
-	 * 
-	 * @return a string representation of the object
-	 */
-  @Override
-	public String toString() {
-		final StringBuffer sb = new StringBuffer("[MemberAttributes: ");
-		sb.append("dcPort=").append(this.dcPort);
-		sb.append(", vmPid=").append(this.vmPid);
-		sb.append(", vmKind=").append(this.vmKind);
-		sb.append(", name=").append(this.name);
-		sb.append(", groups=").append("(");
-    for (int i = 0; i < groups.length; i++) {
-      sb.append(groups[i]);
-    }
-    sb.append(")");
-    sb.append(", durableClientAttributes=").append(this.durableClientAttributes);
-    sb.append("]");
-		return sb.toString();
-	}
-
   /**
    * @return the membership view number in which this member was born
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
index 84a0bd7..3e767ae 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
@@ -230,7 +230,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
 
       SearchState state = searchState;
       
-      long locatorWaitTime = services.getConfig().getLocatorWaitTime() * 1000;
+      long locatorWaitTime = ((long)services.getConfig().getLocatorWaitTime()) * 1000L;
       long timeout = services.getConfig().getJoinTimeout();
       logger.debug("join timeout is set to {}", timeout);
       long retrySleep =  JOIN_RETRY_SLEEP;
@@ -560,7 +560,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     logger.debug("JoinLeave is recording the request to be processed in the next membership view");
     synchronized (viewRequests) {
       viewRequests.add(request);
-      viewRequests.notify();
+      viewRequests.notifyAll();
     }
   }
 
@@ -862,7 +862,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     FindCoordinatorRequest request = new FindCoordinatorRequest(this.localAddress, state.alreadyTried, state.viewId);
     Set<InternalDistributedMember> coordinators = new HashSet<InternalDistributedMember>();
     
-    long giveUpTime = System.currentTimeMillis() + services.getConfig().getLocatorWaitTime() * 1000;
+    long giveUpTime = System.currentTimeMillis() + ((long)services.getConfig().getLocatorWaitTime() * 1000L);
     
     int connectTimeout = (int)services.getConfig().getMemberTimeout() * 2;
     boolean anyResponses = false;
@@ -1055,7 +1055,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
   private void processJoinResponse(JoinResponseMessage rsp) {
     synchronized (joinResponse) {
       joinResponse[0] = rsp;
-      joinResponse.notify();
+      joinResponse.notifyAll();
     }
   }
   
@@ -1149,7 +1149,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
 
       isJoined = true;
       synchronized(joinResponse) {
-        joinResponse.notify();
+        joinResponse.notifyAll();
       }
 
       if (!newView.getCreator().equals(this.localAddress)) {
@@ -1253,7 +1253,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
           && newView.getCreator().equals(localAddress)) { // view-creator logs this
         newView.logCrashedMemberWeights(currentView, logger);
       }
-      int failurePoint = (int) (Math.round(51 * oldWeight) / 100.0);
+      int failurePoint = (int) (Math.round(51.0 * oldWeight) / 100.0);
       if (failedWeight > failurePoint && quorumLostView != newView) {
         quorumLostView = newView;
         logger.warn("total weight lost in this view change is {} of {}.  Quorum has been lost!", failedWeight, oldWeight);
@@ -1437,30 +1437,39 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     logger.debug("processing {}", m);
     switch (m.getDSFID()) {
     case JOIN_REQUEST:
+      assert m instanceof JoinRequestMessage;
       processJoinRequest((JoinRequestMessage) m);
       break;
     case JOIN_RESPONSE:
+      assert m instanceof JoinResponseMessage;
       processJoinResponse((JoinResponseMessage) m);
       break;
     case INSTALL_VIEW_MESSAGE:
+      assert m instanceof InstallViewMessage;
       processViewMessage((InstallViewMessage) m);
       break;
     case VIEW_ACK_MESSAGE:
+      assert m instanceof ViewAckMessage;
       processViewAckMessage((ViewAckMessage) m);
       break;
     case LEAVE_REQUEST_MESSAGE:
+      assert m instanceof LeaveRequestMessage;
       processLeaveRequest((LeaveRequestMessage) m);
       break;
     case REMOVE_MEMBER_REQUEST:
+      assert m instanceof RemoveMemberMessage;
       processRemoveRequest((RemoveMemberMessage) m);
       break;
     case FIND_COORDINATOR_REQ:
+      assert m instanceof FindCoordinatorRequest;
       processFindCoordinatorRequest((FindCoordinatorRequest) m);
       break;
     case FIND_COORDINATOR_RESP:
+      assert m instanceof FindCoordinatorResponse;
       processFindCoordinatorResponse((FindCoordinatorResponse) m);
       break;
     case NETWORK_PARTITION_MESSAGE:
+      assert m instanceof NetworkPartitionMessage;
       processNetworkPartitionMessage((NetworkPartitionMessage) m);
       break;
     default:
@@ -1591,7 +1600,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
       if (notRepliedYet.isEmpty() || (pendingRemovals != null && pendingRemovals.containsAll(notRepliedYet))) {
         logger.debug("All anticipated view responses received - notifying waiting thread");
         waiting = false;
-        notify();
+        notifyAll();
       } else {
         logger.debug("Still waiting for these view replies: {}", notRepliedYet);
       }
@@ -1616,14 +1625,16 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
           }
         }
       } finally {
-        if (!this.waiting) {
-          // if we've set waiting to false due to incoming messages then
-          // we've discounted receiving any other responses from the
-          // remaining members due to leave/crash notification
-          result = pendingRemovals;
-        } else {
-          result.addAll(pendingRemovals);
-          this.waiting = false;
+        synchronized(this) {
+          if (!this.waiting) {
+            // if we've set waiting to false due to incoming messages then
+            // we've discounted receiving any other responses from the
+            // remaining members due to leave/crash notification
+            result = pendingRemovals;
+          } else {
+            result.addAll(pendingRemovals);
+            this.waiting = false;
+          }
         }
       }
       return result;
@@ -1690,7 +1701,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     void shutdown() {
       shutdown = true;
       synchronized (viewRequests) {
-        viewRequests.notify();
+        viewRequests.notifyAll();
         interrupt();
       }
     }
@@ -1906,7 +1917,6 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
         // be reused in an auto-reconnect and get a new vmViewID
         mbrs.addAll(joinReqs);
         newView = new NetView(localAddress, viewNumber, mbrs, leaveReqs, new HashSet<InternalDistributedMember>(removalReqs));
-        int size = joinReqs.size();
         for (InternalDistributedMember mbr: joinReqs) {
           if (mbrs.contains(mbr)) {
             newView.setFailureDetectionPort(mbr, joinPorts.get(mbr));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/HeartbeatMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/HeartbeatMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/HeartbeatMessage.java
index a116913..6662d2c 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/HeartbeatMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/HeartbeatMessage.java
@@ -48,7 +48,7 @@ public class HeartbeatMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  public void process(DistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
  

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/HeartbeatRequestMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/HeartbeatRequestMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/HeartbeatRequestMessage.java
index f7e1009..3c08e33 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/HeartbeatRequestMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/HeartbeatRequestMessage.java
@@ -48,7 +48,7 @@ public class HeartbeatRequestMessage extends HighPriorityDistributionMessage{
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  public void process(DistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }   
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/InstallViewMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/InstallViewMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/InstallViewMessage.java
index 8d4cb4e..91f6918 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/InstallViewMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/InstallViewMessage.java
@@ -76,7 +76,7 @@ public class InstallViewMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  public void process(DistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/JoinResponseMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/JoinResponseMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/JoinResponseMessage.java
index df1b3f6..c01353a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/JoinResponseMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/JoinResponseMessage.java
@@ -36,7 +36,7 @@ public class JoinResponseMessage extends HighPriorityDistributionMessage {
   private NetView currentView;
   private String rejectionMessage;
   private InternalDistributedMember memberID;
-  private Object messengerData;
+  private byte[] messengerData;
   private boolean becomeCoordinator;
   
   public JoinResponseMessage(InternalDistributedMember memberID, NetView view) {
@@ -76,11 +76,11 @@ public class JoinResponseMessage extends HighPriorityDistributionMessage {
     return rejectionMessage;
   }
   
-  public Object getMessengerData() {
+  public byte[] getMessengerData() {
     return this.messengerData;
   }
   
-  public void setMessengerData(Object data) {
+  public void setMessengerData(byte[] data) {
     this.messengerData = data;
   }
 
@@ -114,7 +114,7 @@ public class JoinResponseMessage extends HighPriorityDistributionMessage {
     DataSerializer.writeObject(memberID, out);
     out.writeBoolean(becomeCoordinator);
     DataSerializer.writeString(rejectionMessage, out);
-    DataSerializer.writeObject(messengerData, out);
+    DataSerializer.writeByteArray(messengerData, out);
   }
 
   @Override
@@ -123,7 +123,7 @@ public class JoinResponseMessage extends HighPriorityDistributionMessage {
     memberID = DataSerializer.readObject(in);
     becomeCoordinator = in.readBoolean();
     rejectionMessage = DataSerializer.readString(in);
-    messengerData = DataSerializer.readObject(in);
+    messengerData = DataSerializer.readByteArray(in);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/ViewAckMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/ViewAckMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/ViewAckMessage.java
index 00f31d6..39ade6e 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/ViewAckMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messages/ViewAckMessage.java
@@ -74,7 +74,7 @@ public class ViewAckMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  public void process(DistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/AddressManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/AddressManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/AddressManager.java
index 0fd1c6e..1169044 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/AddressManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/AddressManager.java
@@ -23,10 +23,8 @@ import java.util.List;
 import org.apache.logging.log4j.Logger;
 import org.jgroups.Address;
 import org.jgroups.Event;
-import org.jgroups.Message;
 import org.jgroups.protocols.PingData;
 import org.jgroups.protocols.TP;
-import org.jgroups.protocols.UDP;
 import org.jgroups.stack.IpAddress;
 import org.jgroups.stack.Protocol;
 import org.jgroups.util.Responses;
@@ -55,19 +53,16 @@ public class AddressManager extends Protocol {
   @Override
   public Object up(Event evt) {
     
-//    logger.info("AddressManager.up: " + evt);
-    
     switch (evt.getType()) {
 
     case Event.FIND_MBRS:
       List<Address> missing = (List<Address>)evt.getArg();
-//      logger.debug("AddressManager.FIND_MBRS processing {}", missing);
+
       Responses responses = new Responses(false);
       for (Address laddr: missing) {
         try {
           if (laddr instanceof JGAddress) {
             PingData pd = new PingData(laddr, true, laddr.toString(), newIpAddress(laddr));
-//            logger.debug("AddressManager.FIND_MBRS adding response {}", pd);
             responses.addResponse(pd, false);
             updateUDPCache(pd);
           }
@@ -96,17 +91,13 @@ public class AddressManager extends Protocol {
       findPingDataMethod();
     }
     if (setPingData != null) {
-      Exception problem = null;
       try {
         setPingData.invoke(transport, new Object[]{pd});
-      } catch (InvocationTargetException e) {
-        problem = e;
-      } catch (IllegalAccessException e) {
-        problem = e;
-      }
-      if (problem != null && !warningLogged) {
-        log.warn("Unable to update JGroups address cache - this may affect performance", problem);
-        warningLogged = true;
+      } catch (InvocationTargetException | IllegalAccessException e) {
+        if (!warningLogged) {
+          log.warn("Unable to update JGroups address cache - this may affect performance", e);
+          warningLogged = true;
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSPingPonger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSPingPonger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSPingPonger.java
index fb32254..e2951ee 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSPingPonger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSPingPonger.java
@@ -35,23 +35,31 @@ public class GMSPingPonger {
   }
   
   public void sendPongMessage(JChannel channel, Address src, Address dest) throws Exception {
-    channel.send(createJGMessage(pongInBytes, src, dest, Version.CURRENT_ORDINAL)); 
+    channel.send(createPongMessage(src, dest)); 
   }
   
   public Message createPongMessage(Address src, Address dest) {
 	  return createJGMessage(pongInBytes, src, dest, Version.CURRENT_ORDINAL);
   }
   
+  public Message createPingMessage(Address src, Address dest) {
+    return createJGMessage(pingInBytes, src, dest, Version.CURRENT_ORDINAL);
+  }
+  
   public void sendPingMessage(JChannel channel, Address src, JGAddress dest) throws Exception {
-    channel.send(createJGMessage(pingInBytes, src, dest, Version.CURRENT_ORDINAL));
+    channel.send(createPingMessage(src, dest));
   }
 
   private Message createJGMessage(byte[] msgBytes, Address src, Address dest, short version) {
-	Message msg = new Message();
-	msg.setDest(dest);
-	msg.setSrc(src);
-	msg.setObject(msgBytes);
-	return msg;
+    Message msg = new Message();
+    msg.setDest(dest);
+    msg.setSrc(src);
+    msg.setObject(msgBytes);
+    msg.setFlag(Message.Flag.NO_RELIABILITY);
+    msg.setFlag(Message.Flag.NO_FC);
+    msg.setFlag(Message.Flag.DONT_BUNDLE);
+    msg.setFlag(Message.Flag.OOB);
+    return msg;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGAddress.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGAddress.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGAddress.java
index 1380eb2..6ddafa0 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGAddress.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGAddress.java
@@ -84,22 +84,6 @@ public class JGAddress extends UUID {
   }
 
 
-  private void setAddressToLocalHost() {
-      try {
-          ip_addr=InetAddress.getLocalHost();  // get first NIC found (on multi-homed systems)
-      }
-      catch(Exception e) {
-          ip_addr=null;
-      }
-      if(ip_addr == null) {
-          try {
-              ip_addr=InetAddress.getByName(null);
-          }
-          catch(UnknownHostException e) {
-          }
-      }
-  }
-
   public final InetAddress  getInetAddress()               {return ip_addr;}
   public final int          getPort()                    {return port;}
   
@@ -112,6 +96,7 @@ public class JGAddress extends UUID {
   }
 
 
+  @Override
   public String toString() {
       StringBuilder sb=new StringBuilder();
 
@@ -137,6 +122,7 @@ public class JGAddress extends UUID {
   }
 
 
+  @Override
   public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
       try {
           readFrom(in);
@@ -146,6 +132,7 @@ public class JGAddress extends UUID {
       }
   }
 
+  @Override
   public void writeExternal(ObjectOutput out) throws IOException {
       try {
           writeTo(out);
@@ -155,6 +142,7 @@ public class JGAddress extends UUID {
       }
   }
 
+  @Override
   public void writeTo(DataOutput out) throws Exception {
       if(ip_addr != null) {
           byte[] address=ip_addr.getAddress();  // 4 bytes (IPv4) or 16 bytes (IPv6)
@@ -180,6 +168,7 @@ public class JGAddress extends UUID {
     return leastSigBits;
   }
 
+  @Override
   public void readFrom(DataInput in) throws Exception {
       int len=in.readByte();
       if(len > 0 && (len != Global.IPV4_SIZE && len != Global.IPV6_SIZE))
@@ -202,6 +191,7 @@ public class JGAddress extends UUID {
       leastSigBits = in.readLong();
   }
 
+  @Override
   public int size() {
       // length (1 bytes) + 4 bytes for port
       int tmp_size=Global.BYTE_SIZE+ Global.SHORT_SIZE +Global.SHORT_SIZE
@@ -213,6 +203,7 @@ public class JGAddress extends UUID {
       return tmp_size;
   }
 
+  @Override
   public JGAddress copy() {
     JGAddress result = new JGAddress();
     result.mostSigBits = mostSigBits;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
index 4e68b63..326491a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
@@ -53,6 +53,7 @@ import org.jgroups.Message;
 import org.jgroups.Message.Flag;
 import org.jgroups.Message.TransientFlag;
 import org.jgroups.Receiver;
+import org.jgroups.ReceiverAdapter;
 import org.jgroups.View;
 import org.jgroups.ViewId;
 import org.jgroups.conf.ClassConfigurator;
@@ -64,6 +65,7 @@ import org.jgroups.util.UUID;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.ForcedDisconnectException;
 import com.gemstone.gemfire.GemFireConfigException;
+import com.gemstone.gemfire.GemFireIOException;
 import com.gemstone.gemfire.SystemConnectException;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.DurableClientAttributes;
@@ -100,13 +102,6 @@ public class JGroupsMessenger implements Messenger {
   private static final Logger logger = Services.getLogger();
 
   /**
-   * The system property that specifies the name of a file from which to read
-   * Jgroups configuration information
-   */
-  public static final String JGROUPS_CONFIG = System
-      .getProperty("geode.jgroups_config_file");
-
-  /**
    * The location (in the product) of the locator Jgroups config file.
    */
   private static final String DEFAULT_JGROUPS_TCP_CONFIG = "com/gemstone/gemfire/distributed/internal/membership/gms/messenger/jgroups-config.xml";
@@ -138,7 +133,7 @@ public class JGroupsMessenger implements Messenger {
 
   private GMSPingPonger pingPonger = new GMSPingPonger();
   
-  private volatile long pongsReceived;
+  protected volatile long pongsReceived;
   
   /**
    * A set that contains addresses that we have logged JGroups IOExceptions for in the
@@ -173,23 +168,15 @@ public class JGroupsMessenger implements Messenger {
 
     InputStream is= null;
 
-    if (JGROUPS_CONFIG != null) {
-      File file = new File(JGROUPS_CONFIG);
-      if (!file.exists()) {
-        throw new GemFireConfigException(LocalizedStrings.GroupMembershipService_JGROUPS_CONFIGURATION_FILE_0_DOES_NOT_EXIST.toLocalizedString(JGROUPS_CONFIG));
-      }
+    String r = null;
+    if (transport.isMcastEnabled()) {
+      r = DEFAULT_JGROUPS_MCAST_CONFIG;
+    } else {
+      r = DEFAULT_JGROUPS_TCP_CONFIG;
     }
-    else {
-      String r = null;
-      if (transport.isMcastEnabled()) {
-        r = DEFAULT_JGROUPS_MCAST_CONFIG;
-      } else {
-        r = DEFAULT_JGROUPS_TCP_CONFIG;
-      }
-      is = ClassPathLoader.getLatest().getResourceAsStream(getClass(), r);
-      if (is == null) {
-        throw new GemFireConfigException(LocalizedStrings.GroupMembershipService_CANNOT_FIND_0.toLocalizedString(r));
-      }
+    is = ClassPathLoader.getLatest().getResourceAsStream(getClass(), r);
+    if (is == null) {
+      throw new GemFireConfigException(LocalizedStrings.GroupMembershipService_CANNOT_FIND_0.toLocalizedString(r));
     }
 
     String properties;
@@ -198,11 +185,7 @@ public class JGroupsMessenger implements Messenger {
       //properties = config.getProtocolStackString();
       StringBuffer sb = new StringBuffer(3000);
       BufferedReader br;
-      if (JGROUPS_CONFIG != null) {
-        br = new BufferedReader(new InputStreamReader(is));
-      } else {
-        br = new BufferedReader(new InputStreamReader(is, "US-ASCII"));
-      }
+      br = new BufferedReader(new InputStreamReader(is, "US-ASCII"));
       String input;
       while ((input=br.readLine()) != null) {
         sb.append(input);
@@ -354,7 +337,7 @@ public class JGroupsMessenger implements Messenger {
   public void stop() {
     if (this.myChannel != null) {
       if ((services.isShutdownDueToForcedDisconnect() && services.isAutoReconnectEnabled()) || services.getManager().isReconnectingDS()) {
-        
+        // leave the channel open for reconnect attempts
       }
       else {
         this.myChannel.close();
@@ -396,12 +379,11 @@ public class JGroupsMessenger implements Messenger {
    * recipient.<p>
    * see Transport._send()
    */
-  public void handleJGroupsIOException(IOException e, Message msg, Address dest) {
+  public void handleJGroupsIOException(IOException e, Address dest) {
     if (addressesWithioExceptionsProcessed.contains(dest)) {
       return;
     }
     addressesWithioExceptionsProcessed.add(dest);
-    logger.info("processing JGroups IOException: " + e.getMessage());
     NetView v = this.view;
     JGAddress jgMbr = (JGAddress)dest;
     if (jgMbr != null && v != null) {
@@ -444,18 +426,18 @@ public class JGroupsMessenger implements Messenger {
         logger.info("Unable to find getPhysicallAddress method in UDP - parsing its address instead");
       }
       
-      if (this.jgAddress == null) {
-        String addr = udp.getLocalPhysicalAddress();
-        int cidx = addr.lastIndexOf(':');  // IPv6 literals might have colons
-        String host = addr.substring(0, cidx);
-        int jgport = Integer.parseInt(addr.substring(cidx+1, addr.length()));
-        try {
-          this.jgAddress = new JGAddress(logicalAddress, new IpAddress(InetAddress.getByName(host), jgport));
-        } catch (UnknownHostException e) {
-          myChannel.disconnect();
-          throw new SystemConnectException("unable to initialize jgroups address", e);
-        }
-      }
+//      if (this.jgAddress == null) {
+//        String addr = udp.getLocalPhysicalAddress();
+//        int cidx = addr.lastIndexOf(':');  // IPv6 literals might have colons
+//        String host = addr.substring(0, cidx);
+//        int jgport = Integer.parseInt(addr.substring(cidx+1, addr.length()));
+//        try {
+//          this.jgAddress = new JGAddress(logicalAddress, new IpAddress(InetAddress.getByName(host), jgport));
+//        } catch (UnknownHostException e) {
+//          myChannel.disconnect();
+//          throw new SystemConnectException("unable to initialize jgroups address", e);
+//        }
+//      }
     }
   
     // install the address in the JGroups channel protocols
@@ -563,15 +545,13 @@ public class JGroupsMessenger implements Messenger {
     
     boolean useMcast = false;
     if (services.getConfig().getTransport().isMcastEnabled()) {
-      useMcast = services.getManager().isMulticastAllowed()
-          && (msg.getMulticast() || allDestinations);
+      if (msg.getMulticast() || allDestinations) {
+        useMcast = services.getManager().isMulticastAllowed();
+      }
     }
     
     if (logger.isDebugEnabled() && reliably) {
-      String recips = "multicast";
-      if (!useMcast) {
-        recips = Arrays.toString(msg.getRecipients());
-      }
+      String recips = useMcast? "multicast" : Arrays.toString(msg.getRecipients());
       logger.debug("sending via JGroups: [{}] recipients: {}", msg, recips);
     }
     
@@ -579,22 +559,20 @@ public class JGroupsMessenger implements Messenger {
     
     if (useMcast) {
 
+      long startSer = theStats.startMsgSerialization();
+      Message jmsg = createJGMessage(msg, local, Version.CURRENT_ORDINAL);
+      theStats.endMsgSerialization(startSer);
+
       Exception problem = null;
       try {
-        long startSer = theStats.startMsgSerialization();
-        Message jmsg = createJGMessage(msg, local, Version.CURRENT_ORDINAL);
         jmsg.setTransientFlag(TransientFlag.DONT_LOOPBACK);
         if (!reliably) {
           jmsg.setFlag(Message.Flag.NO_RELIABILITY);
         }
-        theStats.endMsgSerialization(startSer);
         theStats.incSentBytes(jmsg.getLength());
         logger.trace("Sending JGroups message: {}", jmsg);
         myChannel.send(jmsg);
       }
-      catch (IllegalArgumentException e) {
-        problem = e;
-      }
       catch (Exception e) {
         logger.debug("caught unexpected exception", e);
         Throwable cause = e.getCause();
@@ -603,14 +581,12 @@ public class JGroupsMessenger implements Messenger {
         } else {
           problem = e;
         }
-      }
-      if (problem != null) {
         if (services.getShutdownCause() != null) {
-          Throwable cause = services.getShutdownCause();
+          Throwable shutdownCause = services.getShutdownCause();
           // If ForcedDisconnectException occurred then report it as actual
           // problem.
-          if (cause instanceof ForcedDisconnectException) {
-            problem = (Exception) cause;
+          if (shutdownCause instanceof ForcedDisconnectException) {
+            problem = (Exception) shutdownCause;
           } else {
             Throwable ne = problem;
             while (ne.getCause() != null) {
@@ -626,83 +602,83 @@ public class JGroupsMessenger implements Messenger {
     } // useMcast
     else { // ! useMcast
       int len = destinations.length;
-        List<GMSMember> calculatedMembers; // explicit list of members
-        int calculatedLen; // == calculatedMembers.len
-        if (len == 1 && destinations[0] == DistributionMessage.ALL_RECIPIENTS) { // send to all
-          // Grab a copy of the current membership
-          NetView v = services.getJoinLeave().getView();
-          
-          // Construct the list
-          calculatedLen = v.size();
-          calculatedMembers = new LinkedList<GMSMember>();
-          for (int i = 0; i < calculatedLen; i ++) {
-            InternalDistributedMember m = (InternalDistributedMember)v.get(i);
-            calculatedMembers.add((GMSMember)m.getNetMember());
-          }
-        } // send to all
-        else { // send to explicit list
-          calculatedLen = len;
-          calculatedMembers = new LinkedList<GMSMember>();
-          for (int i = 0; i < calculatedLen; i ++) {
-            calculatedMembers.add((GMSMember)destinations[i].getNetMember());
-          }
-        } // send to explicit list
-        Int2ObjectOpenHashMap<Message> messages = new Int2ObjectOpenHashMap<>();
-        long startSer = theStats.startMsgSerialization();
-        boolean firstMessage = true;
-        for (Iterator<GMSMember> it=calculatedMembers.iterator(); it.hasNext(); ) {
-          GMSMember mbr = it.next();
-          short version = mbr.getVersionOrdinal();
-          if ( !messages.containsKey(version) ) {
-            Message jmsg = createJGMessage(msg, local, version);
-            messages.put(version, jmsg);
-            if (firstMessage) {
-              theStats.incSentBytes(jmsg.getLength());
-              firstMessage = false;
-            }
-          }
+      List<GMSMember> calculatedMembers; // explicit list of members
+      int calculatedLen; // == calculatedMembers.len
+      if (len == 1 && destinations[0] == DistributionMessage.ALL_RECIPIENTS) { // send to all
+        // Grab a copy of the current membership
+        NetView v = services.getJoinLeave().getView();
+
+        // Construct the list
+        calculatedLen = v.size();
+        calculatedMembers = new LinkedList<GMSMember>();
+        for (int i = 0; i < calculatedLen; i ++) {
+          InternalDistributedMember m = (InternalDistributedMember)v.get(i);
+          calculatedMembers.add((GMSMember)m.getNetMember());
         }
-        theStats.endMsgSerialization(startSer);
-        Collections.shuffle(calculatedMembers);
-        int i=0;
-        for (GMSMember mbr: calculatedMembers) {
-          JGAddress to = new JGAddress(mbr);
-          short version = mbr.getVersionOrdinal();
-          Message jmsg = (Message)messages.get(version);
-          Exception problem = null;
-          try {
-            Message tmp = (i < (calculatedLen-1)) ? jmsg.copy(true) : jmsg;
-            if (!reliably) {
-              jmsg.setFlag(Message.Flag.NO_RELIABILITY);
-            }
-            tmp.setDest(to);
-            tmp.setSrc(this.jgAddress);
-            logger.trace("Unicasting to {}", to);
-            myChannel.send(tmp);
+      } // send to all
+      else { // send to explicit list
+        calculatedLen = len;
+        calculatedMembers = new LinkedList<GMSMember>();
+        for (int i = 0; i < calculatedLen; i ++) {
+          calculatedMembers.add((GMSMember)destinations[i].getNetMember());
+        }
+      } // send to explicit list
+      Int2ObjectOpenHashMap<Message> messages = new Int2ObjectOpenHashMap<>();
+      long startSer = theStats.startMsgSerialization();
+      boolean firstMessage = true;
+      for (Iterator<GMSMember> it=calculatedMembers.iterator(); it.hasNext(); ) {
+        GMSMember mbr = it.next();
+        short version = mbr.getVersionOrdinal();
+        if ( !messages.containsKey(version) ) {
+          Message jmsg = createJGMessage(msg, local, version);
+          messages.put(version, jmsg);
+          if (firstMessage) {
+            theStats.incSentBytes(jmsg.getLength());
+            firstMessage = false;
           }
-          catch (Exception e) {
-            problem = e;
+        }
+      }
+      theStats.endMsgSerialization(startSer);
+      Collections.shuffle(calculatedMembers);
+      int i=0;
+      for (GMSMember mbr: calculatedMembers) {
+        JGAddress to = new JGAddress(mbr);
+        short version = mbr.getVersionOrdinal();
+        Message jmsg = (Message)messages.get(version);
+        Exception problem = null;
+        try {
+          Message tmp = (i < (calculatedLen-1)) ? jmsg.copy(true) : jmsg;
+          if (!reliably) {
+            jmsg.setFlag(Message.Flag.NO_RELIABILITY);
           }
-          if (problem != null) {
-            if (services.getManager().getShutdownCause() != null) {
-              Throwable cause = services.getManager().getShutdownCause();
-              // If ForcedDisconnectException occurred then report it as actual
-              // problem.
-              if (cause instanceof ForcedDisconnectException) {
-                problem = (Exception) cause;
-              } else {
-                Throwable ne = problem;
-                while (ne.getCause() != null) {
-                  ne = ne.getCause();
-                }
-                ne.initCause(services.getManager().getShutdownCause());
+          tmp.setDest(to);
+          tmp.setSrc(this.jgAddress);
+          logger.trace("Unicasting to {}", to);
+          myChannel.send(tmp);
+        }
+        catch (Exception e) {
+          problem = e;
+        }
+        if (problem != null) {
+          Throwable cause = services.getShutdownCause();
+          if (cause != null) {
+            // If ForcedDisconnectException occurred then report it as actual
+            // problem.
+            if (cause instanceof ForcedDisconnectException) {
+              problem = (Exception) cause;
+            } else {
+              Throwable ne = problem;
+              while (ne.getCause() != null) {
+                ne = ne.getCause();
               }
+              ne.initCause(cause);
             }
+          }
           final String channelClosed = LocalizedStrings.GroupMembershipService_CHANNEL_CLOSED.toLocalizedString();
-//          services.getManager().membershipFailure(channelClosed, problem);
+          //          services.getManager().membershipFailure(channelClosed, problem);
           throw new DistributedSystemDisconnectedException(channelClosed, problem);
-          }
-        } // send individually
+        }
+      } // send individually
     } // !useMcast
 
     // The contract is that every destination enumerated in the
@@ -769,12 +745,16 @@ public class JGroupsMessenger implements Messenger {
       msg.setBuffer(out_stream.toByteArray());
       services.getStatistics().endMsgSerialization(start);
     }
-    catch(IOException ex) {
-        IllegalArgumentException ia = new
-          IllegalArgumentException("Error serializing message");
-        ia.initCause(ex);
-        throw ia;
-        //throw new IllegalArgumentException(ex.toString());
+    catch(IOException | GemFireIOException ex) {
+      logger.warn("Error serializing message", ex);
+      if (ex instanceof GemFireIOException) {
+        throw (GemFireIOException)ex;
+      } else {
+        GemFireIOException ioe = new
+          GemFireIOException("Error serializing message");
+        ioe.initCause(ex);
+        throw ioe;
+      }
     }
     return msg;
   }
@@ -820,19 +800,19 @@ public class JGroupsMessenger implements Messenger {
       GMSMember m = DataSerializer.readObject(dis);
 
       result = DataSerializer.readObject(dis);
-      if (result instanceof DistributionMessage) {
-        DistributionMessage dm = (DistributionMessage)result;
-        // JoinRequestMessages are sent with an ID that may have been
-        // reused from a previous life by way of auto-reconnect,
-        // so we don't want to find a canonical reference for the
-        // request's sender ID
-        if (dm.getDSFID() == JOIN_REQUEST) {
-          sender = ((JoinRequestMessage)dm).getMemberID();
-        } else {
-          sender = getMemberFromView(m, ordinal);
-        }
-        ((DistributionMessage)result).setSender(sender);
+
+      DistributionMessage dm = (DistributionMessage)result;
+      
+      // JoinRequestMessages are sent with an ID that may have been
+      // reused from a previous life by way of auto-reconnect,
+      // so we don't want to find a canonical reference for the
+      // request's sender ID
+      if (dm.getDSFID() == JOIN_REQUEST) {
+        sender = ((JoinRequestMessage)dm).getMemberID();
+      } else {
+        sender = getMemberFromView(m, ordinal);
       }
+      ((DistributionMessage)result).setSender(sender);
       
       services.getStatistics().endMsgDeserialization(start);
     }
@@ -850,17 +830,23 @@ public class JGroupsMessenger implements Messenger {
   
   
   /** look for certain messages that may need to be altered before being sent */
-  private void filterOutgoingMessage(DistributionMessage m) {
+  void filterOutgoingMessage(DistributionMessage m) {
     switch (m.getDSFID()) {
     case JOIN_RESPONSE:
       JoinResponseMessage jrsp = (JoinResponseMessage)m;
       
-      if (jrsp.getRejectionMessage() != null
+      if (jrsp.getRejectionMessage() == null
           &&  services.getConfig().getTransport().isMcastEnabled()) {
         // get the multicast message digest and pass it with the join response
         Digest digest = (Digest)this.myChannel.getProtocolStack()
             .getTopProtocol().down(Event.GET_DIGEST_EVT);
-        jrsp.setMessengerData(digest);
+        HeapDataOutputStream hdos = new HeapDataOutputStream(500, Version.CURRENT);
+        try {
+          digest.writeTo(hdos);
+        } catch (Exception e) {
+          logger.fatal("Unable to serialize JGroups messaging digest", e);
+        }
+        jrsp.setMessengerData(hdos.toByteArray());
       }
       break;
     default:
@@ -868,18 +854,27 @@ public class JGroupsMessenger implements Messenger {
     }
   }
   
-  private void filterIncomingMessage(DistributionMessage m) {
+  void filterIncomingMessage(DistributionMessage m) {
     switch (m.getDSFID()) {
     case JOIN_RESPONSE:
       JoinResponseMessage jrsp = (JoinResponseMessage)m;
       
-      if (jrsp.getRejectionMessage() != null
+      if (jrsp.getRejectionMessage() == null
           &&  services.getConfig().getTransport().isMcastEnabled()) {
-        Digest digest = (Digest)jrsp.getMessengerData();
-        if (digest != null) {
-          logger.trace("installing JGroups message digest {}", digest);
-          this.myChannel.getProtocolStack()
-              .getTopProtocol().down(new Event(Event.SET_DIGEST, digest));
+        byte[] serializedDigest = jrsp.getMessengerData();
+        ByteArrayInputStream bis = new ByteArrayInputStream(serializedDigest);
+        DataInputStream dis = new DataInputStream(bis);
+        try {
+          Digest digest = new Digest();
+          digest.readFrom(dis);
+          if (digest != null) {
+            logger.trace("installing JGroups message digest {}", digest);
+            this.myChannel.getProtocolStack()
+                .getTopProtocol().down(new Event(Event.SET_DIGEST, digest));
+            jrsp.setMessengerData(null);
+          }
+        } catch (Exception e) {
+          logger.fatal("Unable to read JGroups messaging digest", e);
         }
       }
       break;
@@ -894,13 +889,20 @@ public class JGroupsMessenger implements Messenger {
   }
   
   /**
-   * returns the JGroups configuration string
+   * returns the JGroups configuration string, for testing
    */
   public String getJGroupsStackConfig() {
     return this.jgStackConfig;
   }
   
   /**
+   * returns the pinger, for testing
+   */
+  public GMSPingPonger getPingPonger() {
+    return this.pingPonger;
+  }
+  
+  /**
    * for unit testing we need to replace UDP with a fake UDP protocol 
    */
   public void setJGroupsStackConfigForTesting(String config) {
@@ -954,9 +956,10 @@ public class JGroupsMessenger implements Messenger {
     return qc;
   }
   /**
-   * Puller receives incoming JGroups messages and passes them to a handler
+   * JGroupsReceiver receives incoming JGroups messages and passes them to a handler.
+   * It may be accessed through JChannel.getReceiver().
    */
-  class JGroupsReceiver implements Receiver  {
+  class JGroupsReceiver extends ReceiverAdapter  {
   
     @Override
     public void receive(Message jgmsg) {
@@ -970,6 +973,9 @@ public class JGroupsMessenger implements Messenger {
       
       //Respond to ping messages sent from other systems that are in a auto reconnect state
       byte[] contents = jgmsg.getBuffer();
+      if (contents == null) {
+        return;
+      }
       if (pingPonger.isPingMessage(contents)) {
         try {
           pingPonger.sendPongMessage(myChannel, jgAddress, jgmsg.getSrc());
@@ -985,45 +991,27 @@ public class JGroupsMessenger implements Messenger {
       
       Object o = readJGMessage(jgmsg);
       if (o == null) {
-        logger.warn(LocalizedMessage.create(
-            LocalizedStrings.GroupMembershipService_MEMBERSHIP_GEMFIRE_RECEIVED_NULL_MESSAGE_FROM__0, String.valueOf(jgmsg)));
-        logger.warn(LocalizedMessage.create(
-            LocalizedStrings.GroupMembershipService_MEMBERSHIP_MESSAGE_HEADERS__0, jgmsg.printObjectHeaders()));
-        return;
-      } else if ( !(o instanceof DistributionMessage) ) {
-        logger.warn("Received something other than a message from " + jgmsg.getSrc() + ": " + o);
         return;
       }
 
       DistributionMessage msg = (DistributionMessage)o;
+      assert msg.getSender() != null;
       
       // admin-only VMs don't have caches, so we ignore cache operations
       // multicast to them, avoiding deserialization cost and classpath
       // problems
       if ( (services.getConfig().getTransport().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE)
            && (msg instanceof DistributedCacheOperation.CacheOperationMessage)) {
-        if (logger.isTraceEnabled())
-          logger.trace("Membership: admin VM discarding cache operation message {}", jgmsg.getObject());
         return;
       }
 
       msg.resetTimestamp();
       msg.setBytesRead(jgmsg.getLength());
             
-      if (msg.getSender() == null) {
-        Exception e = new Exception(LocalizedStrings.GroupMembershipService_NULL_SENDER.toLocalizedString());
-        logger.warn(LocalizedMessage.create(
-            LocalizedStrings.GroupMembershipService_MEMBERSHIP_GEMFIRE_RECEIVED_A_MESSAGE_WITH_NO_SENDER_ADDRESS), e);
-      }
-      
       try {
-        if (logger.isTraceEnabled()) {
-          logger.trace("JGroupsMessenger dispatching {} from {}", msg, msg.getSender());
-        }
+        logger.trace("JGroupsMessenger dispatching {} from {}", msg, msg.getSender());
         filterIncomingMessage(msg);
-        MessageHandler h = getMessageHandler(msg);
-        logger.trace("Handler for this message is {}", h);
-        h.processMessage(msg);
+        getMessageHandler(msg).processMessage(msg);
       }
       catch (MemberShunnedException e) {
         // message from non-member - ignore
@@ -1053,36 +1041,7 @@ public class JGroupsMessenger implements Messenger {
       }
       return h;
     }
-    
-    
-    @Override
-    public void block() {
-    }
-    
-
-    @Override
-    public void viewAccepted(View new_view) {
-    }
-
-
-    @Override
-    public void getState(OutputStream output) throws Exception {
-    }
-
-    @Override
-    public void setState(InputStream input) throws Exception {
-    }
-
-    @Override
-    public void suspect(Address suspected_mbr) {
-    }
-
-    @Override
-    public void unblock() {
-    }
-    
-        
-  } // Puller class
+  }
   
   
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
index 8ba59b6..1687261 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
@@ -97,7 +97,7 @@ public class Transport extends UDP {
     catch (IOException e) {
       if (messenger != null
           /*&& e.getMessage().contains("Operation not permitted")*/) { // this is the english Oracle JDK exception condition we really want to catch
-        messenger.handleJGroupsIOException(e, msg, dest);
+        messenger.handleJGroupsIOException(e, dest);
       }
     }
     catch(Throwable e) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
index 516fe8d..92793ae 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
@@ -362,7 +362,7 @@ public class TcpServer {
             versionOrdinal = input.readShort();
           }
 
-          if (log.isDebugEnabled()) {
+          if (log.isDebugEnabled() && versionOrdinal != Version.CURRENT_ORDINAL) {
             log.debug("Locator reading request from " + sock.getInetAddress() + " with version " + Version.fromOrdinal(versionOrdinal, false));
           }
           input = new VersionedDataInputStream(input, Version.fromOrdinal(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
index 150b408..7bb97b9 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
@@ -730,9 +730,9 @@ class ParentLocalizedStrings {
   public static final StringId GroupMembershipService_MEMBERSHIP_FAULT_WHILE_PROCESSING_VIEW_REMOVAL_OF__0 = new StringId(1722, "Membership: Fault while processing view removal of  {0}");
   public static final StringId GroupMembershipService_MEMBERSHIP_FINISHED_VIEW_PROCESSING_VIEWID___0 = new StringId(1723, "Membership: Finished view processing viewID =  {0}");
   public static final StringId GroupMembershipService_MEMBERSHIP_GEMFIRE_RECEIVED_A_MESSAGE_WITH_NO_SENDER_ADDRESS = new StringId(1724, "Membership: GemFire received a message with no sender address");
-  public static final StringId GroupMembershipService_MEMBERSHIP_GEMFIRE_RECEIVED_NULL_MESSAGE_FROM__0 = new StringId(1725, "Membership: GemFire received null message from  {0}");
+  // ok to reuse 1725
   public static final StringId GroupMembershipService_MEMBERSHIP_IGNORING_SURPRISE_CONNECT_FROM_SHUNNED_MEMBER_0 = new StringId(1726, "Membership: Ignoring surprise connect from shunned member <{0}>");
-  public static final StringId GroupMembershipService_MEMBERSHIP_MESSAGE_HEADERS__0 = new StringId(1727, "Membership: message headers:  {0}");
+  // ok to reuse 1727
   // ok to reuse 1728
   public static final StringId GroupMembershipService_MEMBERSHIP_PAUSING_TO_ALLOW_OTHER_CONCURRENT_PROCESSES_TO_JOIN_THE_DISTRIBUTED_SYSTEM = new StringId(1729, "Membership: Pausing to allow other concurrent processes to join the distributed system");
   public static final StringId GroupMembershipService_MEMBERSHIP_PROCESSING_ADDITION__0_ = new StringId(1730, "Membership: Processing addition < {0} >");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
index dd6f1fa..9d0f69f 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
@@ -581,7 +581,7 @@ public class ReconnectDUnitTest extends CacheTestCase
     Properties config = getDistributedSystemProperties();
     config.put(DistributionConfig.ROLES_NAME, "");
     config.put(DistributionConfig.LOG_LEVEL_NAME, getDUnitLogLevel());
-    config.put("log-file", "roleLossController.log");
+//    config.put("log-file", "roleLossController.log");
     //creating the DS
     getSystem(config);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
index 82dfdb7..51771cb 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
@@ -347,6 +347,7 @@ public class DistributionManagerDUnitTest extends DistributedTestCase {
       public void afterCreate(EntryEvent event) {
         try {
           if (playDead) {
+            MembershipManagerHelper.beSickMember(system);
             MembershipManagerHelper.playDead(system);
           }
           Thread.sleep(15000);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
index 2ce1ca7..bee2367 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
@@ -16,16 +16,21 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership;
 
+import static org.mockito.Mockito.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.net.InetAddress;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -42,16 +47,28 @@ import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.GemFireConfigException;
 import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.SerialAckedMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.GMSUtil;
 import com.gemstone.gemfire.distributed.internal.membership.gms.ServiceConfig;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.JoinLeave;
 import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.HeartbeatMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.HeartbeatRequestMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.InstallViewMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinRequestMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinResponseMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.LeaveRequestMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.RemoveMemberMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.SuspectMembersMessage;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messages.ViewAckMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.SocketCreator;
@@ -159,6 +176,7 @@ public class MembershipJUnitTest {
     
     MembershipManager m1=null, m2=null;
     Locator l = null;
+    int mcastPort = AvailablePortHelper.getRandomAvailableUDPPort();
     
     try {
       
@@ -175,9 +193,11 @@ public class MembershipJUnitTest {
       // create configuration objects
       Properties nonDefault = new Properties();
       nonDefault.put(DistributionConfig.DISABLE_TCP_NAME, "true");
-      nonDefault.put(DistributionConfig.MCAST_PORT_NAME, "0");
+      nonDefault.put(DistributionConfig.MCAST_PORT_NAME, String.valueOf(mcastPort));
       nonDefault.put(DistributionConfig.LOG_FILE_NAME, "");
-//      nonDefault.put(DistributionConfig.LOG_LEVEL_NAME, "finest");
+      nonDefault.put(DistributionConfig.LOG_LEVEL_NAME, "fine");
+      nonDefault.put(DistributionConfig.GROUPS_NAME, "red, blue");
+      nonDefault.put(DistributionConfig.MEMBER_TIMEOUT_NAME, "2000");
       nonDefault.put(DistributionConfig.LOCATORS_NAME, localHost.getHostName()+'['+port+']');
       DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
       RemoteTransportConfig transport = new RemoteTransportConfig(config,
@@ -222,7 +242,38 @@ public class MembershipJUnitTest {
           }
         }
       }
-        
+      
+      System.out.println("testing multicast availability");
+      assertTrue(m1.testMulticast());
+      
+      System.out.println("multicasting SerialAckedMessage from m1 to m2");
+      SerialAckedMessage msg = new SerialAckedMessage();
+      msg.setRecipient(m2.getLocalMember());
+      msg.setMulticast(true);
+      m1.send(new InternalDistributedMember[] {m2.getLocalMember()}, msg, null);
+      giveUp = System.currentTimeMillis() + 5000;
+      boolean verified = false;
+      Throwable problem = null;
+      while (giveUp > System.currentTimeMillis()) {
+        try {
+          verify(listener2).messageReceived(isA(SerialAckedMessage.class));
+          verified = true;
+          break;
+        } catch (Error e) {
+          problem = e;
+          Thread.sleep(500);
+        }
+      }
+      if (!verified) {
+        if (problem != null) {
+          problem.printStackTrace();
+        }
+        fail("Expected a multicast message to be received");
+      }
+      
+      // let the managers idle for a while and get used to each other
+      Thread.sleep(4000l);
+      
       m2.shutdown();
       assertTrue(!m2.isConnected());
       
@@ -284,5 +335,64 @@ public class MembershipJUnitTest {
     assertEquals(600+4, str.length());
   }
   
+  @Test
+  public void testMessagesThrowExceptionIfProcessed() throws Exception {
+    DistributionManager dm = null;
+    try {
+      new HeartbeatMessage().process(dm);
+      fail("expected an exception to be thrown");
+    } catch (Exception e) {
+      // okay
+    }
+    try {
+      new HeartbeatRequestMessage().process(dm);
+      fail("expected an exception to be thrown");
+    } catch (Exception e) {
+      // okay
+    }
+    try {
+      new InstallViewMessage().process(dm);
+      fail("expected an exception to be thrown");
+    } catch (Exception e) {
+      // okay
+    }
+    try {
+      new JoinRequestMessage().process(dm);
+      fail("expected an exception to be thrown");
+    } catch (Exception e) {
+      // okay
+    }
+    try {
+      new JoinResponseMessage().process(dm);
+      fail("expected an exception to be thrown");
+    } catch (Exception e) {
+      // okay
+    }
+    try {
+      new LeaveRequestMessage().process(dm);
+      fail("expected an exception to be thrown");
+    } catch (Exception e) {
+      // okay
+    }
+    try {
+      new RemoveMemberMessage().process(dm);
+      fail("expected an exception to be thrown");
+    } catch (Exception e) {
+      // okay
+    }
+    try {
+      new SuspectMembersMessage().process(dm);
+      fail("expected an exception to be thrown");
+    } catch (Exception e) {
+      // okay
+    }
+    try {
+      new ViewAckMessage().process(dm);
+      fail("expected an exception to be thrown");
+    } catch (Exception e) {
+      // okay
+    }
+  }
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd43c341/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/MembershipManagerHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/MembershipManagerHelper.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/MembershipManagerHelper.java
index c5141de..f764ef9 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/MembershipManagerHelper.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/MembershipManagerHelper.java
@@ -156,6 +156,7 @@ public class MembershipManagerHelper
   public static void crashDistributedSystem(final DistributedSystem msys) {
     msys.getLogWriter().info("crashing distributed system: " + msys);
     MembershipManagerHelper.inhibitForcedDisconnectLogging(true);
+    MembershipManagerHelper.beSickMember(msys);
     MembershipManagerHelper.playDead(msys);
     GMSMembershipManager mgr = ((GMSMembershipManager)getMembershipManager(msys));
     mgr.forceDisconnect("for testing");


[37/50] [abbrv] incubator-geode git commit: GEODE-590: GMSLocatorRecoveryJUnitTest.testRecoverFromOther Reset gemfire.bind-address system property

Posted by kl...@apache.org.
GEODE-590: GMSLocatorRecoveryJUnitTest.testRecoverFromOther
Reset gemfire.bind-address system property


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

Branch: refs/heads/feature/GEODE-217
Commit: 1e93c6f952a4355da0a04319eaa800cfcbadaac6
Parents: bd43c34
Author: Jason Huynh <hu...@gmail.com>
Authored: Tue Dec 8 09:24:21 2015 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Dec 8 09:24:21 2015 -0800

----------------------------------------------------------------------
 .../internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java       | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1e93c6f9/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
index 86205b9..585ff17 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
@@ -127,6 +127,7 @@ public class GMSHealthMonitorJUnitTest {
   @After
   public void tearDown() {
     gmsHealthMonitor.stop();
+    System.getProperties().remove("gemfire.bind-address");
   }
 
   @Test


[33/50] [abbrv] incubator-geode git commit: GEODE-563: Moving gfsh tests from closed

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
new file mode 100644
index 0000000..83264d5
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
@@ -0,0 +1,1154 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.DiskStore;
+import com.gemstone.gemfire.cache.DiskStoreFactory;
+import com.gemstone.gemfire.cache.EvictionAction;
+import com.gemstone.gemfire.cache.EvictionAttributes;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.query.data.PortfolioPdx;
+import com.gemstone.gemfire.compression.SnappyCompressor;
+import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.SharedConfiguration;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.cache.DiskStoreImpl;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.SnapshotTestUtil;
+import com.gemstone.gemfire.internal.cache.persistence.PersistentMemberManager;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+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 dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.StringTokenizer;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * The DiskStoreCommandsDUnitTest class is a distributed test suite of test cases for testing the disk store commands
+ * that are part of Gfsh. </p>
+ *
+ * @author John Blum
+ * @author David Hoots
+ * @see com.gemstone.gemfire.management.internal.cli.commands.DiskStoreCommands
+ * @see org.junit.Assert
+ * @see org.junit.Test
+ * @since 7.0
+ */
+public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
+  private static final long serialVersionUID = 1L;
+
+  final List<String> filesToBeDeleted = new CopyOnWriteArrayList<String>();
+
+  public DiskStoreCommandsDUnitTest(final String testName) {
+    super(testName);
+  }
+
+  @SuppressWarnings("serial")
+  @Test
+  public void testMissingDiskStore() {
+    final String regionName = "testShowMissingDiskStoreRegion";
+
+    createDefaultSetup(null);
+
+    final VM vm0 = Host.getHost(0).getVM(0);
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1Name = "VM" + vm1.getPid();
+    final String diskStoreName = "DiskStoreCommandsDUnitTest";
+
+    // Default setup creates a cache in the Manager, now create a cache in VM1
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm1Name);
+        getSystem(localProps);
+        Cache cache = getCache();
+      }
+    });
+
+    // Create a disk store and region in the Manager (VM0) and VM1 VMs
+    for (final VM vm : (new VM[]{vm0, vm1})) {
+      final String vmName = "VM" + vm.getPid();
+      vm.invoke(new SerializableRunnable() {
+        public void run() {
+          Cache cache = getCache();
+
+          File diskStoreDirFile = new File(diskStoreName + vm.getPid());
+          diskStoreDirFile.mkdirs();
+
+          DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+          diskStoreFactory.setDiskDirs(new File[]{diskStoreDirFile});
+          diskStoreFactory.setMaxOplogSize(1);
+          diskStoreFactory.setAllowForceCompaction(true);
+          diskStoreFactory.setAutoCompact(false);
+          diskStoreFactory.create(regionName);
+
+          RegionFactory regionFactory = cache.createRegionFactory();
+          regionFactory.setDiskStoreName(regionName);
+          regionFactory.setDiskSynchronous(true);
+          regionFactory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+          regionFactory.setScope(Scope.DISTRIBUTED_ACK);
+          regionFactory.create(regionName);
+        }
+      });
+    }
+
+    // Add data to the region
+    vm0.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(regionName);
+        region.put("A", "B");
+      }
+    });
+
+    // Make sure that everything thus far is okay and there are no missing disk stores
+    CommandResult cmdResult = executeCommand(CliStrings.SHOW_MISSING_DISK_STORE);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertTrue(commandResultToString(cmdResult).contains("No missing disk store found"));
+
+    // Close the region in the Manager (VM0) VM
+    vm0.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(regionName);
+        region.close();
+      }
+    });
+
+    // Add data to VM1 and then close the region
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(regionName);
+        region.put("A", "C");
+        region.close();
+      }
+    });
+
+    // Add the region back to the Manager (VM0) VM
+    vm0.invokeAsync(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+
+        RegionFactory regionFactory = cache.createRegionFactory();
+        regionFactory.setDiskStoreName(regionName);
+        regionFactory.setDiskSynchronous(true);
+        regionFactory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+        regionFactory.setScope(Scope.DISTRIBUTED_ACK);
+        try {
+          regionFactory.create(regionName);
+        } catch (DistributedSystemDisconnectedException e) {
+          // okay to ignore
+        }
+      }
+    });
+
+    // Wait for the region in the Manager (VM0) to come online
+    vm0.invoke(new SerializableRunnable() {
+      public void run() {
+        WaitCriterion waitCriterion = new WaitCriterion() {
+          public boolean done() {
+            Cache cache = getCache();
+            PersistentMemberManager memberManager = ((GemFireCacheImpl) cache).getPersistentMemberManager();
+            return !memberManager.getWaitingRegions().isEmpty();
+          }
+
+          public String description() {
+            return "Waiting for another persistent member to come online";
+          }
+        };
+        waitForCriterion(waitCriterion, 70000, 100, true);
+      }
+    });
+
+    // Validate that there is a missing disk store on VM1
+    cmdResult = executeCommand(CliStrings.SHOW_MISSING_DISK_STORE);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    String stringResult = commandResultToString(cmdResult);
+    System.out.println("command result=" + stringResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Disk Store ID.*Host.*Directory"));
+    assertTrue(stringContainsLine(stringResult, ".*" + diskStoreName + vm1.getPid()));
+
+    // Extract the id from the returned missing disk store
+    String line = getLineFromString(stringResult, 3);
+    assertFalse(line.contains("---------"));
+    StringTokenizer resultTokenizer = new StringTokenizer(line);
+    String id = resultTokenizer.nextToken();
+
+    // Remove the missing disk store and validate the result
+    cmdResult = executeCommand("revoke missing-disk-store --id=" + id);
+    assertNotNull(cmdResult);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertTrue(commandResultToString(cmdResult).contains("Missing disk store successfully revoked"));
+
+    // Do our own cleanup so that the disk store directories can be removed
+    super.destroyDefaultSetup();
+    for (final VM vm : (new VM[]{vm0, vm1})) {
+      final String vmName = "VM" + vm.getPid();
+      vm.invoke(new SerializableRunnable() {
+        public void run() {
+          try {
+            FileUtil.delete((new File(diskStoreName + vm.getPid())));
+          } catch (IOException iex) {
+            // There's nothing else we can do
+          }
+        }
+      });
+    }
+  }
+
+  public void testDescribeOfflineDiskStore() {
+    createDefaultSetup(null);
+
+    final File diskStoreDir = new File(new File(".").getAbsolutePath(), "DiskStoreCommandDUnitDiskStores");
+    diskStoreDir.mkdir();
+    this.filesToBeDeleted.add(diskStoreDir.getAbsolutePath());
+
+    final String diskStoreName1 = "DiskStore1";
+    final String region1 = "Region1";
+    final String region2 = "Region2";
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        final Cache cache = getCache();
+
+        DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+        diskStoreFactory.setDiskDirs(new File[]{diskStoreDir});
+        final DiskStore diskStore1 = diskStoreFactory.create(diskStoreName1);
+        assertNotNull(diskStore1);
+
+        RegionFactory regionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT);
+        regionFactory.setDiskStoreName(diskStoreName1);
+        regionFactory.setDiskSynchronous(true);
+        regionFactory.create(region1);
+
+        regionFactory.setCompressor(SnappyCompressor.getDefaultInstance());
+        regionFactory.create(region2);
+
+        cache.close();
+        assertTrue(new File(diskStoreDir, "BACKUP" + diskStoreName1 + ".if").exists());
+      }
+    });
+
+    CommandResult cmdResult = executeCommand(
+        "describe offline-disk-store --name=" + diskStoreName1 + " --disk-dirs=" + diskStoreDir.getAbsolutePath());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    String stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult,
+        ".*/" + region1 + ": -lru=none -concurrencyLevel=16 -initialCapacity=16 -loadFactor=0.75 -offHeap=false -compressor=none -statisticsEnabled=false"));
+    assertTrue(stringContainsLine(stringResult,
+        ".*/" + region2 + ": -lru=none -concurrencyLevel=16 -initialCapacity=16 -loadFactor=0.75 -offHeap=false -compressor=com.gemstone.gemfire.compression.SnappyCompressor -statisticsEnabled=false"));
+
+    cmdResult = executeCommand(
+        "describe offline-disk-store --name=" + diskStoreName1 + " --disk-dirs=" + diskStoreDir.getAbsolutePath() + " --region=/" + region1);
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(2, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, ".*/" + region1 + ": .*"));
+    assertFalse(stringContainsLine(stringResult, ".*/" + region2 + ": .*"));
+  }
+
+  public void testOfflineDiskStorePdxCommands() {
+    final Properties props = new Properties();
+    props.setProperty("mcast-port", "0");
+    props.setProperty("start-locator", "localhost[" + AvailablePortHelper.getRandomAvailableTCPPort() + "]");
+
+    final File diskStoreDir = new File(new File(".").getAbsolutePath(), "DiskStoreCommandDUnitDiskStores");
+    diskStoreDir.mkdir();
+    this.filesToBeDeleted.add(diskStoreDir.getAbsolutePath());
+
+    final String diskStoreName1 = "DiskStore1";
+    final String region1 = "Region1";
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        final Cache cache = new CacheFactory(props).setPdxPersistent(true).setPdxDiskStore(diskStoreName1).create();
+
+        DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+        diskStoreFactory.setDiskDirs(new File[]{diskStoreDir});
+        final DiskStore diskStore1 = diskStoreFactory.create(diskStoreName1);
+        assertNotNull(diskStore1);
+
+        RegionFactory regionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT);
+        regionFactory.setDiskStoreName(diskStoreName1);
+        regionFactory.setDiskSynchronous(true);
+        Region r1 = regionFactory.create(region1);
+        r1.put("key-1", new PortfolioPdx(1));
+
+        cache.close();
+        assertTrue(new File(diskStoreDir, "BACKUP" + diskStoreName1 + ".if").exists());
+      }
+    });
+
+    CommandResult cmdResult = executeCommand(
+        "describe offline-disk-store --name=" + diskStoreName1 + " --disk-dirs=" + diskStoreDir.getAbsolutePath() + " --pdx=true");
+    String stringResult = commandResultToString(cmdResult);
+    assertTrue(stringContainsLine(stringResult, ".*PDX Types.*"));
+    assertTrue(stringContainsLine(stringResult, ".*com\\.gemstone\\.gemfire\\.cache\\.query\\.data\\.PortfolioPdx.*"));
+    assertTrue(stringContainsLine(stringResult, ".*com\\.gemstone\\.gemfire\\.cache\\.query\\.data\\.PositionPdx.*"));
+    assertTrue(stringContainsLine(stringResult, ".*PDX Enums.*"));
+    assertTrue(
+        stringContainsLine(stringResult, ".*com\\.gemstone\\.gemfire\\.cache\\.query\\.data\\.PortfolioPdx\\$Day.*"));
+  }
+
+
+  public void testValidateDiskStore() {
+    createDefaultSetup(null);
+
+    final File diskStoreDir = new File(new File(".").getAbsolutePath(), "DiskStoreCommandDUnitDiskStores");
+    diskStoreDir.mkdir();
+    this.filesToBeDeleted.add(diskStoreDir.getAbsolutePath());
+
+    final String diskStoreName1 = "DiskStore1";
+    final String region1 = "Region1";
+    final String region2 = "Region2";
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        final Cache cache = getCache();
+
+        DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+        diskStoreFactory.setDiskDirs(new File[]{diskStoreDir});
+        final DiskStore diskStore1 = diskStoreFactory.create(diskStoreName1);
+        assertNotNull(diskStore1);
+
+        RegionFactory regionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT);
+        regionFactory.setDiskStoreName(diskStoreName1);
+        regionFactory.setDiskSynchronous(true);
+        regionFactory.create(region1);
+        regionFactory.create(region2);
+
+        cache.close();
+        assertTrue(new File(diskStoreDir, "BACKUP" + diskStoreName1 + ".if").exists());
+      }
+    });
+    String command = "validate offline-disk-store --name=" + diskStoreName1 + " --disk-dirs=" + diskStoreDir.getAbsolutePath();
+    getLogWriter().info("testValidateDiskStore command: " + command);
+    CommandResult cmdResult = executeCommand(command);
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testValidateDiskStore cmdResult is stringResult " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      assertTrue(stringResult.contains("Total number of region entries in this disk store is"));
+
+    } else {
+      getLogWriter().info("testValidateDiskStore cmdResult is null");
+      fail("Did not get CommandResult in testValidateDiskStore");
+    }
+  }
+
+  public void testExportOfflineDiskStore() throws Exception {
+    createDefaultSetup(null);
+
+    final File diskStoreDir = new File(new File(".").getAbsolutePath(), "DiskStoreCommandDUnitDiskStores");
+    diskStoreDir.mkdir();
+    this.filesToBeDeleted.add(diskStoreDir.getAbsolutePath());
+    final File exportDir = new File(new File(".").getAbsolutePath(), "DiskStoreCommandDUnitExport");
+    exportDir.mkdir();
+    this.filesToBeDeleted.add(exportDir.getAbsolutePath());
+
+    final String diskStoreName1 = "DiskStore1";
+    final String region1 = "Region1";
+    final String region2 = "Region2";
+    final Map<String, String> entries = new HashMap<String, String>();
+    entries.put("key1", "value1");
+    entries.put("key2", "value2");
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        final Cache cache = getCache();
+
+        DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+        diskStoreFactory.setDiskDirs(new File[]{diskStoreDir});
+        final DiskStore diskStore1 = diskStoreFactory.create(diskStoreName1);
+        assertNotNull(diskStore1);
+
+        RegionFactory regionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT);
+        regionFactory.setDiskStoreName(diskStoreName1);
+        regionFactory.setDiskSynchronous(true);
+        Region r1 = regionFactory.create(region1);
+        r1.putAll(entries);
+        Region r2 = regionFactory.create(region2);
+        r2.putAll(entries);
+
+        cache.close();
+        assertTrue(new File(diskStoreDir, "BACKUP" + diskStoreName1 + ".if").exists());
+      }
+    });
+    String command = "export offline-disk-store --name=" + diskStoreName1 + " --disk-dirs=" + diskStoreDir.getAbsolutePath() + " --dir=" + exportDir;
+    getLogWriter().info("testExportDiskStore command" + command);
+    CommandResult cmdResult = executeCommand(command);
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      String stringResult = commandResultToString(cmdResult);
+      SnapshotTestUtil.checkSnapshotEntries(exportDir, entries, diskStoreName1, region1);
+      SnapshotTestUtil.checkSnapshotEntries(exportDir, entries, diskStoreName1, region2);
+
+    } else {
+      getLogWriter().info("testExportOfflineDiskStore cmdResult is null");
+      fail("Did not get CommandResult in testExportOfflineDiskStore");
+    }
+  }
+
+  /**
+   * Asserts that creating and destroying disk stores correctly updates the shared configuration.
+   */
+  public void testCreateDestroyUpdatesSharedConfig() {
+    disconnectAllFromDS();
+
+    final String groupName = "testDiskStoreSharedConfigGroup";
+    final String diskStoreName = "testDiskStoreSharedConfigDiskStore";
+
+    // Start the Locator and wait for shared configuration to be available
+    final int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+
+        final File locatorLogFile = new File("locator-" + locatorPort + ".log");
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, "Locator");
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "fine");
+        locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
+              locatorProps);
+
+          DistributedTestCase.WaitCriterion wc = new DistributedTestCase.WaitCriterion() {
+            @Override
+            public boolean done() {
+              return locator.isSharedConfigurationRunning();
+            }
+
+            @Override
+            public String description() {
+              return "Waiting for shared configuration to be started";
+            }
+          };
+          DistributedTestCase.waitForCriterion(wc, 5000, 500, true);
+        } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        }
+      }
+    });
+
+    // Start the default manager
+    Properties managerProps = new Properties();
+    managerProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    managerProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+    createDefaultSetup(managerProps);
+
+    // Create a cache in VM 1
+    final File diskStoreDir = new File(new File(".").getAbsolutePath(), diskStoreName);
+    this.filesToBeDeleted.add(diskStoreDir.getAbsolutePath());
+    VM vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        diskStoreDir.mkdirs();
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        getSystem(localProps);
+        assertNotNull(getCache());
+      }
+    });
+
+    // Test creating the disk store
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_DISK_STORE);
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__NAME, diskStoreName);
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__GROUP, groupName);
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE, diskStoreDir.getAbsolutePath());
+    CommandResult cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure the disk store exists in the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        String xmlFromConfig;
+        try {
+          xmlFromConfig = sharedConfig.getConfiguration(groupName).getCacheXmlContent();
+          assertTrue(xmlFromConfig.contains(diskStoreName));
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service", e);
+        }
+      }
+    });
+
+    //Restart the cache and make sure it has the diskstore
+    vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableCallable() {
+      @Override
+      public Object call() {
+        getCache().close();
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        localProps.setProperty(DistributionConfig.USE_CLUSTER_CONFIGURATION_NAME, "true");
+        getSystem(localProps);
+        Cache cache = getCache();
+        assertNotNull(cache);
+
+        GemFireCacheImpl gfc = (GemFireCacheImpl) cache;
+        Collection<DiskStoreImpl> diskStoreList = gfc.listDiskStores();
+        assertNotNull(diskStoreList);
+        assertFalse(diskStoreList.isEmpty());
+        assertTrue(diskStoreList.size() == 1);
+
+        for (DiskStoreImpl diskStore : diskStoreList) {
+          assertTrue(diskStore.getName().equals(diskStoreName));
+          break;
+        }
+        return null;
+      }
+    });
+
+    // Test destroying the disk store
+    commandStringBuilder = new CommandStringBuilder(CliStrings.DESTROY_DISK_STORE);
+    commandStringBuilder.addOption(CliStrings.DESTROY_DISK_STORE__NAME, diskStoreName);
+    commandStringBuilder.addOption(CliStrings.DESTROY_DISK_STORE__GROUP, groupName);
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    // Make sure the disk store was removed from the shared config
+    Host.getHost(0).getVM(3).invoke(new SerializableRunnable() {
+      @Override
+      public void run() {
+        SharedConfiguration sharedConfig = ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
+        String xmlFromConfig;
+        try {
+          xmlFromConfig = sharedConfig.getConfiguration(groupName).getCacheXmlContent();
+          assertFalse(xmlFromConfig.contains(diskStoreName));
+        } catch (Exception e) {
+          fail("Error occurred in cluster configuration service", e);
+        }
+      }
+    });
+
+
+    //Restart the cache and make sure it DOES NOT have the diskstore
+    vm = Host.getHost(0).getVM(1);
+    vm.invoke(new SerializableCallable() {
+      @Override
+      public Object call() {
+        getCache().close();
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        localProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, groupName);
+        localProps.setProperty(DistributionConfig.USE_CLUSTER_CONFIGURATION_NAME, "true");
+        getSystem(localProps);
+        Cache cache = getCache();
+        assertNotNull(cache);
+        GemFireCacheImpl gfc = (GemFireCacheImpl) cache;
+        Collection<DiskStoreImpl> diskStores = gfc.listDiskStores();
+        assertNotNull(diskStores);
+        assertTrue(diskStores.isEmpty());
+        return null;
+      }
+    });
+  }
+
+
+  /****
+   * 1) Create a disk-store in a member, get the disk-dirs. 2) Close the member. 3) Execute the command. 4) Restart the
+   * member. 5) Check if the disk-store is altered.
+   *
+   * @throws IOException
+   * @throws ClassNotFoundException
+   */
+  public void testAlterDiskStore() throws ClassNotFoundException, IOException {
+    final String regionName = "region1";
+    final String diskStoreName = "disk-store1";
+    final String diskDirName = "diskStoreDir";
+    final File diskStoreDir = new File(diskDirName);
+    diskStoreDir.deleteOnExit();
+
+    if (!diskStoreDir.exists()) {
+      diskStoreDir.mkdir();
+    }
+
+    final String diskDirPath = diskStoreDir.getCanonicalPath();
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+
+    vm1.invoke(new SerializableCallable() {
+
+      @Override
+      public Object call() throws Exception {
+        getSystem();
+        Region region = createParRegWithPersistence(regionName, diskStoreName, diskDirPath);
+        region.put("a", "QWE");
+        return region.put("b", "ASD");
+      }
+    });
+    //Close the cache and all the connections , so the disk-store can be altered
+    disconnectAllFromDS();
+
+    //Now do the command execution
+    createDefaultSetup(null);
+    Gfsh gfshInstance = Gfsh.getCurrentInstance();
+
+    if (gfshInstance == null) {
+      fail("In testAlterDiskStore command gfshInstance is null");
+    }
+
+    gfshInstance.setDebug(true);
+
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.ALTER_DISK_STORE);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__DISKSTORENAME, diskStoreName);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__REGIONNAME, regionName);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__DISKDIRS, diskDirPath);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__CONCURRENCY__LEVEL, "5");
+    csb.addOption(CliStrings.ALTER_DISK_STORE__INITIAL__CAPACITY, "6");
+    csb.addOption(CliStrings.ALTER_DISK_STORE__LRU__EVICTION__ACTION, "local-destroy");
+    csb.addOption(CliStrings.ALTER_DISK_STORE__COMPRESSOR, "com.gemstone.gemfire.compression.SnappyCompressor");
+    csb.addOption(CliStrings.ALTER_DISK_STORE__STATISTICS__ENABLED, "true");
+
+    String commandString = csb.getCommandString();
+
+    commandString.trim();
+
+    CommandResult cmdResult = executeCommand(commandString);
+    String resultString = commandResultToString(cmdResult);
+    getLogWriter().info("#SB command output : \n" + resultString);
+    assertEquals(true, Result.Status.OK.equals(cmdResult.getStatus()));
+    assertEquals(true, resultString.contains("concurrencyLevel=5"));
+    assertEquals(true, resultString.contains("lruAction=local-destroy"));
+    assertEquals(true, resultString.contains("compressor=com.gemstone.gemfire.compression.SnappyCompressor"));
+    assertEquals(true, resultString.contains("initialCapacity=6"));
+
+    csb = new CommandStringBuilder(CliStrings.ALTER_DISK_STORE);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__DISKSTORENAME, diskStoreName);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__REGIONNAME, regionName);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__DISKDIRS, diskDirPath);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__COMPRESSOR, "none");
+
+    cmdResult = executeCommand(csb.getCommandString().trim());
+    resultString = commandResultToString(cmdResult);
+    assertEquals(true, Result.Status.OK.equals(cmdResult.getStatus()));
+    assertTrue(stringContainsLine(resultString, "-compressor=none"));
+
+    //Alter DiskStore with remove option
+    csb = new CommandStringBuilder(CliStrings.ALTER_DISK_STORE);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__DISKSTORENAME, diskStoreName);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__REGIONNAME, regionName);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__DISKDIRS, diskDirPath);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__REMOVE, "true");
+
+    commandString = csb.getCommandString();
+
+    commandString.trim();
+
+    cmdResult = executeCommand(commandString);
+    resultString = commandResultToString(cmdResult);
+    getLogWriter().info("command output : \n" + resultString);
+    assertEquals(true, Result.Status.OK.equals(cmdResult.getStatus()));
+
+    Object postDestroyValue = vm1.invoke(new SerializableCallable() {
+
+      @Override
+      public Object call() throws Exception {
+        getSystem();
+        Region region = createParRegWithPersistence(regionName, diskStoreName, diskDirPath);
+        return region.get("a");
+      }
+    });
+    assertNull(postDestroyValue);
+
+    csb = new CommandStringBuilder(CliStrings.ALTER_DISK_STORE);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__DISKSTORENAME, diskStoreName);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__REGIONNAME, regionName);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__DISKDIRS, diskDirPath);
+    csb.addOption(CliStrings.ALTER_DISK_STORE__CONCURRENCY__LEVEL, "5");
+    csb.addOption(CliStrings.ALTER_DISK_STORE__REMOVE, "true");
+
+
+    commandString = csb.getCommandString();
+    commandString.trim();
+
+    cmdResult = executeCommand(commandString);
+    resultString = commandResultToString(cmdResult);
+    getLogWriter().info("Alter DiskStore with wrong remove option  : \n" + resultString);
+    assertEquals(true, Result.Status.ERROR.equals(cmdResult.getStatus()));
+
+    filesToBeDeleted.add(diskDirName);
+  }
+
+
+  public void testBackupDiskStoreBackup() throws IOException {
+    final String regionName = "region1";
+    final String fullBackUpName = "fullBackUp";
+    final String controllerName = "controller";
+    final String vm1Name = "vm1";
+    final String diskStoreName = "diskStore";
+    final String controllerDiskDirName = "controllerDiskDir";
+    final String vm1DiskDirName = "vm1DiskDir";
+    final String incrementalBackUpName = "incrementalBackUp";
+    final VM manager = Host.getHost(0).getVM(0);
+    final VM vm1 = Host.getHost(0).getVM(1);
+    createDefaultSetup(null);
+
+
+    File controllerDiskDir = new File(controllerDiskDirName);
+    controllerDiskDir.mkdir();
+    final String controllerDiskDirPath = controllerDiskDir.getCanonicalPath();
+    filesToBeDeleted.add(controllerDiskDirPath);
+
+    File vm1DiskDir = new File(vm1DiskDirName);
+    vm1DiskDir.mkdir();
+    final String vm1DiskDirPath = vm1DiskDir.getCanonicalPath();
+    filesToBeDeleted.add(vm1DiskDirPath);
+
+    File fullBackupDir = new File(fullBackUpName);
+    fullBackupDir.mkdir();
+    final String fullBackupDirPath = fullBackupDir.getCanonicalPath();
+    filesToBeDeleted.add(fullBackupDirPath);
+
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.NAME_NAME, controllerName);
+
+    getSystem(props);
+
+    manager.invoke(new SerializableRunnable() {
+      public void run() {
+        Region region = createParRegWithPersistence(regionName, diskStoreName, controllerDiskDirPath);
+        region.put("A", "1");
+        region.put("B", "2");
+      }
+    });
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm1Name);
+        getSystem(localProps);
+
+        Cache cache = getCache();
+        Region region = createParRegWithPersistence(regionName, diskStoreName, vm1DiskDirPath);
+      }
+    });
+
+
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.BACKUP_DISK_STORE);
+    csb.addOption(CliStrings.BACKUP_DISK_STORE__DISKDIRS, fullBackupDirPath);
+    String commandString = csb.toString();
+
+    CommandResult cmdResult = executeCommand(commandString);
+    String resultAsString = commandResultToString(cmdResult);
+    getLogWriter().info("Result from full backup : \n" + resultAsString);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertEquals(true, resultAsString.contains("Manager"));
+    assertEquals(true, resultAsString.contains(vm1Name));
+
+
+    vm1.invoke(new SerializableRunnable() {
+
+      @Override
+      public void run() {
+        Region region = getCache().getRegion(regionName);
+        //Add some data to the region
+        region.put("F", "231");
+        region.put("D", "ew");
+      }
+    });
+
+    File incrementalBackUpDir = new File(incrementalBackUpName);
+    incrementalBackUpDir.mkdir();
+
+    //Perform an incremental backup 
+    final String incrementalBackUpDirPath = incrementalBackUpDir.getCanonicalPath();
+    filesToBeDeleted.add(incrementalBackUpDirPath);
+
+    csb = new CommandStringBuilder(CliStrings.BACKUP_DISK_STORE);
+    csb.addOption(CliStrings.BACKUP_DISK_STORE__DISKDIRS, incrementalBackUpDirPath);
+    csb.addOption(CliStrings.BACKUP_DISK_STORE__BASELINEDIR, fullBackupDirPath);
+
+    cmdResult = executeCommand(csb.toString());
+    resultAsString = commandResultToString(cmdResult);
+    getLogWriter().info("Result from incremental backup : \n" + resultAsString);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    assertEquals(true, resultAsString.contains("Manager"));
+    assertEquals(true, resultAsString.contains(vm1Name));
+  }
+
+  public void testCreateDiskStore() {
+    final String diskStore1Name = "testCreateDiskStore1";
+    final String diskStore2Name = "testCreateDiskStore2";
+
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group0");
+    createDefaultSetup(localProps);
+
+    CommandResult cmdResult = executeCommand(CliStrings.LIST_DISK_STORE);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertTrue(commandResultToString(cmdResult).contains("No Disk Stores Found"));
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1Name = "VM" + vm1.getPid();
+    final File diskStore1Dir1 = new File(new File(".").getAbsolutePath(), diskStore1Name + ".1");
+    this.filesToBeDeleted.add(diskStore1Dir1.getAbsolutePath());
+    final File diskStore1Dir2 = new File(new File(".").getAbsolutePath(), diskStore1Name + ".2");
+    this.filesToBeDeleted.add(diskStore1Dir2.getAbsolutePath());
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        diskStore1Dir1.mkdirs();
+        diskStore1Dir2.mkdirs();
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm1Name);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+        getSystem(localProps);
+        getCache();
+      }
+    });
+
+    final VM vm2 = Host.getHost(0).getVM(2);
+    final String vm2Name = "VM" + vm2.getPid();
+    final File diskStore2Dir = new File(new File(".").getAbsolutePath(), diskStore2Name);
+    this.filesToBeDeleted.add(diskStore2Dir.getAbsolutePath());
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+        diskStore2Dir.mkdirs();
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm2Name);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(localProps);
+        getCache();
+      }
+    });
+
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_DISK_STORE);
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__NAME, diskStore1Name);
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__GROUP, "Group1");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__ALLOW_FORCE_COMPACTION, "true");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__AUTO_COMPACT, "false");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__COMPACTION_THRESHOLD, "67");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__MAX_OPLOG_SIZE, "355");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__QUEUE_SIZE, "5321");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__TIME_INTERVAL, "2023");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__WRITE_BUFFER_SIZE, "3110");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE,
+        diskStore1Dir1.getAbsolutePath() + "#1452637463");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE, diskStore1Dir2.getAbsolutePath());
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    String stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*Success"));
+
+    // Verify that the disk store was created on the correct member
+    cmdResult = executeCommand(CliStrings.LIST_DISK_STORE);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*" + diskStore1Name + " .*"));
+    assertFalse(stringContainsLine(stringResult, vm2Name + ".*" + diskStore1Name + " .*"));
+
+    // Verify that the disk store files were created in the correct directory.
+    assertEquals(diskStore1Dir1.listFiles().length, 2);
+
+    // Verify that all of the attributes of the disk store were set correctly.
+    commandStringBuilder = new CommandStringBuilder(CliStrings.DESCRIBE_DISK_STORE);
+    commandStringBuilder.addOption(CliStrings.DESCRIBE_DISK_STORE__MEMBER, vm1Name);
+    commandStringBuilder.addOption(CliStrings.DESCRIBE_DISK_STORE__NAME, diskStore1Name);
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertTrue(stringContainsLine(stringResult, "Allow Force Compaction.*Yes"));
+    assertTrue(stringContainsLine(stringResult, "Auto Compaction.*No"));
+    assertTrue(stringContainsLine(stringResult, "Compaction Threshold.*67"));
+    assertTrue(stringContainsLine(stringResult, "Max Oplog Size.*355"));
+    assertTrue(stringContainsLine(stringResult, "Queue Size.*5321"));
+    assertTrue(stringContainsLine(stringResult, "Time Interval.*2023"));
+    assertTrue(stringContainsLine(stringResult, "Write Buffer Size.*3110"));
+    assertTrue(stringContainsLine(stringResult, ".*" + diskStore1Name + ".1 .*1452637463"));
+    assertTrue(stringContainsLine(stringResult, ".*" + diskStore1Name + ".2 .*" + Integer.MAX_VALUE));
+
+    commandStringBuilder = new CommandStringBuilder(CliStrings.CREATE_DISK_STORE);
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__NAME, diskStore2Name);
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__GROUP, "Group2");
+    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE, diskStore2Dir.getAbsolutePath());
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*Success"));
+
+    // Verify that the second disk store was created correctly.
+    cmdResult = executeCommand(CliStrings.LIST_DISK_STORE);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(4, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*" + diskStore1Name + " .*"));
+    assertFalse(stringContainsLine(stringResult, vm2Name + ".*" + diskStore1Name + " .*"));
+    assertFalse(stringContainsLine(stringResult, vm1Name + ".*" + diskStore2Name + " .*"));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + diskStore2Name + " .*"));
+  }
+
+  public void testDestroyDiskStore() {
+    final String diskStore1Name = "testDestroyDiskStore1";
+    final String diskStore2Name = "testDestroyDiskStore2";
+    final String region1Name = "testDestroyDiskStoreRegion1";
+    final String region2Name = "testDestroyDiskStoreRegion2";
+
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group0");
+    createDefaultSetup(localProps);
+
+    CommandResult cmdResult = executeCommand(CliStrings.LIST_DISK_STORE);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertTrue(commandResultToString(cmdResult).contains("No Disk Stores Found"));
+
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1Name = "VM" + vm1.getPid();
+    final File diskStore1Dir1 = new File(new File(".").getAbsolutePath(), diskStore1Name + ".1");
+    this.filesToBeDeleted.add(diskStore1Dir1.getAbsolutePath());
+    final File diskStore2Dir1 = new File(new File(".").getAbsolutePath(), diskStore2Name + ".1");
+    this.filesToBeDeleted.add(diskStore2Dir1.getAbsolutePath());
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        diskStore1Dir1.mkdirs();
+        diskStore2Dir1.mkdirs();
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm1Name);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1,Group2");
+        getSystem(localProps);
+        Cache cache = getCache();
+
+        DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+        diskStoreFactory.setDiskDirs(new File[]{diskStore1Dir1});
+        diskStoreFactory.create(diskStore1Name);
+
+        diskStoreFactory.setDiskDirs(new File[]{diskStore2Dir1});
+        diskStoreFactory.create(diskStore2Name);
+      }
+    });
+
+    final VM vm2 = Host.getHost(0).getVM(2);
+    final String vm2Name = "VM" + vm2.getPid();
+    final File diskStore1Dir2 = new File(new File(".").getAbsolutePath(), diskStore1Name + ".2");
+    this.filesToBeDeleted.add(diskStore1Dir2.getAbsolutePath());
+    final File diskStore2Dir2 = new File(new File(".").getAbsolutePath(), diskStore2Name + ".2");
+    this.filesToBeDeleted.add(diskStore2Dir2.getAbsolutePath());
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+        diskStore1Dir2.mkdirs();
+        diskStore2Dir2.mkdirs();
+
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm2Name);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(localProps);
+        Cache cache = getCache();
+
+        DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+        diskStoreFactory.setDiskDirs(new File[]{diskStore1Dir2});
+        diskStoreFactory.create(diskStore1Name);
+
+        RegionFactory regionFactory = cache.createRegionFactory();
+        regionFactory.setDiskStoreName(diskStore1Name);
+        regionFactory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+        regionFactory.create(region1Name);
+        regionFactory.create(region2Name);
+
+        diskStoreFactory.setDiskDirs(new File[]{diskStore2Dir2});
+        diskStoreFactory.create(diskStore2Name);
+      }
+    });
+
+    // TEST DELETING ON 1 MEMBER
+
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.DESTROY_DISK_STORE);
+    commandStringBuilder.addOption(CliStrings.DESTROY_DISK_STORE__NAME, diskStore1Name);
+    commandStringBuilder.addOption(CliStrings.DESTROY_DISK_STORE__GROUP, "Group1");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    String stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*Success"));
+
+    // Verify that the disk store was destroyed on the correct member
+    cmdResult = executeCommand(CliStrings.LIST_DISK_STORE);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(5, countLinesInString(stringResult, false));
+    assertFalse(stringContainsLine(stringResult, vm1Name + ".*" + diskStore1Name + " .*"));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + diskStore1Name + " .*"));
+
+    // Verify that the disk store files were deleted from the correct directory.
+    assertEquals(0, diskStore1Dir1.listFiles().length);
+    assertEquals(4, diskStore1Dir2.listFiles().length);
+
+    // TEST DELETING ON 2 MEMBERS
+
+    commandStringBuilder = new CommandStringBuilder(CliStrings.DESTROY_DISK_STORE);
+    commandStringBuilder.addOption(CliStrings.DESTROY_DISK_STORE__NAME, diskStore2Name);
+    commandStringBuilder.addOption(CliStrings.DESTROY_DISK_STORE__GROUP, "Group2");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(4, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*Success"));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*Success"));
+
+    // Verify that the disk store was destroyed on the correct member
+    cmdResult = executeCommand(CliStrings.LIST_DISK_STORE);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(3, countLinesInString(stringResult, false));
+    assertFalse(stringContainsLine(stringResult, vm1Name + ".*" + diskStore2Name + " .*"));
+    assertFalse(stringContainsLine(stringResult, vm2Name + ".*" + diskStore2Name + " .*"));
+
+    // Verify that the disk store files were deleted from the correct directories.
+    assertEquals(0, diskStore2Dir1.listFiles().length);
+    assertEquals(0, diskStore2Dir2.listFiles().length);
+
+    // TEST FOR DISK STORE IN USE
+
+    commandStringBuilder = new CommandStringBuilder(CliStrings.DESTROY_DISK_STORE);
+    commandStringBuilder.addOption(CliStrings.DESTROY_DISK_STORE__NAME, diskStore1Name);
+    commandStringBuilder.addOption(CliStrings.DESTROY_DISK_STORE__GROUP, "Group2");
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(4, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*Disk store not found on this member"));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + region1Name + ".*" + region2Name + ".*"));
+
+    // TEST DELETING ON ALL MEMBERS
+
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+
+        Region region = cache.getRegion(region1Name);
+        region.destroyRegion();
+
+        region = cache.getRegion(region2Name);
+        region.destroyRegion();
+      }
+    });
+
+    commandStringBuilder = new CommandStringBuilder(CliStrings.DESTROY_DISK_STORE);
+    commandStringBuilder.addOption(CliStrings.DESTROY_DISK_STORE__NAME, diskStore1Name);
+    cmdResult = executeCommand(commandStringBuilder.toString());
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(5, countLinesInString(stringResult, false));
+    assertEquals(false, stringResult.contains("ERROR"));
+    assertTrue(stringContainsLine(stringResult, "Manager.*Disk store not found on this member"));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*Disk store not found on this member"));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*Success"));
+
+    // Verify that there are no disk stores left.
+    cmdResult = executeCommand(CliStrings.LIST_DISK_STORE);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertTrue(commandResultToString(cmdResult).contains("No Disk Stores Found"));
+
+    // Verify that the disk store files were deleted from the correct directory.
+    assertEquals(0, diskStore1Dir2.listFiles().length);
+  }
+
+  private Region<?, ?> createParRegWithPersistence(String regionName, String diskStoreName, String diskDirName) {
+    Cache cache = getCache();
+    File diskStoreDirFile = new File(diskDirName);
+
+    if (!diskStoreDirFile.exists()) {
+      diskStoreDirFile.mkdirs();
+    }
+
+    DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+    diskStoreFactory.setDiskDirs(new File[]{diskStoreDirFile});
+    diskStoreFactory.setMaxOplogSize(1);
+    diskStoreFactory.setAllowForceCompaction(true);
+    diskStoreFactory.setAutoCompact(false);
+    diskStoreFactory.create(diskStoreName);
+
+    /****
+     * Eviction Attributes
+     */
+    EvictionAttributes ea = EvictionAttributes.createLRUEntryAttributes(1, EvictionAction.OVERFLOW_TO_DISK);
+
+    RegionFactory regionFactory = cache.createRegionFactory();
+    regionFactory.setDiskStoreName(diskStoreName);
+    regionFactory.setDiskSynchronous(true);
+    regionFactory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+    regionFactory.setScope(Scope.DISTRIBUTED_ACK);
+    regionFactory.setEvictionAttributes(ea);
+
+    return regionFactory.create(regionName);
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    for (String path : this.filesToBeDeleted) {
+      try {
+        FileUtil.delete(new File(path));
+      } catch (IOException e) {
+        getLogWriter().error("Unable to delete file", e);
+      }
+    }
+    this.filesToBeDeleted.clear();
+    super.tearDown2();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
new file mode 100644
index 0000000..8c3dd22
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
@@ -0,0 +1,593 @@
+/*
+ * 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.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.cache.functions.TestFunction;
+import com.gemstone.gemfire.management.DistributedRegionMXBean;
+import com.gemstone.gemfire.management.ManagementService;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.json.GfJsonException;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Dunit class for testing gemfire function commands : execute function, destroy function, list function
+ *
+ * @author apande
+ * @author David Hoots
+ */
+public class FunctionCommandsDUnitTest extends CliCommandTestBase {
+  private static final long serialVersionUID = 1L;
+  private static final String REGION_NAME = "FunctionCommandsReplicatedRegion";
+  private static final String REGION_ONE = "RegionOne";
+  private static final String REGION_TWO = "RegionTwo";
+
+  public FunctionCommandsDUnitTest(String name) {
+    super(name);
+  }
+
+  void setupWith2Regions() {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+    createDefaultSetup(null);
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        final Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        FunctionService.registerFunction(function);
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create("RegionOne");
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+        region = dataRegionFactory.create("RegionTwo");
+        for (int i = 0; i < 1000; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+
+
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+        final Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        FunctionService.registerFunction(function);
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create("RegionOne");
+        for (int i = 0; i < 10000; i++) {
+          region.put("key" + (i + 400), "value" + (i + 400));
+        }
+        region = dataRegionFactory.create("Regiontwo");
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+  public void testExecuteFunctionWithNoRegionOnManager() {
+    setupWith2Regions();
+    Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+    FunctionService.registerFunction(function);
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        FunctionService.registerFunction(function);
+      }
+    });
+    try {
+      Thread.sleep(2500);
+    } catch (InterruptedException e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+    }
+    String command = "execute function --id=" + function.getId() + " --region=" + "/" + "RegionOne";
+    getLogWriter().info("testExecuteFunctionWithNoRegionOnManager command : " + command);
+    CommandResult cmdResult = executeCommand(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info("testExecuteFunctionWithNoRegionOnManager stringResult : " + strCmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      assertTrue(strCmdResult.contains("Execution summary"));
+    } else {
+      fail("testExecuteFunctionWithNoRegionOnManager failed as did not get CommandResult");
+    }
+
+  }
+
+  public static String getMemberId() {
+    Cache cache = new FunctionCommandsDUnitTest("test").getCache();
+    return cache.getDistributedSystem().getDistributedMember().getId();
+  }
+
+  public void testExecuteFunctionOnRegion() {
+    createDefaultSetup(null);
+
+    final Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        RegionFactory<Integer, Integer> dataRegionFactory = getCache().createRegionFactory(RegionShortcut.REPLICATE);
+        Region region = dataRegionFactory.create(REGION_NAME);
+        assertNotNull(region);
+        FunctionService.registerFunction(function);
+      }
+    });
+
+    String command = "execute function --id=" + function.getId() + " --region=" + REGION_NAME;
+    getLogWriter().info("testExecuteFunctionOnRegion command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      getLogWriter().info("testExecuteFunctionOnRegion cmdResult=" + cmdResult);
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testExecuteFunctionOnRegion stringResult=" + stringResult);
+      assert (stringResult.contains("Execution summary"));
+    } else {
+      fail("testExecuteFunctionOnRegion did not return CommandResult");
+    }
+  }
+
+  void setupForBug51480() {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    createDefaultSetup(null);
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        final Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        FunctionService.registerFunction(function);
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create(REGION_ONE);
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+  SerializableRunnable checkRegionMBeans = new SerializableRunnable() {
+    @Override
+    public void run() {
+      final WaitCriterion waitForMaangerMBean = new WaitCriterion() {
+        @Override
+        public boolean done() {
+          final ManagementService service = ManagementService.getManagementService(getCache());
+          final DistributedRegionMXBean bean = service.getDistributedRegionMXBean(Region.SEPARATOR + REGION_ONE);
+          if (bean == null) {
+            return false;
+          } else {
+            getLogWriter().info("Probing for checkRegionMBeans testExecuteFunctionOnRegionBug51480 finished");
+            return true;
+          }
+        }
+
+        @Override
+        public String description() {
+          return "Probing for testExecuteFunctionOnRegionBug51480";
+        }
+      };
+      DistributedTestCase.waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
+      DistributedRegionMXBean bean = ManagementService.getManagementService(getCache()).getDistributedRegionMXBean(
+          Region.SEPARATOR + REGION_ONE);
+      assertNotNull(bean);
+    }
+  };
+
+  public void testExecuteFunctionOnRegionBug51480() {
+    setupForBug51480();
+
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+
+    final Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        FunctionService.registerFunction(function);
+      }
+    });
+
+    String command = "execute function --id=" + function.getId() + " --region=" + REGION_ONE;
+
+    getLogWriter().info("testExecuteFunctionOnRegionBug51480 command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    if (cmdResult != null) {
+      getLogWriter().info("testExecuteFunctionOnRegionBug51480 cmdResult=" + cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testExecuteFunctionOnRegionBug51480 stringResult=" + stringResult);
+      assert (stringResult.contains("Execution summary"));
+    } else {
+      fail("testExecuteFunctionOnRegionBug51480 did not return CommandResult");
+
+    }
+  }
+
+  public void testExecuteFunctionOnMember() {
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+    createDefaultSetup(localProps);
+    Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+    FunctionService.registerFunction(function);
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1MemberId = (String) vm1.invoke(FunctionCommandsDUnitTest.class, "getMemberId");
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        RegionFactory<Integer, Integer> dataRegionFactory = getCache().createRegionFactory(RegionShortcut.REPLICATE);
+        Region region = dataRegionFactory.create(REGION_NAME);
+        Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        assertNotNull(region);
+        FunctionService.registerFunction(function);
+      }
+    });
+
+    String command = "execute function --id=" + function.getId() + " --member=" + vm1MemberId;
+    getLogWriter().info("testExecuteFunctionOnMember command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    String stringResult = commandResultToString(cmdResult);
+    getLogWriter().info("testExecuteFunctionOnMember stringResult:" + stringResult);
+    assertTrue(stringResult.contains("Execution summary"));
+  }
+
+  public void testExecuteFunctionOnMembers() {
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+    createDefaultSetup(localProps);
+    Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+    FunctionService.registerFunction(function);
+    final VM vm1 = Host.getHost(0).getVM(1);
+
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        RegionFactory<Integer, Integer> dataRegionFactory = getCache().createRegionFactory(RegionShortcut.REPLICATE);
+        Region region = dataRegionFactory.create(REGION_NAME);
+        Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        assertNotNull(region);
+        FunctionService.registerFunction(function);
+      }
+    });
+    String command = "execute function --id=" + function.getId();
+    getLogWriter().info("testExecuteFunctionOnMembers command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      getLogWriter().info("testExecuteFunctionOnMembers cmdResult:" + cmdResult);
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testExecuteFunctionOnMembers stringResult:" + stringResult);
+      assertTrue(stringResult.contains("Execution summary"));
+    } else {
+      fail("testExecuteFunctionOnMembers did not return CommandResult");
+    }
+  }
+
+  public void testExecuteFunctionOnMembersWithArgs() {
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+    createDefaultSetup(localProps);
+    Function function = new TestFunction(true, TestFunction.TEST_FUNCTION_RETURN_ARGS);
+    FunctionService.registerFunction(function);
+
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        RegionFactory<Integer, Integer> dataRegionFactory = getCache().createRegionFactory(RegionShortcut.REPLICATE);
+        Region region = dataRegionFactory.create(REGION_NAME);
+        Function function = new TestFunction(true, TestFunction.TEST_FUNCTION_RETURN_ARGS);
+        assertNotNull(region);
+        FunctionService.registerFunction(function);
+      }
+    });
+
+    String command = "execute function --id=" + function.getId() + " --arguments=arg1,arg2";
+
+    getLogWriter().info("testExecuteFunctionOnMembersWithArgs command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    if (cmdResult != null) {
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      getLogWriter().info("testExecuteFunctionOnMembersWithArgs cmdResult:" + cmdResult);
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testExecuteFunctionOnMembersWithArgs stringResult:" + stringResult);
+      assertTrue(stringResult.contains("Execution summary"));
+      assertTrue(stringResult.contains("arg1"));
+    } else {
+      fail("testExecuteFunctionOnMembersWithArgs did not return CommandResult");
+    }
+  }
+
+  public void testExecuteFunctionOnGroups() {
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group0");
+    createDefaultSetup(localProps);
+    Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+    FunctionService.registerFunction(function);
+
+    VM vm1 = Host.getHost(0).getVM(1);
+    VM vm2 = Host.getHost(0).getVM(2);
+
+    String vm1id = (String) vm1.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+        getSystem(localProps);
+        Cache cache = getCache();
+        Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        FunctionService.registerFunction(function);
+        return cache.getDistributedSystem().getDistributedMember().getId();
+      }
+    });
+
+    String vm2id = (String) vm2.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(localProps);
+        Cache cache = getCache();
+        Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        FunctionService.registerFunction(function);
+        return cache.getDistributedSystem().getDistributedMember().getId();
+      }
+    });
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        RegionFactory<Integer, Integer> dataRegionFactory = getCache().createRegionFactory(RegionShortcut.REPLICATE);
+        Region region = dataRegionFactory.create(REGION_NAME);
+        Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        assertNotNull(region);
+        FunctionService.registerFunction(function);
+      }
+    });
+
+    String command = "execute function --id=" + TestFunction.TEST_FUNCTION1 + " --groups=Group1,Group2";
+    getLogWriter().info("testExecuteFunctionOnGroups command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    getLogWriter().info("testExecuteFunctionOnGroups cmdResult=" + cmdResult);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+    List<String> members = resultData.retrieveAllValues("Member ID/Name");
+    getLogWriter().info("testExecuteFunctionOnGroups members=" + members);
+    assertTrue(members.size() == 2 && members.contains(vm1id) && members.contains(vm2id));
+  }
+
+
+  public void testDestroyOnMember() {
+    createDefaultSetup(null);
+    Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+    FunctionService.registerFunction(function);
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1MemberId = (String) vm1.invoke(FunctionCommandsDUnitTest.class, "getMemberId");
+    String command = "destroy function --id=" + function.getId() + " --member=" + vm1MemberId;
+    getLogWriter().info("testDestroyOnMember command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      getLogWriter().info("testDestroyOnMember strCmdResult=" + strCmdResult);
+      assertTrue(strCmdResult.contains("Destroyed TestFunction1 Successfully"));
+    } else {
+      fail("testDestroyOnMember failed as did not get CommandResult");
+    }
+  }
+
+  public void testDestroyOnGroups() {
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group0");
+    createDefaultSetup(localProps);
+    Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+    FunctionService.registerFunction(function);
+
+    VM vm1 = Host.getHost(0).getVM(1);
+    VM vm2 = Host.getHost(0).getVM(2);
+
+    String vm1id = (String) vm1.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+        getSystem(localProps);
+        Cache cache = getCache();
+        Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        FunctionService.registerFunction(function);
+        return cache.getDistributedSystem().getDistributedMember().getId();
+      }
+    });
+
+
+    String vm2id = (String) vm2.invoke(new SerializableCallable() {
+      @Override
+      public Object call() throws Exception {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(localProps);
+        Cache cache = getCache();
+        Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        FunctionService.registerFunction(function);
+        return cache.getDistributedSystem().getDistributedMember().getId();
+      }
+    });
+
+    Host.getHost(0).getVM(0).invoke(new SerializableRunnable() {
+      public void run() {
+        Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+        FunctionService.registerFunction(function);
+      }
+    });
+
+    String command = "destroy function --id=" + TestFunction.TEST_FUNCTION1 + " --groups=Group1,Group2";
+    getLogWriter().info("testDestroyOnGroups command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    getLogWriter().info("testDestroyOnGroups cmdResult=" + cmdResult);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    String content = null;
+    try {
+      content = cmdResult.getContent().get("message").toString();
+      getLogWriter().info("testDestroyOnGroups content = " + content);
+    } catch (GfJsonException e) {
+      fail("testDestroyOnGroups exception=" + e);
+    }
+    assertNotNull(content);
+    assertTrue(content.equals(
+        "[\"Destroyed " + TestFunction.TEST_FUNCTION1 + " Successfully on " + vm1id + "," + vm2id + "\"]") || content.equals(
+        "[\"Destroyed " + TestFunction.TEST_FUNCTION1 + " Successfully on " + vm2id + "," + vm1id + "\"]"));
+  }
+
+  public void testListFunction() {
+    // Create the default setup, putting the Manager VM into Group1
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group1");
+    createDefaultSetup(localProps);
+
+    // Find no functions
+    CommandResult cmdResult = executeCommand(CliStrings.LIST_FUNCTION);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertTrue(commandResultToString(cmdResult).contains("No Functions Found"));
+
+    // Add a function in the manager VM (VM 0)
+    final Function function1 = new TestFunction(true, TestFunction.TEST_FUNCTION1);
+    final VM managerVm = Host.getHost(0).getVM(0);
+    managerVm.invoke(new SerializableRunnable() {
+      public void run() {
+        FunctionService.registerFunction(function1);
+      }
+    });
+
+    // Add functions in another VM (VM 1)
+    final Function function2 = new TestFunction(true, TestFunction.TEST_FUNCTION2);
+    final Function function3 = new TestFunction(true, TestFunction.TEST_FUNCTION3);
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1Name = "VM" + vm1.getPid();
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm1Name);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group2");
+        getSystem(localProps);
+        getCache();
+
+        FunctionService.registerFunction(function2);
+        FunctionService.registerFunction(function3);
+      }
+    });
+
+    // Add functions in a third VM (VM 2)
+    final Function function4 = new TestFunction(true, TestFunction.TEST_FUNCTION4);
+    final Function function5 = new TestFunction(true, TestFunction.TEST_FUNCTION5);
+    final Function function6 = new TestFunction(true, TestFunction.TEST_FUNCTION6);
+    final VM vm2 = Host.getHost(0).getVM(2);
+    final String vm2Name = "VM" + vm2.getPid();
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(DistributionConfig.NAME_NAME, vm2Name);
+        localProps.setProperty(DistributionConfig.GROUPS_NAME, "Group3");
+        getSystem(localProps);
+        getCache();
+
+        FunctionService.registerFunction(function4);
+        FunctionService.registerFunction(function5);
+        FunctionService.registerFunction(function6);
+      }
+    });
+
+    // Find all functions
+    cmdResult = executeCommand(CliStrings.LIST_FUNCTION);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    String stringResult = commandResultToString(cmdResult);
+    assertEquals(8, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Member.*Function"));
+    assertTrue(stringContainsLine(stringResult, "Manager.*" + function1.getId()));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*" + function2.getId()));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*" + function3.getId()));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + function4.getId()));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + function5.getId()));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + function6.getId()));
+
+    // Find functions in group Group3
+    cmdResult = executeCommand(CliStrings.LIST_FUNCTION + " --group=Group1,Group3");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(6, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Member.*Function"));
+    assertTrue(stringContainsLine(stringResult, "Manager.*" + function1.getId()));
+    assertFalse(stringContainsLine(stringResult, vm1Name + ".*"));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + function4.getId()));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + function5.getId()));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + function6.getId()));
+
+    // Find functions for Manager member
+    cmdResult = executeCommand(CliStrings.LIST_FUNCTION + " --member=Manager," + vm1Name);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(5, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Member.*Function"));
+    assertTrue(stringContainsLine(stringResult, "Manager.*" + function1.getId()));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*" + function2.getId()));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*" + function3.getId()));
+    assertFalse(stringContainsLine(stringResult, vm2Name + ".*"));
+
+    // Find functions that match a pattern
+    cmdResult = executeCommand(CliStrings.LIST_FUNCTION + " --matches=.*[135]$");
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    stringResult = commandResultToString(cmdResult);
+    assertEquals(5, countLinesInString(stringResult, false));
+    assertTrue(stringContainsLine(stringResult, "Member.*Function"));
+    assertTrue(stringContainsLine(stringResult, "Manager.*" + function1.getId()));
+    assertFalse(stringContainsLine(stringResult, vm2Name + ".*" + function2.getId()));
+    assertTrue(stringContainsLine(stringResult, vm1Name + ".*" + function3.getId()));
+    assertFalse(stringContainsLine(stringResult, vm2Name + ".*" + function4.getId()));
+    assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + function5.getId()));
+    assertFalse(stringContainsLine(stringResult, vm2Name + ".*" + function6.getId()));
+  }
+}


[36/50] [abbrv] incubator-geode git commit: GEODE-563: Moving gfsh tests from closed

Posted by kl...@apache.org.
GEODE-563: Moving gfsh tests from closed


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

Branch: refs/heads/feature/GEODE-217
Commit: eddef322defedea8396697e65c989b85c3d9c433
Parents: 1e93c6f
Author: Jens Deppe <jd...@pivotal.io>
Authored: Thu Dec 3 10:21:59 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Tue Dec 8 09:23:28 2015 -0800

----------------------------------------------------------------------
 gemfire-assembly/build.gradle                   |   13 +
 .../LauncherLifecycleCommandsDUnitTest.java     | 1005 +++++++++
 .../LauncherLifecycleCommandsJUnitTest.java     |  625 ++++++
 .../SharedConfigurationEndToEndDUnitTest.java   |  434 ++++
 .../management/internal/cli/HeadlessGfsh.java   |  376 ++++
 .../internal/cli/HeadlessGfshJUnitTest.java     |   87 +
 .../management/internal/cli/ResultHandler.java  |   23 +
 .../internal/cli/TableBuilderJUnitTest.java     |  183 ++
 .../cli/commands/CliCommandTestBase.java        |  560 +++++
 .../cli/commands/ConfigCommandsDUnitTest.java   |  497 +++++
 ...eateAlterDestroyRegionCommandsDUnitTest.java | 1148 ++++++++++
 .../cli/commands/DeployCommandsDUnitTest.java   |  480 ++++
 .../commands/DiskStoreCommandsDUnitTest.java    | 1154 ++++++++++
 .../cli/commands/FunctionCommandsDUnitTest.java |  593 +++++
 .../commands/GemfireDataCommandsDUnitTest.java  | 2087 ++++++++++++++++++
 ...WithCacheLoaderDuringCacheMissDUnitTest.java |  371 ++++
 .../cli/commands/IndexCommandsDUnitTest.java    |  817 +++++++
 ...stAndDescribeDiskStoreCommandsDUnitTest.java |  197 ++
 .../ListAndDescribeRegionDUnitTest.java         |  320 +++
 .../cli/commands/ListIndexCommandDUnitTest.java |  672 ++++++
 .../cli/commands/MemberCommandsDUnitTest.java   |  286 +++
 .../MiscellaneousCommandsDUnitTest.java         |  492 +++++
 ...laneousCommandsExportLogsPart1DUnitTest.java |  139 ++
 ...laneousCommandsExportLogsPart2DUnitTest.java |  148 ++
 ...laneousCommandsExportLogsPart3DUnitTest.java |  150 ++
 ...laneousCommandsExportLogsPart4DUnitTest.java |  141 ++
 .../cli/commands/QueueCommandsDUnitTest.java    |  385 ++++
 .../SharedConfigurationCommandsDUnitTest.java   |  338 +++
 .../cli/commands/ShellCommandsDUnitTest.java    |  365 +++
 .../cli/commands/ShowDeadlockDUnitTest.java     |  271 +++
 .../cli/commands/ShowMetricsDUnitTest.java      |  347 +++
 .../cli/commands/ShowStackTraceDUnitTest.java   |  149 ++
 .../cli/commands/UserCommandsDUnitTest.java     |  164 ++
 33 files changed, 15017 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index 8de6d4e..514b4a1 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -31,6 +31,8 @@ dependencies {
 
   testCompile project(path: ':gemfire-junit', configuration: 'testOutput')
   testCompile project(path: ':gemfire-core', configuration: 'testOutput')
+
+  testRuntime files("${System.getProperty('java.home')}/../lib/tools.jar")
 }
 
 sourceSets {
@@ -287,6 +289,17 @@ afterEvaluate {
   } 
 }
 
+// Create a configuration closure to configure test targets with the install directory
+def dependOnInstalledProduct = {
+  dependsOn installDist
+  def install = file("$buildDir/install/${distributions.main.baseName}")
+  environment ('GEMFIRE', install)
+}
+
+// Add the configuration closure to the test targets so they depend on the install directory
+test dependOnInstalledProduct
+distributedTest dependOnInstalledProduct
+
 artifacts {
   archives depsJar, gfshDepsJar
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java b/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
new file mode 100644
index 0000000..afb2770
--- /dev/null
+++ b/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
@@ -0,0 +1,1005 @@
+/*
+ * 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.cache.Region;
+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.Pool;
+import com.gemstone.gemfire.cache.client.PoolFactory;
+import com.gemstone.gemfire.cache.client.PoolManager;
+import com.gemstone.gemfire.distributed.AbstractLauncher.ServiceState;
+import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
+import com.gemstone.gemfire.distributed.LocatorLauncher;
+import com.gemstone.gemfire.distributed.LocatorLauncher.Builder;
+import com.gemstone.gemfire.distributed.LocatorLauncher.Command;
+import com.gemstone.gemfire.distributed.LocatorLauncher.LocatorState;
+import com.gemstone.gemfire.distributed.ServerLauncher;
+import com.gemstone.gemfire.distributed.ServerLauncher.ServerState;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.lang.ObjectUtils;
+import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.internal.lang.SystemUtils;
+import com.gemstone.gemfire.internal.process.ProcessType;
+import com.gemstone.gemfire.internal.util.IOUtils;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import org.junit.FixMethodOrder;
+import org.junit.runners.MethodSorters;
+
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.Query;
+import javax.management.QueryExp;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.lang.management.ManagementFactory;
+import java.net.InetAddress;
+import java.nio.charset.Charset;
+import java.text.DateFormat;
+import java.text.MessageFormat;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * The LauncherLifecycleCommandsDUnitTest class is a test suite of integration tests testing the contract and
+ * functionality of the GemFire launcher lifecycle commands inside Gfsh.
+ *
+ * @author John Blum
+ * @see javax.management.MBeanServerConnection
+ * @see javax.management.remote.JMXConnector
+ * @see com.gemstone.gemfire.distributed.AbstractLauncher
+ * @see com.gemstone.gemfire.distributed.LocatorLauncher
+ * @see com.gemstone.gemfire.distributed.ServerLauncher
+ * @see com.gemstone.gemfire.internal.AvailablePortHelper
+ * @see com.gemstone.gemfire.management.internal.cli.shell.Gfsh
+ * @see com.gemstone.gemfire.management.internal.cli.commands.CliCommandTestBase
+ * @see com.gemstone.gemfire.management.internal.cli.commands.LauncherLifecycleCommands
+ * @see com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder
+ * @since 7.0
+ */
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
+
+  protected static final long COMMAND_EXECUTION_TIMEOUT = TimeUnit.MINUTES.toSeconds(2);
+
+  protected static final DateFormat TIMESTAMP = new SimpleDateFormat("yyyyMMddHHmmssSSS");
+
+  private final Queue<Integer> processIds = new ConcurrentLinkedDeque<>();
+
+  public LauncherLifecycleCommandsDUnitTest(final String testName) {
+    super(testName);
+  }
+
+  protected static String getMemberId(final int jmxManagerPort, final String memberName) throws Exception {
+    return getMemberId(InetAddress.getLocalHost().getHostName(), jmxManagerPort, memberName);
+  }
+
+  protected static String getMemberId(final String jmxManagerHost, final int jmxManagerPort,
+      final String memberName) throws Exception {
+    JMXConnector connector = null;
+
+    try {
+      connector = JMXConnectorFactory.connect(new JMXServiceURL(
+          String.format("service:jmx:rmi://%1$s/jndi/rmi://%1$s:%2$d/jmxrmi", jmxManagerHost, jmxManagerPort)));
+
+      MBeanServerConnection connection = connector.getMBeanServerConnection();
+
+      ObjectName objectNamePattern = ObjectName.getInstance("GemFire:type=Member,*");
+
+      QueryExp query = Query.eq(Query.attr("Name"), Query.value(memberName));
+
+      Set<ObjectName> objectNames = connection.queryNames(objectNamePattern, query);
+
+      assertNotNull(objectNames);
+      assertFalse(objectNames.isEmpty());
+      assertEquals(1, objectNames.size());
+
+      //final ObjectName objectName = ObjectName.getInstance("GemFire:type=Member,Name=" + memberName);
+      ObjectName objectName = objectNames.iterator().next();
+
+      //System.err.printf("ObjectName for Member with Name (%1$s) is %2$s%n", memberName, objectName);
+
+      return ObjectUtils.toString(connection.getAttribute(objectName, "Id"));
+    } finally {
+      IOUtils.close(connector);
+    }
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  @Override
+  protected void tearDownAfter() throws Exception {
+    super.tearDownAfter();
+
+    LauncherLifecycleCommands launcherLifecycleCommands = new LauncherLifecycleCommands();
+    Integer pid;
+
+    while ((pid = processIds.poll()) != null) {
+      if (launcherLifecycleCommands.isVmWithProcessIdRunning(pid)) {
+        try {
+          String killCommand = String.format("%1$s %2$d", SystemUtils.isWindows() ? "taskkill /F /PID" : "kill -9",
+              pid);
+          Runtime.getRuntime().exec(killCommand);
+        } catch (Throwable ignore) {
+        }
+      }
+    }
+  }
+
+  @SuppressWarnings("unused")
+  protected void assertStatus(final LocatorState expectedStatus, final LocatorState actualStatus) {
+    assertEquals(expectedStatus.getStatus(), actualStatus.getStatus());
+    assertEquals(expectedStatus.getTimestamp(), actualStatus.getTimestamp());
+    assertEquals(expectedStatus.getServiceLocation(), actualStatus.getServiceLocation());
+    assertTrue(ObjectUtils.equalsIgnoreNull(expectedStatus.getPid(), actualStatus.getPid()));
+    assertEquals(expectedStatus.getUptime(), actualStatus.getUptime());
+    assertEquals(expectedStatus.getWorkingDirectory(), actualStatus.getWorkingDirectory());
+    assertEquals(expectedStatus.getJvmArguments(), actualStatus.getJvmArguments());
+    assertEquals(expectedStatus.getClasspath(), actualStatus.getClasspath());
+    assertEquals(expectedStatus.getGemFireVersion(), actualStatus.getGemFireVersion());
+    assertEquals(expectedStatus.getJavaVersion(), actualStatus.getJavaVersion());
+  }
+
+  protected Integer readPid(final File workingDirectory) throws IOException {
+    assertTrue(String.format("The working directory (%1$s) must exist!", workingDirectory),
+        workingDirectory != null && workingDirectory.isDirectory());
+
+    File[] files = workingDirectory.listFiles(new FileFilter() {
+      @Override
+      public boolean accept(final File pathname) {
+        return (pathname != null && pathname.isFile() && pathname.getAbsolutePath().endsWith(".pid"));
+      }
+    });
+
+    assertNotNull(files);
+    assertTrue(files.length > 0);
+
+    File pidFile = files[0];
+
+    BufferedReader fileReader = null;
+
+    try {
+      fileReader = new BufferedReader(new FileReader(pidFile), 1024);
+      return Integer.parseInt(fileReader.readLine().trim());
+    } catch (Exception ignore) {
+      return null;
+    } finally {
+      IOUtils.close(fileReader);
+    }
+  }
+
+  protected String serviceStateStatusStringNormalized(final ServiceState serviceState) {
+    return serviceStateStatusStringNormalized(serviceState.toString());
+  }
+
+  protected String serviceStateStatusStringNormalized(final String serviceStateStatus) {
+    assertNotNull(serviceStateStatus);
+    assertTrue("serviceStateStatus is missing 'Uptime': " + serviceStateStatus, serviceStateStatus.contains("Uptime"));
+    assertTrue("serviceStateStatus is missing 'JVM Arguments': " + serviceStateStatus,
+        serviceStateStatus.contains("JVM Arguments"));
+
+    return serviceStateStatus.substring(0, serviceStateStatus.indexOf("Uptime")).concat(
+        serviceStateStatus.substring(serviceStateStatus.indexOf("JVM Arguments")));
+  }
+
+  protected Status stopLocator(final File workingDirectory) {
+    return stopLocator(IOUtils.tryGetCanonicalPathElseGetAbsolutePath(workingDirectory));
+  }
+
+  protected Status stopLocator(final String workingDirectory) {
+    return waitForGemFireProcessToStop(
+        new Builder().setCommand(Command.STOP).setWorkingDirectory(workingDirectory).build().stop(), workingDirectory);
+  }
+
+  protected Status stopServer(final File workingDirectory) {
+    return stopServer(IOUtils.tryGetCanonicalPathElseGetAbsolutePath(workingDirectory));
+  }
+
+  protected Status stopServer(final String workingDirectory) {
+    return waitForGemFireProcessToStop(
+        new ServerLauncher.Builder().setCommand(ServerLauncher.Command.STOP).setWorkingDirectory(
+            workingDirectory).build().stop(), workingDirectory);
+  }
+
+  protected String toString(final Result result) {
+    assert result != null : "The Result object from the command execution cannot be null!";
+
+    StringBuilder buffer = new StringBuilder(StringUtils.LINE_SEPARATOR);
+
+    while (result.hasNextLine()) {
+      buffer.append(result.nextLine());
+      buffer.append(StringUtils.LINE_SEPARATOR);
+    }
+
+    return buffer.toString();
+  }
+
+  protected Status waitForGemFireProcessToStop(final ServiceState serviceState, final String workingDirectory) {
+    if (!Status.STOPPED.equals(serviceState.getStatus())) {
+      try {
+        final Integer pid = readPid(new File(workingDirectory));
+
+        if (pid != null) {
+          WaitCriterion waitCriteria = new WaitCriterion() {
+            private LauncherLifecycleCommands launcherLifecycleCommands = new LauncherLifecycleCommands();
+
+            @Override
+            public boolean done() {
+              return !launcherLifecycleCommands.isVmWithProcessIdRunning(pid);
+            }
+
+            @Override
+            public String description() {
+              return String.format("Waiting for GemFire Process with PID (%1$d) to stop.", pid);
+            }
+          };
+
+          waitForCriterion(waitCriteria, TimeUnit.SECONDS.toMillis(15), TimeUnit.SECONDS.toMillis(5), false);
+
+          if (!waitCriteria.done()) {
+            processIds.offer(pid);
+          }
+        }
+      } catch (IOException ignore) {
+      }
+    }
+
+    return serviceState.getStatus();
+  }
+
+  protected void writePid(final File pidFile, final int pid) throws IOException {
+    assertTrue("The PID file must actually exist!", pidFile != null && pidFile.isFile());
+
+    FileWriter writer = null;
+
+    try {
+      writer = new FileWriter(pidFile, false);
+      writer.write(String.valueOf(pid));
+      writer.write(System.getProperty("line.separator"));
+      writer.flush();
+    } finally {
+      IOUtils.close(writer);
+    }
+  }
+
+  public void test000StartLocatorCapturesOutputOnError() throws IOException {
+    final int locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+
+    String pathname = (getClass().getSimpleName() + "_" + testName);
+    File workingDirectory = new File(pathname);
+
+    assertTrue(workingDirectory.isDirectory() || workingDirectory.mkdir());
+
+    File pidFile = new File(workingDirectory, ProcessType.LOCATOR.getPidFileName());
+
+    assertTrue(pidFile.createNewFile());
+
+    writePid(pidFile, getPidOrOne());
+    pidFile.deleteOnExit();
+
+    assertTrue(pidFile.isFile());
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_LOCATOR);
+
+    command.addOption(CliStrings.START_LOCATOR__MEMBER_NAME, pathname);
+    command.addOption(CliStrings.START_LOCATOR__DIR, pathname);
+    command.addOption(CliStrings.START_LOCATOR__PORT, String.valueOf(locatorPort));
+    command.addOption(CliStrings.START_LOCATOR__ENABLE__SHARED__CONFIGURATION, Boolean.FALSE.toString());
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.http-service-port=0");
+    command.addOption(CliStrings.START_LOCATOR__J,
+        "-Dgemfire.jmx-manager-port=" + AvailablePortHelper.getRandomAvailableTCPPort());
+
+    CommandResult result = executeCommand(command.toString());
+
+    assertNotNull(result);
+    assertEquals(Result.Status.ERROR, result.getStatus());
+
+    String resultString = toString(result);
+
+    assertTrue(resultString, resultString.contains(
+        "Exception in thread \"main\" java.lang.RuntimeException: A PID file already exists and a Locator may be running in " + IOUtils.tryGetCanonicalFileElseGetAbsoluteFile(
+            workingDirectory)));
+    assertTrue(resultString, resultString.contains(
+        "Caused by: com.gemstone.gemfire.internal.process.FileAlreadyExistsException: Pid file already exists: " + IOUtils.tryGetCanonicalFileElseGetAbsoluteFile(
+            pidFile)));
+  }
+
+  /*
+   * This method makes an effort to get the PID of the running process. If it is unable to determine accurately, it
+   * simply returns 1.
+   */
+  private int getPidOrOne() {
+    int pid = 1;
+    String[] name = ManagementFactory.getRuntimeMXBean().getName().split("@");
+    if (name.length > 1) {
+      try {
+        pid = Integer.parseInt(name[0]);
+      } catch (NumberFormatException nex) {
+        // Ignored
+      }
+    }
+
+    return pid;
+  }
+
+  public void test001StartLocatorFailsFastOnMissingGemFirePropertiesFile() {
+    String gemfirePropertiesPathname = "/path/to/missing/gemfire.properties";
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_LOCATOR);
+
+    command.addOption(CliStrings.START_LOCATOR__MEMBER_NAME, getClass().getSimpleName().concat("_").concat(testName));
+    command.addOption(CliStrings.START_LOCATOR__PORT, "0");
+    command.addOption(CliStrings.START_LOCATOR__PROPERTIES, gemfirePropertiesPathname);
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.http-service-port=0");
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.jmx-manager=false");
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.jmx-manager-port=0");
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.jmx-manager-start=false");
+
+    CommandResult result = executeCommand(command.toString());
+
+    assertNotNull(result);
+    assertEquals(Result.Status.ERROR, result.getStatus());
+
+    String resultString = toString(result);
+
+    assertTrue(resultString, resultString.contains(
+        MessageFormat.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, StringUtils.EMPTY_STRING,
+            gemfirePropertiesPathname)));
+  }
+
+  public void test002StartLocatorFailsFastOnMissingGemFireSecurityPropertiesFile() {
+    String gemfireSecurityPropertiesPathname = "/path/to/missing/gemfire-security.properties";
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_LOCATOR);
+
+    command.addOption(CliStrings.START_LOCATOR__MEMBER_NAME, getClass().getSimpleName().concat("_").concat(testName));
+    command.addOption(CliStrings.START_LOCATOR__PORT, "0");
+    command.addOption(CliStrings.START_LOCATOR__SECURITY_PROPERTIES, gemfireSecurityPropertiesPathname);
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.http-service-port=0");
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.jmx-manager=false");
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.jmx-manager-port=0");
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.jmx-manager-start=false");
+
+    CommandResult result = executeCommand(command.toString());
+
+    assertNotNull(result);
+    assertEquals(Result.Status.ERROR, result.getStatus());
+
+    String resultString = toString(result);
+
+    assertTrue(resultString, resultString.contains(
+        MessageFormat.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ",
+            gemfireSecurityPropertiesPathname)));
+  }
+
+  public void test003StartServerFailsFastOnMissingCacheXmlFile() {
+    String cacheXmlPathname = "/path/to/missing/cache.xml";
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_SERVER);
+
+    command.addOption(CliStrings.START_SERVER__NAME, getClass().getSimpleName().concat("_").concat(testName));
+    command.addOption(CliStrings.START_SERVER__CACHE_XML_FILE, cacheXmlPathname);
+
+    CommandResult result = executeCommand(command.toString());
+
+    assertNotNull(result);
+    assertEquals(Result.Status.ERROR, result.getStatus());
+
+    String resultString = toString(result);
+
+    assertTrue(resultString,
+        resultString.contains(MessageFormat.format(CliStrings.CACHE_XML_NOT_FOUND_MESSAGE, cacheXmlPathname)));
+  }
+
+  public void test004StartServerFailsFastOnMissingGemFirePropertiesFile() {
+    String gemfirePropertiesFile = "/path/to/missing/gemfire.properties";
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_SERVER);
+
+    command.addOption(CliStrings.START_SERVER__NAME, getClass().getSimpleName().concat("_").concat(testName));
+    command.addOption(CliStrings.START_SERVER__PROPERTIES, gemfirePropertiesFile);
+
+    CommandResult result = executeCommand(command.toString());
+
+    assertNotNull(result);
+    assertEquals(Result.Status.ERROR, result.getStatus());
+
+    String resultString = toString(result);
+
+    assertTrue(resultString, resultString.contains(
+        MessageFormat.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, StringUtils.EMPTY_STRING,
+            gemfirePropertiesFile)));
+  }
+
+  public void test005StartServerFailsFastOnMissingGemFireSecurityPropertiesFile() {
+    String gemfireSecuritiesPropertiesFile = "/path/to/missing/gemfire-securities.properties";
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_SERVER);
+
+    command.addOption(CliStrings.START_SERVER__NAME, getClass().getSimpleName().concat("_").concat(testName));
+    command.addOption(CliStrings.START_SERVER__SECURITY_PROPERTIES, gemfireSecuritiesPropertiesFile);
+
+    CommandResult result = executeCommand(command.toString());
+
+    assertNotNull(result);
+    assertEquals(Result.Status.ERROR, result.getStatus());
+
+    String resultString = toString(result);
+
+    assertTrue(resultString, resultString.contains(
+        MessageFormat.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ",
+            gemfireSecuritiesPropertiesFile)));
+  }
+
+  public void test006StartLocatorInRelativeDirectory() {
+    final int locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+
+    String pathname = (getClass().getSimpleName() + "_" + testName);
+    File workingDirectory = new File(pathname);
+
+    assertTrue(workingDirectory.isDirectory() || workingDirectory.mkdir());
+
+    try {
+      CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_LOCATOR);
+
+      command.addOption(CliStrings.START_LOCATOR__MEMBER_NAME, pathname);
+      command.addOption(CliStrings.START_LOCATOR__CONNECT, Boolean.FALSE.toString());
+      command.addOption(CliStrings.START_LOCATOR__DIR, pathname);
+      command.addOption(CliStrings.START_LOCATOR__PORT, String.valueOf(locatorPort));
+      command.addOption(CliStrings.START_LOCATOR__ENABLE__SHARED__CONFIGURATION, Boolean.FALSE.toString());
+      command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.http-service-port=0");
+      command.addOption(CliStrings.START_LOCATOR__J,
+          "-Dgemfire.jmx-manager-port=" + AvailablePortHelper.getRandomAvailableTCPPort());
+
+      CommandResult result = executeCommand(command.toString());
+
+      assertNotNull(result);
+      assertEquals(Result.Status.OK, result.getStatus());
+
+      String locatorOutput = toString(result);
+
+      assertNotNull(locatorOutput);
+      assertTrue("Locator output was: " + locatorOutput,
+          locatorOutput.contains("Locator in " + IOUtils.tryGetCanonicalFileElseGetAbsoluteFile(workingDirectory)));
+    } finally {
+      stopLocator(workingDirectory);
+    }
+  }
+
+  public void test007StatusLocatorUsingMemberNameIDWhenGfshIsNotConnected() {
+    CommandResult result = executeCommand(CliStrings.STATUS_LOCATOR + " --name=" + testName);
+
+    assertNotNull(result);
+    assertEquals(Result.Status.ERROR, result.getStatus());
+    assertEquals(CliStrings.format(CliStrings.STATUS_SERVICE__GFSH_NOT_CONNECTED_ERROR_MESSAGE, "Locator"),
+        StringUtils.trim(toString(result)));
+  }
+
+  public void test008StatusLocatorUsingMemberName() {
+    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+
+    final int jmxManagerPort = ports[0];
+    final int locatorPort = ports[1];
+
+    String pathname = (getClass().getSimpleName() + "_" + testName);
+    File workingDirectory = new File(pathname);
+
+    assertTrue(workingDirectory.isDirectory() || workingDirectory.mkdir());
+
+    try {
+      CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_LOCATOR);
+
+      command.addOption(CliStrings.START_LOCATOR__MEMBER_NAME, pathname);
+      command.addOption(CliStrings.START_LOCATOR__CONNECT, Boolean.FALSE.toString());
+      command.addOption(CliStrings.START_LOCATOR__DIR, pathname);
+      command.addOption(CliStrings.START_LOCATOR__PORT, String.valueOf(locatorPort));
+      command.addOption(CliStrings.START_LOCATOR__ENABLE__SHARED__CONFIGURATION, Boolean.FALSE.toString());
+      command.addOption(CliStrings.START_LOCATOR__FORCE, Boolean.TRUE.toString());
+      command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.http-service-port=0");
+      command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.jmx-manager-port=" + jmxManagerPort);
+
+      CommandResult result = executeCommand(command.toString());
+
+      assertNotNull(result);
+      assertEquals(Result.Status.OK, result.getStatus());
+
+      LocatorLauncher locatorLauncher = new LocatorLauncher.Builder().setCommand(
+          LocatorLauncher.Command.STATUS).setBindAddress(null).setPort(locatorPort).setWorkingDirectory(
+          workingDirectory.getPath()).build();
+
+      assertNotNull(locatorLauncher);
+
+      LocatorState expectedLocatorState = locatorLauncher.waitOnStatusResponse(60, 10, TimeUnit.SECONDS);
+
+      assertNotNull(expectedLocatorState);
+      assertEquals(Status.ONLINE, expectedLocatorState.getStatus());
+
+      result = executeCommand(String.format("%1$s --locator=localhost[%2$d]", CliStrings.CONNECT, locatorPort));
+
+      assertNotNull(result);
+      assertEquals(Result.Status.OK, result.getStatus());
+
+      result = executeCommand(String.format("%1$s --name=invalidLocatorMemberName", CliStrings.STATUS_LOCATOR));
+
+      assertNotNull(result);
+      assertEquals(Result.Status.ERROR, result.getStatus());
+      assertEquals(CliStrings.format(CliStrings.STATUS_LOCATOR__NO_LOCATOR_FOUND_FOR_MEMBER_ERROR_MESSAGE,
+          "invalidLocatorMemberName"), StringUtils.trim(toString(result)));
+
+      result = executeCommand(String.format("%1$s --name=%2$s", CliStrings.STATUS_LOCATOR, pathname));
+
+      assertNotNull(result);
+      assertEquals(Result.Status.OK, result.getStatus());
+      assertTrue(serviceStateStatusStringNormalized(toString(result)).contains(
+          serviceStateStatusStringNormalized(expectedLocatorState)));
+    } finally {
+      stopLocator(workingDirectory);
+    }
+  }
+
+  public void test009StatusLocatorUsingMemberId() throws Exception {
+    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+
+    final int jmxManagerPort = ports[0];
+    final int locatorPort = ports[1];
+
+    String pathname = (getClass().getSimpleName() + "_" + testName);
+    File workingDirectory = new File(pathname);
+
+    assertTrue(workingDirectory.isDirectory() || workingDirectory.mkdir());
+
+    try {
+      CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_LOCATOR);
+
+      command.addOption(CliStrings.START_LOCATOR__MEMBER_NAME, pathname);
+      command.addOption(CliStrings.START_LOCATOR__CONNECT, Boolean.FALSE.toString());
+      command.addOption(CliStrings.START_LOCATOR__DIR, pathname);
+      command.addOption(CliStrings.START_LOCATOR__PORT, String.valueOf(locatorPort));
+      command.addOption(CliStrings.START_LOCATOR__ENABLE__SHARED__CONFIGURATION, Boolean.FALSE.toString());
+      command.addOption(CliStrings.START_LOCATOR__FORCE, Boolean.TRUE.toString());
+      command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.http-service-port=0");
+      command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.jmx-manager-port=" + jmxManagerPort);
+
+      CommandResult result = executeCommand(command.toString());
+
+      assertNotNull(result);
+      assertEquals(Result.Status.OK, result.getStatus());
+
+      LocatorLauncher locatorLauncher = new LocatorLauncher.Builder().setCommand(
+          LocatorLauncher.Command.STATUS).setBindAddress(null).setPort(locatorPort).setWorkingDirectory(
+          workingDirectory.getPath()).build();
+
+      assertNotNull(locatorLauncher);
+
+      LocatorState expectedLocatorState = locatorLauncher.waitOnStatusResponse(60, 10, TimeUnit.SECONDS);
+
+      assertNotNull(expectedLocatorState);
+      assertEquals(Status.ONLINE, expectedLocatorState.getStatus());
+
+      result = executeCommand(String.format("%1$s --locator=localhost[%2$d]", CliStrings.CONNECT, locatorPort));
+
+      assertNotNull(result);
+      assertEquals(Result.Status.OK, result.getStatus());
+
+      result = executeCommand(
+          String.format("%1$s --name=%2$s", CliStrings.STATUS_LOCATOR, getMemberId(jmxManagerPort, pathname)));
+
+      assertNotNull(result);
+      assertEquals(Result.Status.OK, result.getStatus());
+      assertTrue(serviceStateStatusStringNormalized(toString(result)).contains(
+          serviceStateStatusStringNormalized(expectedLocatorState)));
+    } finally {
+      stopLocator(workingDirectory);
+    }
+  }
+
+  public void test010StopLocatorUsingMemberNameIDWhenGfshIsNotConnected() {
+    CommandResult result = executeCommand(CliStrings.STOP_LOCATOR + " --name=" + testName);
+
+    assertNotNull(result);
+    assertEquals(Result.Status.ERROR, result.getStatus());
+    assertEquals(CliStrings.format(CliStrings.STOP_SERVICE__GFSH_NOT_CONNECTED_ERROR_MESSAGE, "Locator"),
+        StringUtils.trim(toString(result)));
+  }
+
+  public void test011StopLocatorUsingMemberName() {
+    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+
+    final int jmxManagerPort = ports[0];
+    final int locatorPort = ports[1];
+
+    String pathname = (getClass().getSimpleName() + "_" + testName);
+    File workingDirectory = new File(pathname);
+
+    assertTrue(workingDirectory.isDirectory() || workingDirectory.mkdir());
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_LOCATOR);
+
+    command.addOption(CliStrings.START_LOCATOR__MEMBER_NAME, pathname);
+    command.addOption(CliStrings.START_LOCATOR__CONNECT, Boolean.FALSE.toString());
+    command.addOption(CliStrings.START_LOCATOR__DIR, pathname);
+    command.addOption(CliStrings.START_LOCATOR__PORT, String.valueOf(locatorPort));
+    command.addOption(CliStrings.START_LOCATOR__ENABLE__SHARED__CONFIGURATION, Boolean.FALSE.toString());
+    command.addOption(CliStrings.START_LOCATOR__FORCE, Boolean.TRUE.toString());
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.http-service-port=0");
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.jmx-manager-port=" + jmxManagerPort);
+
+    CommandResult result = executeCommand(command.toString());
+
+    assertNotNull(result);
+    assertEquals(Result.Status.OK, result.getStatus());
+
+    final LocatorLauncher locatorLauncher = new LocatorLauncher.Builder().setCommand(
+        LocatorLauncher.Command.STOP).setBindAddress(null).setPort(locatorPort).setWorkingDirectory(
+        workingDirectory.getPath()).build();
+
+    assertNotNull(locatorLauncher);
+
+    LocatorState locatorStatus = locatorLauncher.waitOnStatusResponse(60, 10, TimeUnit.SECONDS);
+
+    assertNotNull(locatorStatus);
+    assertEquals(Status.ONLINE, locatorStatus.getStatus());
+
+    result = executeCommand(String.format("%1$s --locator=localhost[%2$d]", CliStrings.CONNECT, locatorPort));
+
+    assertNotNull(result);
+    assertEquals(Result.Status.OK, result.getStatus());
+
+    result = executeCommand(String.format("%1$s --name=invalidLocatorMemberName", CliStrings.STOP_LOCATOR));
+
+    assertNotNull(result);
+    assertEquals(Result.Status.ERROR, result.getStatus());
+    assertEquals(CliStrings.format(CliStrings.STOP_LOCATOR__NO_LOCATOR_FOUND_FOR_MEMBER_ERROR_MESSAGE,
+        "invalidLocatorMemberName"), StringUtils.trim(toString(result)));
+
+    locatorStatus = locatorLauncher.status();
+
+    assertNotNull(locatorStatus);
+    assertEquals(Status.ONLINE, locatorStatus.getStatus());
+
+    result = executeCommand(String.format("%1$s --name=%2$s", CliStrings.STOP_LOCATOR, pathname));
+
+    assertNotNull(result);
+    assertEquals(Result.Status.OK, result.getStatus());
+
+    // TODO figure out what output to assert and validate on now that 'stop locator' uses Gfsh's logger
+    // and standard err/out...
+    //assertEquals(CliStrings.format(CliStrings.STOP_LOCATOR__SHUTDOWN_MEMBER_MESSAGE, pathname),
+    //  StringUtils.trim(toString(result)));
+
+    WaitCriterion waitCriteria = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        final LocatorState locatorStatus = locatorLauncher.status();
+        return (locatorStatus != null && Status.NOT_RESPONDING.equals(locatorStatus.getStatus()));
+      }
+
+      @Override
+      public String description() {
+        return "wait for the Locator to stop; the Locator will no longer respond after it stops";
+      }
+    };
+
+    waitForCriterion(waitCriteria, 15 * 1000, 5000, true);
+
+    locatorStatus = locatorLauncher.status();
+
+    assertNotNull(locatorStatus);
+    assertEquals(Status.NOT_RESPONDING, locatorStatus.getStatus());
+  }
+
+  // @see Trac Bug # 46760
+  public void test012StopLocatorUsingMemberId() throws Exception {
+    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+
+    final int jmxManagerPort = ports[0];
+    final int locatorPort = ports[1];
+
+    String pathname = (getClass().getSimpleName() + "_" + testName);
+    File workingDirectory = new File(pathname);
+
+    assertTrue(workingDirectory.isDirectory() || workingDirectory.mkdir());
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_LOCATOR);
+
+    command.addOption(CliStrings.START_LOCATOR__MEMBER_NAME, pathname);
+    command.addOption(CliStrings.START_LOCATOR__CONNECT, Boolean.FALSE.toString());
+    command.addOption(CliStrings.START_LOCATOR__DIR, pathname);
+    command.addOption(CliStrings.START_LOCATOR__PORT, String.valueOf(locatorPort));
+    command.addOption(CliStrings.START_LOCATOR__ENABLE__SHARED__CONFIGURATION, Boolean.FALSE.toString());
+    command.addOption(CliStrings.START_LOCATOR__FORCE, Boolean.TRUE.toString());
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.http-service-port=0");
+    command.addOption(CliStrings.START_LOCATOR__J, "-Dgemfire.jmx-manager-port=" + jmxManagerPort);
+
+    CommandResult result = executeCommand(command.toString());
+
+    assertNotNull(result);
+    assertEquals(Result.Status.OK, result.getStatus());
+
+    final LocatorLauncher locatorLauncher = new LocatorLauncher.Builder().setCommand(
+        LocatorLauncher.Command.STOP).setBindAddress(null).setPort(locatorPort).setWorkingDirectory(
+        workingDirectory.getPath()).build();
+
+    assertNotNull(locatorLauncher);
+
+    LocatorState locatorState = locatorLauncher.waitOnStatusResponse(60, 10, TimeUnit.SECONDS);
+
+    assertNotNull(locatorState);
+    assertEquals(Status.ONLINE, locatorState.getStatus());
+
+    result = executeCommand(String.format("%1$s --locator=localhost[%2$d]", CliStrings.CONNECT, locatorPort));
+
+    assertNotNull(result);
+    assertEquals(Result.Status.OK, result.getStatus());
+
+    String memberId = getMemberId(jmxManagerPort, pathname);
+
+    result = executeCommand(String.format("%1$s --name=%2$s", CliStrings.STOP_LOCATOR, memberId));
+
+    assertNotNull(result);
+    assertEquals(Result.Status.OK, result.getStatus());
+
+    // TODO figure out what output to assert and validate on now that 'stop locator' uses Gfsh's logger
+    // and standard err/out...
+    //assertEquals(CliStrings.format(CliStrings.STOP_LOCATOR__SHUTDOWN_MEMBER_MESSAGE, memberId),
+    //  StringUtils.trim(toString(result)));
+
+    WaitCriterion waitCriteria = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        LocatorState locatorState = locatorLauncher.status();
+        return (locatorState != null && Status.NOT_RESPONDING.equals(locatorState.getStatus()));
+      }
+
+      @Override
+      public String description() {
+        return "wait for the Locator to stop; the Locator will no longer respond after it stops";
+      }
+    };
+
+    waitForCriterion(waitCriteria, 15 * 1000, 5000, true);
+
+    locatorState = locatorLauncher.status();
+
+    assertNotNull(locatorState);
+    assertEquals(Status.NOT_RESPONDING, locatorState.getStatus());
+  }
+
+  public void test013StartServerWithSpring() {
+    String pathname = (getClass().getSimpleName() + "_" + testName);
+    File workingDirectory = new File(pathname);
+
+    assertTrue(workingDirectory.isDirectory() || workingDirectory.mkdir());
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_SERVER);
+
+    command.addOption(CliStrings.START_SERVER__NAME, getClass().getSimpleName().concat("_").concat(testName));
+    command.addOption(CliStrings.START_SERVER__USE_CLUSTER_CONFIGURATION, Boolean.FALSE.toString());
+    command.addOption(CliStrings.START_SERVER__LOG_LEVEL, "config");
+    command.addOption(CliStrings.START_SERVER__INCLUDE_SYSTEM_CLASSPATH);
+    command.addOption(CliStrings.START_SERVER__DISABLE_DEFAULT_SERVER);
+    command.addOption(CliStrings.START_SERVER__DIR, pathname);
+    command.addOption(CliStrings.START_SERVER__SPRING_XML_LOCATION, "spring/spring-gemfire-context.xml");
+
+    CommandResult result = executeCommand(command.toString());
+
+    assertNotNull(result);
+    assertEquals(Result.Status.OK, result.getStatus());
+
+    final ServerLauncher springGemFireServer = new ServerLauncher.Builder().setCommand(
+        ServerLauncher.Command.STATUS).setWorkingDirectory(
+        IOUtils.tryGetCanonicalPathElseGetAbsolutePath(workingDirectory)).build();
+
+    assertNotNull(springGemFireServer);
+
+    ServerState serverState = springGemFireServer.status();
+
+    assertNotNull(serverState);
+    assertEquals(Status.ONLINE, serverState.getStatus());
+
+    // Now that the GemFire Server bootstrapped with Spring started up OK, stop it!
+    stopServer(springGemFireServer.getWorkingDirectory());
+
+    WaitCriterion waitCriteria = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        ServerState serverState = springGemFireServer.status();
+        return (serverState != null && Status.NOT_RESPONDING.equals(serverState.getStatus()));
+      }
+
+      @Override
+      public String description() {
+        return "wait for the Locator to stop; the Locator will no longer respond after it stops";
+      }
+    };
+
+    waitForCriterion(waitCriteria, TimeUnit.SECONDS.toMillis(15), TimeUnit.SECONDS.toMillis(5), true);
+
+    serverState = springGemFireServer.status();
+
+    assertNotNull(serverState);
+    assertEquals(Status.NOT_RESPONDING, serverState.getStatus());
+  }
+
+  public void test014GemFireServerJvmProcessTerminatesOnOutOfMemoryError() throws Exception {
+    int ports[] = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+    final int serverPort = ports[0];
+    final int locatorPort = ports[1];
+
+    String pathname = getClass().getSimpleName().concat("_").concat(testName);
+    File workingDirectory = new File(pathname);
+
+    assertTrue(workingDirectory.isDirectory() || workingDirectory.mkdir());
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_SERVER);
+
+    command.addOption(CliStrings.START_SERVER__NAME, pathname + TIMESTAMP.format(Calendar.getInstance().getTime()));
+    command.addOption(CliStrings.START_SERVER__SERVER_PORT, String.valueOf(serverPort));
+    command.addOption(CliStrings.START_SERVER__USE_CLUSTER_CONFIGURATION, Boolean.FALSE.toString());
+    command.addOption(CliStrings.START_SERVER__MAXHEAP, "10M");
+    command.addOption(CliStrings.START_SERVER__LOG_LEVEL, "config");
+    command.addOption(CliStrings.START_SERVER__DIR, pathname);
+    command.addOption(CliStrings.START_SERVER__CACHE_XML_FILE,
+        IOUtils.tryGetCanonicalPathElseGetAbsolutePath(writeAndGetCacheXmlFile(workingDirectory)));
+    command.addOption(CliStrings.START_SERVER__INCLUDE_SYSTEM_CLASSPATH);
+    command.addOption(CliStrings.START_SERVER__J,
+        "-Dgemfire." + DistributionConfig.START_LOCATOR_NAME + "=localhost[" + locatorPort + "]");
+
+
+    CommandResult result = executeCommand(command.toString());
+    System.out.println("result=" + result);
+
+    assertNotNull(result);
+    assertEquals(Result.Status.OK, result.getStatus());
+
+    ServerLauncher serverLauncher = new ServerLauncher.Builder().setCommand(
+        ServerLauncher.Command.STATUS).setWorkingDirectory(
+        IOUtils.tryGetCanonicalPathElseGetAbsolutePath(workingDirectory)).build();
+
+    assertNotNull(serverLauncher);
+
+    ServerState serverState = serverLauncher.status();
+
+    assertNotNull(serverState);
+    assertEquals(Status.ONLINE, serverState.getStatus());
+
+    // Verify our GemFire Server JVM process is running!
+    assertTrue(new LauncherLifecycleCommands().isVmWithProcessIdRunning(serverState.getPid()));
+
+    ClientCache clientCache = setupClientCache(pathname + String.valueOf(serverPort), serverPort);
+
+    assertNotNull(clientCache);
+
+    try {
+      Region<Long, String> exampleRegion = clientCache.getRegion("/Example");
+      // run the GemFire Server "out-of-town" with an OutOfMemoryError!
+      for (long index = 0; index < Long.MAX_VALUE; index++) {
+        exampleRegion.put(index, String.valueOf(index));
+      }
+    } catch (Exception ignore) {
+      System.err.printf("%1$s: %2$s%n", ignore.getClass().getName(), ignore.getMessage());
+    } finally {
+      clientCache.close();
+
+      final int serverPid = serverState.getPid();
+
+      WaitCriterion waitCriteria = new WaitCriterion() {
+        private LauncherLifecycleCommands launcherLifecycleCommands = new LauncherLifecycleCommands();
+
+        @Override
+        public boolean done() {
+          return !launcherLifecycleCommands.isVmWithProcessIdRunning(serverPid);
+        }
+
+        @Override
+        public String description() {
+          return "Wait for the GemFire Server JVM process that ran out-of-memory to exit.";
+        }
+      };
+
+      waitForCriterion(waitCriteria, TimeUnit.SECONDS.toMillis(30), TimeUnit.SECONDS.toMillis(10), true);
+
+      // Verify our GemFire Server JVM process is was terminated!
+      assertFalse(new LauncherLifecycleCommands().isVmWithProcessIdRunning(serverState.getPid()));
+
+      serverState = serverLauncher.status();
+
+      assertNotNull(serverState);
+      assertEquals(Status.NOT_RESPONDING, serverState.getStatus());
+    }
+  }
+
+  private File writeAndGetCacheXmlFile(final File workingDirectory) throws IOException {
+    File cacheXml = new File(workingDirectory, "cache.xml");
+    StringBuilder buffer = new StringBuilder("<?xml version=\"1.0\" encoding=\"UTF-8\"?>");
+
+    buffer.append(StringUtils.LINE_SEPARATOR);
+    buffer.append("<!DOCTYPE cache PUBLIC  \"-//GemStone Systems, Inc.//GemFire Declarative Caching 7.0//EN\"");
+    buffer.append(StringUtils.LINE_SEPARATOR);
+    buffer.append("  \"http://www.gemstone.com/dtd/cache7_0.dtd\">");
+    buffer.append(StringUtils.LINE_SEPARATOR);
+    buffer.append("<cache>");
+    buffer.append(StringUtils.LINE_SEPARATOR);
+    buffer.append("  <region name=\"Example\" refid=\"REPLICATE\"/>");
+    buffer.append(StringUtils.LINE_SEPARATOR);
+    buffer.append("</cache>");
+
+    BufferedWriter fileWriter = null;
+
+    try {
+      fileWriter = new BufferedWriter(
+          new OutputStreamWriter(new FileOutputStream(cacheXml, false), Charset.forName("UTF-8").newEncoder()));
+      fileWriter.write(buffer.toString());
+      fileWriter.flush();
+    } finally {
+      IOUtils.close(fileWriter);
+    }
+
+    return cacheXml;
+  }
+
+  private ClientCache setupClientCache(final String durableClientId, final int serverPort) {
+    ClientCache clientCache = new ClientCacheFactory().set("durable-client-id", durableClientId).create();
+
+    PoolFactory poolFactory = PoolManager.createFactory();
+
+    poolFactory.setMaxConnections(10);
+    poolFactory.setMinConnections(1);
+    poolFactory.setReadTimeout(5000);
+    poolFactory.addServer("localhost", serverPort);
+
+    Pool pool = poolFactory.create("serverConnectionPool");
+
+    assertNotNull("The 'serverConnectionPool' was not properly configured and initialized!", pool);
+
+    ClientRegionFactory<Long, String> regionFactory = clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY);
+
+    regionFactory.setPoolName(pool.getName());
+    regionFactory.setKeyConstraint(Long.class);
+    regionFactory.setValueConstraint(String.class);
+
+    Region<Long, String> exampleProxy = regionFactory.create("Example");
+
+    assertNotNull("The 'Example' Client Region was not properly configured and initialized", exampleProxy);
+
+    return clientCache;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eddef322/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsJUnitTest.java b/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsJUnitTest.java
new file mode 100755
index 0000000..d7e7970
--- /dev/null
+++ b/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsJUnitTest.java
@@ -0,0 +1,625 @@
+/*
+ * 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.GemFireException;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.ServerLauncher;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.DistributionLocator;
+import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.internal.lang.SystemUtils;
+import com.gemstone.gemfire.internal.util.IOUtils;
+import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.Stack;
+import java.util.jar.Attributes;
+import java.util.jar.Attributes.Name;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
+
+import static org.junit.Assert.*;
+
+/**
+ * The LauncherLifecycleCommandsJUnitTest class is a test suite of test cases testing the contract and functionality of
+ * the lifecycle launcher GemFire shell (Gfsh) commands.
+ *
+ * @author John Blum
+ * @see com.gemstone.gemfire.management.internal.cli.commands.LauncherLifecycleCommands
+ * @see org.junit.Assert
+ * @see org.junit.Test
+ * @since 7.0
+ */
+@SuppressWarnings("unused")
+@Category(UnitTest.class)
+public class LauncherLifecycleCommandsJUnitTest {
+
+  private static final String GFSH_DEPENDENCIES_JAR_PATHNAME = IOUtils.appendToPath(System.getenv("GEMFIRE"), "lib",
+      "gfsh-dependencies.jar");
+
+  private LauncherLifecycleCommands launcherCommands;
+
+  @Before
+  public void setup() {
+    launcherCommands = new LauncherLifecycleCommands();
+  }
+
+  @After
+  public void tearDown() {
+    launcherCommands = null;
+  }
+
+  protected LauncherLifecycleCommands getLauncherLifecycleCommands() {
+    return launcherCommands;
+  }
+
+  protected void writePid(final File pidFile, final int pid) throws IOException {
+    final FileWriter fileWriter = new FileWriter(pidFile, false);
+    fileWriter.write(String.valueOf(pid));
+    fileWriter.write("\n");
+    fileWriter.flush();
+    IOUtils.close(fileWriter);
+  }
+
+  @Test
+  public void testAddGemFirePropertyFileToCommandLine() {
+    final List<String> commandLine = new ArrayList<>();
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addGemFirePropertyFile(commandLine, null);
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addGemFirePropertyFile(commandLine, StringUtils.EMPTY_STRING);
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addGemFirePropertyFile(commandLine, " ");
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addGemFirePropertyFile(commandLine, "/path/to/gemfire.properties");
+
+    assertFalse(commandLine.isEmpty());
+    assertTrue(commandLine.contains("-DgemfirePropertyFile=/path/to/gemfire.properties"));
+  }
+
+  @Test
+  public void testAddGemFireSystemPropertiesToCommandLine() {
+    final List<String> commandLine = new ArrayList<>();
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addGemFireSystemProperties(commandLine, new Properties());
+
+    assertTrue(commandLine.isEmpty());
+
+    final Properties gemfireProperties = new Properties();
+
+    gemfireProperties.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[11235]");
+    gemfireProperties.setProperty(DistributionConfig.LOG_LEVEL_NAME, "config");
+    gemfireProperties.setProperty(DistributionConfig.LOG_FILE_NAME, StringUtils.EMPTY_STRING);
+    gemfireProperties.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    gemfireProperties.setProperty(DistributionConfig.NAME_NAME, "tidepool");
+
+    getLauncherLifecycleCommands().addGemFireSystemProperties(commandLine, gemfireProperties);
+
+    assertFalse(commandLine.isEmpty());
+    assertEquals(4, commandLine.size());
+
+    for (final String propertyName : gemfireProperties.stringPropertyNames()) {
+      final String propertyValue = gemfireProperties.getProperty(propertyName);
+      if (StringUtils.isBlank(propertyValue)) {
+        for (final String systemProperty : commandLine) {
+          assertFalse(systemProperty.startsWith("-Dgemfire.".concat(propertyName).concat("=")));
+        }
+      } else {
+        assertTrue(commandLine.contains("-Dgemfire.".concat(propertyName).concat("=").concat(propertyValue)));
+      }
+    }
+  }
+
+  @Test
+  public void testAddInitialHeapToCommandLine() {
+    final List<String> commandLine = new ArrayList<>();
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addInitialHeap(commandLine, null);
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addInitialHeap(commandLine, StringUtils.EMPTY_STRING);
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addInitialHeap(commandLine, " ");
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addInitialHeap(commandLine, "512M");
+
+    assertFalse(commandLine.isEmpty());
+    assertEquals("-Xms512M", commandLine.get(0));
+  }
+
+  @Test
+  public void testAddJvmArgumentsAndOptionsToCommandLine() {
+    final List<String> commandLine = new ArrayList<>();
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addJvmArgumentsAndOptions(commandLine, null);
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addJvmArgumentsAndOptions(commandLine, new String[]{});
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addJvmArgumentsAndOptions(commandLine,
+        new String[]{"-DmyProp=myVal", "-d64", "-server", "-Xprof"});
+
+    assertFalse(commandLine.isEmpty());
+    assertEquals(4, commandLine.size());
+    assertEquals("-DmyProp=myVal", commandLine.get(0));
+    assertEquals("-d64", commandLine.get(1));
+    assertEquals("-server", commandLine.get(2));
+    assertEquals("-Xprof", commandLine.get(3));
+  }
+
+  // Fix for Bug #47192 - "Making GemFire (JVM) to exit in case of OutOfMemory"
+  @Test
+  public void testAddJvmOptionsForOutOfMemoryErrors() {
+    final List<String> jvmOptions = new ArrayList<>(1);
+
+    getLauncherLifecycleCommands().addJvmOptionsForOutOfMemoryErrors(jvmOptions);
+
+    if (SystemUtils.isHotSpotVM()) {
+      if (SystemUtils.isWindows()) {
+        assertTrue(jvmOptions.contains("-XX:OnOutOfMemoryError=taskkill /F /PID %p"));
+      } else {
+        assertTrue(jvmOptions.contains("-XX:OnOutOfMemoryError=kill -KILL %p"));
+      }
+    } else if (SystemUtils.isJ9VM()) {
+      assertEquals(1, jvmOptions.size());
+      assertTrue(jvmOptions.contains("-Xcheck:memory"));
+    } else if (SystemUtils.isJRockitVM()) {
+      assertEquals(1, jvmOptions.size());
+      assertTrue(jvmOptions.contains("-XXexitOnOutOfMemory"));
+    } else {
+      assertTrue(jvmOptions.isEmpty());
+    }
+  }
+
+  @Test
+  public void testAddMaxHeapToCommandLine() {
+    final List<String> commandLine = new ArrayList<>();
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addMaxHeap(commandLine, null);
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addMaxHeap(commandLine, StringUtils.EMPTY_STRING);
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addMaxHeap(commandLine, "  ");
+
+    assertTrue(commandLine.isEmpty());
+
+    getLauncherLifecycleCommands().addMaxHeap(commandLine, "1024M");
+
+    assertFalse(commandLine.isEmpty());
+    assertEquals(3, commandLine.size());
+    assertEquals("-Xmx1024M", commandLine.get(0));
+    assertEquals("-XX:+UseConcMarkSweepGC", commandLine.get(1));
+    assertEquals("-XX:CMSInitiatingOccupancyFraction=" + LauncherLifecycleCommands.CMS_INITIAL_OCCUPANCY_FRACTION,
+        commandLine.get(2));
+  }
+
+  @Test(expected = AssertionError.class)
+  public void testReadPidWithNull() {
+    try {
+      getLauncherLifecycleCommands().readPid(null);
+    } catch (AssertionError expected) {
+      assertEquals("The file from which to read the process ID (pid) cannot be null!", expected.getMessage());
+      throw expected;
+    }
+  }
+
+  @Test
+  public void testReadPidWithNonExistingFile() {
+    assertEquals(LauncherLifecycleCommands.INVALID_PID,
+        getLauncherLifecycleCommands().readPid(new File("/path/to/non_existing/pid.file")));
+  }
+
+  @Test
+  public void testReadPid() throws IOException {
+    final int expectedPid = 12345;
+
+    File pidFile = new File(getClass().getSimpleName().concat("_testReadPid.pid"));
+
+    assertTrue(pidFile.createNewFile());
+
+    pidFile.deleteOnExit();
+    writePid(pidFile, expectedPid);
+
+    final int actualPid = getLauncherLifecycleCommands().readPid(pidFile);
+
+    assertEquals(expectedPid, actualPid);
+  }
+
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testGetClasspath() {
+    assertEquals(System.getProperty("java.class.path"), getLauncherLifecycleCommands().getClasspath(null));
+  }
+
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testGetClasspathWithUserDefinedClasspath() {
+    assertEquals(System.getProperty("java.class.path") + File.pathSeparator + "/path/to/user/classes",
+        getLauncherLifecycleCommands().getClasspath("/path/to/user/classes"));
+  }
+
+  @Test
+  public void testGemFireCoreClasspath() throws IOException {
+    File coreDependenciesJar = new File(LauncherLifecycleCommands.CORE_DEPENDENCIES_JAR_PATHNAME);
+
+    assertNotNull(coreDependenciesJar);
+    assertTrue(coreDependenciesJar + " is not a file", coreDependenciesJar.isFile());
+
+    Collection<String> expectedJarDependencies = Arrays.asList("antlr", "commons-io", "commons-lang", "commons-logging",
+        "gemfire", "jackson-annotations", "jackson-core", "jackson-databind", "jansi", "jline", "snappy-java",
+        "spring-core", "spring-shell", "jetty-server", "jetty-servlet", "jetty-webapp", "jetty-util", "jetty-http",
+        "servlet-api", "jetty-io", "jetty-security", "jetty-xml"
+
+    );
+
+    assertJarFileManifestClassPath(coreDependenciesJar, expectedJarDependencies);
+  }
+
+  @Test
+  public void testGetSpringJars() {
+    List<String> actualSpringJarPathnames = new LauncherLifecycleCommands().getSpringJars();
+
+    assertNotNull(actualSpringJarPathnames);
+    assertEquals(LauncherLifecycleCommands.SPRING_JAR_NAME_PREFIXES.size(), actualSpringJarPathnames.size());
+
+    int springCoreVersion = -1;
+    int springDataCommonsVersion = -1;
+    int springDataGemFireVersion = -1;
+
+    Set<String> expectedSpringJarNames = new HashSet<>(LauncherLifecycleCommands.SPRING_JAR_NAME_PREFIXES);
+
+    assertFalse(expectedSpringJarNames.isEmpty());
+
+    for (String springJarPathname : actualSpringJarPathnames) {
+      String springJarName = springJarPathname.substring(springJarPathname.lastIndexOf(File.separator) + 1);
+      String springJarNamePrefix = springJarName.substring(0, springJarName.lastIndexOf("-"));
+
+      switch (springJarNamePrefix) {
+        case LauncherLifecycleCommands.SPRING_BEANS_JAR_NAME_PREFIX:
+          springCoreVersion = Integer.parseInt(StringUtils.getDigitsOnly(springJarName));
+          break;
+        case LauncherLifecycleCommands.SPRING_DATA_COMMONS_JAR_NAME_PREFIX:
+          springDataCommonsVersion = Integer.parseInt(StringUtils.getDigitsOnly(springJarName));
+          break;
+        case LauncherLifecycleCommands.SPRING_DATA_GEMFIRE_JAR_NAME_PREFIX:
+          springDataGemFireVersion = Integer.parseInt(StringUtils.getDigitsOnly(springJarName));
+          break;
+      }
+
+      expectedSpringJarNames.remove(springJarNamePrefix);
+    }
+
+    assertTrue(String.format("Expected empty; but was (%1$s)", expectedSpringJarNames),
+        expectedSpringJarNames.isEmpty());
+    assertEquals(3212, springCoreVersion);
+    assertEquals(191, springDataCommonsVersion);
+    assertEquals(151, springDataGemFireVersion);
+  }
+
+  @Test
+  public void testGetSystemClasspath() {
+    assertEquals(System.getProperty("java.class.path"), getLauncherLifecycleCommands().getSystemClasspath());
+  }
+
+  @Test
+  public void testLocatorClasspathOrder() {
+    String userClasspath = "/path/to/user/lib/app.jar:/path/to/user/classes";
+
+    String expectedClasspath = launcherCommands.getGemFireJarPath().concat(File.pathSeparator).concat(
+        userClasspath).concat(File.pathSeparator).concat(System.getProperty("java.class.path")).concat(
+        File.pathSeparator).concat(LauncherLifecycleCommands.CORE_DEPENDENCIES_JAR_PATHNAME);
+
+    String actualClasspath = launcherCommands.getLocatorClasspath(true, userClasspath);
+
+    assertEquals(expectedClasspath, actualClasspath);
+  }
+
+  @Test
+  public void testServerClasspathOrder() {
+    String userClasspath = "/path/to/user/lib/app.jar:/path/to/user/classes";
+
+    String expectedClasspath = launcherCommands.getGemFireJarPath().concat(File.pathSeparator).concat(
+        userClasspath).concat(File.pathSeparator).concat(
+        LauncherLifecycleCommands.CORE_DEPENDENCIES_JAR_PATHNAME).concat(File.pathSeparator).concat(
+        toPath(launcherCommands.getSpringJars().toArray()));
+
+    String actualClasspath = launcherCommands.getServerClasspath(false, true, userClasspath);
+
+    assertEquals(expectedClasspath, actualClasspath);
+  }
+
+  private String toPath(Object... pathElements) {
+    String path = "";
+
+    for (Object pathElement : pathElements) {
+      path += (path.isEmpty() ? StringUtils.EMPTY_STRING : File.pathSeparator);
+      path += pathElement;
+    }
+
+    return path;
+  }
+
+  @Test
+  public void testToClasspath() {
+    final boolean EXCLUDE_SYSTEM_CLASSPATH = false;
+    final boolean INCLUDE_SYSTEM_CLASSPATH = true;
+
+    String[] jarFilePathnames = {"/path/to/user/libs/A.jar", "/path/to/user/libs/B.jar", "/path/to/user/libs/C.jar"};
+
+    String[] userClasspaths = {"/path/to/classes:/path/to/libs/1.jar:/path/to/libs/2.jar", "/path/to/ext/libs/1.jar:/path/to/ext/classes:/path/to/ext/lib/10.jar"};
+
+    String expectedClasspath = LauncherLifecycleCommands.GEMFIRE_JAR_PATHNAME.concat(File.pathSeparator).concat(
+        toClasspath(userClasspaths)).concat(File.pathSeparator).concat(toClasspath(jarFilePathnames));
+
+    assertEquals(expectedClasspath,
+        getLauncherLifecycleCommands().toClasspath(EXCLUDE_SYSTEM_CLASSPATH, jarFilePathnames, userClasspaths));
+
+    expectedClasspath = LauncherLifecycleCommands.GEMFIRE_JAR_PATHNAME.concat(File.pathSeparator).concat(
+        toClasspath(userClasspaths)).concat(File.pathSeparator).concat(System.getProperty("java.class.path")).concat(
+        File.pathSeparator).concat(toClasspath(jarFilePathnames));
+
+    assertEquals(expectedClasspath,
+        getLauncherLifecycleCommands().toClasspath(INCLUDE_SYSTEM_CLASSPATH, jarFilePathnames, userClasspaths));
+
+    expectedClasspath = LauncherLifecycleCommands.GEMFIRE_JAR_PATHNAME.concat(File.pathSeparator).concat(
+        System.getProperty("java.class.path"));
+
+    assertEquals(expectedClasspath,
+        getLauncherLifecycleCommands().toClasspath(INCLUDE_SYSTEM_CLASSPATH, null, (String[]) null));
+
+    assertEquals(LauncherLifecycleCommands.GEMFIRE_JAR_PATHNAME,
+        getLauncherLifecycleCommands().toClasspath(EXCLUDE_SYSTEM_CLASSPATH, null, (String[]) null));
+
+    assertEquals(LauncherLifecycleCommands.GEMFIRE_JAR_PATHNAME,
+        getLauncherLifecycleCommands().toClasspath(EXCLUDE_SYSTEM_CLASSPATH, new String[0], ""));
+  }
+
+  @Test
+  public void testToClassPathOrder() {
+    String userClasspathOne = "/path/to/user/lib/a.jar:/path/to/user/classes";
+    String userClasspathTwo = "/path/to/user/lib/x.jar:/path/to/user/lib/y.jar:/path/to/user/lib/z.jar";
+
+    String expectedClasspath = launcherCommands.getGemFireJarPath().concat(File.pathSeparator).concat(
+        userClasspathOne).concat(File.pathSeparator).concat(userClasspathTwo).concat(File.pathSeparator).concat(
+        System.getProperty("java.class.path")).concat(File.pathSeparator).concat(
+        LauncherLifecycleCommands.CORE_DEPENDENCIES_JAR_PATHNAME).concat(File.pathSeparator).concat(
+        LauncherLifecycleCommands.CORE_DEPENDENCIES_JAR_PATHNAME);
+
+    String actualClasspath = launcherCommands.toClasspath(true,
+        new String[]{LauncherLifecycleCommands.CORE_DEPENDENCIES_JAR_PATHNAME, LauncherLifecycleCommands.CORE_DEPENDENCIES_JAR_PATHNAME},
+        userClasspathOne, userClasspathTwo);
+
+    assertEquals(expectedClasspath, actualClasspath);
+  }
+
+  private void assertJarFileManifestClassPath(final File dependenciesJar,
+      final Collection<String> expectedJarDependencies) throws IOException {
+    JarFile dependenciesJarFile = new JarFile(dependenciesJar);
+    Manifest manifest = dependenciesJarFile.getManifest();
+
+    assertNotNull(manifest);
+
+    Attributes attributes = manifest.getMainAttributes();
+
+    assertNotNull(attributes);
+    assertTrue(attributes.containsKey(Name.CLASS_PATH));
+
+    String[] actualJarDependencies = attributes.getValue(Name.CLASS_PATH).split(" ");
+
+    assertNotNull(actualJarDependencies);
+    assertTrue(String.format("Expected the actual number of JAR dependencies to be (%1$d); but was (%2$d)!",
+        expectedJarDependencies.size(), actualJarDependencies.length),
+        actualJarDependencies.length >= expectedJarDependencies.size());
+    //assertTrue(Arrays.asList(actualJarDependencies).containsAll(expectedJarDependencies));
+
+    List<String> actualJarDependenciesList = new ArrayList<>(Arrays.asList(actualJarDependencies));
+    List<String> missingExpectedJarDependenciesList = new ArrayList<>(expectedJarDependencies.size());
+
+    for (String expectedJarDependency : expectedJarDependencies) {
+      boolean containsExpectedJar = false;
+
+      for (int index = 0, size = actualJarDependenciesList.size(); index < size; index++) {
+        if (actualJarDependenciesList.get(index).toLowerCase().contains(expectedJarDependency.toLowerCase())) {
+          actualJarDependenciesList.remove(index);
+          containsExpectedJar = true;
+          break;
+        }
+      }
+
+      if (!containsExpectedJar) {
+        missingExpectedJarDependenciesList.add(expectedJarDependency);
+      }
+    }
+
+    assertTrue(String.format(
+        "GemFire dependencies JAR file (%1$s) does not contain the expected dependencies (%2$s) in the Manifest Class-Path attribute (%3$s)!",
+        dependenciesJar, missingExpectedJarDependenciesList, attributes.getValue(Name.CLASS_PATH)),
+        missingExpectedJarDependenciesList.isEmpty());
+  }
+
+  private String toClasspath(final String... jarFilePathnames) {
+    String classpath = StringUtils.EMPTY_STRING;
+
+    if (jarFilePathnames != null) {
+      for (final String jarFilePathname : jarFilePathnames) {
+        classpath += (classpath.isEmpty() ? StringUtils.EMPTY_STRING : File.pathSeparator);
+        classpath += jarFilePathname;
+      }
+    }
+
+    return classpath;
+  }
+
+  @Test
+  public void testGetJavaPathname() {
+    assertEquals(IOUtils.appendToPath(System.getProperty("java.home"), "bin",
+        "java" + LauncherLifecycleCommands.getExecutableSuffix()),
+        getLauncherLifecycleCommands().getJdkToolPathname("java" + LauncherLifecycleCommands.getExecutableSuffix(),
+            new GemFireException() {
+            }));
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testGetJdkToolPathnameWithNullPathnames() {
+    try {
+      getLauncherLifecycleCommands().getJdkToolPathname((Stack<String>) null, new GemFireException() {
+      });
+    } catch (NullPointerException expected) {
+      assertEquals("The JDK tool executable pathnames cannot be null!", expected.getMessage());
+      throw expected;
+    }
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testGetJdkToolPathnameWithNullGemFireException() {
+    try {
+      getLauncherLifecycleCommands().getJdkToolPathname(new Stack<String>(), null);
+    } catch (NullPointerException expected) {
+      assertEquals("The GemFireException cannot be null!", expected.getMessage());
+      throw expected;
+    }
+  }
+
+  @Test
+  public void testGetJdkToolPathnameForNonExistingTool() {
+    try {
+      final GemFireException expected = new GemFireException() {
+        @Override
+        public String getMessage() {
+          return "expected";
+        }
+      };
+
+      getLauncherLifecycleCommands().getJdkToolPathname("nonExistingTool.exe", expected);
+    } catch (GemFireException expected) {
+      assertEquals("expected", expected.getMessage());
+    }
+  }
+
+  @Test
+  public void testGetLocatorId() {
+    assertEquals("tidepool[11235]", getLauncherLifecycleCommands().getLocatorId("tidepool", 11235));
+    assertEquals("tidepool.gemstone.com[11235]",
+        getLauncherLifecycleCommands().getLocatorId("tidepool.gemstone.com", 11235));
+    assertEquals("tidepool[" + DistributionLocator.DEFAULT_LOCATOR_PORT + "]",
+        getLauncherLifecycleCommands().getLocatorId("tidepool", null));
+  }
+
+  @Test
+  public void testGetServerId() {
+    assertEquals("tidepool[12480]", getLauncherLifecycleCommands().getServerId("tidepool", 12480));
+    assertEquals("tidepool.vmware.com[12480]",
+        getLauncherLifecycleCommands().getServerId("tidepool.vmware.com", 12480));
+    assertEquals("tidepool[" + CacheServer.DEFAULT_PORT + "]",
+        getLauncherLifecycleCommands().getServerId("tidepool", null));
+  }
+
+  @Test
+  public void testCreateJmxServerUrlWithMemberName() {
+    assertEquals("service:jmx:rmi://localhost:8192/jndi/rmi://localhost:8192/jmxrmi",
+        getLauncherLifecycleCommands().getJmxServiceUrlAsString("localhost[8192]"));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testCreateJmxServiceUrlWithInvalidMemberName() {
+    try {
+      System.err.println(getLauncherLifecycleCommands().getJmxServiceUrlAsString("memberOne[]"));
+    } catch (IllegalArgumentException expected) {
+      assertEquals(CliStrings.START_JCONSOLE__CONNECT_BY_MEMBER_NAME_ID_ERROR_MESSAGE, expected.getMessage());
+      throw expected;
+    }
+  }
+
+  @Test
+  public void testCreateServerCommandLine() throws Exception {
+    ServerLauncher serverLauncher = new ServerLauncher.Builder().setCommand(
+        ServerLauncher.Command.START).setDisableDefaultServer(true).setMemberName(
+        "testCreateServerCommandLine").setRebalance(true)
+        //.setServerBindAddress("localhost")
+        .setServerPort(41214).setCriticalHeapPercentage(95.5f).setEvictionHeapPercentage(85.0f).build();
+
+    String[] commandLineElements = launcherCommands.createStartServerCommandLine(serverLauncher, null, null,
+        new Properties(), null, false, new String[0], false, null, null);
+
+    assertNotNull(commandLineElements);
+    assertTrue(commandLineElements.length > 0);
+
+    Set<String> expectedCommandLineElements = new HashSet<>(6);
+
+    expectedCommandLineElements.add(serverLauncher.getCommand().getName());
+    expectedCommandLineElements.add("--disable-default-server");
+    expectedCommandLineElements.add(serverLauncher.getMemberName().toLowerCase());
+    expectedCommandLineElements.add("--rebalance");
+    //expectedCommandLineElements.add(String.format("--server-bind-address=%1$s", serverLauncher.getServerBindAddress().getHostName()));
+    expectedCommandLineElements.add(String.format("--server-port=%1$d", serverLauncher.getServerPort()));
+    expectedCommandLineElements.add(
+        String.format("--critical-heap-percentage=%1$s", serverLauncher.getCriticalHeapPercentage()));
+    expectedCommandLineElements.add(
+        String.format("--eviction-heap-percentage=%1$s", serverLauncher.getEvictionHeapPercentage()));
+
+    for (String commandLineElement : commandLineElements) {
+      expectedCommandLineElements.remove(commandLineElement.toLowerCase());
+    }
+
+    assertTrue(String.format("Expected ([]); but was (%1$s)", expectedCommandLineElements),
+        expectedCommandLineElements.isEmpty());
+  }
+
+}


[04/50] [abbrv] incubator-geode git commit: GEODE-606: Add some logging to war file detection

Posted by kl...@apache.org.
GEODE-606: Add some logging to war file detection


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

Branch: refs/heads/feature/GEODE-217
Commit: 5a9fdb8ac6dfde9d1822a429ce94f6af05353a86
Parents: 38dd3ed
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed Nov 25 14:53:08 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Tue Dec 1 09:54:09 2015 -0800

----------------------------------------------------------------------
 .../gemfire/management/internal/AgentUtil.java  | 56 ++++++++++----------
 1 file changed, 29 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5a9fdb8a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/AgentUtil.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/AgentUtil.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/AgentUtil.java
index e5b88ad..bb160ea 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/AgentUtil.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/AgentUtil.java
@@ -38,8 +38,6 @@ public class AgentUtil {
   private static final String LIB_WEB_API_WAR = "/lib/gemfire-web-api-";
   private static final String TOOLS_WEB_WAR = "/tools/Extensions/gemfire-web-";
   private static final String LIB_WEB_WAR = "/lib/gemfire-web-";
-  private static final String TOOLS_API_WAR = "/tools/Extensions/gemfire-api-";
-  private static final String LIB_API_WAR = "/lib/gemfire-api-";
   private static final String TOOLS_PULSE_WAR = "/tools/Pulse/pulse.war";
   private static final String LIB_PULSE_WAR = "/lib/pulse.war";
   public static final String ERROR_VARIABLE_NOT_SET = "The GEMFIRE environment variable must be set!";
@@ -53,13 +51,18 @@ public class AgentUtil {
   public String getGemFireWebApiWarLocation() {
     String gemfireHome = getGemFireHome();
     assert !StringUtils.isBlank(gemfireHome) : ERROR_VARIABLE_NOT_SET;
-    logger.warn(gemfireHome + TOOLS_WEB_API_WAR + gemfireVersion + ".war");
 
-    if (new File(gemfireHome + TOOLS_WEB_API_WAR + gemfireVersion + ".war").isFile()) {
-      return gemfireHome + TOOLS_WEB_API_WAR + gemfireVersion + ".war";
-    } else if (new File(gemfireHome + LIB_WEB_API_WAR + gemfireVersion + ".war").isFile()) {
-      return gemfireHome + LIB_WEB_API_WAR + gemfireVersion + ".war";
+    String toolsWebApiWar = gemfireHome + TOOLS_WEB_API_WAR + gemfireVersion + ".war";
+    String libWebApiWar = gemfireHome + LIB_WEB_API_WAR + gemfireVersion + ".war";
+
+    if (new File(toolsWebApiWar).isFile()) {
+      logger.info("GemFire Dev REST API war: {}", toolsWebApiWar);
+      return toolsWebApiWar;
+    } else if (new File(libWebApiWar).isFile()) {
+      logger.info("GemFire Dev REST API war: {}", libWebApiWar);
+      return libWebApiWar;
     } else {
+      logger.warn("GemFire Dev REST API war not found - neither {} or {} exist", toolsWebApiWar, libWebApiWar);
       return null;
     }
   }
@@ -74,24 +77,17 @@ public class AgentUtil {
     String gemfireHome = getGemFireHome();
     assert !StringUtils.isBlank(gemfireHome) : ERROR_VARIABLE_NOT_SET;
 
-    if (new File(gemfireHome + TOOLS_WEB_WAR + gemfireVersion + ".war").isFile()) {
-      return gemfireHome + TOOLS_WEB_WAR + gemfireVersion + ".war";
-    } else if (new File(gemfireHome + LIB_WEB_WAR + gemfireVersion + ".war").isFile()) {
-      return gemfireHome + LIB_WEB_WAR + gemfireVersion + ".war";
-    } else {
-      return null;
-    }
-  }
-
-  public String getGemfireApiWarLocation() {
-    String gemfireHome = getGemFireHome();
-    assert !StringUtils.isBlank(gemfireHome) :ERROR_VARIABLE_NOT_SET;
+    String toolsWebWar = gemfireHome + TOOLS_WEB_WAR + gemfireVersion + ".war";
+    String libWebWar = gemfireHome + LIB_WEB_WAR + gemfireVersion + ".war";
 
-    if (new File(gemfireHome + TOOLS_API_WAR + gemfireVersion + ".war").isFile()) {
-      return gemfireHome + TOOLS_API_WAR + gemfireVersion + ".war";
-    } else if (new File(gemfireHome + LIB_API_WAR + gemfireVersion + ".war").isFile()) {
-      return gemfireHome + LIB_API_WAR + gemfireVersion + ".war";
+    if (new File(toolsWebWar).isFile()) {
+      logger.info("GemFire Admin REST war: {}", toolsWebWar);
+      return toolsWebWar;
+    } else if (new File(libWebWar).isFile()) {
+      logger.info("GemFire Admin REST war: {}", libWebWar);
+      return libWebWar;
     } else {
+      logger.warn("GemFire Admin REST war not found - neither {} or {} exist", toolsWebWar, libWebWar);
       return null;
     }
   }
@@ -104,11 +100,17 @@ public class AgentUtil {
     String gemfireHome = getGemFireHome();
     assert !StringUtils.isBlank(gemfireHome) : ERROR_VARIABLE_NOT_SET;
 
-    if (new File(gemfireHome + TOOLS_PULSE_WAR).isFile()) {
-      return gemfireHome + TOOLS_PULSE_WAR;
-    } else if (new File(gemfireHome + LIB_PULSE_WAR).isFile()) {
-      return gemfireHome + LIB_PULSE_WAR;
+    String toolsPulseWar = gemfireHome + TOOLS_PULSE_WAR;
+    String libPulseWar = gemfireHome + LIB_PULSE_WAR;
+
+    if (new File(toolsPulseWar).isFile()) {
+      logger.info("GemFire Pulse war: {}", toolsPulseWar);
+      return toolsPulseWar;
+    } else if (new File(libPulseWar).isFile()) {
+      logger.info("GemFire Pulse war: {}", libPulseWar);
+      return libPulseWar;
     } else {
+      logger.warn("GemFire Pulse war not found - neither {} or {} exist", toolsPulseWar, libPulseWar);
       return null;
     }
   }


[50/50] [abbrv] incubator-geode git commit: Fix JDK 1.8 inference problems. Remove unused class.

Posted by kl...@apache.org.
Fix JDK 1.8 inference problems. Remove unused class.


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

Branch: refs/heads/feature/GEODE-217
Commit: 0e62bc04b4fce84e9310103403cc0f37aa5756ae
Parents: edc29d0
Author: Kirk Lund <kl...@pivotal.io>
Authored: Fri Dec 11 13:21:01 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Fri Dec 11 13:21:01 2015 -0800

----------------------------------------------------------------------
 .../distributed/DistributedMemberDUnitTest.java |  56 +++++-----
 .../gemfire/test/dunit/DUnitTestRule.java       |  21 ++--
 .../test/dunit/DistributedSystemSupport.java    | 102 -------------------
 .../gemfire/test/dunit/DistributedTestCase.java |  21 ++--
 4 files changed, 57 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0e62bc04/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedMemberDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedMemberDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedMemberDUnitTest.java
index b3bb599..94abc83 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedMemberDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedMemberDUnitTest.java
@@ -16,13 +16,20 @@
  */
 package com.gemstone.gemfire.distributed;
 
-import static com.gemstone.gemfire.test.dunit.DUnitTestRule.*;
-import static com.googlecode.catchexception.CatchException.*;
-import static com.jayway.awaitility.Awaitility.*;
-import static com.jayway.awaitility.Duration.*;
-import static java.util.concurrent.TimeUnit.*;
-import static org.assertj.core.api.StrictAssertions.*;
-import static org.hamcrest.Matchers.*;
+import static com.gemstone.gemfire.test.dunit.DUnitTestRule.disconnectAllFromDS;
+import static com.gemstone.gemfire.test.dunit.DUnitTestRule.getSystem;
+import static com.googlecode.catchexception.CatchException.catchException;
+import static com.googlecode.catchexception.CatchException.caughtException;
+import static com.jayway.awaitility.Awaitility.with;
+import static com.jayway.awaitility.Duration.TWO_HUNDRED_MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.assertj.core.api.StrictAssertions.assertThat;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isEmptyString;
+import static org.hamcrest.Matchers.isIn;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -30,12 +37,6 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.gemstone.gemfire.IncompatibleSystemException;
-import com.gemstone.gemfire.distributed.internal.DM;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
-
 import java.io.Serializable;
 import java.net.InetAddress;
 import java.util.Arrays;
@@ -54,14 +55,19 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import com.gemstone.gemfire.test.junit.categories.MembershipTest;
+import com.gemstone.gemfire.IncompatibleSystemException;
+import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.test.dunit.DUnitTestRule;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.RMIException;
 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.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.MembershipTest;
 
 /**
  * Tests the functionality of the {@link DistributedMember} class.
@@ -109,9 +115,9 @@ public class DistributedMemberDUnitTest implements Serializable {
     final InternalDistributedMember member = system.getDistributedMember();
 
     // assert
-    assertThat(system.getConfig().getRoles(), is(DistributionConfig.DEFAULT_ROLES));
-    assertThat(system.getConfig().getGroups(), is(DistributionConfig.DEFAULT_ROLES));
-    assertThat(system.getConfig().getName(), is(DistributionConfig.DEFAULT_NAME));
+    assertThat(system.getConfig().getRoles(), equalTo(DistributionConfig.DEFAULT_ROLES));
+    assertThat(system.getConfig().getGroups(), equalTo(DistributionConfig.DEFAULT_ROLES));
+    assertThat(system.getConfig().getName(), equalTo(DistributionConfig.DEFAULT_NAME));
     
     assertThat(member.getRoles(), is(empty()));
     assertThat(member.getName(), isEmptyString());
@@ -129,8 +135,8 @@ public class DistributedMemberDUnitTest implements Serializable {
     final InternalDistributedMember member = system.getDistributedMember();
 
     // assert
-    assertThat(system.getConfig().getName(), is("nondefault"));
-    assertThat(member.getName(), is("nondefault"));
+    assertThat(system.getConfig().getName(), equalTo("nondefault"));
+    assertThat(member.getName(), equalTo("nondefault"));
   }
 
   /**
@@ -153,13 +159,13 @@ public class DistributedMemberDUnitTest implements Serializable {
     final InternalDistributedMember member = system.getDistributedMember();
 
     // assert
-    assertThat(system.getConfig().getRoles(), is(roles));
-    assertThat(system.getConfig().getGroups(), is(groups));
-    assertThat(member.getRoles().size(), is(rolesAndGroups.size()));
+    assertThat(system.getConfig().getRoles(), equalTo(roles));
+    assertThat(system.getConfig().getGroups(), equalTo(groups));
+    assertThat(member.getRoles().size(), equalTo(rolesAndGroups.size()));
     for (Role role : member.getRoles()) {
       assertThat(role.getName(), isIn(rolesAndGroups));
     }
-    assertThat(member.getGroups(), is(rolesAndGroups));
+    assertThat(member.getGroups(), equalTo(rolesAndGroups));
   }
 
   @Test
@@ -364,7 +370,7 @@ public class DistributedMemberDUnitTest implements Serializable {
     final DistributedMember member = system.getDistributedMember();
     
     // assert
-    assertThat(system.getMemberId(), is(member.getId()));
+    assertThat(system.getMemberId(), equalTo(member.getId()));
     assertThat(member.getId(), containsString("foobar"));
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0e62bc04/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java
index e994167..a66fad5 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java
@@ -22,6 +22,7 @@ import com.gemstone.gemfire.admin.internal.AdminDistributedSystemImpl;
 import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HoplogConfig;
 import com.gemstone.gemfire.cache.query.QueryTestUtils;
 import com.gemstone.gemfire.cache.query.internal.QueryObserverHolder;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.GlobalLockingDUnitTest;
 import com.gemstone.gemfire.cache30.MultiVMRegionTestCase;
 import com.gemstone.gemfire.cache30.RegionTestCase;
@@ -35,19 +36,18 @@ import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.InternalInstantiator;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.admin.ClientStatsManager;
+import com.gemstone.gemfire.internal.cache.DiskStoreObserver;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.InitialImageOperation;
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.tier.sockets.DataSerializerPropogationDUnitTest;
+import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.management.internal.cli.LogWrapper;
 import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 import com.gemstone.gemfire.test.junit.rules.SerializableTestRule;
-import com.gemstone.org.jgroups.stack.IpAddress;
-import com.gemstone.org.jgroups.stack.Protocol;
-import com.gemstone.org.jgroups.util.GemFireTracer;
 
 /**
  * TestRule for DistributedTest. Use this for DUnit tests using JUnit 4.x 
@@ -678,8 +678,10 @@ public class DUnitTestRule implements SerializableTestRule {
   }
   
   private static void cleanupThisVM() {
-    IpAddress.resolve_dns = true;
     SocketCreator.resolve_dns = true;
+    CacheCreation.clearThreadLocals();
+    System.clearProperty("gemfire.log-level");
+    System.clearProperty("jgroups.resolve_dns");
     InitialImageOperation.slowImageProcessing = 0;
     DistributionMessageObserver.setInstance(null);
     QueryTestUtils.setCache(null);
@@ -689,13 +691,16 @@ public class DUnitTestRule implements SerializableTestRule {
     LogWrapper.close();
     ClientProxyMembershipID.system = null;
     MultiVMRegionTestCase.CCRegion = null;
-    InternalBridgeMembership.unregisterAllListeners();
+    InternalClientMembership.unregisterAllListeners();
     ClientStatsManager.cleanupForTests();
+    ClientServerTestCase.AUTO_LOAD_BALANCE = false;
     unregisterInstantiatorsInThisVM();
-    GemFireTracer.DEBUG = Boolean.getBoolean("DistributionManager.DEBUG_JAVAGROUPS");
-    Protocol.trace = GemFireTracer.DEBUG;
     DistributionMessageObserver.setInstance(null);
     QueryObserverHolder.reset();
+    DiskStoreObserver.setInstance(null);
+    System.clearProperty("gemfire.log-level");
+    System.clearProperty("jgroups.resolve_dns");
+    
     if (InternalDistributedSystem.systemAttemptingReconnect != null) {
       InternalDistributedSystem.systemAttemptingReconnect.stopReconnecting();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0e62bc04/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedSystemSupport.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedSystemSupport.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedSystemSupport.java
deleted file mode 100755
index 9c52e46..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedSystemSupport.java
+++ /dev/null
@@ -1,102 +0,0 @@
-package com.gemstone.gemfire.test.dunit;
-
-import static com.gemstone.gemfire.test.dunit.Wait.waitForCriterion;
-
-import java.io.File;
-
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.distributed.internal.membership.jgroup.MembershipManagerHelper;
-import com.gemstone.org.jgroups.Event;
-import com.gemstone.org.jgroups.JChannel;
-import com.gemstone.org.jgroups.stack.Protocol;
-
-public class DistributedSystemSupport {
-
-  protected DistributedSystemSupport() {
-  }
-  
-  /**
-   * Crash the cache in the given VM in such a way that it immediately stops communicating with
-   * peers.  This forces the VM's membership manager to throw a ForcedDisconnectException by
-   * forcibly terminating the JGroups protocol stack with a fake EXIT event.<p>
-   * 
-   * NOTE: if you use this method be sure that you clean up the VM before the end of your
-   * test with disconnectFromDS() or disconnectAllFromDS().
-   */
-  public static boolean crashDistributedSystem(VM vm) { // TODO: move
-    return (Boolean)vm.invoke(new SerializableCallable("crash distributed system") {
-      public Object call() throws Exception {
-        DistributedSystem msys = InternalDistributedSystem.getAnyInstance();
-        crashDistributedSystem(msys);
-        return true;
-      }
-    });
-  }
-  
-  /**
-   * Crash the cache in the given VM in such a way that it immediately stops communicating with
-   * peers.  This forces the VM's membership manager to throw a ForcedDisconnectException by
-   * forcibly terminating the JGroups protocol stack with a fake EXIT event.<p>
-   * 
-   * NOTE: if you use this method be sure that you clean up the VM before the end of your
-   * test with disconnectFromDS() or disconnectAllFromDS().
-   */
-  public static void crashDistributedSystem(final DistributedSystem msys) { // TODO: move
-    MembershipManagerHelper.inhibitForcedDisconnectLogging(true);
-    MembershipManagerHelper.playDead(msys);
-    JChannel c = MembershipManagerHelper.getJChannel(msys);
-    Protocol udp = c.getProtocolStack().findProtocol("UDP");
-    udp.stop();
-    udp.passUp(new Event(Event.EXIT, new RuntimeException("killing member's ds")));
-    try {
-      MembershipManagerHelper.getJChannel(msys).waitForClose();
-    }
-    catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      // attempt rest of work with interrupt bit set
-    }
-    MembershipManagerHelper.inhibitForcedDisconnectLogging(false);
-    WaitCriterion wc = new WaitCriterion() {
-      public boolean done() {
-        return !msys.isConnected();
-      }
-      public String description() {
-        return "waiting for distributed system to finish disconnecting: " + msys;
-      }
-    };
-//    try {
-      waitForCriterion(wc, 10000, 1000, true);
-//    } finally {
-//      dumpMyThreads(getLogWriter());
-//    }
-  }
-
-  /** get the host name to use for a server cache in client/server dunit
-   * testing
-   * @param host
-   * @return the host name
-   */
-  public static String getServerHostName(Host host) {
-    return System.getProperty("gemfire.server-bind-address") != null?
-        System.getProperty("gemfire.server-bind-address")
-        : host.getHostName();
-  }
-
-  /** 
-   * Delete locator state files.  Use this after getting a random port
-   * to ensure that an old locator state file isn't picked up by the
-   * new locator you're starting.
-   * 
-   * @param ports
-   */
-  public static void deleteLocatorStateFile(final int... ports) {
-    for (int i=0; i<ports.length; i++) {
-      final File stateFile = new File("locator"+ports[i]+"state.dat");
-      if (stateFile.exists()) {
-        stateFile.delete();
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0e62bc04/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
index def74b8..2a63ee3 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
@@ -19,6 +19,7 @@ import com.gemstone.gemfire.admin.internal.AdminDistributedSystemImpl;
 import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HoplogConfig;
 import com.gemstone.gemfire.cache.query.QueryTestUtils;
 import com.gemstone.gemfire.cache.query.internal.QueryObserverHolder;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.GlobalLockingDUnitTest;
 import com.gemstone.gemfire.cache30.MultiVMRegionTestCase;
 import com.gemstone.gemfire.cache30.RegionTestCase;
@@ -32,19 +33,18 @@ import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.InternalInstantiator;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.admin.ClientStatsManager;
+import com.gemstone.gemfire.internal.cache.DiskStoreObserver;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.InitialImageOperation;
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.tier.sockets.DataSerializerPropogationDUnitTest;
+import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LogWriterLogger;
 import com.gemstone.gemfire.management.internal.cli.LogWrapper;
-import com.gemstone.org.jgroups.stack.IpAddress;
-import com.gemstone.org.jgroups.stack.Protocol;
-import com.gemstone.org.jgroups.util.GemFireTracer;
 
 import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 
@@ -609,8 +609,10 @@ public abstract class DistributedTestCase implements java.io.Serializable {
   }
 
   private static void cleanupThisVM() {
-    IpAddress.resolve_dns = true;
     SocketCreator.resolve_dns = true;
+    CacheCreation.clearThreadLocals();
+    System.clearProperty("gemfire.log-level");
+    System.clearProperty("jgroups.resolve_dns");
     InitialImageOperation.slowImageProcessing = 0;
     DistributionMessageObserver.setInstance(null);
     QueryTestUtils.setCache(null);
@@ -620,13 +622,16 @@ public abstract class DistributedTestCase implements java.io.Serializable {
     LogWrapper.close();
     ClientProxyMembershipID.system = null;
     MultiVMRegionTestCase.CCRegion = null;
-    InternalBridgeMembership.unregisterAllListeners();
+    InternalClientMembership.unregisterAllListeners();
     ClientStatsManager.cleanupForTests();
+    ClientServerTestCase.AUTO_LOAD_BALANCE = false;
     unregisterInstantiatorsInThisVM();
-    GemFireTracer.DEBUG = Boolean.getBoolean("DistributionManager.DEBUG_JAVAGROUPS");
-    Protocol.trace = GemFireTracer.DEBUG;
     DistributionMessageObserver.setInstance(null);
     QueryObserverHolder.reset();
+    DiskStoreObserver.setInstance(null);
+    System.clearProperty("gemfire.log-level");
+    System.clearProperty("jgroups.resolve_dns");
+    
     if (InternalDistributedSystem.systemAttemptingReconnect != null) {
       InternalDistributedSystem.systemAttemptingReconnect.stopReconnecting();
     }


[11/50] [abbrv] incubator-geode git commit: GEODE-621: Added BucketCountLoadProbe to sanctionedDataSerializables

Posted by kl...@apache.org.
GEODE-621: Added BucketCountLoadProbe to sanctionedDataSerializables

Fixing a failing in AnalyzeSerializables due to the changes for
GEODE-581


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

Branch: refs/heads/feature/GEODE-217
Commit: ae8c475de698e87305092aa711be953ee2daafb0
Parents: b7030d1
Author: Dan Smith <up...@apache.org>
Authored: Wed Dec 2 10:07:33 2015 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Wed Dec 2 10:07:33 2015 -0800

----------------------------------------------------------------------
 .../gemfire/codeAnalysis/sanctionedDataSerializables.txt         | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ae8c475d/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
index af0ee36..d553ad9 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
@@ -1569,6 +1569,10 @@ com/gemstone/gemfire/internal/cache/partitioned/BucketBackupMessage,2
 fromData,16,2a2bb7001d2a2bb9001e0100b50003b1
 toData,16,2a2bb7001f2b2ab40003b900200200b1
 
+com/gemstone/gemfire/internal/cache/partitioned/BucketCountLoadProbe,2
+fromData,1,b1
+toData,1,b1
+
 com/gemstone/gemfire/internal/cache/partitioned/BucketProfileUpdateMessage,2
 fromData,47,2a2bb700232a2bb900240100b500052a2bb900240100b500062a2bb900240100b500022a2bb80025c00026b50007b1
 toData,44,2a2bb700272b2ab40005b9002802002b2ab40006b9002802002b2ab40002b9002802002ab400072bb80029b1


[07/50] [abbrv] incubator-geode git commit: GEODE-581: LoadProbe that balances based on bucket count

Posted by kl...@apache.org.
GEODE-581: LoadProbe that balances based on bucket count

Adding a load probe that balances partitioned regions based on bucket
count, rather than on the amount of data in bytes on each node.


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

Branch: refs/heads/feature/GEODE-217
Commit: 442faa06f3b3a7cb85065ee5b0585ed679c80ba0
Parents: bff59d1
Author: Dan Smith <ds...@pivotal.io>
Authored: Thu Dec 2 19:44:38 2010 +0000
Committer: Dan Smith <up...@apache.org>
Committed: Tue Dec 1 13:21:48 2015 -0800

----------------------------------------------------------------------
 .../gemstone/gemfire/internal/DSFIDFactory.java |   2 +
 .../internal/DataSerializableFixedID.java       |   3 +-
 .../cache/control/InternalResourceManager.java  |  11 +-
 .../cache/partitioned/BucketCountLoadProbe.java |  75 ++++++++++
 .../control/RebalanceOperationDUnitTest.java    | 141 ++++++++++++++++++-
 5 files changed, 227 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/442faa06/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
index 24dd181..b77dfdb 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
@@ -302,6 +302,7 @@ import com.gemstone.gemfire.internal.cache.partitioned.AllBucketProfilesUpdateMe
 import com.gemstone.gemfire.internal.cache.partitioned.BecomePrimaryBucketMessage;
 import com.gemstone.gemfire.internal.cache.partitioned.BecomePrimaryBucketMessage.BecomePrimaryBucketReplyMessage;
 import com.gemstone.gemfire.internal.cache.partitioned.BucketBackupMessage;
+import com.gemstone.gemfire.internal.cache.partitioned.BucketCountLoadProbe;
 import com.gemstone.gemfire.internal.cache.partitioned.BucketProfileUpdateMessage;
 import com.gemstone.gemfire.internal.cache.partitioned.BucketSizeMessage;
 import com.gemstone.gemfire.internal.cache.partitioned.BucketSizeMessage.BucketSizeReplyMessage;
@@ -1052,6 +1053,7 @@ public final class DSFIDFactory implements DataSerializableFixedID {
         DestroyRegionOnDataStoreMessage.class);
     registerDSFID(SHUTDOWN_ALL_GATEWAYHUBS_REQUEST,
         ShutdownAllGatewayHubsRequest.class);
+    registerDSFID(BUCKET_COUNT_LOAD_PROBE, BucketCountLoadProbe.class);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/442faa06/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
index 8dd8f00..7b263bf 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
@@ -746,8 +746,9 @@ public interface DataSerializableFixedID extends SerializationVersions {
   public static final short SERIAL_ACKED_MESSAGE = 2001;
   public static final short CLIENT_DATASERIALIZER_MESSAGE=2002;
   
-  //2003..2099 unused
+  //2003..2098 unused
   
+  public static final short BUCKET_COUNT_LOAD_PROBE = 2099;
   public static final short PERSISTENT_MEMBERSHIP_VIEW_REQUEST =2100;
   public static final short PERSISTENT_MEMBERSHIP_VIEW_REPLY = 2101;
   public static final short PERSISTENT_STATE_QUERY_REQUEST = 2102;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/442faa06/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/InternalResourceManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/InternalResourceManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/InternalResourceManager.java
index 98e1f25..ef9c502 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/InternalResourceManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/InternalResourceManager.java
@@ -89,7 +89,7 @@ public class InternalResourceManager implements ResourceManager {
 
   final GemFireCacheImpl cache;
   
-  private final LoadProbe loadProbe;
+  private LoadProbe loadProbe;
   
   private final ResourceManagerStats stats;
   private final ResourceAdvisor resourceAdvisor;
@@ -545,6 +545,15 @@ public class InternalResourceManager implements ResourceManager {
     return this.loadProbe;
   }
 
+  /**
+   * This method is test purposes only.
+   */
+  public LoadProbe setLoadProbe(LoadProbe probe) {
+    LoadProbe old = this.loadProbe;
+    this.loadProbe = probe;
+    return old;
+  }
+
   /* (non-Javadoc)
    * @see com.gemstone.gemfire.cache.control.ResourceManager#setEvictionHeapPercentage(int)
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/442faa06/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/BucketCountLoadProbe.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/BucketCountLoadProbe.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/BucketCountLoadProbe.java
new file mode 100644
index 0000000..07c9a1f
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/BucketCountLoadProbe.java
@@ -0,0 +1,75 @@
+/*
+ * 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.partitioned;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.cache.BucketAdvisor;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
+
+/**
+ * A load probe which calculates the load of a pr using
+ * the just the number of buckets on a member.
+ * 
+ */
+public class BucketCountLoadProbe implements LoadProbe, DataSerializableFixedID {
+  private static final long serialVersionUID = 7040814060882774875L;
+
+  public PRLoad getLoad(PartitionedRegion pr) {
+    PartitionedRegionDataStore ds = pr.getDataStore();
+    int configuredBucketCount = pr.getTotalNumberOfBuckets();
+    PRLoad prLoad = new PRLoad(
+        configuredBucketCount, pr.getLocalMaxMemory());
+    
+    // key: bid, value: size
+    for(Integer bidInt : ds.getAllLocalBucketIds()) {
+      int bid = bidInt.intValue();
+      
+      BucketAdvisor bucketAdvisor = pr.getRegionAdvisor().
+      getBucket(bid).getBucketAdvisor();
+      //Wait for a primary to exist for this bucket, because
+      //it might be this member.
+      bucketAdvisor.getPrimary();
+      boolean isPrimary = pr.getRegionAdvisor().
+          getBucket(bid).getBucketAdvisor().isPrimary();
+      prLoad.addBucket(bid, 1, isPrimary ? 1 : 0);
+    }
+    
+    return prLoad;
+  }
+
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+  }
+
+  public void toData(DataOutput out) throws IOException {
+  }
+
+  public int getDSFID() {
+    return BUCKET_COUNT_LOAD_PROBE;
+  }
+
+  @Override
+  public Version[] getSerializationVersions() {
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/442faa06/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
index 9029b8d..88516fe 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
@@ -43,6 +43,7 @@ import com.gemstone.gemfire.cache.DiskStoreFactory;
 import com.gemstone.gemfire.cache.EntryEvent;
 import com.gemstone.gemfire.cache.EvictionAction;
 import com.gemstone.gemfire.cache.EvictionAttributes;
+import com.gemstone.gemfire.cache.GemFireCache;
 import com.gemstone.gemfire.cache.LoaderHelper;
 import com.gemstone.gemfire.cache.PartitionAttributes;
 import com.gemstone.gemfire.cache.PartitionAttributesFactory;
@@ -66,9 +67,12 @@ import com.gemstone.gemfire.internal.cache.BucketRegion;
 import com.gemstone.gemfire.internal.cache.ColocationHelper;
 import com.gemstone.gemfire.internal.cache.DiskStoreImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.InternalCache;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.control.InternalResourceManager.ResourceObserverAdapter;
+import com.gemstone.gemfire.internal.cache.partitioned.BucketCountLoadProbe;
+import com.gemstone.gemfire.internal.cache.partitioned.LoadProbe;
 
 import dunit.AsyncInvocation;
 import dunit.Host;
@@ -2810,13 +2814,144 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
     moveBucketsWithUnrecoveredValuesRedundancy(false);
   }
   
+  public void testBalanceBucketsByCountSimulation() {
+    balanceBucketsByCount(true);
+  }
+  
+  public void testBalanceBucketsByCount() {
+    balanceBucketsByCount(false);
+  }
+  
   /**
    * Check to make sure that we balance
-   * buckets between two hosts with no redundancy,
-   * 
-   * even if the values have not yet been faulted in from disk.
+   * buckets between two hosts with no redundancy.
    * @param simulate
    */
+  public void balanceBucketsByCount(final boolean simulate) {
+    Host host = Host.getHost(0);
+    VM vm0 = host.getVM(0);
+    VM vm1 = host.getVM(1);
+
+    LoadProbe oldProbe = setLoadProbe(vm0, new BucketCountLoadProbe());
+    try {
+      SerializableRunnable createPrRegion = new SerializableRunnable("createRegion") {
+        public void run()
+        {
+          Cache cache = getCache();
+          AttributesFactory attr = new AttributesFactory();
+          PartitionAttributesFactory paf = new PartitionAttributesFactory();
+          paf.setRedundantCopies(0);
+          paf.setRecoveryDelay(-1);
+          paf.setStartupRecoveryDelay(-1);
+          PartitionAttributes prAttr = paf.create();
+          attr.setPartitionAttributes(prAttr);
+          attr.setCacheLoader(new Bug40228Loader());
+          cache.createRegion("region1", attr.create());
+        }
+      };
+
+      //Create the region in only 1 VM
+      vm0.invoke(createPrRegion);
+
+      //Create some buckets with very uneven sizes
+      vm0.invoke(new SerializableRunnable("createSomeBuckets") {
+
+        public void run() {
+          Cache cache = getCache();
+          Region region = cache.getRegion("region1");
+          region.put(Integer.valueOf(1), new byte[1024 * 1024]);
+          region.put(Integer.valueOf(2), "A");
+          region.put(Integer.valueOf(3), "A");
+          region.put(Integer.valueOf(4), "A");
+          region.put(Integer.valueOf(5), "A");
+          region.put(Integer.valueOf(6), "A");
+        }
+      });
+
+      //Create the region in the other VM (should have no effect)
+      vm1.invoke(createPrRegion);
+
+      //Now simulate a rebalance
+      vm0.invoke(new SerializableRunnable("simulateRebalance") {
+
+        public void run() {
+          Cache cache = getCache();
+          ResourceManager manager = cache.getResourceManager();
+          RebalanceResults results = doRebalance(simulate, manager);
+          assertEquals(0, results.getTotalBucketCreatesCompleted());
+          assertEquals(0, results.getTotalPrimaryTransfersCompleted());
+          assertEquals(3, results.getTotalBucketTransfersCompleted());
+          assertTrue(0 < results.getTotalBucketTransferBytes());
+          Set<PartitionRebalanceInfo> detailSet = results.getPartitionRebalanceDetails();
+          assertEquals(1, detailSet.size());
+          PartitionRebalanceInfo details = detailSet.iterator().next();
+          assertEquals(0, details.getBucketCreatesCompleted());
+          assertEquals(0, details.getPrimaryTransfersCompleted());
+          assertTrue(0 < details.getBucketTransferBytes());
+          assertEquals(3, details.getBucketTransfersCompleted());
+
+          Set<PartitionMemberInfo> afterDetails = details.getPartitionMemberDetailsAfter();
+          assertEquals(2, afterDetails.size());
+          for(PartitionMemberInfo memberDetails: afterDetails) {
+            assertEquals(3, memberDetails.getBucketCount());
+            assertEquals(3, memberDetails.getPrimaryCount());
+          }
+          if(!simulate) {
+            verifyStats(manager, results);
+          }
+        }
+      });
+
+      if(!simulate) {
+        SerializableRunnable checkRedundancyFixed = new SerializableRunnable("checkRedundancyFixed") {
+
+          public void run() {
+            Cache cache = getCache();
+            Region region = cache.getRegion("region1");
+            PartitionRegionInfo details = PartitionRegionHelper.getPartitionRegionInfo(region);
+            assertEquals(6, details.getCreatedBucketCount());
+            assertEquals(0,details.getActualRedundantCopies());
+            assertEquals(0,details.getLowRedundancyBucketCount());
+            assertEquals(2, details.getPartitionMemberInfo().size());
+            for(PartitionMemberInfo memberDetails: details.getPartitionMemberInfo()) {
+              assertEquals(3, memberDetails.getBucketCount());
+              assertEquals(3, memberDetails.getPrimaryCount());
+            }
+
+            //check to make sure that moving buckets didn't close the cache loader
+            Bug40228Loader loader = (Bug40228Loader) cache.getRegion("region1").getAttributes().getCacheLoader();
+            assertFalse(loader.isClosed());
+          }
+        };
+
+        vm0.invoke(checkRedundancyFixed);
+        vm1.invoke(checkRedundancyFixed);
+      }
+    } finally {
+      setLoadProbe(vm0, oldProbe);
+    }
+  }
+  
+  private LoadProbe setLoadProbe(VM vm, final LoadProbe probe) {
+    LoadProbe oldProbe = (LoadProbe) vm.invoke(new SerializableCallable("set load probe") {
+      
+      public Object call() {
+        GemFireCacheImpl cache = (GemFireCacheImpl) getCache();
+        InternalResourceManager mgr = cache.getResourceManager();
+        return mgr.setLoadProbe(probe);
+      }
+    });
+    
+    return oldProbe;
+  }
+
+  /** 
+   * Test to ensure that we wait for
+   * in progress write operations before moving a primary.
+   * @throws InterruptedException 
+   * @throws CancellationException 
+   * @throws TimeoutException 
+   */
   public void moveBucketsWithUnrecoveredValuesRedundancy(final boolean simulate) {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);