You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ab...@apache.org on 2016/02/08 18:03:16 UTC

[01/50] [abbrv] incubator-geode git commit: GEODE-708: Add stats for Geode membership health monitor

Repository: incubator-geode
Updated Branches:
  refs/heads/master 690246593 -> 008309dd6


GEODE-708: Add stats for Geode membership health monitor


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

Branch: refs/heads/master
Commit: 91b438971a33b444c8de826bb18a3fc2cff4f8b4
Parents: 81eafcc
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Tue Dec 29 09:38:43 2015 -0800
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Tue Dec 29 09:38:43 2015 -0800

----------------------------------------------------------------------
 .../gemfire/distributed/internal/DMStats.java   |  80 +++++++
 .../distributed/internal/DistributionStats.java | 237 +++++++++++++++++++
 .../internal/LonerDistributionManager.java      |  64 +++++
 .../membership/gms/fd/GMSHealthMonitor.java     |  93 +++++---
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |  42 +++-
 5 files changed, 482 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/91b43897/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DMStats.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DMStats.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DMStats.java
index e79a40b..7bf5b80 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DMStats.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DMStats.java
@@ -536,4 +536,84 @@ public interface DMStats {
 
   public void endPdxInstanceDeserialization(long start);
   public void incPdxInstanceCreations();
+  
+  //Stats for GMSHealthMonitor
+  public long getHeartbeatRequestsSent();
+  
+  public void incHeartbeatRequestsSent();
+  
+  public long getHeartbeatRequestsReceived();
+  
+  public void incHeartbeatRequestsReceived();
+  
+  public long getHeartbeatsSent();
+  
+  public void incHeartbeatsSent();
+
+  public long getHeartbeatsReceived();
+  
+  public void incHeartbeatsReceived();
+  
+
+  public long getSuspectsSent();
+  
+  public void incSuspectsSent();
+
+  public long getSuspectsReceived();
+  
+  public void incSuspectsReceived();
+  
+  
+  public long getFinalCheckRequestsSent();
+  
+  public void incFinalCheckRequestsSent();
+  
+  public long getFinalCheckRequestsReceived();
+  
+  public void incFinalCheckRequestsReceived();
+  
+  public long getFinalCheckResponsesSent();
+  
+  public void incFinalCheckResponsesSent();
+  
+  public long getFinalCheckResponsesReceived();
+  
+  public void incFinalCheckResponsesReceived();
+  
+  
+  public long getTcpFinalCheckRequestsSent();
+  
+  public void incTcpFinalCheckRequestsSent();
+
+  public long getTcpFinalCheckRequestsReceived();
+  
+  public void incTcpFinalCheckRequestsReceived();
+  
+  public long getTcpFinalCheckResponsesSent();
+  
+  public void incTcpFinalCheckResponsesSent();
+
+  public long getTcpFinalCheckResponsesReceived();
+  
+  public void incTcpFinalCheckResponsesReceived();
+
+  
+  public long getUdpFinalCheckRequestsSent();
+  
+  public void incUdpFinalCheckRequestsSent();
+  
+//  UDP final check is implemented using HeartbeatRequestMessage and HeartbeatMessage
+//  So the following code is commented out.
+  
+//  public long getUdpFinalCheckRequestsReceived();
+//  
+//  public void incUdpFinalCheckRequestsReceived();
+//  
+//  public long getUdpFinalCheckResponsesSent();
+//  
+//  public void incUdpFinalCheckResponsesSent();
+
+  public long getUdpFinalCheckResponsesReceived();
+  
+  public void incUdpFinalCheckResponsesReceived();
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/91b43897/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
index 804b507..92b0fcb 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
@@ -215,6 +215,26 @@ public class DistributionStats implements DMStats {
   private final static int eldersId;
   private final static int initialImageMessagesInFlightId;
   private final static int initialImageRequestsInProgressId;
+  
+  //For GMSHealthMonitor
+  private final static int heartbeatRequestsSentId;  
+  private final static int heartbeatRequestsReceivedId;  
+  private final static int heartbeatsSentId;  
+  private final static int heartbeatsReceivedId;  
+  private final static int suspectsSentId;  
+  private final static int suspectsReceivedId;
+  private final static int finalCheckRequestsSentId;  
+  private final static int finalCheckRequestsReceivedId;  
+  private final static int finalCheckResponsesSentId;  
+  private final static int finalCheckResponsesReceivedId;  
+  private final static int tcpFinalCheckRequestsSentId; 
+  private final static int tcpFinalCheckRequestsReceivedId;  
+  private final static int tcpFinalCheckResponsesSentId;  
+  private final static int tcpFinalCheckResponsesReceivedId;
+  private final static int udpFinalCheckRequestsSentId;
+  private final static int udpFinalCheckRequestsReceivedId;
+  private final static int udpFinalCheckResponsesSentId;
+  private final static int udpFinalCheckResponsesReceivedId;
 
   static {
     String statName = "DistributionStats";
@@ -317,6 +337,33 @@ public class DistributionStats implements DMStats {
     final String initialImageMessagesInFlightDesc = "The number of messages with initial image data sent from this member that have not yet been acknowledged.";
     final String initialImageRequestsInProgressDesc = "The number of initial images this member is currently receiving.";
 
+    //For GMSHealthMonitor
+    final String heartbeatRequestsSentDesc = "The number of heartbeat request messages that this member has sent.";
+    final String heartbeatRequestsReceivedDesc = "The number of heartbeat request messages that this member has received.";
+    
+    final String heartbeatsSentDesc = "The number of heartbeat messages that this member has sent.";
+    final String heartbeatsReceivedDesc = "The number of heartbeat messages that this member has received.";
+    
+    final String suspectsSentDesc = "The number of suspect member messages that this member has sent.";
+    final String suspectsReceivedDesc = "The number of suspect member messages that this member has received.";
+    
+    final String finalCheckRequestsSentDesc = "The number of final check requests that this member has sent.";
+    final String finalCheckRequestsReceivedDesc = "The number of final check requests that this member has received.";
+    
+    final String finalCheckResponsesSentDesc = "The number of final check responses that this member has sent.";
+    final String finalCheckResponsesReceivedDesc = "The number of final check responses that this member has received.";    
+    
+    final String tcpFinalCheckRequestsSentDesc = "The number of TCP final check requests that this member has sent.";
+    final String tcpFinalCheckRequestsReceivedDesc = "The number of TCP final check requests that this member has received.";
+    
+    final String tcpFinalCheckResponsesSentDesc = "The number of TCP final check responses that this member has sent.";
+    final String tcpFinalCheckResponsesReceivedDesc = "The number of TCP final check responses that this member has received.";
+
+    final String udpFinalCheckRequestsSentDesc = "The number of UDP final checks that this member has sent.";
+    final String udpFinalCheckRequestsReceivedDesc = "The number of UDP final check requests that this member has received.";
+    
+    final String udpFinalCheckResponsesSentDesc = "The number of UDP final check responses that this member has sent.";
+    final String udpFinalCheckResponsesReceivedDesc = "The number of UDP final check responses that this member has received.";
 
     StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
 
@@ -486,6 +533,26 @@ public class DistributionStats implements DMStats {
         f.createIntGauge("elders", eldersDesc, "elders"),
         f.createIntGauge("initialImageMessagesInFlight", initialImageMessagesInFlightDesc, "messages"),
         f.createIntGauge("initialImageRequestsInProgress", initialImageRequestsInProgressDesc, "requests"),
+        
+        //For GMSHealthMonitor
+        f.createLongCounter("heartbeatRequestsSent", heartbeatRequestsSentDesc, "messages"),
+        f.createLongCounter("heartbeatRequestsReceived", heartbeatRequestsReceivedDesc, "messages"),
+        f.createLongCounter("heartbeatsSent", heartbeatsSentDesc, "messages"),
+        f.createLongCounter("heartbeatsReceived", heartbeatsReceivedDesc, "messages"),
+        f.createLongCounter("suspectsSent", suspectsSentDesc, "messages"),
+        f.createLongCounter("suspectsReceived", suspectsReceivedDesc, "messages"),
+        f.createLongCounter("finalCheckRequestsSent", finalCheckRequestsSentDesc, "messages"),
+        f.createLongCounter("finalCheckRequestsReceived", finalCheckRequestsReceivedDesc, "messages"),
+        f.createLongCounter("finalCheckResponsesSent", finalCheckResponsesSentDesc, "messages"),
+        f.createLongCounter("finalCheckResponsesReceived", finalCheckResponsesReceivedDesc, "messages"),
+        f.createLongCounter("tcpFinalCheckRequestsSent", tcpFinalCheckRequestsSentDesc, "nanoseconds", false),
+        f.createLongCounter("tcpFinalCheckRequestsReceived", tcpFinalCheckRequestsReceivedDesc, "nanoseconds", false),
+        f.createLongCounter("tcpFinalCheckResponsesSent", tcpFinalCheckResponsesSentDesc, "nanoseconds", false),
+        f.createLongCounter("tcpFinalCheckResponsesReceived", tcpFinalCheckResponsesReceivedDesc, "nanoseconds", false),
+        f.createLongCounter("udpFinalCheckRequestsSent", udpFinalCheckRequestsSentDesc, "messages"),
+        f.createLongCounter("udpFinalCheckRequestsReceived", udpFinalCheckRequestsReceivedDesc, "messages"),
+        f.createLongCounter("udpFinalCheckResponsesSent", udpFinalCheckResponsesSentDesc, "messages"),
+        f.createLongCounter("udpFinalCheckResponsesReceived", udpFinalCheckResponsesReceivedDesc, "messages"),
       }
     );
 
@@ -654,6 +721,26 @@ public class DistributionStats implements DMStats {
     eldersId = type.nameToId("elders");
     initialImageMessagesInFlightId = type.nameToId("initialImageMessagesInFlight");
     initialImageRequestsInProgressId = type.nameToId("initialImageRequestsInProgress");
+    
+    //For GMSHealthMonitor
+    heartbeatRequestsSentId = type.nameToId("heartbeatRequestsSent");
+    heartbeatRequestsReceivedId = type.nameToId("heartbeatRequestsReceived");
+    heartbeatsSentId = type.nameToId("heartbeatsSent");
+    heartbeatsReceivedId = type.nameToId("heartbeatsReceived");
+    suspectsSentId = type.nameToId("suspectsSent");
+    suspectsReceivedId = type.nameToId("suspectsReceived");
+    finalCheckRequestsSentId = type.nameToId("finalCheckRequestsSent");
+    finalCheckRequestsReceivedId = type.nameToId("finalCheckRequestsReceived");
+    finalCheckResponsesSentId = type.nameToId("finalCheckResponsesSent");
+    finalCheckResponsesReceivedId = type.nameToId("finalCheckResponsesReceived");
+    tcpFinalCheckRequestsSentId = type.nameToId("tcpFinalCheckRequestsSent");
+    tcpFinalCheckRequestsReceivedId = type.nameToId("tcpFinalCheckRequestsReceived");
+    tcpFinalCheckResponsesSentId = type.nameToId("tcpFinalCheckResponsesSent");
+    tcpFinalCheckResponsesReceivedId = type.nameToId("tcpFinalCheckResponsesReceived");
+    udpFinalCheckRequestsSentId = type.nameToId("udpFinalCheckRequestsSent");
+    udpFinalCheckRequestsReceivedId = type.nameToId("udpFinalCheckRequestsReceived");
+    udpFinalCheckResponsesSentId = type.nameToId("udpFinalCheckResponsesSent");
+    udpFinalCheckResponsesReceivedId = type.nameToId("udpFinalCheckResponsesReceived");
   }
 
   /** The Statistics object that we delegate most behavior to */
@@ -1811,4 +1898,154 @@ public class DistributionStats implements DMStats {
   public Statistics getStats(){
     return stats;
   }
+  
+  //For GMSHealthMonitor
+  public long getHeartbeatRequestsSent() {
+    return this.stats.getLong(heartbeatRequestsSentId);
+  }
+  
+  public void incHeartbeatRequestsSent() {
+    this.stats.incLong(heartbeatRequestsSentId, 1L);
+  }
+  
+  public long getHeartbeatRequestsReceived() {
+    return this.stats.getLong(heartbeatRequestsReceivedId);
+  }
+  
+  public void incHeartbeatRequestsReceived() {
+    this.stats.incLong(heartbeatRequestsReceivedId, 1L);
+  }
+  
+  public long getHeartbeatsSent() {
+    return this.stats.getLong(heartbeatsSentId);
+  }
+  
+  public void incHeartbeatsSent() {
+    this.stats.incLong(heartbeatsSentId, 1L);
+  }
+  
+  public long getHeartbeatsReceived() {
+    return this.stats.getLong(heartbeatsReceivedId);
+  }
+  
+  public void incHeartbeatsReceived() {
+    this.stats.incLong(heartbeatsReceivedId, 1L);
+  }
+  
+  public long getSuspectsSent() {
+    return this.stats.getLong(suspectsSentId);
+  }
+  
+  public void incSuspectsSent() {
+    this.stats.incLong(suspectsSentId, 1L);
+  }
+
+  public long getSuspectsReceived() {
+    return this.stats.getLong(suspectsReceivedId);
+  }
+  
+  public void incSuspectsReceived() {
+    this.stats.incLong(suspectsReceivedId, 1L);
+  }
+  
+  public long getFinalCheckRequestsSent() {
+    return this.stats.getLong(finalCheckRequestsSentId);
+  }
+  
+  public void incFinalCheckRequestsSent() {
+    this.stats.incLong(finalCheckRequestsSentId, 1L);
+  }
+  
+  public long getFinalCheckRequestsReceived() {
+    return this.stats.getLong(finalCheckRequestsReceivedId);
+  }
+  
+  public void incFinalCheckRequestsReceived() {
+    this.stats.incLong(finalCheckRequestsReceivedId, 1L);
+  }
+  
+  public long getFinalCheckResponsesSent() {
+    return this.stats.getLong(finalCheckResponsesSentId);
+  }
+  
+  public void incFinalCheckResponsesSent() {
+    this.stats.incLong(finalCheckResponsesSentId, 1L);
+  }
+  
+  public long getFinalCheckResponsesReceived() {
+    return this.stats.getLong(finalCheckResponsesReceivedId);
+  }
+  
+  public void incFinalCheckResponsesReceived() {
+    this.stats.incLong(finalCheckResponsesReceivedId, 1L);
+  }
+  
+///
+  public long getTcpFinalCheckRequestsSent() {
+    return this.stats.getLong(tcpFinalCheckRequestsSentId);
+  }
+  
+  public void incTcpFinalCheckRequestsSent() {
+    this.stats.incLong(tcpFinalCheckRequestsSentId, 1L);
+  }
+  
+  public long getTcpFinalCheckRequestsReceived() {
+    return this.stats.getLong(tcpFinalCheckRequestsReceivedId);
+  }
+  
+  public void incTcpFinalCheckRequestsReceived() {
+    this.stats.incLong(tcpFinalCheckRequestsReceivedId, 1L);
+  }
+  
+  public long getTcpFinalCheckResponsesSent() {
+    return this.stats.getLong(tcpFinalCheckResponsesSentId);
+  }
+  
+  public void incTcpFinalCheckResponsesSent() {
+    this.stats.incLong(tcpFinalCheckResponsesSentId, 1L);
+  }
+  
+  public long getTcpFinalCheckResponsesReceived() {
+    return this.stats.getLong(tcpFinalCheckResponsesReceivedId);
+  }
+  
+  public void incTcpFinalCheckResponsesReceived() {
+    this.stats.incLong(tcpFinalCheckResponsesReceivedId, 1L);
+  }
+ 
+///
+  public long getUdpFinalCheckRequestsSent() {
+    return this.stats.getLong(udpFinalCheckRequestsSentId);
+  }
+  
+  public void incUdpFinalCheckRequestsSent() {
+    this.stats.incLong(udpFinalCheckRequestsSentId, 1L);
+  }
+
+//  UDP final check is implemented using HeartbeatRequestMessage and HeartbeatMessage
+//  So the following code is commented out  
+//  public long getUdpFinalCheckRequestsReceived() {
+//    return this.stats.getLong(udpFinalCheckRequestsReceivedId);
+//  }
+//  
+//  public void incUdpFinalCheckRequestsReceived() {
+//    this.stats.incLong(udpFinalCheckRequestsReceivedId, 1L);
+//  }
+//  
+//  public long getUdpFinalCheckResponsesSent() {
+//    return this.stats.getLong(udpFinalCheckResponsesSentId);
+//  }
+//  
+//  public void incUdpFinalCheckResponsesSent() {
+//    this.stats.incLong(udpFinalCheckResponsesSentId, 1L);
+//  }
+  
+  public long getUdpFinalCheckResponsesReceived() {
+    return this.stats.getLong(udpFinalCheckResponsesReceivedId);
+  }
+  
+  public void incUdpFinalCheckResponsesReceived() {
+    this.stats.incLong(udpFinalCheckResponsesReceivedId, 1L);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/91b43897/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
index 60158d1..b55fe88 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
@@ -573,6 +573,70 @@ public class LonerDistributionManager implements DM {
     @Override
     public void incThreadOwnedReceivers(long value, int dominoCount) {
     }
+    @Override
+    public long getHeartbeatRequestsSent() {return 0;}
+    @Override
+    public void incHeartbeatRequestsSent() {}
+    @Override
+    public long getHeartbeatRequestsReceived() {return 0;}
+    @Override
+    public void incHeartbeatRequestsReceived() {}
+    @Override
+    public long getHeartbeatsSent() {return 0;}
+    @Override
+    public void incHeartbeatsSent() {}
+    @Override
+    public long getHeartbeatsReceived() {return 0;}
+    @Override
+    public void incHeartbeatsReceived() {}
+    @Override
+    public long getSuspectsSent() {return 0;}
+    @Override
+    public void incSuspectsSent() {}
+    @Override
+    public long getSuspectsReceived() {return 0;}
+    @Override
+    public void incSuspectsReceived() {}
+    @Override
+    public long getFinalCheckRequestsSent() {return 0;}
+    @Override
+    public void incFinalCheckRequestsSent() {}
+    @Override
+    public long getFinalCheckRequestsReceived() {return 0;}
+    @Override
+    public void incFinalCheckRequestsReceived() {}
+    @Override
+    public long getFinalCheckResponsesSent() {return 0;}
+    @Override
+    public void incFinalCheckResponsesSent() {}
+    @Override
+    public long getFinalCheckResponsesReceived() {return 0;}
+    @Override
+    public void incFinalCheckResponsesReceived() {}
+    @Override    
+    public long getTcpFinalCheckRequestsSent() {return 0;}
+    @Override
+    public void incTcpFinalCheckRequestsSent() {}
+    @Override
+    public long getTcpFinalCheckRequestsReceived() {return 0;}
+    @Override
+    public void incTcpFinalCheckRequestsReceived() {}
+    @Override
+    public long getTcpFinalCheckResponsesSent() {return 0;}
+    @Override
+    public void incTcpFinalCheckResponsesSent() {}
+    @Override
+    public long getTcpFinalCheckResponsesReceived() {return 0;}
+    @Override
+    public void incTcpFinalCheckResponsesReceived() {}
+    @Override
+    public long getUdpFinalCheckRequestsSent() {return 0;}
+    @Override
+    public void incUdpFinalCheckRequestsSent() {}
+    @Override
+    public long getUdpFinalCheckResponsesReceived() {return 0;}
+    @Override
+    public void incUdpFinalCheckResponsesReceived() {}
   }
   protected static class DummyExecutor implements ExecutorService {
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/91b43897/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 005b0ed..b6f6c12 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
@@ -32,7 +32,6 @@ import java.net.SocketTimeoutException;
 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;
@@ -59,6 +58,7 @@ 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.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.NetView;
@@ -139,10 +139,10 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
    */
   final private ConcurrentHashMap<InternalDistributedMember, NetView> suspectedMemberInView = new ConcurrentHashMap<>();
   
-  /**
-   * Members undergoing final checks
-   */
-  final private List<InternalDistributedMember> membersInFinalCheck = Collections.synchronizedList(new ArrayList<>(30));
+//  /**
+//   * Members undergoing final checks
+//   */
+//  final private List<InternalDistributedMember> membersInFinalCheck = Collections.synchronizedList(new ArrayList<>(30));
 
   /**
    * Replies to messages
@@ -175,6 +175,9 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
   static final int ERROR = 0x00;  
   private volatile int socketPort;
   private volatile ServerSocket serverSocket;
+  
+  /** Statistics about health monitor */
+  protected DMStats stats;
 
   /**
    * this class is to avoid garbage
@@ -282,6 +285,8 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
         int  vmViewId = in.readInt();
         long uuidLSBs = in.readLong();
         long uuidMSBs = in.readLong();
+        GMSHealthMonitor.this.stats.incFinalCheckRequestsReceived();
+        GMSHealthMonitor.this.stats.incTcpFinalCheckRequestsReceived();
         boolean debug = logger.isDebugEnabled();
         GMSMember gmbr = (GMSMember) GMSHealthMonitor.this.localAddress.getNetMember();
         UUID myUUID = gmbr.getUUID();
@@ -311,6 +316,8 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
           out.write(OK);
           out.flush();
           socket.shutdownOutput();
+          GMSHealthMonitor.this.stats.incFinalCheckResponsesSent();
+          GMSHealthMonitor.this.stats.incTcpFinalCheckResponsesSent();
           if (debug) {
             logger.debug("GMSHealthMonitor server socket replied OK.");
           }
@@ -320,6 +327,8 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
           out.write(ERROR);
           out.flush();
           socket.shutdownOutput();
+          GMSHealthMonitor.this.stats.incFinalCheckResponsesSent();
+          GMSHealthMonitor.this.stats.incTcpFinalCheckResponsesSent();
           if (debug) {
             logger.debug("GMSHealthMonitor server socket replied ERROR.");
           }
@@ -358,9 +367,9 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
   public void contactedBy(InternalDistributedMember sender) {
     TimeStamp cTS = new TimeStamp(currentTimeStamp);
     cTS = memberTimeStamps.putIfAbsent(sender, cTS);
-    if (cTS != null) {
-      cTS.setTimeStamp(currentTimeStamp);
-    }
+//    if (cTS != null) {
+//      cTS.setTimeStamp(currentTimeStamp);
+//    }
     if (suspectedMemberInView.remove(sender) != null) {
       logger.info("No longer suspecting {}", sender);
     }
@@ -369,10 +378,10 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
 
   private HeartbeatRequestMessage constructHeartbeatRequestMessage(final InternalDistributedMember mbr) {
     final int reqId = requestId.getAndIncrement();
-    final HeartbeatRequestMessage prm = new HeartbeatRequestMessage(mbr, reqId);
-    prm.setRecipient(mbr);
+    final HeartbeatRequestMessage hrm = new HeartbeatRequestMessage(mbr, reqId);
+    hrm.setRecipient(mbr);
 
-    return prm;
+    return hrm;
   }
 
   private void checkMember(final InternalDistributedMember mbr) {
@@ -424,11 +433,12 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
       return true;
     }
     logger.trace("Checking member {}", member);
-    final HeartbeatRequestMessage prm = constructHeartbeatRequestMessage(member);
+    final HeartbeatRequestMessage hrm = constructHeartbeatRequestMessage(member);
     final Response pingResp = new Response();
-    requestIdVsResponse.put(prm.getRequestId(), pingResp);
+    requestIdVsResponse.put(hrm.getRequestId(), pingResp);
     try {
-      Set<InternalDistributedMember> membersNotReceivedMsg = this.services.getMessenger().send(prm);
+      Set<InternalDistributedMember> membersNotReceivedMsg = this.services.getMessenger().send(hrm);
+      this.stats.incHeartbeatRequestsSent();
       if (membersNotReceivedMsg != null && membersNotReceivedMsg.contains(member)) {
         // member is not part of current view.
         logger.trace("Member {} is not part of current view.", member);
@@ -446,6 +456,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
             return false;
           } else {
             logger.trace("received heartbeat from {}", member);
+            this.stats.incHeartbeatsReceived();
             if (ts != null) {
               ts.setTimeStamp(System.currentTimeMillis());
             }
@@ -456,7 +467,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
     } catch (InterruptedException e) {
       logger.debug("GMSHealthMonitor checking thread interrupted, while waiting for response from member: {} .", member);
     } finally {
-      requestIdVsResponse.remove(prm.getRequestId());
+      requestIdVsResponse.remove(hrm.getRequestId());
     }
     return false;
   }
@@ -502,8 +513,12 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
         GMSMember gmbr = (GMSMember) suspectMember.getNetMember();
         writeMemberToStream(gmbr, out);
         clientSocket.shutdownOutput();
+        this.stats.incFinalCheckRequestsSent();
+        this.stats.incTcpFinalCheckRequestsSent();
         logger.debug("Connected - reading response from suspect member {}", suspectMember);
         int b = in.read();
+        this.stats.incFinalCheckResponsesReceived();
+        this.stats.incTcpFinalCheckResponsesReceived();
         logger.debug("Received {}", (b == OK ? "OK" : (b == ERROR ? "ERROR" : b)), suspectMember);
         if (b == OK) {
           TimeStamp ts = memberTimeStamps.get(suspectMember);
@@ -535,12 +550,6 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
     out.flush();
   }
   
-  /*
-   * (non-Javadoc)
-   * 
-   * @see com.gemstone.gemfire.distributed.internal.membership.gms.fd.HealthMonitor#suspectMember(com.gemstone.gemfire.distributed.DistributedMember,
-   * java.lang.String)
-   */
   @Override
   public void suspect(InternalDistributedMember mbr, String reason) {
     initiateSuspicion(mbr, reason);
@@ -722,6 +731,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
               return;
             }
             services.getMessenger().sendUnreliably(message);
+            GMSHealthMonitor.this.stats.incHeartbeatsSent();
           } catch (CancelException e) {
             return;
           }
@@ -748,6 +758,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
           message.setRecipient(mbr);
           try {
             services.getMessenger().sendUnreliably(message);
+            GMSHealthMonitor.this.stats.incHeartbeatsSent();
             numSent++;
             if (numSent >= NUM_HEARTBEATS) {
               break;
@@ -843,11 +854,12 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
     isStopping = false;
     services = s;
     memberTimeout = s.getConfig().getMemberTimeout();
+    this.stats = services.getStatistics();
     services.getMessenger().addHandler(HeartbeatRequestMessage.class, this);
     services.getMessenger().addHandler(HeartbeatMessage.class, this);
     services.getMessenger().addHandler(SuspectMembersMessage.class, this);
   }
-
+  
   @Override
   public void started() {
     setLocalAddress( services.getMessenger().getMemberID());
@@ -993,6 +1005,8 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
 
   private void processHeartbeatRequest(HeartbeatRequestMessage m) {
     
+    this.stats.incHeartbeatRequestsReceived();
+    
     if (this.isStopping || this.playingDead) {
       return;
     }
@@ -1001,9 +1015,10 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
     InternalDistributedMember me = localAddress;
 
     if (me.getVmViewId() >= 0  &&  m.getTarget().equals(me)) {
-      HeartbeatMessage prm = new HeartbeatMessage(m.getRequestId());
-      prm.setRecipient(m.getSender());
-      Set<InternalDistributedMember> membersNotReceivedMsg = services.getMessenger().send(prm);
+      HeartbeatMessage hm = new HeartbeatMessage(m.getRequestId());
+      hm.setRecipient(m.getSender());
+      Set<InternalDistributedMember> membersNotReceivedMsg = services.getMessenger().send(hm);
+      this.stats.incHeartbeatsSent();
       if (membersNotReceivedMsg != null && membersNotReceivedMsg.contains(m.getSender())) {
         logger.debug("Unable to send heartbeat to member: {}", m.getSender());
       }
@@ -1013,6 +1028,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
   }
 
   private void processHeartbeat(HeartbeatMessage m) {
+    this.stats.incHeartbeatsReceived();
     if (m.getRequestId() < 0) {
       // a periodic heartbeat
       contactedBy(m.getSender());
@@ -1037,6 +1053,9 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
    * @param incomingRequest
    */
   private void processSuspectMembersRequest(SuspectMembersMessage incomingRequest) {
+    
+    this.stats.incSuspectsReceived();
+    
     NetView cv = currentView;
 
     if (cv == null) {
@@ -1062,6 +1081,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
           message.setRecipient(sender);
           try {
             services.getMessenger().send(message);
+            this.stats.incHeartbeatsSent();
             it.remove();
           } catch (CancelException e) {
             return;
@@ -1132,13 +1152,13 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
   private void doFinalCheck(final InternalDistributedMember initiator,
       List<SuspectRequest> sMembers, NetView cv, InternalDistributedMember localAddress) {
 
-    List<InternalDistributedMember> membersChecked = new ArrayList<>(10);
+//    List<InternalDistributedMember> membersChecked = new ArrayList<>(10);
     try {
       for (int i = 0; i < sMembers.size(); i++) {
         final SuspectRequest sr = sMembers.get(i);
         final InternalDistributedMember mbr = sr.getSuspectMember();
 
-        if (!cv.contains(mbr) || membersInFinalCheck.contains(mbr)) {
+        if (!cv.contains(mbr) /*|| membersInFinalCheck.contains(mbr)*/) {
           continue;
         }
 
@@ -1146,7 +1166,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
           continue;// self
         }
         
-        membersChecked.add(mbr);
+//        membersChecked.add(mbr);
 
         // suspectMemberInView is now set by the heartbeat monitoring code
         // to allow us to move on from watching members we've already
@@ -1181,6 +1201,12 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
                   logger.debug("\ncurrent view: {}\nports: {}", view, Arrays.toString(view.getFailureDetectionPorts()));
                 }
                 pinged = GMSHealthMonitor.this.doCheckMember(mbr);
+                GMSHealthMonitor.this.stats.incFinalCheckRequestsSent();
+                GMSHealthMonitor.this.stats.incUdpFinalCheckRequestsSent();
+                if (pinged) {
+                  GMSHealthMonitor.this.stats.incFinalCheckResponsesReceived();
+                  GMSHealthMonitor.this.stats.incUdpFinalCheckResponsesReceived();
+                }
               } else {
                 pinged = GMSHealthMonitor.this.doTCPCheckMember(mbr, port);
               }
@@ -1214,7 +1240,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
         //      }// scheduling for final check and removing it..
       }
     } finally {
-      membersInFinalCheck.removeAll(membersChecked);
+//      membersInFinalCheck.removeAll(membersChecked);
     }
   }
   @Override
@@ -1253,10 +1279,11 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
       recipients = currentView.getMembers();
 //    }
 
-    SuspectMembersMessage rmm = new SuspectMembersMessage(recipients, requests);
+    SuspectMembersMessage smm = new SuspectMembersMessage(recipients, requests);
     Set<InternalDistributedMember> failedRecipients;
     try {
-      failedRecipients = services.getMessenger().send(rmm);
+      failedRecipients = services.getMessenger().send(smm);
+      this.stats.incSuspectsSent();
     } catch (CancelException e) {
       return;
     }
@@ -1299,4 +1326,8 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
     }
     
   }
+  
+  public DMStats getStats() {
+    return this.stats;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/91b43897/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 d539374..82ac316 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
@@ -39,6 +39,7 @@ import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Properties;
 
 import org.jgroups.util.UUID;
 import org.junit.After;
@@ -49,8 +50,11 @@ import org.junit.experimental.categories.Category;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 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;
@@ -80,6 +84,7 @@ public class GMSHealthMonitorJUnitTest {
   private JoinLeave joinLeave;
   private GMSHealthMonitor gmsHealthMonitor;
   private Manager manager;
+  private long statsId = 123;
   final long memberTimeout = 1000l;
   private int[] portRange= new int[]{0, 65535};
 
@@ -92,7 +97,20 @@ public class GMSHealthMonitorJUnitTest {
     joinLeave = mock(JoinLeave.class);
     manager = mock(Manager.class);
     services = mock(Services.class);
-    Stopper stopper = mock(Stopper.class);
+    Stopper stopper = mock(Stopper.class); 
+    
+    Properties nonDefault = new Properties();
+    nonDefault.put(DistributionConfig.ACK_WAIT_THRESHOLD_NAME, "1");
+    nonDefault.put(DistributionConfig.ACK_SEVERE_ALERT_THRESHOLD_NAME, "10");
+    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]");
+    DM dm = mock(DM.class);    
+    InternalDistributedSystem system = InternalDistributedSystem.newInstanceForTesting(dm, nonDefault);
 
     when(mockConfig.getDistributionConfig()).thenReturn(mockDistConfig);
     when(mockConfig.getMemberTimeout()).thenReturn(memberTimeout);
@@ -102,8 +120,8 @@ public class GMSHealthMonitorJUnitTest {
     when(services.getJoinLeave()).thenReturn(joinLeave);
     when(services.getCancelCriterion()).thenReturn(stopper);
     when(services.getManager()).thenReturn(manager);
+    when(services.getStatistics()).thenReturn(new DistributionStats(system, statsId));
     when(stopper.isCancelInProgress()).thenReturn(false);
-    
 
     if (mockMembers == null) {
       mockMembers = new ArrayList<InternalDistributedMember>();
@@ -142,6 +160,8 @@ public class GMSHealthMonitorJUnitTest {
 
     gmsHealthMonitor.processMessage(new HeartbeatRequestMessage(mbr, 1));
     verify(messenger, atLeastOnce()).send(any(HeartbeatMessage.class));
+    Assert.assertEquals(1, gmsHealthMonitor.getStats().getHeartbeatRequestsReceived());
+    Assert.assertEquals(1, gmsHealthMonitor.getStats().getHeartbeatsSent());
   }
 
   /**
@@ -187,7 +207,7 @@ public class GMSHealthMonitorJUnitTest {
     // neighbor should change to 5th
     System.out.println("testHMNextNeighborAfterTimeout ending");
     Assert.assertEquals("expected " + expected + " but found " + neighbor
-        + ".  view="+v, expected, neighbor);
+        + ".  view="+v, expected, neighbor);  
   }
 
   /**
@@ -239,6 +259,8 @@ public class GMSHealthMonitorJUnitTest {
 
     System.out.println("testSuspectMembersCalledThroughMemberCheckThread ending");
     assertTrue(gmsHealthMonitor.isSuspectMember(mockMembers.get(4)));
+    Assert.assertTrue(gmsHealthMonitor.getStats().getHeartbeatRequestsSent() > 0);
+    Assert.assertTrue(gmsHealthMonitor.getStats().getSuspectsSent() > 0);
   }
 
   /***
@@ -282,6 +304,8 @@ public class GMSHealthMonitorJUnitTest {
     Thread.sleep(GMSHealthMonitor.MEMBER_SUSPECT_COLLECTION_INTERVAL + 1000);
 
     verify(messenger, atLeastOnce()).send(any(SuspectMembersMessage.class));
+    
+    Assert.assertTrue(gmsHealthMonitor.getStats().getSuspectsSent() > 0);
   }
 
   /***
@@ -306,6 +330,8 @@ public class GMSHealthMonitorJUnitTest {
     }
 
     verify(messenger, atLeastOnce()).send(isA(SuspectMembersMessage.class));
+    
+    Assert.assertTrue(gmsHealthMonitor.getStats().getSuspectsSent() > 0);
   }
 
   /***
@@ -338,6 +364,7 @@ public class GMSHealthMonitorJUnitTest {
 
     System.out.println("testRemoveMemberCalled ending");
     verify(joinLeave, atLeastOnce()).remove(any(InternalDistributedMember.class), any(String.class));
+    Assert.assertTrue(gmsHealthMonitor.getStats().getSuspectsReceived() > 0);
   }
 
   /***
@@ -373,6 +400,7 @@ public class GMSHealthMonitorJUnitTest {
 
     System.out.println("testRemoveMemberNotCalledBeforeTimeout ending");
     verify(joinLeave, never()).remove(any(InternalDistributedMember.class), any(String.class));
+    Assert.assertTrue(gmsHealthMonitor.getStats().getSuspectsReceived() > 0);
   }
 
   /***
@@ -407,6 +435,7 @@ public class GMSHealthMonitorJUnitTest {
     Thread.sleep(memberTimeout + 200);
 
     verify(joinLeave, atLeastOnce()).remove(any(InternalDistributedMember.class), any(String.class));
+    Assert.assertTrue(gmsHealthMonitor.getStats().getSuspectsReceived() > 0);
   }
 
   /***
@@ -548,6 +577,9 @@ public class GMSHealthMonitorJUnitTest {
     DataInputStream dis = new DataInputStream(new ByteArrayInputStream(outputStream.toByteArray()));
     int byteReply = dis.read();
     Assert.assertEquals(expectedResult, byteReply);
+    
+    Assert.assertTrue(gmsHealthMonitor.getStats().getFinalCheckResponsesSent() > 0);
+    Assert.assertTrue(gmsHealthMonitor.getStats().getTcpFinalCheckResponsesSent() > 0);
   }
   
   @Test
@@ -610,6 +642,10 @@ public class GMSHealthMonitorJUnitTest {
     when(fakeSocket.isConnected()).thenReturn(true);
     
     Assert.assertEquals(expectedResult, gmsHealthMonitor.doTCPCheckMember(otherMember, fakeSocket));
+    Assert.assertTrue(gmsHealthMonitor.getStats().getFinalCheckRequestsSent() > 0);
+    Assert.assertTrue(gmsHealthMonitor.getStats().getTcpFinalCheckRequestsSent() > 0);
+    Assert.assertTrue(gmsHealthMonitor.getStats().getFinalCheckResponsesReceived() > 0);
+    Assert.assertTrue(gmsHealthMonitor.getStats().getTcpFinalCheckResponsesReceived() > 0);
     
     //we can check to see if the gms member information was written out by the tcp check
     byte[] bytesWritten = outputStream.toByteArray();



[23/50] [abbrv] incubator-geode git commit: GEODE-731: fix suspect string in CompressionCacheConfigDUnitTest

Posted by ab...@apache.org.
GEODE-731: fix suspect string in CompressionCacheConfigDUnitTest


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

Branch: refs/heads/master
Commit: fb38186b142b4d10bdeb069b6255dac1cd2d9bc6
Parents: f36310d
Author: Swapnil Bawaskar <sb...@pivotal.io>
Authored: Wed Jan 6 09:46:48 2016 -0800
Committer: Swapnil Bawaskar <sb...@pivotal.io>
Committed: Wed Jan 6 09:48:19 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fb38186b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
index 9f01ef8..9eb51c5 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
@@ -27,10 +27,7 @@ import com.gemstone.gemfire.compression.Compressor;
 import com.gemstone.gemfire.compression.SnappyCompressor;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 
-import dunit.Host;
-import dunit.SerializableCallable;
-import dunit.SerializableRunnable;
-import dunit.VM;
+import dunit.*;
 
 /**
  * Tests configured and badly configured cache.xml files with regards to compression.
@@ -97,8 +94,13 @@ public class CompressionCacheConfigDUnitTest extends CacheTestCase {
    */
   public void testCreateCacheWithBadCompressor() throws Exception {
     File cacheXml = createCacheXml(BAD_COMPRESSOR);
-    assertFalse(createCacheOnVM(getVM(0),cacheXml.getCanonicalPath()));
-    cacheXml.delete();
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
+    try {
+      assertFalse(createCacheOnVM(getVM(0), cacheXml.getCanonicalPath()));
+    } finally {
+      expectedException.remove();
+      cacheXml.delete();
+    }
   }
   
   /**


[20/50] [abbrv] incubator-geode git commit: GEODE-718 GEODE-725: Ensure that test resources are cleaned up from prior tests

Posted by ab...@apache.org.
GEODE-718 GEODE-725: Ensure that test resources are cleaned up from prior 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/a69a0dc0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/a69a0dc0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/a69a0dc0

Branch: refs/heads/master
Commit: a69a0dc0ea1b64f34920b18da654545de7d12268
Parents: 13a3adf
Author: Jens Deppe <jd...@pivotal.io>
Authored: Tue Jan 5 11:20:33 2016 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Tue Jan 5 11:20:48 2016 -0800

----------------------------------------------------------------------
 .../gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a69a0dc0/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
index 77e8785..3d4615a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
@@ -46,6 +46,7 @@ public class GfshHistoryJUnitTest {
 
   @Before
   public void setUp() throws Exception {
+    teardown();
     gfshHistoryFile = tempDirectory.newFile("historyFile");
     gfshConfig = new GfshConfig(gfshHistoryFile.getAbsolutePath(),
         "",                                         // defaultPrompt


[31/50] [abbrv] incubator-geode git commit: GEODE-610: Add bundled source dependencies to LICENSE

Posted by ab...@apache.org.
GEODE-610: Add bundled source dependencies to LICENSE

Add the source dependencies to LICENSE. Clean up NOTICE file.


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

Branch: refs/heads/master
Commit: fdd0e470d1056baf833f45fc99a2c3170c200c23
Parents: 7492dce
Author: Anthony Baker <ab...@pivotal.io>
Authored: Sat Jan 2 10:20:08 2016 -0600
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Wed Jan 6 17:37:43 2016 -0800

----------------------------------------------------------------------
 LICENSE |  77 ++++++++++++++
 NOTICE  | 334 -----------------------------------------------------------
 2 files changed, 77 insertions(+), 334 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdd0e470/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index d645695..25b301c 100644
--- a/LICENSE
+++ b/LICENSE
@@ -200,3 +200,80 @@
    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.
+
+===========================================================================
+
+Apache Geode includes a number of components and libraries with separate 
+copyright notices and license terms. Your use of those components are 
+subject to the terms and conditions of the following licenses. 
+
+---------------------------------------------------------------------------
+Backbone (http://backbonejs.org)
+---------------------------------------------------------------------------
+This product bundles Backbone which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+Handlebars (http://handlebarsjs.com)
+---------------------------------------------------------------------------
+This product bundles Handlebars which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+highlight.js (https://highlightjs.org)
+---------------------------------------------------------------------------
+This product bundles highlight.js which is available under a BSD 3 clauses license.  
+For details see http://opensource.org/licenses/BSD-3-Clause.
+
+---------------------------------------------------------------------------
+JOpt Simple (http://pholser.github.io/jopt-simple/)
+---------------------------------------------------------------------------
+This product bundles JOpt Simple which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+JOpt Simple (http://pholser.github.io/jopt-simple/)
+---------------------------------------------------------------------------
+This product bundles JOpt Simple which is available under a BSD-like license.  
+For details see http://json.org/license.html.
+
+---------------------------------------------------------------------------
+jQuery (https://jquery.com)
+---------------------------------------------------------------------------
+This product bundles jQuery which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+jQuery BBQ (http://benalman.com/projects/jquery-bbq-plugin/)
+---------------------------------------------------------------------------
+This product bundles jQuery BBQ which is available under an MIT and GPL dual license.  
+For details see http://benalman.com/about/license.
+
+---------------------------------------------------------------------------
+jQuery-slideto (https://github.com/Sleavely/jQuery-slideto)
+---------------------------------------------------------------------------
+This product bundles jQuery-slideto which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+jQuery Wiggle (https://github.com/wilhelm-murdoch/jQuery-Wiggle)
+---------------------------------------------------------------------------
+This product bundles jQuery Wiggle which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+reset.css (http://meyerweb.com/eric/tools/css/reset/)
+---------------------------------------------------------------------------
+This product bundles reset.css which is available in the public domain.  
+
+---------------------------------------------------------------------------
+shred (https://github.com/pandastrike/shred)
+---------------------------------------------------------------------------
+This product bundles JOpt Simple which is available under an MIT or ISC license.  
+For details see https://github.com/pandastrike/shred/blob/master/LICENSE.
+
+---------------------------------------------------------------------------
+underscore (http://underscorejs.org)
+---------------------------------------------------------------------------
+This product bundles JOpt Simple which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdd0e470/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index a63dcd5..e73e3db 100644
--- a/NOTICE
+++ b/NOTICE
@@ -3,337 +3,3 @@ Copyright 2015 The Apache Software Foundation.
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
-
-The following is a listing of the open source components detailed in
-this document. This list is provided for your convenience; please read
-further if you wish to review the copyright notice(s) and the full text
-of the license associated with each component.
-
-SECTION 1: BSD-STYLE, MIT-STYLE, OR SIMILAR STYLE LICENSES
-
-   >>> dom4j-1.6.1
-   >>> jopt-simple-4.4
-   >>> json-none
-   >>> spymemcached-2.9.0
-
-
-
-SECTION 2: Apache License, V2.0
-
-   >>> fastutil-6.6.1
-   >>> hadoop-common-2.4.0
-   >>> swagger-ui-2.0.17
-
-
-
-SECTION 3: GNU Lesser General Public License, V2.1
-
-   >>> jgroups-2.2.9
-
-
-
-APPENDIX. Standard License Files
-
-   >>> Apache License, V2.0
-
-   >>> GNU Lesser General Public License, V2.1
-
-
-
-
---------------- SECTION 1:  BSD-STYLE, MIT-STYLE, OR SIMILAR STYLE LICENSES ----------
-
-BSD-STYLE, MIT-STYLE, OR SIMILAR STYLE LICENSES are applicable to the following component(s).
-
-
->>> dom4j-1.6.1
-
-Copyright 2001-2005 (C) MetaStuff, Ltd. All Rights Reserved.
-
-Redistribution and use of this software and associated documentation
-("Software"), with or without modification, are permitted provided
-that the following conditions are met:
-
-1. Redistributions of source code must retain copyright
-   statements and notices.  Redistributions must also contain a
-   copy of this document.
- 
-2. Redistributions in binary form must reproduce the
-   above copyright notice, this list of conditions and the
-   following disclaimer in the documentation and/or other
-   materials provided with the distribution.
- 
-3. The name "DOM4J" must not be used to endorse or promote
-   products derived from this Software without prior written
-   permission of MetaStuff, Ltd.  For written permission,
-   please contact dom4j-info@metastuff.com.
- 
-4. Products derived from this Software may not be called "DOM4J"
-   nor may "DOM4J" appear in their names without prior written
-   permission of MetaStuff, Ltd. DOM4J is a registered
-   trademark of MetaStuff, Ltd.
- 
-5. Due credit should be given to the DOM4J Project - 
-   http://www.dom4j.org
- 
-THIS SOFTWARE IS PROVIDED BY METASTUFF, LTD. AND CONTRIBUTORS
-``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT
-NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
-FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL
-METASTUFF, LTD. OR ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT,
-INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
-(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
-SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
-HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
-STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
-ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
-OF THE POSSIBILITY OF SUCH DAMAGE.
-
-
->>> jopt-simple-4.4
-
-The MIT License
-
- Copyright (c) 2004-2011 Paul R. Holser, Jr.
-
- Permission is hereby granted, free of charge, to any person obtaining
- a copy of this software and associated documentation files (the
- "Software"), to deal in the Software without restriction, including
- without limitation the rights to use, copy, modify, merge, publish,
- distribute, sublicense, and/or sell copies of the Software, and to
- permit persons to whom the Software is furnished to do so, subject to
- the following conditions:
-
- The above copyright notice and this permission notice shall be
- included in all copies or substantial portions of the Software.
-
- THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
- EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
- MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
- NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
- LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
- OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
- WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
-
-ADDITIONAL LICENSE INFORMATION:
-
-> Apache 2.0
-
-pholser-jopt-simple-jopt-simple-4.3-11-gdf866e0.zip\pholser-jopt-simple-df866e0\src\site\resources\scripts\prettify.js
-
-Copyright (C) 2006 Google Inc.
-
-Licensed 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.
-
-
->>> json-none
-
-Copyright (c) 2002 JSON.org
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in all
-copies or substantial portions of the Software.
-
-The Software shall be used for Good, not Evil.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
-SOFTWARE.
-
-
->>> spymemcached-2.9.0
-
-Copyright (C) 2006-2009 Dustin Sallings
- * Copyright (C) 2009-2011 Couchbase, Inc.
- *
- * Permission is hereby granted, free of charge, to any person obtaining a copy
- * of this software and associated documentation files (the "Software"), to deal
- * in the Software without restriction, including without limitation the rights
- * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
- * copies of the Software, and to permit persons to whom the Software is
- * furnished to do so, subject to the following conditions:
- *
- * The above copyright notice and this permission notice shall be included in
- * all copies or substantial portions of the Software.
- *
- * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
- * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
- * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
- * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
- * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
- * FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALING
- * IN THE SOFTWARE.
-
-
---------------- SECTION 2: Apache License, V2.0 ----------
-
-Apache License, V2.0 is applicable to the following component(s).
-
-
->>> fastutil-6.6.1
-
-* Copyright (C) 2002-2014 Sebastiano Vigna 
- *
- * Licensed 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.
-
-
->>> hadoop-common-2.4.0
-
-Licensed 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. See accompanying LICENSE file.
-
-
->>> swagger-ui-2.0.17
-
-License: Apache 2.0
-
-
-
-ADDITIONAL LICENSE INFORMATION:
-
-
-> MIT
-
-
-swagger-ui-2.0.17.jar/ META-INF/ resources/ webjars/ swagger-ui/ 2.0.17/ lib/ jquery.ba-bbq.min.js
-
-
-[PLEASE NOTE:  WE ELECT TO USE AND DISTRIBUTE THIS COMPONENT UNDER THE TERMS OF THE MIT LICENSE.  THE ORIGINAL LICENSE TERMS ARE REPRODUCED BELOW ONLY AS A REFERENCE.]
-
-
-
-jQuery BBQ: Back Button & Query Library - v1.2.1 - 2/17/2010
- * http://benalman.com/projects/jquery-bbq-plugin/
- * 
- * Copyright (c) 2010 "Cowboy" Ben Alman
- * Dual licensed under the MIT and GPL licenses.
- * http://benalman.com/about/license/
-
-
---------------- SECTION 3: GNU Lesser General Public License, V2.1 ----------
-
-GNU Lesser General Public License, V2.1 is applicable to the following component(s).
-
-
->>> jgroups-2.2.9
-
-License: LGPL 2.1
-
-
-ADDITIONAL LICENSE INFORMATION:
-
-
-> Apache 1.1
-
-JGroups-2.2.9.src.zip\JGroups-2.2.9.src\lib\commons-logging.jar\META-INF\LICENSE.txt
-
-
-The Apache Software License, Version 1.1
- *
- * Copyright (c) 1999-2003 The Apache Software Foundation.  All rights
- * reserved.
- *
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions
- * are met:
- *
- * 1. Redistributions of source code must retain the above copyright
- *    notice, this list of conditions and the following disclaimer.
- *
- * 2. Redistributions in binary form must reproduce the above copyright
- *    notice, this list of conditions and the following disclaimer in
- *    the documentation and/or other materials provided with the
- *    distribution.
- *
- * 3. The end-user documentation included with the redistribution, if
- *    any, must include the following acknowlegement:
- *       "This product includes software developed by the
- *        Apache Software Foundation (http://www.apache.org/)."
- *    Alternately, this acknowlegement may appear in the software itself,
- *    if and wherever such third-party acknowlegements normally appear.
- *
- * 4. The names "The Jakarta Project", "Commons", and "Apache Software
- *    Foundation" must not be used to endorse or promote products derived
- *    from this software without prior written permission. For written
- *    permission, please contact apache@apache.org.
- *
- * 5. Products derived from this software may not be called "Apache"
- *    nor may "Apache" appear in their names without prior written
- *    permission of the Apache Group.
- *
- * THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED
- * WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
- * DISCLAIMED.  IN NO EVENT SHALL THE APACHE SOFTWARE FOUNDATION OR
- * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF
- * USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
- * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
- * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
- * OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
- * SUCH DAMAGE.
- * ====================================================================
- *
- * This software consists of voluntary contributions made by many
- * individuals on behalf of the Apache Software Foundation.  For more
- * information on the Apache Software Foundation, please see
- * <http://www.apache.org/>.
-
-
-ant-optional.jar
-ant.jar
-
-
-> MIT
-
-JGroups-2.2.9.src.zip/ JGroups-2.2.9.src/ lib/bcprov-jdk14-117.jar/ org/ bouncycastle/ LICENSE.class
-	
-
-
-Copyright (c) 2000 The Legion Of The Bouncy Castle (http://www.bouncycastle.org)  GH line.separator IJ^
-
-
-Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so,subject to the following conditions:The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED,INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE [LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.


[47/50] [abbrv] incubator-geode git commit: Update README instructions for building with gradle instead of gradlew

Posted by ab...@apache.org.
Update README instructions for building with gradle instead of gradlew


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

Branch: refs/heads/master
Commit: a6774c4263f54b2e03abffb8a2add1d1295c64ed
Parents: 90efd88
Author: Anthony Baker <ab...@apache.org>
Authored: Mon Jan 25 12:35:55 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Wed Jan 27 10:56:25 2016 -0800

----------------------------------------------------------------------
 README.md | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6774c42/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index e90b769..57f5e4d 100755
--- a/README.md
+++ b/README.md
@@ -32,11 +32,10 @@ Geode includes the following features:
 
 # Geode in 5 minutes
 
-With JDK 1.8 or a more recent version installed, obtain the source archive. 
-Extract and build from source:
+With both a recent version of Gradle and JDK 1.8 or a more recent version installed, obtain the source archive.
+Extract the source archive and build from the expanded directory:
 
-    $ cd geode
-    $ ./gradlew build installDist
+    $ gradle build installDist
 
 Start a locator and server:
 


[28/50] [abbrv] incubator-geode git commit: Add DISCLAIMER file to meet ASF incubator branding requirements

Posted by ab...@apache.org.
Add DISCLAIMER file to meet ASF incubator branding requirements

Incubating projects should include a DISCLAIMER file along with the
LICENSE and NOTICE files in each distribution.  See
http://incubator.apache.org/guides/branding.html for more details.


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

Branch: refs/heads/master
Commit: 27e94365069ebe660474cd667b70fee940764dcf
Parents: 7c28132
Author: Anthony Baker <ab...@pivotal.io>
Authored: Wed Jan 6 16:27:13 2016 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Wed Jan 6 16:28:36 2016 -0800

----------------------------------------------------------------------
 DISCLAIMER                                | 6 ++++++
 gemfire-assembly/src/main/dist/DISCLAIMER | 6 ++++++
 2 files changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/27e94365/DISCLAIMER
----------------------------------------------------------------------
diff --git a/DISCLAIMER b/DISCLAIMER
new file mode 100644
index 0000000..43c734d
--- /dev/null
+++ b/DISCLAIMER
@@ -0,0 +1,6 @@
+Apache Geode is an effort undergoing incubation at The Apache Software Foundation (ASF),
+sponsored by the Apache 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.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/27e94365/gemfire-assembly/src/main/dist/DISCLAIMER
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/DISCLAIMER b/gemfire-assembly/src/main/dist/DISCLAIMER
new file mode 100644
index 0000000..43c734d
--- /dev/null
+++ b/gemfire-assembly/src/main/dist/DISCLAIMER
@@ -0,0 +1,6 @@
+Apache Geode is an effort undergoing incubation at The Apache Software Foundation (ASF),
+sponsored by the Apache 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.


[11/50] [abbrv] incubator-geode git commit: Fix source header

Posted by ab...@apache.org.
Fix source header


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

Branch: refs/heads/master
Commit: d19bb41ec2f51dc216926a5a6f6b41cbbf57096c
Parents: 18a17d7
Author: Anthony Baker <ab...@pivotal.io>
Authored: Mon Jan 4 14:29:02 2016 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Mon Jan 4 14:29:45 2016 -0800

----------------------------------------------------------------------
 .../InstantiatorPropagationDUnitTest.java       | 37 ++++++++------------
 1 file changed, 15 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d19bb41e/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
index 7c5cf53..72bbdbd 100644
--- 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
@@ -1,9 +1,18 @@
-/*=========================================================================
- * 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.
- *=========================================================================
+/*
+ * 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.tier.sockets;
 
@@ -41,22 +50,6 @@ import dunit.DistributedTestCase;
 import dunit.Host;
 import dunit.VM;
 
-/*
- * 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.
- */
 public class InstantiatorPropagationDUnitTest extends DistributedTestCase {
   private static Cache cache = null;
 


[09/50] [abbrv] incubator-geode git commit: GEODE-715: Move dunit.standalone under com.gemstone.gemfire.test

Posted by ab...@apache.org.
GEODE-715: Move dunit.standalone under com.gemstone.gemfire.test

* Move dunit.BounceResult and dunit.RemoteDUnitVMIF to dunit.standalone
* Move dunit.standalone to com.gemstone.gemfire.test.dunit.standalone
* Remove hydra dependencies from com.gemstone.gemfire.test.dunit.standalone
except for MethExecutor


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

Branch: refs/heads/master
Commit: c57a88c6f2437a8d74f51abbe2edc3355eb1806b
Parents: df6d7ff
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Dec 28 15:10:27 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Mon Jan 4 13:59:51 2016 -0800

----------------------------------------------------------------------
 .../cache/PartitionedRegionDUnitTestCase.java   |   2 +-
 .../CompressionRegionConfigDUnitTest.java       |   2 +-
 .../test/dunit/standalone/BounceResult.java     |  36 ++
 .../gemfire/test/dunit/standalone/ChildVM.java  |  81 ++++
 .../test/dunit/standalone/DUnitLauncher.java    | 461 ++++++++++++++++++
 .../test/dunit/standalone/ProcessManager.java   | 259 +++++++++++
 .../test/dunit/standalone/RemoteDUnitVM.java    | 142 ++++++
 .../test/dunit/standalone/RemoteDUnitVMIF.java  |  36 ++
 .../dunit/standalone/StandAloneDUnitEnv.java    |  75 +++
 .../test/dunit/tests/BasicDUnitTest.java        | 132 ++++++
 .../gemfire/test/dunit/tests/TestFailure.java   |  50 ++
 .../gemfire/test/dunit/tests/VMDUnitTest.java   | 237 ++++++++++
 .../src/test/java/dunit/BounceResult.java       |  36 --
 gemfire-core/src/test/java/dunit/DUnitEnv.java  |   2 +
 gemfire-core/src/test/java/dunit/Host.java      |   2 +
 .../src/test/java/dunit/RemoteDUnitVMIF.java    |  36 --
 gemfire-core/src/test/java/dunit/VM.java        |   7 +-
 .../src/test/java/dunit/standalone/ChildVM.java |  82 ----
 .../java/dunit/standalone/DUnitLauncher.java    | 463 -------------------
 .../java/dunit/standalone/ProcessManager.java   | 261 -----------
 .../java/dunit/standalone/RemoteDUnitVM.java    | 144 ------
 .../dunit/standalone/StandAloneDUnitEnv.java    |  75 ---
 .../test/java/dunit/tests/BasicDUnitTest.java   | 132 ------
 .../src/test/java/dunit/tests/TestFailure.java  |  50 --
 .../src/test/java/dunit/tests/VMDUnitTest.java  | 237 ----------
 25 files changed, 1520 insertions(+), 1520 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
index a4d32f2..74f201d 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
@@ -30,10 +30,10 @@ import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogWriterImpl;
 import com.gemstone.gemfire.internal.logging.PureLogWriter;
+import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 
 import dunit.Host;
 import dunit.SerializableRunnable;
-import dunit.standalone.DUnitLauncher;
 
 /**
  * This class is extended by some PartitionedRegion related DUnit test cases 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java
index af7d07f..1fb22c6 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java
@@ -27,13 +27,13 @@ import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.compression.Compressor;
 import com.gemstone.gemfire.compression.SnappyCompressor;
+import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 
 import dunit.DistributedTestCase;
 import dunit.Host;
 import dunit.SerializableCallable;
 import dunit.SerializableRunnable;
 import dunit.VM;
-import dunit.standalone.DUnitLauncher;
 
 /**
  * Sanity checks on a number of basic cluster configurations with compression turned on.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java
new file mode 100644
index 0000000..e117004
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java
@@ -0,0 +1,36 @@
+/*
+ * 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.dunit.standalone;
+
+public class BounceResult {
+  private final int newPid;
+  private final RemoteDUnitVMIF newClient;
+  
+  public BounceResult(int newPid, RemoteDUnitVMIF newClient) {
+    this.newPid = newPid;
+    this.newClient = newClient;
+  }
+
+  public int getNewPid() {
+    return newPid;
+  }
+
+  public RemoteDUnitVMIF getNewClient() {
+    return newClient;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java
new file mode 100644
index 0000000..49b53e8
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java
@@ -0,0 +1,81 @@
+/*
+ * 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.dunit.standalone;
+
+import hydra.HydraRuntimeException;
+import hydra.Log;
+
+import java.rmi.Naming;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.internal.OSProcess;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher.MasterRemote;
+
+/**
+ * @author dsmith
+ *
+ */
+public class ChildVM {
+  
+  private static boolean stopMainLoop = false;
+  
+  /**
+   * tells the main() loop to exit
+   */
+  public static void stopVM() {
+    stopMainLoop = true;
+  }
+  
+  static {
+    createHydraLogWriter();
+  }
+  
+  private final static Logger logger = LogService.getLogger();
+  private static RemoteDUnitVM dunitVM;
+  
+  public static void main(String[] args) throws Throwable {
+    try {
+      int namingPort = Integer.getInteger(DUnitLauncher.RMI_PORT_PARAM).intValue();
+      int vmNum = Integer.getInteger(DUnitLauncher.VM_NUM_PARAM).intValue();
+      int pid = OSProcess.getId();
+      logger.info("VM" + vmNum + " is launching" + (pid > 0? " with PID " + pid : ""));
+      MasterRemote holder = (MasterRemote) Naming.lookup("//localhost:" + namingPort + "/" + DUnitLauncher.MASTER_PARAM);
+      DUnitLauncher.init(holder);
+      DUnitLauncher.locatorPort = holder.getLocatorPort();
+      dunitVM = new RemoteDUnitVM();
+      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 (!stopMainLoop) {
+        holder.ping();
+        Thread.sleep(1000);
+      }
+    } catch (Throwable t) {
+      t.printStackTrace();
+      System.exit(1);
+    }
+  }
+
+  private static void createHydraLogWriter() {
+    try {
+      Log.createLogWriter("dunit-childvm", "fine");
+    } catch (HydraRuntimeException ignore) {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
new file mode 100644
index 0000000..1358722
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
@@ -0,0 +1,461 @@
+/*
+ * 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.dunit.standalone;
+
+import hydra.Log;
+import hydra.MethExecutorResult;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.URISyntaxException;
+import java.nio.channels.FileChannel;
+import java.nio.charset.Charset;
+import java.rmi.AccessException;
+import java.rmi.AlreadyBoundException;
+import java.rmi.NotBoundException;
+import java.rmi.Remote;
+import java.rmi.RemoteException;
+import java.rmi.registry.LocateRegistry;
+import java.rmi.registry.Registry;
+import java.rmi.server.UnicastRemoteObject;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.appender.FileAppender;
+import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.apache.logging.log4j.core.layout.PatternLayout;
+import org.junit.Assert;
+
+import batterytest.greplogs.ExpectedStrings;
+import batterytest.greplogs.LogConsumer;
+
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+import dunit.DUnitEnv;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.VM;
+
+/**
+ * A class to build a fake test configuration and launch some DUnit VMS.
+ * 
+ * For use within eclipse. This class completely skips hydra and just starts
+ * some vms directly, creating a fake test configuration
+ * 
+ * Also, it's a good idea to set your working directory, because the test code
+ * a lot of files that it leaves around.
+ * 
+ * @author dsmith
+ *
+ */
+public class DUnitLauncher {
+
+  /** change this to use a different log level in unit tests */
+  public static final String LOG_LEVEL = System.getProperty("logLevel", "info");
+  
+  static int locatorPort;
+
+  private static final int NUM_VMS = 4;
+  private static final int DEBUGGING_VM_NUM = -1;
+  private static final int LOCATOR_VM_NUM = -2;
+
+  static final long STARTUP_TIMEOUT = 30 * 1000;
+  private static final String SUSPECT_FILENAME = "dunit_suspect.log";
+  private static File DUNIT_SUSPECT_FILE;
+
+  public static final String DUNIT_DIR = "dunit";
+  public static final String WORKSPACE_DIR_PARAM = "WORKSPACE_DIR";
+  public static final boolean LOCATOR_LOG_TO_DISK = Boolean.getBoolean("locatorLogToDisk");
+
+  static final String MASTER_PARAM = "DUNIT_MASTER";
+  static final String RMI_PORT_PARAM = "gemfire.DUnitLauncher.RMI_PORT";
+  static final String VM_NUM_PARAM = "gemfire.DUnitLauncher.VM_NUM";
+
+  private static final String LAUNCHED_PROPERTY = "gemfire.DUnitLauncher.LAUNCHED";
+
+  private static Master master;
+
+  private DUnitLauncher() {
+  }
+  
+  private static boolean isHydra() {
+    try {
+      //TODO - this is hacky way to test for a hydra environment - see
+      //if there is registered test configuration object.
+      Class<?> clazz = Class.forName("hydra.TestConfig");
+      Method getInstance = clazz.getMethod("getInstance", new Class[0]);
+      getInstance.invoke(null);
+      return true;
+    } catch (Exception e) {
+      return false;
+    }
+  }
+  /**
+   * Launch DUnit. If the unit test was launched through
+   * the hydra framework, leave the test alone.
+   */
+  public static void launchIfNeeded() {
+    if(System.getProperties().contains(VM_NUM_PARAM)) {
+      //we're a dunit child vm, do nothing.
+      return;
+    }
+
+    if(!isHydra() &&!isLaunched()) {
+      try {
+        launch();
+      } catch (Exception e) {
+        throw new RuntimeException("Unable to launch dunit VMS", e);
+      }
+    }
+  }
+  
+  /**
+   * Test it see if the eclise dunit environment is launched.
+   */
+  public static boolean isLaunched() {
+    return Boolean.getBoolean(LAUNCHED_PROPERTY);
+  }
+  
+  public static String getLocatorString() {
+    return "localhost[" + locatorPort + "]";
+  }
+
+  
+  private static void launch() throws URISyntaxException, AlreadyBoundException, IOException, InterruptedException, NotBoundException  {
+//  initialize the log writer that hydra uses
+    Log.createLogWriter( "dunit-master", LOG_LEVEL );
+
+    DUNIT_SUSPECT_FILE = new File(SUSPECT_FILENAME);
+    DUNIT_SUSPECT_FILE.delete();
+    DUNIT_SUSPECT_FILE.deleteOnExit();
+    
+    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+     
+    //create an RMI registry and add an object to share our tests config
+    int namingPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    Registry registry = LocateRegistry.createRegistry(namingPort);
+
+    final ProcessManager processManager = new ProcessManager(namingPort, registry);
+    master = new Master(registry, processManager);
+    registry.bind(MASTER_PARAM, master);
+
+    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();
+      }
+    });
+    
+    //Create a VM for the locator
+    processManager.launchVM(LOCATOR_VM_NUM);
+    
+    //Launch an initial set of VMs
+    for(int i=0; i < NUM_VMS; i++) {
+      processManager.launchVM(i);
+    }
+    
+    //wait for the VMS to start up
+    if(!processManager.waitForVMs(STARTUP_TIMEOUT)) {
+      throw new RuntimeException("VMs did not start up with 30 seconds");
+    }
+    
+    //populate the Host class with our stubs. The tests use this host class
+    DUnitHost host = new DUnitHost(InetAddress.getLocalHost().getCanonicalHostName(), processManager);
+    host.init(registry, NUM_VMS);
+
+    init(master);
+    
+    startLocator(registry);
+  }
+  
+  public static Properties getDistributedSystemProperties() {
+    Properties p = new Properties();
+    p.setProperty("locators", getLocatorString());
+    p.setProperty("mcast-port", "0");
+    p.setProperty("enable-cluster-configuration", "false");
+    p.setProperty("use-cluster-configuration", "false");
+    p.setProperty("log-level", LOG_LEVEL);
+    return p;
+  }
+
+  /**
+   * Add an appender to Log4j which sends all INFO+ messages to a separate file
+   * which will be used later to scan for suspect strings.  The pattern of the
+   * messages conforms to the original log format so that hydra will be able
+   * to parse them.
+   */
+  private static void addSuspectFileAppender(final String workspaceDir) {
+    final String suspectFilename = new File(workspaceDir, SUSPECT_FILENAME).getAbsolutePath();
+
+    final LoggerContext appenderContext = ((org.apache.logging.log4j.core.Logger)
+        LogManager.getLogger(LogService.BASE_LOGGER_NAME)).getContext();
+
+    final PatternLayout layout = PatternLayout.createLayout(
+        "[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} <%thread> tid=%tid] %message%n%throwable%n", null, null,
+        Charset.defaultCharset(), true, false, "", "");
+    
+    final FileAppender fileAppender = FileAppender.createAppender(suspectFilename, "true", "false",
+        DUnitLauncher.class.getName(), "true", "false", "false", "0", layout, null, null, null, appenderContext.getConfiguration());
+    fileAppender.start();
+
+    LoggerConfig loggerConfig = appenderContext.getConfiguration().getLoggerConfig(LogService.BASE_LOGGER_NAME);
+    loggerConfig.addAppender(fileAppender, Level.INFO, null);
+  }
+  
+  private static void startLocator(Registry registry) throws IOException, NotBoundException {
+    RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + LOCATOR_VM_NUM);
+    final File locatorLogFile =
+        LOCATOR_LOG_TO_DISK ? new File("locator-" + locatorPort + ".log") : new File(""); 
+    MethExecutorResult result = remote.executeMethodOnObject(new SerializableCallable() {
+      public Object call() throws IOException {
+        Properties p = getDistributedSystemProperties();
+        // I never want this locator to end up starting a jmx manager
+        // since it is part of the unit test framework
+        p.setProperty("jmx-manager", "false");
+        //Disable the shared configuration on this locator.
+        //Shared configuration tests create their own locator
+        p.setProperty("enable-cluster-configuration", "false");
+        //Tell the locator it's the first in the system for
+        //faster boot-up
+        
+        System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
+        try {
+          Locator.startLocatorAndDS(locatorPort, locatorLogFile, p);
+        } finally {
+          System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
+        }
+        
+        return null;
+      }
+    }, "call");
+    if(result.getException() != null) {
+      RuntimeException ex = new RuntimeException("Failed to start locator", result.getException());
+      ex.printStackTrace();
+      throw ex;
+    }
+  }
+
+  public static void init(MasterRemote master) {
+    DUnitEnv.set(new StandAloneDUnitEnv(master));
+    //fake out tests that are using a bunch of hydra stuff
+    String workspaceDir = System.getProperty(DUnitLauncher.WORKSPACE_DIR_PARAM) ;
+    workspaceDir = workspaceDir == null ? new File(".").getAbsolutePath() : workspaceDir;
+    
+    addSuspectFileAppender(workspaceDir);
+    
+    //Free off heap memory when disconnecting from the distributed system
+    System.setProperty("gemfire.free-off-heap-memory", "true");
+    
+    //indicate that this CM is controlled by the eclipse dunit.
+    System.setProperty(LAUNCHED_PROPERTY, "true");
+  }
+  
+  public static void closeAndCheckForSuspects() {
+    if (isLaunched()) {
+      final boolean skipLogMsgs = ExpectedStrings.skipLogMsgs("dunit");
+      final List<?> expectedStrings = ExpectedStrings.create("dunit");
+      final LogConsumer logConsumer = new LogConsumer(skipLogMsgs, expectedStrings, "log4j", 5);
+
+      final StringBuilder suspectStringBuilder = new StringBuilder();
+
+      BufferedReader buffReader = null;
+      FileChannel fileChannel = null;
+      try {
+        fileChannel = new FileOutputStream(DUNIT_SUSPECT_FILE, true).getChannel();
+        buffReader = new BufferedReader(new FileReader(DUNIT_SUSPECT_FILE));
+      } catch (FileNotFoundException e) {
+        System.err.println("Could not find the suspect string output file: " + e);
+        return;
+      }
+      try {
+        String line;
+        try {
+          while ((line = buffReader.readLine()) != null) {
+            final StringBuilder builder = logConsumer.consume(line);
+            if (builder != null) {
+              suspectStringBuilder.append(builder);
+            }
+          }
+        } catch (IOException e) {
+          System.err.println("Could not read the suspect string output file: " + e);
+        }
+        
+        try {
+          fileChannel.truncate(0);
+        } catch (IOException e) {
+          System.err.println("Could not truncate the suspect string output file: " + e);
+        }
+        
+      } finally {
+        try {
+          buffReader.close();
+          fileChannel.close();
+        } catch (IOException e) {
+          System.err.println("Could not close the suspect string output file: " + e);
+        }
+      }
+
+      if (suspectStringBuilder.length() != 0) {
+        System.err.println("Suspicious strings were written to the log during this run.\n"
+            + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n"
+            + suspectStringBuilder);
+        
+        Assert.fail("Suspicious strings were written to the log during this run.\n"
+            + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n"
+            + suspectStringBuilder);
+      }
+    }
+  }
+
+  public interface MasterRemote extends Remote {
+    public int getLocatorPort() throws RemoteException;
+    public void signalVMReady() throws RemoteException;
+    public void ping() throws RemoteException;
+    public BounceResult bounce(int pid) throws RemoteException;
+  }
+  
+  public static class Master extends UnicastRemoteObject implements MasterRemote {
+    private static final long serialVersionUID = 1178600200232603119L;
+    
+    private final Registry registry;
+    private final ProcessManager processManager;
+
+
+    public Master(Registry registry, ProcessManager processManager) throws RemoteException {
+      this.processManager = processManager;
+      this.registry = registry;
+    }
+
+    public int getLocatorPort()  throws RemoteException{
+      return locatorPort;
+    }
+
+    public synchronized void signalVMReady() {
+      processManager.signalVMReady();
+    }
+    
+    public void ping() {
+      //do nothing
+    }
+
+    @Override
+    public BounceResult bounce(int pid) {
+      processManager.bounce(pid);
+      
+      try {
+        if(!processManager.waitForVMs(STARTUP_TIMEOUT)) {
+          throw new RuntimeException("VMs did not start up with 30 seconds");
+        }
+        RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + pid);
+        return new BounceResult(pid, remote);
+      } catch (RemoteException | NotBoundException e) {
+        throw new RuntimeException("could not lookup name", e);
+      } catch (InterruptedException e) {
+        throw new RuntimeException("Failed waiting for VM", e);
+      }
+    }
+  }
+  
+  private static class DUnitHost extends Host {
+    private static final long serialVersionUID = -8034165624503666383L;
+    
+    private transient final VM debuggingVM;
+
+    private transient ProcessManager processManager;
+    
+    public DUnitHost(String hostName, ProcessManager processManager) throws RemoteException {
+      super(hostName);
+      this.debuggingVM = new VM(this, -1, new RemoteDUnitVM());
+      this.processManager = processManager;
+    }
+    
+    public void init(Registry registry, int numVMs) throws AccessException, RemoteException, NotBoundException, InterruptedException {
+      for(int i = 0; i < numVMs; i++) {
+        RemoteDUnitVMIF remote = processManager.getStub(i);
+        addVM(i, remote);
+      }
+      
+      addLocator(LOCATOR_VM_NUM, processManager.getStub(LOCATOR_VM_NUM));
+      
+      addHost(this);
+    }
+
+    @Override
+    public VM getVM(int n) {
+      
+      if(n == DEBUGGING_VM_NUM) {
+        //for ease of debugging, pass -1 to get the local VM
+        return debuggingVM;
+      }
+
+      int oldVMCount = getVMCount();
+      if(n >= oldVMCount) {
+        //If we don't have a VM with that number, dynamically create it.
+        try {
+          for(int i = oldVMCount; i <= n; i++) {
+            processManager.launchVM(i);
+          }
+          processManager.waitForVMs(STARTUP_TIMEOUT);
+
+          for(int i = oldVMCount; i <= n; i++) {
+            addVM(i, processManager.getStub(i));
+          }
+
+        } catch (IOException | InterruptedException | NotBoundException e) {
+          throw new RuntimeException("Could not dynamically launch vm + " + n, e);
+        }
+      }
+      
+      return super.getVM(n);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
new file mode 100644
index 0000000..7b053b6
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
@@ -0,0 +1,259 @@
+/*
+ * 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.dunit.standalone;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintStream;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.rmi.AccessException;
+import java.rmi.NotBoundException;
+import java.rmi.RemoteException;
+import java.rmi.registry.Registry;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.io.FileUtils;
+
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+/**
+ * @author dsmith
+ *
+ */
+public class ProcessManager {
+  private int namingPort;
+  private Map<Integer, ProcessHolder> processes = new HashMap<Integer, ProcessHolder>();
+  private File log4jConfig;
+  private int pendingVMs;
+  private Registry registry;
+  private int debugPort = Integer.getInteger("dunit.debug.basePort", 0);
+  private int suspendVM = Integer.getInteger("dunit.debug.suspendVM", -100);
+
+  public ProcessManager(int namingPort, Registry registry) {
+    this.namingPort = namingPort;
+    this.registry = registry;
+  }
+  
+  public void launchVMs() throws IOException, NotBoundException {
+    log4jConfig = LogService.findLog4jConfigInCurrentDir();
+  }
+
+  public synchronized void launchVM(int vmNum) throws IOException {
+    if(processes.containsKey(vmNum)) {
+      throw new IllegalStateException("VM " + vmNum + " is already running.");
+    }
+    
+    String[] cmd = buildJavaCommand(vmNum, namingPort);
+    System.out.println("Executing " + Arrays.asList(cmd));
+    File workingDir = getVMDir(vmNum);
+    try {
+      FileUtil.delete(workingDir);
+    } catch(IOException e) {
+      //This delete is occasionally failing on some platforms, maybe due to a lingering
+      //process. Allow the process to be launched anyway.
+      System.err.println("Unable to delete " + workingDir + ". Currently contains " 
+                          + Arrays.asList(workingDir.list()));
+    }
+    workingDir.mkdirs();
+    if (log4jConfig != null) {
+      FileUtils.copyFileToDirectory(log4jConfig, workingDir);
+    }
+    
+    //TODO - delete directory contents, preferably with commons io FileUtils
+    Process process = Runtime.getRuntime().exec(cmd, null, workingDir);
+    pendingVMs++;
+    ProcessHolder holder = new ProcessHolder(process);
+    processes.put(vmNum, holder);
+    linkStreams(vmNum, holder, process.getErrorStream(), System.err);
+    linkStreams(vmNum, holder, process.getInputStream(), System.out);
+  }
+
+  public static File getVMDir(int vmNum) {
+    return new File(DUnitLauncher.DUNIT_DIR, "vm" + vmNum);
+  }
+  
+  public synchronized void killVMs() {
+    for(ProcessHolder process : processes.values()) {
+      if(process != null) {
+        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);
+    }
+    try {
+      ProcessHolder holder = processes.remove(vmNum);
+      holder.kill();
+      holder.getProcess().waitFor();
+      launchVM(vmNum);
+    } catch (InterruptedException | IOException e) {
+      throw new RuntimeException("Unable to restart VM " + vmNum, e);
+    }
+  }
+   
+  private void linkStreams(final int vmNum, final ProcessHolder holder, final InputStream in, final PrintStream out) {
+    Thread ioTransport = new Thread() {
+      public void run() {
+        BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+        String vmName = (vmNum==-2)? "[locator]" : "[vm_"+vmNum+"]";
+        try {
+          String line = reader.readLine();
+          while(line != null) {
+            if (line.length() == 0) {
+              out.println();
+            } else {
+              out.print(vmName);
+              out.println(line);
+            }
+            line = reader.readLine();
+          }
+        } catch(Exception e) {
+          if(!holder.isKilled()) {
+            out.println("Error transporting IO from child process");
+            e.printStackTrace(out);
+          }
+        }
+      }
+    };
+
+    ioTransport.setDaemon(true);
+    ioTransport.start();
+  }
+
+  private String[] buildJavaCommand(int vmNum, int namingPort) {
+    String cmd = System.getProperty( "java.home" ) + File.separator + "bin" + File.separator + "java";
+    String classPath = System.getProperty("java.class.path");
+    //String tmpDir = System.getProperty("java.io.tmpdir");
+    String agent = getAgentString();
+
+    String jdkDebug = "";
+    if (debugPort > 0) {
+      jdkDebug += ",address=" + debugPort;
+      debugPort++;
+    }
+
+    String jdkSuspend = vmNum == suspendVM ? "y" : "n";
+
+    return new String[] {
+      cmd, "-classpath", classPath,
+      "-D" + DUnitLauncher.RMI_PORT_PARAM + "=" + namingPort,
+      "-D" + DUnitLauncher.VM_NUM_PARAM + "=" + vmNum,
+      "-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=" + jdkSuspend + jdkDebug,
+      "-XX:+HeapDumpOnOutOfMemoryError",
+      "-Xmx512m",
+      "-Dgemfire.DEFAULT_MAX_OPLOG_SIZE=10",
+      "-Dgemfire.disallowMcastDefaults=true",
+      "-ea",
+      agent,
+      "dunit.standalone.ChildVM"
+    };
+  }
+  
+  /**
+   * Get the java agent passed to this process and pass it to the child VMs.
+   * This was added to support jacoco code coverage reports
+   */
+  private String getAgentString() {
+    RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
+    if (runtimeBean != null) {
+      for(String arg: runtimeBean.getInputArguments()) {
+        if(arg.contains("-javaagent:")) {
+          //HACK for gradle bug  GRADLE-2859. Jacoco is passing a relative path
+          //That won't work when we pass this to dunit VMs in a different 
+          //directory
+          arg = arg.replace("-javaagent:..", "-javaagent:" + System.getProperty("user.dir") + File.separator + "..");
+          arg = arg.replace("destfile=..", "destfile=" + System.getProperty("user.dir") + File.separator + "..");
+          return arg;
+        }
+      }
+    }
+    
+    return "-DdummyArg=true";
+  }
+
+  synchronized void signalVMReady() {
+    pendingVMs--;
+    this.notifyAll();
+  }
+  
+  public synchronized boolean waitForVMs(long timeout) throws InterruptedException {
+    long end = System.currentTimeMillis() + timeout;
+    while(pendingVMs > 0) {
+      long remaining = end - System.currentTimeMillis();
+      if(remaining <= 0) {
+        return false;
+      }
+      this.wait(remaining);
+    }
+    
+    return true;
+  }
+  
+  private static class ProcessHolder {
+    private final Process process;
+    private volatile boolean killed = false;
+    
+    public ProcessHolder(Process process) {
+      this.process = process;
+    }
+
+    public void kill() {
+      this.killed = true;
+      process.destroy();
+      
+    }
+
+    public Process getProcess() {
+      return process;
+    }
+
+    public boolean isKilled() {
+      return killed;
+    }
+    
+    public boolean isAlive() {
+      return !killed && process.isAlive();
+    }
+  }
+
+  public RemoteDUnitVMIF getStub(int i) throws AccessException, RemoteException, NotBoundException, InterruptedException {
+    waitForVMs(DUnitLauncher.STARTUP_TIMEOUT);
+    return (RemoteDUnitVMIF) registry.lookup("vm" + i);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java
new file mode 100644
index 0000000..51c6177
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java
@@ -0,0 +1,142 @@
+/*
+ * 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.dunit.standalone;
+
+import java.rmi.RemoteException;
+import java.rmi.server.UnicastRemoteObject;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.internal.logging.LogService;
+
+import hydra.MethExecutor;
+import hydra.MethExecutorResult;
+
+/**
+ * @author dsmith
+ *
+ */
+public class RemoteDUnitVM extends UnicastRemoteObject implements RemoteDUnitVMIF {
+  
+  private static final Logger logger = LogService.getLogger();
+  
+  public RemoteDUnitVM() throws RemoteException {
+    super();
+  }
+
+  /** 
+   * Called remotely by the master controller to cause the client to execute 
+   * the instance method on the object.  Does this synchronously (does not spawn
+   * a thread).  This method is used by the unit test framework, dunit.
+   *
+   * @param obj the object to execute the method on
+   * @param methodName the name of the method to execute
+   * @return the result of method execution
+   */ 
+   public MethExecutorResult executeMethodOnObject( Object obj, String methodName ) {
+     String name = obj.getClass().getName() + "." + methodName + 
+       " on object: " + obj;
+     logger.info("Received method: " + name);
+     long start = System.currentTimeMillis();
+     MethExecutorResult result = MethExecutor.executeObject( obj, methodName );
+     long delta = System.currentTimeMillis() - start;
+     logger.info( "Got result: " + result.toString().trim()  + " from " +
+               name + " (took " + delta + " ms)");
+     return result;
+   }
+
+   /**
+    * Executes a given instance method on a given object with the given
+    * arguments. 
+    */
+   public MethExecutorResult executeMethodOnObject(Object obj,
+                                                   String methodName,
+                                                   Object[] args) {
+     String name = obj.getClass().getName() + "." + methodName + 
+              (args != null ? " with " + args.length + " args": "") +
+       " on object: " + obj;
+     logger.info("Received method: " + name);
+     long start = System.currentTimeMillis();
+     MethExecutorResult result = 
+       MethExecutor.executeObject(obj, methodName, args);
+     long delta = System.currentTimeMillis() - start;
+     logger.info( "Got result: " + result.toString() + " from " + name + 
+               " (took " + delta + " ms)");
+     return result;
+   }
+
+  /** 
+   * Called remotely by the master controller to cause the client to execute 
+   * the method on the class.  Does this synchronously (does not spawn a thread).
+   * This method is used by the unit test framework, dunit.
+   *
+   * @param className the name of the class execute
+   * @param methodName the name of the method to execute
+   * @return the result of method execution
+   */ 
+   public MethExecutorResult executeMethodOnClass( String className, String methodName ) {
+     String name = className + "." + methodName;
+     logger.info("Received method: " +  name);
+     long start = System.currentTimeMillis();
+     MethExecutorResult result = MethExecutor.execute( className, methodName );
+     long delta = System.currentTimeMillis() - start;
+     logger.info( "Got result: " + result.toString() + " from " + name + 
+               " (took " + delta + " ms)");
+     
+     return result;
+   }
+
+   /**
+    * Executes a given static method in a given class with the given
+    * arguments. 
+    */
+   public MethExecutorResult executeMethodOnClass(String className,
+                                                  String methodName,
+                                                  Object[] args) {
+     String name = className + "." + methodName + 
+       (args != null ? " with " + args.length + " args": "");
+     logger.info("Received method: " + name);
+     long start = System.currentTimeMillis();
+     MethExecutorResult result = 
+       MethExecutor.execute(className, methodName, args);
+     long delta = System.currentTimeMillis() - start;
+     logger.info( "Got result: " + result.toString() + " from " + name +
+               " (took " + delta + " ms)");
+     return result;
+   }
+
+  public void executeTask(int tsid, int type, int index) throws RemoteException {
+    throw new UnsupportedOperationException();
+    
+  }
+  
+  public void runShutdownHook() throws RemoteException {
+    
+  }
+
+  public void notifyDynamicActionComplete(int actionId) throws RemoteException {
+    throw new UnsupportedOperationException();
+    
+  }
+
+  public void shutDownVM() throws RemoteException {
+    ChildVM.stopVM();
+  }
+
+  public void disconnectVM() throws RemoteException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java
new file mode 100644
index 0000000..849e2f2
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java
@@ -0,0 +1,36 @@
+/*
+ * 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.dunit.standalone;
+
+import hydra.MethExecutorResult;
+
+import java.rmi.Remote;
+import java.rmi.RemoteException;
+
+public interface RemoteDUnitVMIF extends Remote {
+
+  MethExecutorResult executeMethodOnObject(Object o, String methodName) throws RemoteException;
+
+  MethExecutorResult executeMethodOnObject(Object o, String methodName,
+      Object[] args) throws RemoteException;
+
+  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/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java
new file mode 100644
index 0000000..eef24fe
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.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.test.dunit.standalone;
+
+import java.io.File;
+import java.rmi.RemoteException;
+import java.util.Properties;
+
+import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher.MasterRemote;
+
+import dunit.DUnitEnv;
+
+public class StandAloneDUnitEnv extends DUnitEnv {
+
+  private MasterRemote master;
+
+  public StandAloneDUnitEnv(MasterRemote master) {
+    this.master = master;
+  }
+
+  @Override
+  public String getLocatorString() {
+    return DUnitLauncher.getLocatorString();
+  }
+
+  @Override
+  public String getLocatorAddress() {
+    return "localhost";
+  }
+  
+  @Override
+  public int getLocatorPort() {
+    return DUnitLauncher.locatorPort;
+  }
+
+  @Override
+  public Properties getDistributedSystemProperties() {
+    return DUnitLauncher.getDistributedSystemProperties();
+  }
+
+  @Override
+  public int getPid() {
+    return Integer.getInteger(DUnitLauncher.VM_NUM_PARAM, -1).intValue();
+  }
+
+  @Override
+  public int getVMID() {
+    return getPid();
+  }
+
+  @Override
+  public BounceResult bounce(int pid) throws RemoteException {
+    return master.bounce(pid);
+  }
+
+  @Override
+  public File getWorkingDirectory(int pid) {
+    return ProcessManager.getVMDir(pid);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
new file mode 100644
index 0000000..76faf93
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.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.test.dunit.tests;
+
+import java.util.Properties;
+
+import dunit.AsyncInvocation;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.RMIException;
+import dunit.VM;
+
+/**
+ * This class tests the basic functionality of the distributed unit
+ * test framework.
+ */
+public class BasicDUnitTest extends DistributedTestCase {
+
+  public BasicDUnitTest(String name) {
+    super(name);
+  }
+
+  ////////  Test Methods
+
+  /**
+   * Tests how the Hydra framework handles an error
+   */
+  public void _testDontCatchRemoteException() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    vm.invoke(this.getClass(), "remoteThrowException");
+  }
+
+  public void testRemoteInvocationWithException() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invoke(this.getClass(), "remoteThrowException");
+      fail("Should have thrown a BasicTestException");
+
+    } catch (RMIException ex) {
+      assertTrue(ex.getCause() instanceof BasicTestException);
+    }
+  } 
+
+  static class BasicTestException extends RuntimeException {
+    BasicTestException() {
+      this("Test exception.  Please ignore.");
+    }
+
+    BasicTestException(String s) {
+      super(s);
+    }
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   *
+   */
+  protected static void remoteThrowException() {
+    String s = "Test exception.  Please ignore.";
+    throw new BasicTestException(s);
+  }
+
+  public void _testRemoteInvocationBoolean() {
+
+  }
+
+  public void testRemoteInvokeAsync() throws InterruptedException {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    String name = this.getUniqueName();
+    String value = "Hello";
+
+    AsyncInvocation ai =
+      vm.invokeAsync(this.getClass(), "remoteBind", 
+                     new Object[] { name, value });
+    ai.join();
+    // TODO shouldn't we call fail() here?
+    if (ai.exceptionOccurred()) {
+      fail("remoteBind failed", ai.getException());
+    }
+
+    ai = vm.invokeAsync(this.getClass(), "remoteValidateBind",
+                        new Object[] {name, value });
+    ai.join();
+    if (ai.exceptionOccurred()) {
+      fail("remoteValidateBind failed", ai.getException());
+    }
+  }
+
+  private static Properties bindings = new Properties();
+  private static void remoteBind(String name, String s) {
+    new BasicDUnitTest("bogus").getSystem(); // forces connection
+    bindings.setProperty(name, s);
+  }
+
+  private static void remoteValidateBind(String name, String expected)
+  {
+    assertEquals(expected, bindings.getProperty(name));
+  }
+
+  public void testRemoteInvokeAsyncWithException() 
+    throws InterruptedException {
+
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+//    String name = this.getUniqueName();
+//    String value = "Hello";
+
+    AsyncInvocation ai =
+      vm.invokeAsync(this.getClass(), "remoteThrowException");
+    ai.join();
+    assertTrue(ai.exceptionOccurred());
+    Throwable ex = ai.getException();
+    assertTrue(ex instanceof BasicTestException);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java
new file mode 100644
index 0000000..abbe229
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java
@@ -0,0 +1,50 @@
+/*
+ * 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.dunit.tests;
+
+import dunit.*;
+
+/**
+ * The tests in this class always fail.  It is used when developing
+ * DUnit to give us an idea of how test failure are logged, etc.
+ *
+ * @author David Whitlock
+ *
+ * @since 3.0
+ */
+public class TestFailure extends DistributedTestCase {
+
+  public TestFailure(String name) {
+    super(name);
+  }
+
+  ////////  Test Methods
+
+  public void testFailure() {
+    assertTrue("Test Failure", false);
+  }
+
+  public void testError() {
+    String s = "Test Error";
+    throw new Error(s);
+  }
+
+  public void testHang() throws InterruptedException {
+    Thread.sleep(100000 * 1000);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java
new file mode 100644
index 0000000..3562f86
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java
@@ -0,0 +1,237 @@
+/*
+ * 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.dunit.tests;
+
+import dunit.*;
+
+import java.io.Serializable;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * This class tests the functionality of the {@link VM} class.
+ */
+public class VMDUnitTest extends DistributedTestCase {
+
+  private static final boolean BOOLEAN_VALUE = true;
+  private static final byte BYTE_VALUE = (byte) 40;
+  private static final long LONG_VALUE = 42L;
+  private static final String STRING_VALUE = "BLAH BLAH BLAH";
+
+  public VMDUnitTest(String name) {
+    super(name);
+  }
+
+  ////////  Test Methods
+
+  public void notestInvokeNonExistentMethod() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invoke(VMDUnitTest.class, "nonExistentMethod");
+      fail("Should have thrown an RMIException");
+
+    } catch (RMIException ex) {
+      String s = "Excepted a NoSuchMethodException, got a " +
+        ex.getCause();;
+      assertTrue(s, ex.getCause() instanceof NoSuchMethodException);
+    }
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   * @return
+   */
+  protected static byte remoteByteMethod() {
+    return BYTE_VALUE;
+  }
+
+  public void notestInvokeStaticBoolean() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(BOOLEAN_VALUE,
+                 vm.invokeBoolean(VMDUnitTest.class, "remoteBooleanMethod")); 
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   * @return
+   */
+  protected static boolean remoteBooleanMethod() {
+    return BOOLEAN_VALUE;
+  }
+
+  public void notestInvokeStaticBooleanNotBoolean() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invokeBoolean(VMDUnitTest.class, "remoteByteMethod");
+      fail("Should have thrown an IllegalArgumentException");
+
+    } catch (IllegalArgumentException ex) {
+      
+    }
+  }
+
+  public void notestInvokeStaticLong() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(LONG_VALUE,
+                 vm.invokeLong(VMDUnitTest.class, "remoteLongMethod")); 
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   * @return
+   */
+  protected static long remoteLongMethod() {
+    return LONG_VALUE;
+  }
+
+  public void notestInvokeStaticLongNotLong() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invokeLong(VMDUnitTest.class, "remoteByteMethod");
+      fail("Should have thrown an IllegalArgumentException");
+
+    } catch (IllegalArgumentException ex) {
+      
+    }
+  }
+
+  protected static class ClassWithLong implements Serializable {
+    public long getLong() {
+      return LONG_VALUE;
+    }
+  }
+
+  protected static class ClassWithByte implements Serializable {
+    public byte getByte() {
+      return BYTE_VALUE;
+    }
+  }
+
+  public void notestInvokeInstanceLong() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(LONG_VALUE,
+                 vm.invokeLong(new ClassWithLong(), "getLong"));
+  }
+
+  public void notestInvokeInstanceLongNotLong() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invokeLong(new ClassWithByte(), "getByte");
+      fail("Should have thrown an IllegalArgumentException");
+
+    } catch (IllegalArgumentException ex) {
+
+    }
+  }
+
+  protected static class InvokeRunnable
+    implements Serializable, Runnable {
+
+    public void run() {
+      throw new BasicDUnitTest.BasicTestException();
+    }
+  }
+
+  protected static class ClassWithString implements Serializable {
+    public String getString() {
+      return STRING_VALUE;
+    }
+  }
+
+  public void notestInvokeInstance() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(STRING_VALUE,
+                 vm.invoke(new ClassWithString(), "getString"));
+  }
+
+  public void notestInvokeRunnable() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invoke(new InvokeRunnable());
+      fail("Should have thrown a BasicTestException");
+
+    } catch (RMIException ex) {
+      assertTrue(ex.getCause() instanceof BasicDUnitTest.BasicTestException);
+    }
+  }
+  
+  private static final AtomicInteger COUNTER = new AtomicInteger();
+  public static Integer getAndIncStaticCount() {
+    return new Integer(COUNTER.getAndIncrement());
+  }
+  public static Integer incrementStaticCount(Integer inc) {
+    return new Integer(COUNTER.addAndGet(inc.intValue()));
+  }
+  public static void incStaticCount() {
+    COUNTER.incrementAndGet();
+  }
+  public static class VMTestObject implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final AtomicInteger val;
+    public VMTestObject(int init) {
+      this.val = new AtomicInteger(init);
+    }
+    public Integer get() {
+      return new Integer(this.val.get());
+    }
+    public Integer incrementAndGet() {
+      return new Integer(this.val.incrementAndGet());
+    }
+    public void set(Integer newVal) {
+      this.val.set(newVal.intValue());
+    }
+  }
+  public void testReturnValue() throws Exception {
+    final Host host = Host.getHost(0);
+    final VM vm = host.getVM(0);
+    // Assert class static invocation works
+    AsyncInvocation a1 = vm.invokeAsync(getClass(), "getAndIncStaticCount");
+    a1.join();
+    assertEquals(new Integer(0), a1.getReturnValue());
+    // Assert class static invocation with args works
+    a1 = vm.invokeAsync(getClass(), "incrementStaticCount", new Object[] {new Integer(2)});
+    a1.join();
+    assertEquals(new Integer(3), a1.getReturnValue());
+    // Assert that previous values are not returned when invoking method w/ no return val
+    a1 = vm.invokeAsync(getClass(), "incStaticCount");
+    a1.join();
+    assertNull(a1.getReturnValue());
+    // Assert that previous null returns are over-written 
+    a1 = vm.invokeAsync(getClass(), "getAndIncStaticCount");
+    a1.join();
+    assertEquals(new Integer(4), a1.getReturnValue());
+
+    // Assert object method invocation works with zero arg method
+    final VMTestObject o = new VMTestObject(0);
+    a1 = vm.invokeAsync(o, "incrementAndGet", new Object[] {});
+    a1.join();
+    assertEquals(new Integer(1), a1.getReturnValue());
+    // Assert object method invocation works with no return
+    a1 = vm.invokeAsync(o, "set", new Object[] {new Integer(3)});
+    a1.join();
+    assertNull(a1.getReturnValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/BounceResult.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/BounceResult.java b/gemfire-core/src/test/java/dunit/BounceResult.java
deleted file mode 100644
index d8ac186..0000000
--- a/gemfire-core/src/test/java/dunit/BounceResult.java
+++ /dev/null
@@ -1,36 +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 dunit;
-
-public class BounceResult {
-  private final int newPid;
-  private final RemoteDUnitVMIF newClient;
-  
-  public BounceResult(int newPid, RemoteDUnitVMIF newClient) {
-    this.newPid = newPid;
-    this.newClient = newClient;
-  }
-
-  public int getNewPid() {
-    return newPid;
-  }
-
-  public RemoteDUnitVMIF getNewClient() {
-    return newClient;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/DUnitEnv.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/DUnitEnv.java b/gemfire-core/src/test/java/dunit/DUnitEnv.java
index 19c3635..54fe67f 100644
--- a/gemfire-core/src/test/java/dunit/DUnitEnv.java
+++ b/gemfire-core/src/test/java/dunit/DUnitEnv.java
@@ -23,6 +23,8 @@ import java.io.File;
 import java.rmi.RemoteException;
 import java.util.Properties;
 
+import com.gemstone.gemfire.test.dunit.standalone.BounceResult;
+
 
 /**
  * This class provides an abstraction over the environment

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/Host.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/Host.java b/gemfire-core/src/test/java/dunit/Host.java
index cc41316..0c69783 100644
--- a/gemfire-core/src/test/java/dunit/Host.java
+++ b/gemfire-core/src/test/java/dunit/Host.java
@@ -18,6 +18,8 @@ package dunit;
 
 import java.util.*;
 
+import com.gemstone.gemfire.test.dunit.standalone.RemoteDUnitVMIF;
+
 /**
  * <P>This class represents a host on which a remote method may be
  * invoked.  It provides access to the VMs and GemFire systems that

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/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
deleted file mode 100644
index 5dffa47..0000000
--- a/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java
+++ /dev/null
@@ -1,36 +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 dunit;
-
-import hydra.MethExecutorResult;
-
-import java.rmi.Remote;
-import java.rmi.RemoteException;
-
-public interface RemoteDUnitVMIF extends Remote {
-
-  MethExecutorResult executeMethodOnObject(Object o, String methodName) throws RemoteException;
-
-  MethExecutorResult executeMethodOnObject(Object o, String methodName,
-      Object[] args) throws RemoteException;
-
-  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/c57a88c6/gemfire-core/src/test/java/dunit/VM.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/VM.java b/gemfire-core/src/test/java/dunit/VM.java
index 3c0a7b7..f4cde93 100644
--- a/gemfire-core/src/test/java/dunit/VM.java
+++ b/gemfire-core/src/test/java/dunit/VM.java
@@ -16,8 +16,6 @@
  */
 package dunit;
 
-import hydra.MethExecutorResult;
-
 import java.io.File;
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -26,6 +24,11 @@ import java.util.concurrent.Callable;
 //import java.util.Iterator;
 //import java.util.Vector;
 
+import hydra.MethExecutorResult;
+
+import com.gemstone.gemfire.test.dunit.standalone.BounceResult;
+import com.gemstone.gemfire.test.dunit.standalone.RemoteDUnitVMIF;
+
 /**
  * This class represents a Java Virtual Machine that runs on a host.
  *

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/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
deleted file mode 100644
index 45a236a..0000000
--- a/gemfire-core/src/test/java/dunit/standalone/ChildVM.java
+++ /dev/null
@@ -1,82 +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 dunit.standalone;
-
-import hydra.HydraRuntimeException;
-import hydra.Log;
-
-import java.rmi.Naming;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.internal.OSProcess;
-import com.gemstone.gemfire.internal.logging.LogService;
-
-import dunit.standalone.DUnitLauncher.MasterRemote;
-
-/**
- * @author dsmith
- *
- */
-public class ChildVM {
-  
-  private static boolean stopMainLoop = false;
-  
-  /**
-   * tells the main() loop to exit
-   */
-  public static void stopVM() {
-    stopMainLoop = true;
-  }
-  
-  static {
-    createHydraLogWriter();
-  }
-  
-  private final static Logger logger = LogService.getLogger();
-  private static RemoteDUnitVM dunitVM;
-  
-  public static void main(String[] args) throws Throwable {
-    try {
-      int namingPort = Integer.getInteger(DUnitLauncher.RMI_PORT_PARAM).intValue();
-      int vmNum = Integer.getInteger(DUnitLauncher.VM_NUM_PARAM).intValue();
-      int pid = OSProcess.getId();
-      logger.info("VM" + vmNum + " is launching" + (pid > 0? " with PID " + pid : ""));
-      MasterRemote holder = (MasterRemote) Naming.lookup("//localhost:" + namingPort + "/" + DUnitLauncher.MASTER_PARAM);
-      DUnitLauncher.init(holder);
-      DUnitLauncher.locatorPort = holder.getLocatorPort();
-      dunitVM = new RemoteDUnitVM();
-      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 (!stopMainLoop) {
-        holder.ping();
-        Thread.sleep(1000);
-      }
-    } catch (Throwable t) {
-      t.printStackTrace();
-      System.exit(1);
-    }
-  }
-
-  private static void createHydraLogWriter() {
-    try {
-      Log.createLogWriter("dunit-childvm", "fine");
-    } catch (HydraRuntimeException ignore) {
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/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
deleted file mode 100644
index 72c33d6..0000000
--- a/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
+++ /dev/null
@@ -1,463 +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 dunit.standalone;
-
-import hydra.Log;
-import hydra.MethExecutorResult;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.net.InetAddress;
-import java.net.URISyntaxException;
-import java.nio.channels.FileChannel;
-import java.nio.charset.Charset;
-import java.rmi.AccessException;
-import java.rmi.AlreadyBoundException;
-import java.rmi.NotBoundException;
-import java.rmi.Remote;
-import java.rmi.RemoteException;
-import java.rmi.registry.LocateRegistry;
-import java.rmi.registry.Registry;
-import java.rmi.server.UnicastRemoteObject;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.core.LoggerContext;
-import org.apache.logging.log4j.core.appender.FileAppender;
-import org.apache.logging.log4j.core.config.LoggerConfig;
-import org.apache.logging.log4j.core.layout.PatternLayout;
-import org.junit.Assert;
-
-import batterytest.greplogs.ExpectedStrings;
-import batterytest.greplogs.LogConsumer;
-
-import com.gemstone.gemfire.distributed.Locator;
-import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.internal.logging.LogService;
-
-import dunit.BounceResult;
-import dunit.DUnitEnv;
-import dunit.Host;
-import dunit.RemoteDUnitVMIF;
-import dunit.SerializableCallable;
-import dunit.VM;
-
-/**
- * A class to build a fake test configuration and launch some DUnit VMS.
- * 
- * For use within eclipse. This class completely skips hydra and just starts
- * some vms directly, creating a fake test configuration
- * 
- * Also, it's a good idea to set your working directory, because the test code
- * a lot of files that it leaves around.
- * 
- * @author dsmith
- *
- */
-public class DUnitLauncher {
-
-  /** change this to use a different log level in unit tests */
-  public static final String LOG_LEVEL = System.getProperty("logLevel", "info");
-  
-  static int locatorPort;
-
-  private static final int NUM_VMS = 4;
-  private static final int DEBUGGING_VM_NUM = -1;
-  private static final int LOCATOR_VM_NUM = -2;
-
-  static final long STARTUP_TIMEOUT = 30 * 1000;
-  private static final String SUSPECT_FILENAME = "dunit_suspect.log";
-  private static File DUNIT_SUSPECT_FILE;
-
-  public static final String DUNIT_DIR = "dunit";
-  public static final String WORKSPACE_DIR_PARAM = "WORKSPACE_DIR";
-  public static final boolean LOCATOR_LOG_TO_DISK = Boolean.getBoolean("locatorLogToDisk");
-
-  static final String MASTER_PARAM = "DUNIT_MASTER";
-  static final String RMI_PORT_PARAM = "gemfire.DUnitLauncher.RMI_PORT";
-  static final String VM_NUM_PARAM = "gemfire.DUnitLauncher.VM_NUM";
-
-  private static final String LAUNCHED_PROPERTY = "gemfire.DUnitLauncher.LAUNCHED";
-
-  private static Master master;
-
-  private DUnitLauncher() {
-  }
-  
-  private static boolean isHydra() {
-    try {
-      //TODO - this is hacky way to test for a hydra environment - see
-      //if there is registered test configuration object.
-      Class<?> clazz = Class.forName("hydra.TestConfig");
-      Method getInstance = clazz.getMethod("getInstance", new Class[0]);
-      getInstance.invoke(null);
-      return true;
-    } catch (Exception e) {
-      return false;
-    }
-  }
-  /**
-   * Launch DUnit. If the unit test was launched through
-   * the hydra framework, leave the test alone.
-   */
-  public static void launchIfNeeded() {
-    if(System.getProperties().contains(VM_NUM_PARAM)) {
-      //we're a dunit child vm, do nothing.
-      return;
-    }
-
-    if(!isHydra() &&!isLaunched()) {
-      try {
-        launch();
-      } catch (Exception e) {
-        throw new RuntimeException("Unable to launch dunit VMS", e);
-      }
-    }
-  }
-  
-  /**
-   * Test it see if the eclise dunit environment is launched.
-   */
-  public static boolean isLaunched() {
-    return Boolean.getBoolean(LAUNCHED_PROPERTY);
-  }
-  
-  public static String getLocatorString() {
-    return "localhost[" + locatorPort + "]";
-  }
-
-  
-  private static void launch() throws URISyntaxException, AlreadyBoundException, IOException, InterruptedException, NotBoundException  {
-//  initialize the log writer that hydra uses
-    Log.createLogWriter( "dunit-master", LOG_LEVEL );
-
-    DUNIT_SUSPECT_FILE = new File(SUSPECT_FILENAME);
-    DUNIT_SUSPECT_FILE.delete();
-    DUNIT_SUSPECT_FILE.deleteOnExit();
-    
-    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
-     
-    //create an RMI registry and add an object to share our tests config
-    int namingPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    Registry registry = LocateRegistry.createRegistry(namingPort);
-
-    final ProcessManager processManager = new ProcessManager(namingPort, registry);
-    master = new Master(registry, processManager);
-    registry.bind(MASTER_PARAM, master);
-
-    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();
-      }
-    });
-    
-    //Create a VM for the locator
-    processManager.launchVM(LOCATOR_VM_NUM);
-    
-    //Launch an initial set of VMs
-    for(int i=0; i < NUM_VMS; i++) {
-      processManager.launchVM(i);
-    }
-    
-    //wait for the VMS to start up
-    if(!processManager.waitForVMs(STARTUP_TIMEOUT)) {
-      throw new RuntimeException("VMs did not start up with 30 seconds");
-    }
-    
-    //populate the Host class with our stubs. The tests use this host class
-    DUnitHost host = new DUnitHost(InetAddress.getLocalHost().getCanonicalHostName(), processManager);
-    host.init(registry, NUM_VMS);
-
-    init(master);
-    
-    startLocator(registry);
-  }
-  
-  public static Properties getDistributedSystemProperties() {
-    Properties p = new Properties();
-    p.setProperty("locators", getLocatorString());
-    p.setProperty("mcast-port", "0");
-    p.setProperty("enable-cluster-configuration", "false");
-    p.setProperty("use-cluster-configuration", "false");
-    p.setProperty("log-level", LOG_LEVEL);
-    return p;
-  }
-
-  /**
-   * Add an appender to Log4j which sends all INFO+ messages to a separate file
-   * which will be used later to scan for suspect strings.  The pattern of the
-   * messages conforms to the original log format so that hydra will be able
-   * to parse them.
-   */
-  private static void addSuspectFileAppender(final String workspaceDir) {
-    final String suspectFilename = new File(workspaceDir, SUSPECT_FILENAME).getAbsolutePath();
-
-    final LoggerContext appenderContext = ((org.apache.logging.log4j.core.Logger)
-        LogManager.getLogger(LogService.BASE_LOGGER_NAME)).getContext();
-
-    final PatternLayout layout = PatternLayout.createLayout(
-        "[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} <%thread> tid=%tid] %message%n%throwable%n", null, null,
-        Charset.defaultCharset(), true, false, "", "");
-    
-    final FileAppender fileAppender = FileAppender.createAppender(suspectFilename, "true", "false",
-        DUnitLauncher.class.getName(), "true", "false", "false", "0", layout, null, null, null, appenderContext.getConfiguration());
-    fileAppender.start();
-
-    LoggerConfig loggerConfig = appenderContext.getConfiguration().getLoggerConfig(LogService.BASE_LOGGER_NAME);
-    loggerConfig.addAppender(fileAppender, Level.INFO, null);
-  }
-  
-  private static void startLocator(Registry registry) throws IOException, NotBoundException {
-    RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + LOCATOR_VM_NUM);
-    final File locatorLogFile =
-        LOCATOR_LOG_TO_DISK ? new File("locator-" + locatorPort + ".log") : new File(""); 
-    MethExecutorResult result = remote.executeMethodOnObject(new SerializableCallable() {
-      public Object call() throws IOException {
-        Properties p = getDistributedSystemProperties();
-        // I never want this locator to end up starting a jmx manager
-        // since it is part of the unit test framework
-        p.setProperty("jmx-manager", "false");
-        //Disable the shared configuration on this locator.
-        //Shared configuration tests create their own locator
-        p.setProperty("enable-cluster-configuration", "false");
-        //Tell the locator it's the first in the system for
-        //faster boot-up
-        
-        System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
-        try {
-          Locator.startLocatorAndDS(locatorPort, locatorLogFile, p);
-        } finally {
-          System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
-        }
-        
-        return null;
-      }
-    }, "call");
-    if(result.getException() != null) {
-      RuntimeException ex = new RuntimeException("Failed to start locator", result.getException());
-      ex.printStackTrace();
-      throw ex;
-    }
-  }
-
-  public static void init(MasterRemote master) {
-    DUnitEnv.set(new StandAloneDUnitEnv(master));
-    //fake out tests that are using a bunch of hydra stuff
-    String workspaceDir = System.getProperty(DUnitLauncher.WORKSPACE_DIR_PARAM) ;
-    workspaceDir = workspaceDir == null ? new File(".").getAbsolutePath() : workspaceDir;
-    
-    addSuspectFileAppender(workspaceDir);
-    
-    //Free off heap memory when disconnecting from the distributed system
-    System.setProperty("gemfire.free-off-heap-memory", "true");
-    
-    //indicate that this CM is controlled by the eclipse dunit.
-    System.setProperty(LAUNCHED_PROPERTY, "true");
-  }
-  
-  public static void closeAndCheckForSuspects() {
-    if (isLaunched()) {
-      final boolean skipLogMsgs = ExpectedStrings.skipLogMsgs("dunit");
-      final List<?> expectedStrings = ExpectedStrings.create("dunit");
-      final LogConsumer logConsumer = new LogConsumer(skipLogMsgs, expectedStrings, "log4j", 5);
-
-      final StringBuilder suspectStringBuilder = new StringBuilder();
-
-      BufferedReader buffReader = null;
-      FileChannel fileChannel = null;
-      try {
-        fileChannel = new FileOutputStream(DUNIT_SUSPECT_FILE, true).getChannel();
-        buffReader = new BufferedReader(new FileReader(DUNIT_SUSPECT_FILE));
-      } catch (FileNotFoundException e) {
-        System.err.println("Could not find the suspect string output file: " + e);
-        return;
-      }
-      try {
-        String line;
-        try {
-          while ((line = buffReader.readLine()) != null) {
-            final StringBuilder builder = logConsumer.consume(line);
-            if (builder != null) {
-              suspectStringBuilder.append(builder);
-            }
-          }
-        } catch (IOException e) {
-          System.err.println("Could not read the suspect string output file: " + e);
-        }
-        
-        try {
-          fileChannel.truncate(0);
-        } catch (IOException e) {
-          System.err.println("Could not truncate the suspect string output file: " + e);
-        }
-        
-      } finally {
-        try {
-          buffReader.close();
-          fileChannel.close();
-        } catch (IOException e) {
-          System.err.println("Could not close the suspect string output file: " + e);
-        }
-      }
-
-      if (suspectStringBuilder.length() != 0) {
-        System.err.println("Suspicious strings were written to the log during this run.\n"
-            + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n"
-            + suspectStringBuilder);
-        
-        Assert.fail("Suspicious strings were written to the log during this run.\n"
-            + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n"
-            + suspectStringBuilder);
-      }
-    }
-  }
-
-  public interface MasterRemote extends Remote {
-    public int getLocatorPort() throws RemoteException;
-    public void signalVMReady() throws RemoteException;
-    public void ping() throws RemoteException;
-    public BounceResult bounce(int pid) throws RemoteException;
-  }
-  
-  public static class Master extends UnicastRemoteObject implements MasterRemote {
-    private static final long serialVersionUID = 1178600200232603119L;
-    
-    private final Registry registry;
-    private final ProcessManager processManager;
-
-
-    public Master(Registry registry, ProcessManager processManager) throws RemoteException {
-      this.processManager = processManager;
-      this.registry = registry;
-    }
-
-    public int getLocatorPort()  throws RemoteException{
-      return locatorPort;
-    }
-
-    public synchronized void signalVMReady() {
-      processManager.signalVMReady();
-    }
-    
-    public void ping() {
-      //do nothing
-    }
-
-    @Override
-    public BounceResult bounce(int pid) {
-      processManager.bounce(pid);
-      
-      try {
-        if(!processManager.waitForVMs(STARTUP_TIMEOUT)) {
-          throw new RuntimeException("VMs did not start up with 30 seconds");
-        }
-        RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + pid);
-        return new BounceResult(pid, remote);
-      } catch (RemoteException | NotBoundException e) {
-        throw new RuntimeException("could not lookup name", e);
-      } catch (InterruptedException e) {
-        throw new RuntimeException("Failed waiting for VM", e);
-      }
-    }
-  }
-  
-  private static class DUnitHost extends Host {
-    private static final long serialVersionUID = -8034165624503666383L;
-    
-    private transient final VM debuggingVM;
-
-    private transient ProcessManager processManager;
-    
-    public DUnitHost(String hostName, ProcessManager processManager) throws RemoteException {
-      super(hostName);
-      this.debuggingVM = new VM(this, -1, new RemoteDUnitVM());
-      this.processManager = processManager;
-    }
-    
-    public void init(Registry registry, int numVMs) throws AccessException, RemoteException, NotBoundException, InterruptedException {
-      for(int i = 0; i < numVMs; i++) {
-        RemoteDUnitVMIF remote = processManager.getStub(i);
-        addVM(i, remote);
-      }
-      
-      addLocator(LOCATOR_VM_NUM, processManager.getStub(LOCATOR_VM_NUM));
-      
-      addHost(this);
-    }
-
-    @Override
-    public VM getVM(int n) {
-      
-      if(n == DEBUGGING_VM_NUM) {
-        //for ease of debugging, pass -1 to get the local VM
-        return debuggingVM;
-      }
-
-      int oldVMCount = getVMCount();
-      if(n >= oldVMCount) {
-        //If we don't have a VM with that number, dynamically create it.
-        try {
-          for(int i = oldVMCount; i <= n; i++) {
-            processManager.launchVM(i);
-          }
-          processManager.waitForVMs(STARTUP_TIMEOUT);
-
-          for(int i = oldVMCount; i <= n; i++) {
-            addVM(i, processManager.getStub(i));
-          }
-
-        } catch (IOException | InterruptedException | NotBoundException e) {
-          throw new RuntimeException("Could not dynamically launch vm + " + n, e);
-        }
-      }
-      
-      return super.getVM(n);
-    }
-  }
-}



[44/50] [abbrv] incubator-geode git commit: GEODE-781: Another nexus / Jenkins fix for uploading artifacts

Posted by ab...@apache.org.
GEODE-781: Another nexus / Jenkins fix for uploading artifacts


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

Branch: refs/heads/master
Commit: f06a43fe8dbaee00b49d892d6b5207dd02c3fae6
Parents: fa60ac7
Author: Anthony Baker <ab...@apache.org>
Authored: Fri Jan 15 11:36:34 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Sat Jan 16 09:36:06 2016 -0800

----------------------------------------------------------------------
 build.gradle | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f06a43fe/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 6004226..90d4a60 100755
--- a/build.gradle
+++ b/build.gradle
@@ -521,10 +521,8 @@ subprojects {
   // from gradle.  However, we must match the repository id which the nexus plugin is not exposing.
   apply plugin: 'maven-publish-auth'
   afterEvaluate {
-    tasks.getByName('uploadArchives').repositories.mavenDeployer() {
-      if (project.nexus.snapshotRepositoryUrl) {
-        repository(id: 'apache.snapshots.https', url: project.nexus.snapshotRepositoryUrl)
-      }
+    tasks.getByName('uploadArchives').doFirst {
+      repositories.each { it.snapshotRepository.id = 'apache.snapshots.https' }
     }
   }
 


[25/50] [abbrv] incubator-geode git commit: Change GMS health monitor stats description per suggestion from Darrel

Posted by ab...@apache.org.
Change GMS health monitor stats description per suggestion from Darrel


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

Branch: refs/heads/master
Commit: 61a16b170678302591028a90702d3387162bdc00
Parents: b6a89ad
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Wed Jan 6 12:26:21 2016 -0800
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Wed Jan 6 12:26:21 2016 -0800

----------------------------------------------------------------------
 .../distributed/internal/DistributionStats.java | 36 ++++++++++----------
 1 file changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/61a16b17/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
index 92b0fcb..28ca380 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
@@ -338,32 +338,32 @@ public class DistributionStats implements DMStats {
     final String initialImageRequestsInProgressDesc = "The number of initial images this member is currently receiving.";
 
     //For GMSHealthMonitor
-    final String heartbeatRequestsSentDesc = "The number of heartbeat request messages that this member has sent.";
-    final String heartbeatRequestsReceivedDesc = "The number of heartbeat request messages that this member has received.";
+    final String heartbeatRequestsSentDesc = "Heartbeat request messages that this member has sent.";
+    final String heartbeatRequestsReceivedDesc = "Heartbeat request messages that this member has received.";
     
-    final String heartbeatsSentDesc = "The number of heartbeat messages that this member has sent.";
-    final String heartbeatsReceivedDesc = "The number of heartbeat messages that this member has received.";
+    final String heartbeatsSentDesc = "Heartbeat messages that this member has sent.";
+    final String heartbeatsReceivedDesc = "Heartbeat messages that this member has received.";
     
-    final String suspectsSentDesc = "The number of suspect member messages that this member has sent.";
-    final String suspectsReceivedDesc = "The number of suspect member messages that this member has received.";
+    final String suspectsSentDesc = "Suspect member messages that this member has sent.";
+    final String suspectsReceivedDesc = "Suspect member messages that this member has received.";
     
-    final String finalCheckRequestsSentDesc = "The number of final check requests that this member has sent.";
-    final String finalCheckRequestsReceivedDesc = "The number of final check requests that this member has received.";
+    final String finalCheckRequestsSentDesc = "Final check requests that this member has sent.";
+    final String finalCheckRequestsReceivedDesc = "Final check requests that this member has received.";
     
-    final String finalCheckResponsesSentDesc = "The number of final check responses that this member has sent.";
-    final String finalCheckResponsesReceivedDesc = "The number of final check responses that this member has received.";    
+    final String finalCheckResponsesSentDesc = "Final check responses that this member has sent.";
+    final String finalCheckResponsesReceivedDesc = "Final check responses that this member has received.";    
     
-    final String tcpFinalCheckRequestsSentDesc = "The number of TCP final check requests that this member has sent.";
-    final String tcpFinalCheckRequestsReceivedDesc = "The number of TCP final check requests that this member has received.";
+    final String tcpFinalCheckRequestsSentDesc = "TCP final check requests that this member has sent.";
+    final String tcpFinalCheckRequestsReceivedDesc = "TCP final check requests that this member has received.";
     
-    final String tcpFinalCheckResponsesSentDesc = "The number of TCP final check responses that this member has sent.";
-    final String tcpFinalCheckResponsesReceivedDesc = "The number of TCP final check responses that this member has received.";
+    final String tcpFinalCheckResponsesSentDesc = "TCP final check responses that this member has sent.";
+    final String tcpFinalCheckResponsesReceivedDesc = "TCP final check responses that this member has received.";
 
-    final String udpFinalCheckRequestsSentDesc = "The number of UDP final checks that this member has sent.";
-    final String udpFinalCheckRequestsReceivedDesc = "The number of UDP final check requests that this member has received.";
+    final String udpFinalCheckRequestsSentDesc = "UDP final check requests that this member has sent.";
+    final String udpFinalCheckRequestsReceivedDesc = "UDP final check requests that this member has received.";
     
-    final String udpFinalCheckResponsesSentDesc = "The number of UDP final check responses that this member has sent.";
-    final String udpFinalCheckResponsesReceivedDesc = "The number of UDP final check responses that this member has received.";
+    final String udpFinalCheckResponsesSentDesc = "UDP final check responses that this member has sent.";
+    final String udpFinalCheckResponsesReceivedDesc = "UDP final check responses that this member has received.";
 
     StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
 


[39/50] [abbrv] incubator-geode git commit: GEODE-776: Add KEYS file containing code signing keys for releases

Posted by ab...@apache.org.
GEODE-776: Add KEYS file containing code signing keys for releases


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

Branch: refs/heads/master
Commit: a48bac410be8417dd4d3df0be1fb1f3ec21c7d41
Parents: 5e8eed5
Author: Anthony Baker <ab...@apache.org>
Authored: Tue Jan 12 13:29:02 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Tue Jan 12 13:40:45 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a48bac41/KEYS
----------------------------------------------------------------------
diff --git a/KEYS b/KEYS
new file mode 100644
index 0000000..9d13f70
--- /dev/null
+++ b/KEYS
@@ -0,0 +1,236 @@
+This file contains the PGP keys of various developers.
+
+Users: pgp < KEYS
+       gpg --import KEYS
+Developers: 
+        pgp -kxa <your name> and append it to this file.
+        (pgpk -ll <your name> && pgpk -xa <your name>) >> this file.
+        (gpg --list-sigs <your name>
+             && gpg --armor --export <your name>) >> this file.
+
+pub   1024D/9475BD5D 2010-10-08
+uid                  Roman V Shaposhnik (CODE SIGNING KEY) <rv...@apache.org>
+sig 3        9475BD5D 2011-11-01  Roman V Shaposhnik (CODE SIGNING KEY) <rv...@apache.org>
+uid                  Roman V Shaposhnik <rv...@apache.org>
+sig 3        9475BD5D 2011-11-01  Roman V Shaposhnik (CODE SIGNING KEY) <rv...@apache.org>
+uid                  Roman V Shaposhnik <ro...@shaposhnik.org>
+sig 3        9475BD5D 2010-10-08  Roman V Shaposhnik (CODE SIGNING KEY) <rv...@apache.org>
+sig          61A073E9 2010-10-28  [User ID not found]
+sub   2048g/DD0FB547 2010-10-08
+sig          9475BD5D 2010-10-08  Roman V Shaposhnik (CODE SIGNING KEY) <rv...@apache.org>
+
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Version: GnuPG v1.4.10 (GNU/Linux)
+
+mQGiBEyvi2cRBAC0n2WpJwVJL+LIwZ6rCxJDZPRSxUvs8GrKKPp3Wa8hAgz37vl3
+U8RilZGcIHnMJlKrAqsh4TNgeWBaHk2duKsoupP+a3t70gJZs+9h2DK2Xj72nk1K
++RGKLUw5lRKKkye53v8cqNt6kdTNpxAlswNx68e5+X4yDFJXJmqd1uG6QwCg4ytn
+AlyjvSmQ39OpAw71fRn+t3sD/1+jYOyJJADbZpDQQ+l9iVe66ExGNNUJYrHlCPKr
+4bwhln/hSr7SCoEBpKi0xAokRG2wpebOZLUukoy+YaSsPJdCQLy3/ymsx6VwGDdo
+Zu9Vs7b4akRpVmKPfKb1QZBW5DatcUcWpDManum5gcu8SfgohAJggLal71tdp8Om
+SoShBACZ96LURWHVU7ws6eHnVoQ68SSONwRe8oxVhLXA16Dbs2Hfl8wVYSbvp8CL
++8Q3pEEjb1A6mUYzsiQskdM3x5/bAkcvTWXjRrdBmsV9C4BTLhadrDsh8vTCGdTG
+rqiu17Ata1+vfckHbfMOYHY3viuMejvAxV0uUEQ7K95yZG7uPLQjUm9tYW4gViBT
+aGFwb3NobmlrIDxydnNAYXBhY2hlLm9yZz6IYgQTEQgAIgUCTq9/BAIbAwYLCQgH
+AwIGFQgCCQoLBBYCAwECHgECF4AACgkQE5cdo5R1vV0gagCgn5aLDgWI7rqpdFlk
+/s5GF9wmvhIAmgO5akcO6GwSrNNWDPZuABL3QP+htClSb21hbiBWIFNoYXBvc2hu
+aWsgPHJvbWFuQHNoYXBvc2huaWsub3JnPohgBBMRAgAgBQJMr4tnAhsDBgsJCAcD
+AgQVAggDBBYCAwECHgECF4AACgkQE5cdo5R1vV0K0ACgsW7UXcMQTTPzhxJONZqr
+JsfwCBcAoKAY48OyUl7f5vglnEkNOlw/l0/6iQEcBBABAgAGBQJMyMKhAAoJEC7U
+XWZhoHPpxVYH/0Knah53F+WseeDyZT/UCdG5R0VclY6dPEuxRnSBzb118WuuuIHR
+9yH0bfSWAilijOJR/an3+MTXnhmcOYJfiNwzC2g6z5k5y6dTj4I8a4mOq3MDe5KA
+aGovomUWiA6Oe2B8ITRUOK+DtmN83XF25yv4bvStg/bPftbf89SBl3ZfcyppQb5G
+tr1tAsYOkTJyOfnGAIstYwM68UxOjNLdGAJHkqYRNlNYLUqw+L24/gFNwCl9STl/
+OGx3lPY0zqCvNm0MtChU7b5WWdh6dmKh797oGro/LDKCQoZrLqYnRdraZ3iYwmEO
+t2nc6/tE3fRIyNYPCHXsFBYGxJdrAd/sfe60NlJvbWFuIFYgU2hhcG9zaG5payAo
+Q09ERSBTSUdOSU5HIEtFWSkgPHJ2c0BhcGFjaGUub3JnPohiBBMRCAAiBQJOr3/7
+AhsDBgsJCAcDAgYVCAIJCgsEFgIDAQIeAQIXgAAKCRATlx2jlHW9XZAeAJ94Ahm4
+NYnycI3A9eTQhij/4J4QEQCgpjIu/ZBcg+GcooA0ZDZuoLw+EBS5Ag0ETK+LZxAI
+AOL8ROZxL/T9vbQ7hWa4CUH1IktnnmMIhN8O80y5C0oVT5cXPghiiVzRHJ+5+a7a
+Mcf7jlGtv3fr/2x+/cUkwvPdN0Zk+Mc1YOqyvPXWY1eR95fzt6XL6ovX2HvKDHYM
+V4AnAjCS/e66Z+FDRvD4I6Nv48cJscAdcZuScmp3cdrsN+9h2fhokkhe32OpBukq
+HiEDADNAFwkpuNo5Hi7LNurmZqCj+bEIXq0AjV07tvpzzt5j2wIk+Gcq4TLCITGE
+vtmo6QcDVo7x+JxpZ4d71NXJHMxLEoClYBzsv2C+lemeoD4nO2KqOPnUizI1rjMc
+Xuk/aNVpCTWQZrJLEplVVgMAAwUH/j21ya6hWWTVio1szPCI7dAZs+b1g3kdktwn
+xYMTVo4id0Q5BNhujy9xSTjqqPsxwQrzT1e+NEKWmUBrzAg9VxEXkB3BBz+WdmFt
+Q7W3tAB80UNY0OGJKWFgkoA/9UNXYkwPcLYHAyZqunqv8N/uRPuQVIZvkEdERzQq
+XCfnxmm2zW8XrYnLWBwJo5FopeuxC9EYObWn/zeZh3DBFcBGjiGN7e117TIx7KQA
+eve9u4BwwEwFfpqyN/sMTMVOtelpE2BCM1ZlzzwgASPjX206uqqqQa+6EXv8MykX
+GGWQp0yK9HU+Dzk1qaOLmsz/wDioIdDIObQvbf569bYf+KOGMrSISQQYEQIACQUC
+TK+LZwIbDAAKCRATlx2jlHW9XRwLAJ0eqbLTvHBdI0le1CSI+pf1y7m8hwCfU+KF
+yMSU5BkirXWZDazip1CeMuY=
+=jokp
+
+
+-----END PGP PUBLIC KEY BLOCK-----
+
+pub  4096R/C72CFB64 2015-10-01            
+	 Fingerprint=948E 8234 14BE 693A 7F74  ABBE 19DB CAEE C72C FB64 
+
+uid Anthony Baker <ab...@apache.org>
+sig  sig3  C72CFB64 2015-10-01 __________ 2019-10-01 [selfsig]
+sig  sig3  8B6CF8DD 2015-10-09 2019-10-09 __________ Anthony Baker <ab...@pivotal.io>
+sig  sig3  5AE83723 2015-10-09 2019-10-09 __________ William Markito Oliveira <ma...@apache.org>
+
+sub  4096R/872FE71E 2015-10-01            
+sig sbind  C72CFB64 2015-10-01 __________ 2019-10-01 []
+
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Version: SKS 1.1.3
+
+mQINBFYNZ2ABEADjf61xzAIwBkkvY806h98hYZrtJ399QjjimxICzF+IOkWw+o1qSxxt4TDM
+jIp2SiOX2flLYJNLj9MFKd6PwAGGEaXjQylAGGHRH/SP47g0hzsF3ZqMD8wNkT6tXLc22Ru6
+CjB1dAIRqWxY9AMkg1+Ds04EGeFlMjDy0hMQMfqG00E8IyGmYSF8bA2XULl+F4/nu403aQ2P
+31Iw2LLpEAzDzhGztI8Yvio+AeJ9arnZtt/6+1shdeJYLFUVWxianJBuWHnvZYJPVkQyL7np
+2hxjg+dGISICIKyDu9AgTaRlkGnhzkEJHd5heqcD2kaQ0ihKJitcNXlRgINfnrmwPlF/zQjY
+H2uU0oseoHL88xC7aXZjNUkQsX95bOVMhoj+1rEdXI7+puWZvojK8JIqA4Uxciba/B/W7SEj
+KUZK1UUIpj2DfKQZMtLGMxe+J9OQQNrpOCoDFeD3tTqJLnQou9ylTn393Vs4RQbRheTbp94e
+WK+T1NUqQqqubLBtlujABQO9qKZplycV4m1mko1luldt8anrIvLXLsNBYOysAHEllrGaaRYa
+bQQgRgalbm9+ZWkqx2gl8zuiAYLknru3cz2Gagw/NUuK2KNWGMJql0UWdXMChY6F4RZWbBwT
+moN74rS4u+ScROGYuwNqSIozez93kNQW+cgMeyR084nUBxes8wARAQABtCFBbnRob255IEJh
+a2VyIDxhYmFrZXJAYXBhY2hlLm9yZz6JAiIEEwEKAAwFAlYX61gFgweGH4AACgkQJo9QMIts
++N0o3RAAn+2WTK4zqeX1AYGa+OqXbeCvsZTfLMdSIU9AhHjQEcOrGB867YvXOfNr0r2Wg47I
+xpQeuLcj6ZVR3k2v/c/kBvHHLumkevwaK05sHV0Q9mJvXCd4LnFXVFZvV+e1uJx3mrSvqgB0
+d0Bk/mkV4HFjwyIHlkvKENX9oqy29L9uUpmAgBmSKSZfYYiHBu/WG1kY2UIlAivjv+IR75d/
+OUi4gs8Ku0gw9EfkCLKUpPai2se6VjgyOF7CzU1s4x5wgfdnBk5p4yCqT1yOduXDFgVm4IW2
+Htqn9ZbrdU9rWM4yJFC1M60BkxFvDeKBtR35gpmO+dJXME6oZaiOiHLp60AZmJnA8euA2XZF
+4ZgkMLOO+0vOfl574xaOWdZmdC+1l9iz/8JspheFQ3r0wSHSHGfVndQMEKsQDKRx9DNo1AEQ
+lRm6GJhazfNTRLx2ernrnbPEf3pPQ4SDMceTYXkEt7gfJRaHUXY6JEyPH4LhH1NRlHTuqkGC
+/Jq807lF3V08bkie9KWn7wEM5nuoK+p5obZV2xLhcDrd9dLxokReCqXPVlARYUBmL1QTsjIw
+y8DbwXVDXWYJW52aijNeH7HTONvqzaIbeFK+NElx5HvD6EiMaUzIARpVViN7zno6nI/5EPsI
+YCBp+jCgZab2TLvlJFXUV1KTgIjM/234P4BT3FrCyVCJAiIEEwEKAAwFAlYX8JEFgweGH4AA
+CgkQnrZMFlroNyPLdg//TOiwaTGl05G/f5+W06s18xPorGfR1f03QmLq3dL95LETvWblDNvC
+xDt8UxH4Y1SVCmkHk9MeOu2pF7ZeKgtjxf3Jb29wjcurYluVYw8WjPe3AMEEOhCpn2FVMI30
+KQyxPYJGeLP2pHUf+dphCOh4GPokZPyzm5a/eOq0/HXlMPrInql5WPlinn8ZsXEww+DvJJIc
+lF3tblaCfwuKKJyVxZkCIO3zh7qIIOBPmxZQEzymNF1Io4q+UCk5O1WMroiw4HHhXDtJ03//
+gQJG4O2t3HoCgVzefgqhT8qgEabv+jQkpPYoAZcp6MzteKcXsUfMhqpJrV39WLNW8WbwORM1
+ZcXSq6g2SOT5xdIs9R4qfhP5xDBvWjAm1TSV2Zf7lI7Ma8zU+vk4LqOEe4NnIO+g3T303Xxt
+y6fwWZzabv4pjP8td+qsbIJfVjAFfmIw2quBTpSpMWkVSH3HUa3leBlrfHH9Cyh82eN88jpc
+VPh02UhBkRzAUro/N2wHqmv1Z7MT8VOO51MWxmIbq8JYYhzkG4AIsMjUh/QRbrxPCpaKZP+2
+1a8HuOxsWGsHfJFurAEKap5mlxQzEFWMR2jLKuKlRVBzqSZRmAT76Ap0+1ngm4lOY+NENDMU
+7eFxvLNe0YKetYdVy6vr2dYat7rNAwJZvmachcRKLI55PPcPHMhd04GJAj0EEwEKACcFAlYN
+Z2ACGwMFCQeGH4AFCwkIBwMFFQoJCAsFFgIDAQACHgECF4AACgkQGdvK7scs+2Sb1A/5AcfX
+FX1NiFGYhUIbjc3ePzkXNVCzUOIZFU9mR5K7l2hNqhOnwyZb/ig2dgyccSBCrtA3Ct2oU883
+otZVYfaa6gbton63y3klcguWFiC/TNnm63ONd3mcbPZ2573M2j56q1H1+gOJcWtxYNzgvDbN
+m0vKRY8C3hNsY9cbAvO97rBQSDR1YAwBTPIR9cL8Wf4fOdSCRJZIkiHYKC7JKoC6PalrXSTM
+ImRaMmhdyaLlvrb4Vv/6dFJuRLwD2LUx0Q9+MSHMAgpCMGaVryeEzewJQpL3SVchUR9PgYhE
+63FWRbtliEVcvytgCYrd8smVe7ovhF5fr1brhE1fHmhl5P46vC11bIWjyYmhCqYig7bC3UuH
+X8m+cxyY+oNIiqPnYjIpVn0lyd3vFQX8PYKjV9uvptfN2dO7E/w0Oehhv0zaqoUhvAhUtOPN
+yyrtA17yYEzLpqV2ef/yRMipLDeo2v6BJDihTRr6h+7Q23mohBE3YE77f7a2sMyCjOUDEZCV
+zp9LrEau5EH0Tl8Rm/d4gdMqhdMjjagbV4cjtm6svpTg2DkCv9gyHnVh8INdTo5XdWOC3JNr
+zHgRzBXljxAFF8gdgDHR0EAtZCh6/094rfBfQRmvOD2Yey77yxdVlI1uqLKtr6BrIjTKTED9
+/3V6C0bESB+aD3AlSCFTHsKxk6XGBdi5Ag0EVg1nYAEQANsty5ed4KggkzO8LhJ3+vcydZUz
+zAnnzzcQnvmHjul9OUoPhu4vmPLZG8J/eku/zHgd6qdf291VMjNPYLt8HzDvmwBpYyJEVq2s
+cZWK4Nyk71Zm0+C32qwsr1S17/pAS7RaDCI8HyMIT+IcMV7/5WUYHvDTminFvkG/D7wIoOla
+dAHrG9VZenhm3jH4oMXbQoJ/2CQNb3s7zuU514/rTTTEg8mK0rT/l/BW6zVwTWHaqdvEGKEK
+/iMj7rs4fxuY6UvZGUPAHHAMM125rKBsRa6dbvRCrI9vM4EQQYhjeXSD0jEDSuXDCsJAkYRd
+r8uI8b39smrlmhecbVubgOu47rPF20pObVvxlgh6Yt9Wu1RZuI7wVcCweV/r8/79ijbYrV6X
+zTwiy15Hd3ktAdsnpak13Y7xQ/TXhKVo3xPrMMfo5JPAHLnmH7cfoeFn3awsEwY5wuuuh0pf
+lWcVkCOMXPItG+FpByOIvgHYbeIZaFPb3yAFLFdrn/m4/ZtI2wLDQq5oS1X8AQetqUe8Z41l
+3tD6fYQATM2gdNdCDhww0CwGWKyqerT+nYnWzjgxOCo1zxsNnmwYwwwOBtMx2KA5n5uQu8V4
+dE8Z8MRMiED/UErFiN2Jx1Nr71v1Vknwi8xkuLqN5UUTkEqdU2/QOl23oKMT13re1OP82qEU
+1MFcq6g1ABEBAAGJAiUEGAEKAA8FAlYNZ2ACGwwFCQeGH4AACgkQGdvK7scs+2Rhug/+J+Na
+gAwbRQvk+SYakaNLIfTf8t4kIAyrVjiiMbkWTBuRx6VhGSoe/75MhXOEO46CAzK0iY3TQJu6
+4J/901v5V1EhODNn3QTd1s9O5a3CFwcvUu3sH5SNhkc5LW/9MxhX2jBec7HQcFy5y4L9gnfH
+yAe/GlWlPgElxhPe72A7pBmei67J/koINSmpRVycPTJSAxaygHAmlKfq/eIuPO66eU1J5q5d
+ISs2ZIuArJey/X30SwriHkcDMMGCxdiYgLRMREsL8S3cXIeHUkpV2LBr3ZE9Rt9uApu/Ztzr
+uOR7vRhTkrOPs0Yaa90prNMi4DMhS/WI5C+LxUzDwgdDgSJLNVOSIDsnfmOyPg0pNA+ZJVTS
+p6RhrsLKN/x3m+SH9Shn+dBTzGqoSPsBw71PtLog0ky0dDEJOppqfAabdHz+111fu68W+foU
+z6Zc9BafO78KBmy+6zdakwCGaYe7sMikuprZm0WwimUStt24WKD0Nw2tlImSVUQqQYc5Hdkt
+ull1SOvBxT0VTHAwamzYKq+5RtVkAgj0p5xndHd3IlzmUK9ha7d4eL/wvjjpLY13as1fJw0T
+FxGl07BDMt7Qx0igQPO8pu5DW0cBEjuQpWoUrc1VyLCW36aUHFTR7SJQR/v1pOktdC9xYc0s
+8Uow+sc8jl2OC1j5NZLEB8QQY7K3myI=
+=1p7v
+-----END PGP PUBLIC KEY BLOCK-----
+
+pub   rsa2048/456B8A09 2016-01-12
+uid         [ultimate] Nitin Lamba (CODE SIGNING KEY) <ni...@ampool.io>
+sub   rsa2048/F6891D30 2016-01-12
+
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Version: GnuPG v2
+
+mQENBFaVRnMBCADPrMJBXXWLflywdgBR6EYnSU704zDa1M92snHBhs6ensz7SnMQ
+wR4XaWovs9HTMtDkBUkG/y6yHHI9OG/rzDkR/KzmtwxOvpbtgIQDE/w+r4brR95Q
+pBZ7j5Mw+ylyUKRLsF/pX7tupOpR82U69JQ6G2yRn2XVjcVQ+TVm6gzB1Iq6pbfH
+/RE7QhpdzheEoOI+Ufat6UNQkY4CDcjh4CozMWCB1Zl9+g2oXPU6C4kB4KGIOwhB
+HVwYJO4KFkm84rDqw3Yv6A3/xFBfYjBVBF0iERx1S2DBZTTp92agYnqM6Yw1xwsa
+iYK5Xo7p3VrhAFMjkS/j23Myrl4lPotKSRArABEBAAG0ME5pdGluIExhbWJhIChD
+T0RFIFNJR05JTkcgS0VZKSA8bml0aW5AYW1wb29sLmlvPokBNwQTAQoAIQUCVpVG
+cwIbAwULCQgHAwUVCgkICwUWAgMBAAIeAQIXgAAKCRBlCFWCRWuKCWccB/9t0lAA
+sVOzkvx5ukDqgvRpiq42Xy77u6Wat6ICqsZmgkIpGQ4SZv3DsU+GC+cYTM5FiZco
+mXFvAZL0Frz2W4jfT3h/1f3ukjqf+KX8rNpGgBL0FZ9KYEFiUSc8+iFZjvcYFGaD
+Ymal38+WzrZcdt4kK0mba2Udtu4i48CizX+CoBg0YYJaFKg1Jy/AXdpJT3eHSbru
+1F6SflOs5tkr9y94WnG5hvb5CQmYqvjbyiLotp3vxvD9czTSeg+1GR0/6ZvQaGxP
+sKnbJrkME47LhJoB8lJS5k5R8RiGcNWRyDr/u4OOAkDKtmW6ihxsY+3NQ13bi8+6
+0SE6YY88g4d3vwEquQENBFaVRnMBCAC2osXaufdp9PulEaVaZ0Qc/RimnY3xKw45
+7Md3BWH/ViIGWR2D5NWsiwiTqE6V3bjCy2A/oTlsa+0X1cE3aXikk6hNdOprZQpS
+ITzmZ7ScWeyB9Sn0g3wI2w/ynrqevqUF8WqUyQ/le9bq9wYmMgCoyrIy4V+7rgKL
+pgtNqaUhd9hRTToJ68IMxmmyIMtLwmrLJt8l5cGrVTBB1tccVKRWbeZgTe3YsesA
+9rSIZPm0qzoMvcPj5iR2VSqLtdpp1PA48ZYPSuvRapl4/Ipk3WK3TVywfdp6kZF4
+0TnuS+0pNZ3TIi2eMOrp8U5ztMUZtAkx7AYYToGUczE1US+ieIP5ABEBAAGJAR8E
+GAEKAAkFAlaVRnMCGwwACgkQZQhVgkVrigkA5Qf+P5AsgHookRYBDYuud1Wyfm1M
+lsf3AUIfLq7pB5mF72yuK79jCAfOQR4FlmM75oswFbPtRcy5+dRm5/cFXPGsnCb9
+BbnHvAOMOREvRzmxwjkoV2KfSik71q7M7YC7GLErM/up1/wdhvzwhdNfacE6am/g
+ee2pf/GwYYqtCQZya20eyuEWnD+gZWEo6vpBwTxVkjJ+SW9mW58SopN7OqX02Ciz
+shSSlzzAmNUUrnbepXrAM3claSldiV0HRDc1lvIqgK+8AWsN5RTAN3j+kLB3ZXas
+02E85NJfruSVQdfsF/70ETCQW4Q25w1a6s8TkNOuAyLkgvbfnsg9X9q2Difj9g==
+=+T/b
+-----END PGP PUBLIC KEY BLOCK——
+
+pub   4096R/A17B74E4 2016-01-12 [expires: 2020-01-11]
+uid       [ultimate] Mark Bretl (ASF) <mb...@apache.org>
+sub   4096R/10AB2063 2016-01-12 [expires: 2020-01-11]
+
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Version: GnuPG v2
+
+mQINBFaVX4ABEACwyJ4K3upyZCtc05fFwxKbU9p/0oNqsFAF1WoQf5CO/LGMg+vE
+LH4bl60Umshk2UHNvmvLlNbMxaLurmPW9hnmsEVFH16fTu1OB3r4y4q2PoWMdtjv
+2MS7hhaZyY7bS4iCjhnjYZHgDABKCBO7QK20A/KCYBGhDHc6LpqfGKxrxkkkJU6K
+7ZP0v075TFzWFJzp4YR1wBfRkY6f6CmtPlvkGYLP4XCLC3NQzjcl3m0kA66nuJLf
+ikyQ8U3AGrjFe5pbSNRTyY9ZiocJ2PRemzbicLLKIgzp4hDv5REo26WjDTAwhBtf
+JhIU7ywM8+/OMLGAIkGl77CUdgDAnWYbqBZB3ZN2AQf+2nXLHYVGus3tL84WZike
+79S+4o3Orx5J7yIXoV2GOvcmPXODzdqgHdL7Mn2IT6W1oCl3n/CxroFpyMTPDjqb
+SxWUWZg/GqhJ6prZqW2RmCTPj6QF+Ct9uU7VrtWNhfm7KgwY1C/ru93ewLeGMzN7
+SHjBqDvVJbYOHuPdVhcEf9mAXEus2EQKlNwT0Be3VnnRDz0ZN9sJLUUCxEsSgUfT
+Zu7XqdEQxSxU9RKmn55OMw/yU2zH4QouTQ3ACFVhD8pdHVjMsT1+VWYFE0RzmGDs
+Ew+s0mD/1pwQU8K2t507pCPaHZSZo4iuCXeD8geqb81ahSBQ9TLdWf8TCQARAQAB
+tCRNYXJrIEJyZXRsIChBU0YpIDxtYnJldGxAYXBhY2hlLm9yZz6JAj8EEwEIACkF
+AlaVX4ACGwMFCQeEzgAHCwkIBwMCAQYVCAIJCgsEFgIDAQIeAQIXgAAKCRA7b2j0
+oXt05Ev7EACTDs+ENZvJGHzTN5PMZzNjTc5nrcPRGW6u/GTu7wEhhL1l6MICsROG
+b2owRQWJnkxmbQFBuuA52d+jMecv+Put9SG9O4+UDsofLFMyNNKpbbPJwGcZO2Hp
+q8q2ZhQy5dGDCKx5IcfvbQqSAIOiIne1sD8Akd2qc2pRZKrfp3p+jLQCPrOfELTZ
+sco+ffThNfggXBRHPjC3pD1V5M+bNMbPCmFqpSTwgkdTCHmUFuvUfg3A7HycmBXC
+kr/5pYkWLtfSC5ObBPKgDYtFqc2Z2izH2WsK0CuCZef83E93CAIy16ciOwb7OsKr
+qaqH5l0OzmPre0kg6Bz5tSzxHeXt0kBFoaaKwxUi3eVilnKyTojNjNG+HTHGSuCX
+DJC5hliSMU8c3EouXsuO5sr1kiIKRiOVRax8xxKEvdyRhW2+K4MNBcZ1DyTwu/PQ
+HU0FI9ephk3lx6b4/KVgfyLYeBb+lB3W8mCT7xge4bIHz5R7RQnGqZHzWp1hLSYI
+mRp5dTy9SXgtuh8iyOOim9Z4Sk4zJfFvrKuTFMIAB4pHijv0BOS5AERpwm9jhJ2g
+vNh63UcY05hOq+Q/8btN7IwZBMHqbwfP4GQE+3oC+BZE4iEffajx1v2RAV1JFEOp
+vA+m6SNrUdgMvDVmZUOZqJyvypRCFBX/QxkpwU/TUUIrXCBg7F+rmbkCDQRWlV+A
+ARAAqIxvIrGKRH5pO+8dHgfLYW3KVA4Y1GaBpZA4Imn4YeFlVlMbeyE7N5OWRaWx
+6n5vzQSsuaBDh+72ELHQow04pCqX2XN+P5NmScUJEx0X/GSkzn1ha/EdFKEez1sj
+twMvQJcZnr0KsCY9kciAGpLtTQmzwsPFpt0j/P7ZnNTHDGU2MX7TReDHtfo5+VyH
+NX5mtbOKqtBFmw8e5ex3Fe8Tb5TkSvyr27cjp/q7qU+sXEj2+ZHLFghKDE4AvtAh
+6fbsi6jDoc2tl0EnaMSl59XLPStW3Bxc7KkfZ004a4tZBOp6VQ9c/F6RTbc0RWwB
+BokwRexszlpM01xIlhDw175VuxqFB/7ID099WIc1cKbHsZNNtK2kMzezBMJlc1MR
+NHIXsyeyijibB1Sbu2dOTFY6n2yRjZggniwZqNSh93Qx+MTXQqK0gwd6+51Dlhhr
+tBvcRERkl5eZMDzCY3h84c76RHVoa1YYCkxqytiFV83ia6KWNGs4C/q+/ZSmEfDR
+bJA34+Xpbl2L6GB7q2M/XXo5jg/naZsd1tIIn65FrLS+jkbrCgmT2383arItpRfT
+Dw0y5X7UGLvY/qbr2KlCx+v1H98RdU856P0GmyPvSYxT0IX8//nilfIy9peOCDPc
+gzK0Ze0grKxiXnJi0HvK8O2B42K6oKWmKJJO8wst5NY4bTMAEQEAAYkCJQQYAQgA
+DwUCVpVfgAIbDAUJB4TOAAAKCRA7b2j0oXt05EWmD/9HX3nX4Ck//HJA4K8q9lsA
+c8DM4l29Jyzn0x8JRSnA8D09weqhLr9dBKiVPn+WQO5dmdS0E9MaIWo8GeOCfk/U
+CYJPPq5pjjWyLrkXf7B7QqvbgDOufvSSBWV0guFuZOe+oI9aFkokHiBNUMiW+Oug
+aQXG+1fJXW5vFglIPF7qkTcihg1dBUxq0yGTL+r93Pvb02jDQTxX45kO1bmNahlG
+qbOP2vncjrreZ14EbXKQwqnhXfAxCUXMHwxeVHY1VHQTmaThh0/XZujYwTeGEj+0
+AYdSleMntdSKXS8A5hSsSWoJKhZwkDSo31dt5FLhpUN2L8IfNBnCm+mLJIsBlrci
+yQ36xQtIGQI8Sz86+89BZNMHEV6GuIiwUUBDCj9w0borcINEXN/eRQehO6mGlaJd
++wYEgHX+nb8/a1MgmhFWCFfKXlVrsOGTFP96aWHdEVFaXNCgo3ON2hQkV9z79eNd
+O4B75yVROzCowbDLAF53euUN4VQ5Ju2QAGsEYyEW1U16k5wbA1PJc34YmZTQPpX/
+ryI7LgeqIr7qfWgaCcezSSrEV3zynQ5ymSAzn34bQVpwrAQW1y0hFzF8x00lUx0w
+lwsXAU7cyVeLPjtrbLCDDBStjBWpD6MZDiDg0qmvesrpOa2cnIbHAPGOFGOLMML8
+KnUrsgPcFqltB5lYS0vJ3g==
+=323X
+-----END PGP PUBLIC KEY BLOCK-----
\ No newline at end of file


[18/50] [abbrv] incubator-geode git commit: Revert "GEODE-714: Modify all tests to use JUnit Categories"

Posted by ab...@apache.org.
Revert "GEODE-714: Modify all tests to use JUnit Categories"

This reverts commit ca6148aa99faf8fcb278fd7a840f69f622e6fc3f.


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

Branch: refs/heads/master
Commit: cb7dbd0b77507ed383e37eb7dd0925533d6dcbcf
Parents: dfba327
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Jan 4 16:28:38 2016 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Mon Jan 4 16:28:38 2016 -0800

----------------------------------------------------------------------
 build.gradle                                    |   21 +-
 .../test/java/com/gemstone/gemfire/BadTest.java |   42 +
 .../cache30/Bug40255JUnitDisabledTest.java      |  139 +++
 .../gemfire/cache30/Bug40255JUnitTest.java      |  143 ---
 .../cache30/Bug40662JUnitDisabledTest.java      |   92 ++
 .../gemfire/cache30/Bug40662JUnitTest.java      |   93 --
 .../locks/CollaborationJUnitDisabledTest.java   |  562 ++++++++++
 .../internal/locks/CollaborationJUnitTest.java  |  617 -----------
 .../DiskRegionPerfJUnitPerformanceTest.java     |    6 +-
 ...HARegionQueueStartStopJUnitDisabledTest.java |  123 +++
 .../cache/tier/sockets/HAInterestBaseTest.java  | 1015 +++++++++++++++++
 .../tier/sockets/HAInterestPart1DUnitTest.java  |  102 +-
 .../tier/sockets/HAInterestPart2DUnitTest.java  |  116 +-
 .../cache/tier/sockets/HAInterestTestCase.java  | 1018 ------------------
 .../tier/sockets/command/CommitCommandTest.java |    6 -
 .../logging/LogWriterPerformanceTest.java       |    6 -
 .../log4j/Log4J2DisabledPerformanceTest.java    |    6 -
 .../logging/log4j/Log4J2PerformanceTest.java    |    9 +-
 .../LogWriterLoggerDisabledPerformanceTest.java |    6 -
 .../log4j/LogWriterLoggerPerformanceTest.java   |    9 +-
 .../test/java/dunit/DistributedTestCase.java    |    3 -
 ...IndexRepositoryImplJUnitPerformanceTest.java |  437 ++++++++
 .../IndexRepositoryImplPerformanceTest.java     |  439 --------
 23 files changed, 2537 insertions(+), 2473 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 4563590..f464dc3 100755
--- a/build.gradle
+++ b/build.gradle
@@ -345,7 +345,7 @@ subprojects {
   }
 
   test {
-    include '**/*Test.class'
+    include '**/*JUnitTest.class'
     useJUnit {
       includeCategories 'com.gemstone.gemfire.test.junit.categories.UnitTest'
       excludeCategories 'com.gemstone.gemfire.test.junit.categories.IntegrationTest'
@@ -364,12 +364,10 @@ subprojects {
   //This target does not run any tests. Rather, it validates that there are no
   //tests that are missing a category annotation
   task checkMissedTests(type: Test) {
-    include '**/*Test.class'
+    include '**/*JUnitTest.class'
     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'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.PerformanceTest'
     }    
 
     beforeTest { descriptor ->
@@ -379,7 +377,7 @@ subprojects {
   }
 
   task integrationTest(type:Test) {
-    include '**/*Test.class'
+    include '**/*JUnitTest.class'
     useJUnit {
       excludeCategories 'com.gemstone.gemfire.test.junit.categories.UnitTest'
       includeCategories 'com.gemstone.gemfire.test.junit.categories.IntegrationTest'
@@ -394,16 +392,17 @@ subprojects {
   }
   
   task distributedTest(type:Test) {
-    include '**/*Test.class'
+    include '**/*DUnitTest.class'
     
     // maxParallelForks = 2
     // maxParallelForks = Runtime.runtime.availableProcessors()
     
-    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'
-    }    
+// 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'
+//    }    
     
     //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/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/BadTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/BadTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/BadTest.java
new file mode 100644
index 0000000..46dc799
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/BadTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+import junit.framework.*;
+
+/**
+ * This test provides examples of a test failing and a test getting an
+ * error.  We use it to test JUnit failure reporting.
+ */
+public class BadTest extends TestCase {
+
+  public BadTest(String name) {
+    super(name);
+  }
+
+  ////////  Test Methods
+
+  public void testFailure() {
+    fail("I'm failing away...");
+  }
+
+  public void testError() {
+    String s = "I've failed";
+    throw new RuntimeException(s);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitDisabledTest.java
new file mode 100644
index 0000000..4fe8a49
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitDisabledTest.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.cache30;
+
+import java.io.File;
+import java.util.Properties;
+
+import junit.framework.TestCase;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+
+/**
+ * @author Shobhit Agarwal
+ *
+ */
+public class Bug40255JUnitDisabledTest extends TestCase{
+
+  private static final String BUG_40255_XML = Bug40255JUnitDisabledTest.class.getResource("bug40255xmlparameterization.xml").getFile();
+  private static final String BUG_40255_PROPS = Bug40255JUnitDisabledTest.class.getResource("bug40255_gemfire.properties").getFile();
+
+  private static final String ATTR_PROPERTY_STRING = "region.disk.store";
+
+  private static final String ATTR_PROPERTY_VALUE = "teststore";
+
+  private static final String NESTED_ATTR_PROPERTY_STRING = "custom-nested.test";
+
+  private static final String NESTED_ATTR_PROPERTY_VALUE = "disk";
+
+  private static final String ELEMENT_PROPERTY_STRING = "custom-string.element";
+
+  private static final String ELEMENT_PROPERTY_VALUE = "example-string";
+
+  private static final String CONCAT_ELEMENT_PROPERTY_STRING = "concat.test";
+
+  private static final String CONCAT_ELEMENT_PROPERTY_VALUE = "-name";
+
+  private static final String ELEMENT_KEY_VALUE = "example-value";
+
+  DistributedSystem ds;
+  Cache cache;
+
+  @Override
+  public void setName(String name) {
+    super.setName(name);
+  }
+
+  public void testResolveReplacePropertyStringForLonerCache(){
+    Properties props = new Properties();
+    props.setProperty("mcast-port", "0");
+    props.setProperty("locators", "");
+    System.setProperty("gemfirePropertyFile", BUG_40255_PROPS);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40255_XML);
+    System.setProperty(NESTED_ATTR_PROPERTY_STRING, NESTED_ATTR_PROPERTY_VALUE);
+    System.setProperty(ATTR_PROPERTY_STRING, ATTR_PROPERTY_VALUE);
+    System.setProperty(ELEMENT_PROPERTY_STRING, ELEMENT_PROPERTY_VALUE);
+    System.setProperty(CONCAT_ELEMENT_PROPERTY_STRING, CONCAT_ELEMENT_PROPERTY_VALUE);
+    
+    // create the directory where data is going to be stored
+    File dir = new File("persistData1");
+    dir.mkdir();
+
+    this.ds = DistributedSystem.connect(props);
+    this.cache = CacheFactory.create(this.ds);
+
+    Region exampleRegion = this.cache.getRegion("example-region");
+    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
+
+    //Check if disk store got same name as passed in system properties in setup().
+    assertEquals(attrs.getDiskStoreName(), System.getProperty(ATTR_PROPERTY_STRING));
+    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE));
+    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE), ELEMENT_KEY_VALUE);
+    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE));
+    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE), CONCAT_ELEMENT_PROPERTY_VALUE);
+  }
+
+  public void testResolveReplacePropertyStringForNonLonerCache(){
+    Properties props = new Properties();
+    props.setProperty("mcast-port", "10333");
+    props.setProperty("locators", "");
+    System.setProperty("gemfirePropertyFile", BUG_40255_PROPS);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40255_XML);
+    System.setProperty(NESTED_ATTR_PROPERTY_STRING, NESTED_ATTR_PROPERTY_VALUE);
+    System.setProperty(ATTR_PROPERTY_STRING, ATTR_PROPERTY_VALUE);
+    System.setProperty(ELEMENT_PROPERTY_STRING, ELEMENT_PROPERTY_VALUE);
+    System.setProperty(CONCAT_ELEMENT_PROPERTY_STRING, CONCAT_ELEMENT_PROPERTY_VALUE);
+    
+    // create the directory where data is going to be stored
+    File dir = new File("persistData1");
+    dir.mkdir();
+
+    this.ds = DistributedSystem.connect(props);
+    this.cache = CacheFactory.create(this.ds);
+
+    Region exampleRegion = this.cache.getRegion("example-region");
+    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
+
+    //Check if disk store got same name as passed in system properties in setup().
+    assertEquals(attrs.getDiskStoreName(), System.getProperty(ATTR_PROPERTY_STRING));
+    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE));
+    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE), ELEMENT_KEY_VALUE);
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    super.tearDown();
+    if (this.cache != null) {
+      this.cache.close();
+      this.cache = null;
+    }
+    if (this.ds != null) {
+      this.ds.disconnect();
+      this.ds = null;
+    }
+  }
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitTest.java
deleted file mode 100644
index 92bfbe7..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitTest.java
+++ /dev/null
@@ -1,143 +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.cache30;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.util.Properties;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-/**
- * @author Shobhit Agarwal
- *
- */
-@Category(IntegrationTest.class)
-@Ignore("Test is broken and was named Bug40255JUnitDisabledTest")
-public class Bug40255JUnitTest {
-
-  private static final String BUG_40255_XML = Bug40255JUnitTest.class.getResource("bug40255xmlparameterization.xml").getFile();
-  private static final String BUG_40255_PROPS = Bug40255JUnitTest.class.getResource("bug40255_gemfire.properties").getFile();
-
-  private static final String ATTR_PROPERTY_STRING = "region.disk.store";
-
-  private static final String ATTR_PROPERTY_VALUE = "teststore";
-
-  private static final String NESTED_ATTR_PROPERTY_STRING = "custom-nested.test";
-
-  private static final String NESTED_ATTR_PROPERTY_VALUE = "disk";
-
-  private static final String ELEMENT_PROPERTY_STRING = "custom-string.element";
-
-  private static final String ELEMENT_PROPERTY_VALUE = "example-string";
-
-  private static final String CONCAT_ELEMENT_PROPERTY_STRING = "concat.test";
-
-  private static final String CONCAT_ELEMENT_PROPERTY_VALUE = "-name";
-
-  private static final String ELEMENT_KEY_VALUE = "example-value";
-
-  DistributedSystem ds;
-  Cache cache;
-
-  @Test
-  public void testResolveReplacePropertyStringForLonerCache(){
-    Properties props = new Properties();
-    props.setProperty("mcast-port", "0");
-    props.setProperty("locators", "");
-    System.setProperty("gemfirePropertyFile", BUG_40255_PROPS);
-    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40255_XML);
-    System.setProperty(NESTED_ATTR_PROPERTY_STRING, NESTED_ATTR_PROPERTY_VALUE);
-    System.setProperty(ATTR_PROPERTY_STRING, ATTR_PROPERTY_VALUE);
-    System.setProperty(ELEMENT_PROPERTY_STRING, ELEMENT_PROPERTY_VALUE);
-    System.setProperty(CONCAT_ELEMENT_PROPERTY_STRING, CONCAT_ELEMENT_PROPERTY_VALUE);
-    
-    // create the directory where data is going to be stored
-    File dir = new File("persistData1");
-    dir.mkdir();
-
-    this.ds = DistributedSystem.connect(props);
-    this.cache = CacheFactory.create(this.ds);
-
-    Region exampleRegion = this.cache.getRegion("example-region");
-    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
-
-    //Check if disk store got same name as passed in system properties in setup().
-    assertEquals(attrs.getDiskStoreName(), System.getProperty(ATTR_PROPERTY_STRING));
-    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE));
-    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE), ELEMENT_KEY_VALUE);
-    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE));
-    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE), CONCAT_ELEMENT_PROPERTY_VALUE);
-  }
-
-  @Test
-  public void testResolveReplacePropertyStringForNonLonerCache(){
-    Properties props = new Properties();
-    props.setProperty("mcast-port", "10333");
-    props.setProperty("locators", "");
-    System.setProperty("gemfirePropertyFile", BUG_40255_PROPS);
-    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40255_XML);
-    System.setProperty(NESTED_ATTR_PROPERTY_STRING, NESTED_ATTR_PROPERTY_VALUE);
-    System.setProperty(ATTR_PROPERTY_STRING, ATTR_PROPERTY_VALUE);
-    System.setProperty(ELEMENT_PROPERTY_STRING, ELEMENT_PROPERTY_VALUE);
-    System.setProperty(CONCAT_ELEMENT_PROPERTY_STRING, CONCAT_ELEMENT_PROPERTY_VALUE);
-    
-    // create the directory where data is going to be stored
-    File dir = new File("persistData1");
-    dir.mkdir();
-
-    this.ds = DistributedSystem.connect(props);
-    this.cache = CacheFactory.create(this.ds);
-
-    Region exampleRegion = this.cache.getRegion("example-region");
-    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
-
-    //Check if disk store got same name as passed in system properties in setup().
-    assertEquals(attrs.getDiskStoreName(), System.getProperty(ATTR_PROPERTY_STRING));
-    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE));
-    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE), ELEMENT_KEY_VALUE);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (this.cache != null) {
-      this.cache.close();
-      this.cache = null;
-    }
-    if (this.ds != null) {
-      this.ds.disconnect();
-      this.ds = null;
-    }
-  }
-
-  @Before
-  public void setUp() throws Exception {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitDisabledTest.java
new file mode 100644
index 0000000..9455641
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitDisabledTest.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * 
+ */
+package com.gemstone.gemfire.cache30;
+
+import java.util.Properties;
+
+import junit.framework.TestCase;
+
+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.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+
+/**
+ * Test for Bug no. 40662. To verify the default action being set in eviction
+ * attributes by CacheXmlParser when cache.xml has eviction attributes with no
+ * eviction action specified. which was being set to EvictionAction.NONE
+ * 
+ * @author shoagarwal
+ * @since 6.6
+ */
+public class Bug40662JUnitDisabledTest extends TestCase {
+
+  private static final String BUG_40662_XML = Bug40662JUnitDisabledTest.class.getResource("bug40662noevictionaction.xml").getFile();
+
+  DistributedSystem ds;
+  Cache cache;
+
+  @Override
+  public void setName(String name) {
+    super.setName(name);
+  }
+
+  /**
+   * Test for checking eviction action in eviction attributes if no evicition
+   * action is specified in cache.xml
+   */
+  public void testEvictionActionSetLocalDestroyPass() {
+    Region exampleRegion = this.cache.getRegion("example-region");
+    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
+    EvictionAttributes evicAttrs = attrs.getEvictionAttributes();
+
+    //Default eviction action is LOCAL_DESTROY always. 
+    assertEquals(EvictionAction.LOCAL_DESTROY, evicAttrs.getAction());
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    super.tearDown();
+    if (this.cache != null) {
+      this.cache.close();
+      this.cache = null;
+    }
+    if (this.ds != null) {
+      this.ds.disconnect();
+      this.ds = null;
+    }
+  }
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    Properties props = new Properties();
+    props.setProperty("mcast-port", "0");
+    props.setProperty("locators", "");
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40662_XML);
+    this.ds = DistributedSystem.connect(props);
+    this.cache = CacheFactory.create(this.ds);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitTest.java
deleted file mode 100644
index 0d62127..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitTest.java
+++ /dev/null
@@ -1,93 +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.cache30;
-
-import static org.junit.Assert.*;
-
-import java.util.Properties;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.experimental.categories.Category;
-
-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.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-/**
- * Test for Bug no. 40662. To verify the default action being set in eviction
- * attributes by CacheXmlParser when cache.xml has eviction attributes with no
- * eviction action specified. which was being set to EvictionAction.NONE
- * 
- * @author shoagarwal
- * @since 6.6
- */
-@Category(IntegrationTest.class)
-@Ignore("Test is broken and was named Bug40662JUnitDisabledTest")
-public class Bug40662JUnitTest {
-
-  private static final String BUG_40662_XML = Bug40662JUnitTest.class.getResource("bug40662noevictionaction.xml").getFile();
-
-  DistributedSystem ds;
-  Cache cache;
-
-  /**
-   * Test for checking eviction action in eviction attributes if no evicition
-   * action is specified in cache.xml
-   */
-  public void testEvictionActionSetLocalDestroyPass() {
-    Region exampleRegion = this.cache.getRegion("example-region");
-    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
-    EvictionAttributes evicAttrs = attrs.getEvictionAttributes();
-
-    //Default eviction action is LOCAL_DESTROY always. 
-    assertEquals(EvictionAction.LOCAL_DESTROY, evicAttrs.getAction());
-  }
-
-  @After
-  protected void tearDown() throws Exception {
-    if (this.cache != null) {
-      this.cache.close();
-      this.cache = null;
-    }
-    if (this.ds != null) {
-      this.ds.disconnect();
-      this.ds = null;
-    }
-  }
-
-  @Before
-  protected void setUp() throws Exception {
-    Properties props = new Properties();
-    props.setProperty("mcast-port", "0");
-    props.setProperty("locators", "");
-    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40662_XML);
-    this.ds = DistributedSystem.connect(props);
-    this.cache = CacheFactory.create(this.ds);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitDisabledTest.java
new file mode 100755
index 0000000..7b93a36
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitDisabledTest.java
@@ -0,0 +1,562 @@
+/*
+ * 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.locks;
+
+import junit.framework.*;
+import java.util.*;
+
+import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.internal.logging.LocalLogWriter;
+import com.gemstone.gemfire.internal.logging.InternalLogWriter;
+
+import dunit.DistributedTestCase;
+import dunit.DistributedTestCase.WaitCriterion;
+
+/**
+ * Tests the Collaboration Lock used internally by dlock service.
+ *
+ * @author Kirk Lund
+ * @since 4.1.1
+ */
+public class CollaborationJUnitDisabledTest extends TestCase {
+
+  protected LogWriter log = new LocalLogWriter(InternalLogWriter.INFO_LEVEL);
+  protected Collaboration collaboration;
+  
+  public CollaborationJUnitDisabledTest(String name) {
+    super(name);
+  }
+  
+  public void setUp() throws Exception {
+    this.collaboration = new Collaboration(new CancelCriterion() {
+
+      public String cancelInProgress() {
+        // TODO Auto-generated method stub
+        return null;
+      }
+
+      public RuntimeException generateCancelledException(Throwable e) {
+        // TODO Auto-generated method stub
+        return null;
+      }
+      
+    });
+  }
+  
+  public void tearDown() throws Exception {
+    this.collaboration = null;
+  }
+  
+  protected volatile boolean flagTestBlocksUntilRelease = false;
+  protected volatile boolean threadBStartedTestBlocksUntilRelease = false;
+  public void testBlocksUntilRelease() throws Exception {
+    this.log.info("[testBlocksUntilRelease]");
+    Thread threadA = new Thread(group, new Runnable() {
+      public void run() {
+        collaboration.acquireUninterruptibly("topicA");
+        try {
+          flagTestBlocksUntilRelease = true;
+          while(flagTestBlocksUntilRelease) {
+            try {
+              Thread.sleep(10);
+            }
+            catch (InterruptedException ignore) {fail("interrupted");}
+          }
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    
+    // thread one acquires
+    threadA.start();
+    WaitCriterion ev = new WaitCriterion() {
+      public boolean done() {
+        return CollaborationJUnitDisabledTest.this.flagTestBlocksUntilRelease;
+      }
+      public String description() {
+        return "waiting for thread";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
+    assertTrue(this.collaboration.hasCurrentTopic(threadA));
+    
+    // thread two blocks until one releeases
+    Thread threadB = new Thread(group, new Runnable() {
+      public void run() {
+        threadBStartedTestBlocksUntilRelease = true;
+        collaboration.acquireUninterruptibly("topicB");
+        try {
+          flagTestBlocksUntilRelease = true;
+          WaitCriterion ev2 = new WaitCriterion() {
+            public boolean done() {
+              return !flagTestBlocksUntilRelease;
+            }
+            public String description() {
+              return "waiting for release";
+            }
+          };
+          DistributedTestCase.waitForCriterion(ev2, 20 * 1000, 200, true);
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    
+    // start up threadB
+    threadB.start();
+    ev = new WaitCriterion() {
+      public boolean done() {
+        return threadBStartedTestBlocksUntilRelease;
+      }
+      public String description() {
+        return "waiting for thread b";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
+    
+    // threadA holds topic and threadB is waiting...
+    assertTrue(this.collaboration.hasCurrentTopic(threadA));
+    assertFalse(this.collaboration.hasCurrentTopic(threadB));
+
+    // let threadA release so that threadB gets lock
+    this.flagTestBlocksUntilRelease = false;
+    DistributedTestCase.join(threadA, 30 * 1000, null);
+    
+    // make sure threadB is doing what it's supposed to do...
+    ev = new WaitCriterion() {
+      public boolean done() {
+        return flagTestBlocksUntilRelease;
+      }
+      public String description() {
+        return "threadB";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
+    // threadB must have lock now... let threadB release
+    assertTrue(this.collaboration.hasCurrentTopic(threadB));
+    this.flagTestBlocksUntilRelease = false;
+    DistributedTestCase.join(threadB, 30 * 1000, null);
+
+    // collaboration should be free now    
+    assertFalse(this.collaboration.hasCurrentTopic(threadA));
+    assertFalse(this.collaboration.hasCurrentTopic(threadB));
+    assertFalse(this.collaboration.hasCurrentTopic());
+  }
+  
+  protected volatile boolean threadAFlag_TestLateComerJoinsIn = false;
+  protected volatile boolean threadBFlag_TestLateComerJoinsIn = false;
+  protected volatile boolean threadCFlag_TestLateComerJoinsIn = true;
+  protected volatile boolean threadDFlag_TestLateComerJoinsIn = false;
+  public void testLateComerJoinsIn() throws Exception {
+    this.log.info("[testLateComerJoinsIn]");
+    
+    final Object topicA = "topicA";
+    final Object topicB = "topicB";
+    
+    // threads one and two acquire
+    Thread threadA = new Thread(group, new Runnable() {
+      public void run() {
+        collaboration.acquireUninterruptibly(topicA);
+        try {
+          threadAFlag_TestLateComerJoinsIn = true;
+          WaitCriterion ev = new WaitCriterion() {
+            public boolean done() {
+              return !threadAFlag_TestLateComerJoinsIn;
+            }
+            public String description() {
+              return null;
+            }
+          };
+          DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    threadA.start();
+    WaitCriterion ev = new WaitCriterion() {
+      public boolean done() {
+        return threadAFlag_TestLateComerJoinsIn;
+      }
+      public String description() {
+        return "wait for ThreadA";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 30 * 1000, 200, true);
+    assertTrue(this.collaboration.hasCurrentTopic(threadA));
+    assertTrue(this.collaboration.isCurrentTopic(topicA));
+    
+    Thread threadB = new Thread(group, new Runnable() {
+      public void run() {
+        collaboration.acquireUninterruptibly(topicA);
+        try {
+          threadBFlag_TestLateComerJoinsIn = true;
+          WaitCriterion ev2 = new WaitCriterion() {
+            public boolean done() {
+              return !threadBFlag_TestLateComerJoinsIn;
+            }
+            public String description() {
+              return null;
+            }
+          };
+          DistributedTestCase.waitForCriterion(ev2, 60 * 1000, 200, true);
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    threadB.start();
+    ev = new WaitCriterion() {
+      public boolean done() {
+        return threadBFlag_TestLateComerJoinsIn;
+      }
+      public String description() {
+        return "";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+    assertTrue(this.collaboration.hasCurrentTopic(threadB));
+    
+    // thread three blocks for new topic
+    Thread threadC = new Thread(group, new Runnable() {
+      public void run() {
+        threadCFlag_TestLateComerJoinsIn = false;
+        collaboration.acquireUninterruptibly(topicB);
+        try {
+          threadCFlag_TestLateComerJoinsIn = true;
+          WaitCriterion ev2 = new WaitCriterion() {
+            public boolean done() {
+              return !threadCFlag_TestLateComerJoinsIn;
+            }
+            public String description() {
+              return null;
+            }
+          };
+          DistributedTestCase.waitForCriterion(ev2, 60 * 1000, 200, true);
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    threadC.start();
+    ev = new WaitCriterion() {
+      public boolean done() {
+        return threadCFlag_TestLateComerJoinsIn;
+      }
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+    assertFalse(this.collaboration.hasCurrentTopic(threadC));
+    assertFalse(this.collaboration.isCurrentTopic(topicB));
+    
+    // thread four (lateComer) acquires current topic immediately
+    Thread threadD = new Thread(group, new Runnable() {
+      public void run() {
+        collaboration.acquireUninterruptibly(topicA);
+        try {
+          threadDFlag_TestLateComerJoinsIn = true;
+          while(threadDFlag_TestLateComerJoinsIn) {
+            try {
+              Thread.sleep(10);
+            }
+            catch (InterruptedException ignore) {fail("interrupted");}
+          }
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    threadD.start();
+    ev = new WaitCriterion() {
+      public boolean done() {
+        return threadDFlag_TestLateComerJoinsIn;
+      }
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+    assertTrue(this.collaboration.hasCurrentTopic(threadD));
+    
+    // release threadA
+    this.threadAFlag_TestLateComerJoinsIn = false;
+    DistributedTestCase.join(threadA, 30 * 1000, null);
+    assertFalse(this.collaboration.hasCurrentTopic(threadA));
+    assertTrue(this.collaboration.hasCurrentTopic(threadB));
+    assertFalse(this.collaboration.hasCurrentTopic(threadC));
+    assertTrue(this.collaboration.hasCurrentTopic(threadD));
+    assertTrue(this.collaboration.isCurrentTopic(topicA));
+    assertFalse(this.collaboration.isCurrentTopic(topicB));
+    
+    // release threadB
+    this.threadBFlag_TestLateComerJoinsIn = false;
+    DistributedTestCase.join(threadB, 30 * 1000, null);
+    assertFalse(this.collaboration.hasCurrentTopic(threadB));
+    assertFalse(this.collaboration.hasCurrentTopic(threadC));
+    assertTrue(this.collaboration.hasCurrentTopic(threadD));
+    assertTrue(this.collaboration.isCurrentTopic(topicA));
+    assertFalse(this.collaboration.isCurrentTopic(topicB));
+    
+    // release threadD
+    this.threadDFlag_TestLateComerJoinsIn = false;
+    DistributedTestCase.join(threadD, 30 * 1000, null);
+    ev = new WaitCriterion() {
+      public boolean done() {
+        return threadCFlag_TestLateComerJoinsIn;
+      }
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+    assertTrue(this.collaboration.hasCurrentTopic(threadC));
+    assertFalse(this.collaboration.hasCurrentTopic(threadD));
+    assertFalse(this.collaboration.isCurrentTopic(topicA));
+    assertTrue(this.collaboration.isCurrentTopic(topicB));
+    
+    // release threadC
+    this.threadCFlag_TestLateComerJoinsIn = false;
+    DistributedTestCase.join(threadC, 30 * 1000, null);
+    assertFalse(this.collaboration.hasCurrentTopic(threadC));
+    assertFalse(this.collaboration.isCurrentTopic(topicA));
+    assertFalse(this.collaboration.isCurrentTopic(topicB));
+  }
+  
+  protected List waitingList = Collections.synchronizedList(new ArrayList());
+  protected List fairnessList = Collections.synchronizedList(new ArrayList());
+  protected volatile boolean runTestFairnessStressfully = true;
+  public void testFairnessStressfully() throws Exception {
+    this.log.info("[testFairnessStressfully]");
+    final int numThreads = 20;
+    Thread threads[] = new Thread[numThreads];
+    
+    Runnable run = new Runnable() {
+      public void run() {
+        boolean released = false;
+        try {
+          String uniqueTopic = Thread.currentThread().getName();
+          while(runTestFairnessStressfully) {
+            waitingList.add(uniqueTopic);
+            collaboration.acquireUninterruptibly(uniqueTopic);
+            try {
+              released = false;
+              fairnessList.add(uniqueTopic);
+              waitingList.remove(uniqueTopic);
+            }
+            finally {
+              // wait for the other threads to line up...
+              WaitCriterion ev = new WaitCriterion() {
+                public boolean done() {
+                  return !runTestFairnessStressfully || waitingList.size() >= numThreads - 1;
+                }
+                public String description() {
+                  return "other threads lining up";
+                }
+              };
+              DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+              collaboration.release();
+              released = true;
+            }
+          }
+        }
+        finally {
+          if (!released) {
+            collaboration.release();
+          }
+        }
+      }
+    };
+    
+    try {
+      // many threads loop: acquire and release with unique topic
+      for (int t = 0; t < threads.length; t++) {
+        threads[t] = new Thread(group, run, String.valueOf(t));
+        threads[t].start();
+      }
+
+      log.info("Started all threads... waiting for test to complete.");
+            
+      // wait for numThreads * 10
+      WaitCriterion ev = new WaitCriterion() {
+        public boolean done() {
+          return fairnessList.size() >= numThreads * 20;
+        }
+        public String description() {
+          return "waiting for numThreads * 10";
+        }
+      };
+      DistributedTestCase.waitForCriterion(ev, 5 * 60 * 1000, 200, true);
+    }
+    finally {
+      if (this.runTestFairnessStressfully) {
+        this.runTestFairnessStressfully = false;
+      }
+    }
+    
+    for (int t = 0; t < threads.length; t++) {
+      DistributedTestCase.join(threads[t], 30 * 1000, null);
+    }
+    
+    // assert that all topics are acquired in order
+    // count number of occurrences of each thread
+    int count[] = new int[numThreads];
+    for (int i = 0; i < count.length; i++) { // shouldn't be necessary
+      count[i] = 0;
+    }
+    synchronized(this.fairnessList) {
+      for (Iterator iter = this.fairnessList.iterator(); iter.hasNext();) {
+        int id = Integer.valueOf((String)iter.next()).intValue();
+        count[id] = count[id]+1;
+      }
+    }
+    
+    int totalLocks = 0;
+    int minLocks = Integer.MAX_VALUE;
+    int maxLocks = 0;
+    for (int i = 0; i < count.length; i++) {
+      int locks = count[i];
+      this.log.fine("testFairnessStressfully thread-" + i + " acquired topic " + 
+        locks + " times.");
+      if (locks < minLocks) minLocks = locks;
+      if (locks > maxLocks) maxLocks = locks;
+      totalLocks = totalLocks + locks;
+    }
+
+    this.log.info("[testFairnessStressfully] totalLocks=" + totalLocks + 
+                  " minLocks=" + minLocks +
+                  " maxLocks=" + maxLocks);
+
+    int expectedLocks = (totalLocks / numThreads) + 1;
+    
+    // NOTE: if you turn on fine logs, this deviation may be too small...
+    // slower machines may also fail depending on thread scheduling
+    int deviation = (int)(expectedLocks * 0.25);
+    int lowThreshold = expectedLocks - deviation;
+    int highThreshold = expectedLocks + deviation;
+
+    this.log.info("[testFairnessStressfully] deviation=" + deviation +
+                  " expectedLocks=" + expectedLocks + 
+                  " lowThreshold=" + lowThreshold +
+                  " highThreshold=" + highThreshold);
+                        
+    // if these assertions keep failing we'll have to rewrite the test
+    // to handle scheduling of the threads...
+                  
+    assertTrue("minLocks is less than lowThreshold",
+               minLocks >= lowThreshold);
+    assertTrue("maxLocks is greater than highThreshold",
+               maxLocks <= highThreshold);
+  }
+  
+  public void testHasCurrentTopic() throws Exception {
+    this.log.info("[testHasCurrentTopic]");
+    assertTrue(!this.collaboration.hasCurrentTopic());
+    this.collaboration.acquireUninterruptibly("testHasCurrentTopic");
+    try {
+      assertTrue(this.collaboration.hasCurrentTopic());
+    }
+    finally {
+      this.collaboration.release();
+    }
+    assertTrue(!this.collaboration.hasCurrentTopic());
+  }
+  
+  protected volatile boolean flagTestThreadHasCurrentTopic = false;
+  public void testThreadHasCurrentTopic() throws Exception {
+    this.log.info("[testThreadHasCurrentTopic]");
+    Thread thread = new Thread(group, new Runnable() {
+      public void run() {
+        collaboration.acquireUninterruptibly("testThreadHasCurrentTopic");
+        try {
+          flagTestThreadHasCurrentTopic = true;
+          WaitCriterion ev = new WaitCriterion() {
+            public boolean done() {
+              return !flagTestThreadHasCurrentTopic;
+            }
+            public String description() {
+              return null;
+            }
+          };
+          DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    
+    // before starting thread, hasCurrentTopic(thread) returns false
+    assertTrue(!this.collaboration.hasCurrentTopic(thread));
+    thread.start();
+    WaitCriterion ev = new WaitCriterion() {
+      public boolean done() {
+        return flagTestThreadHasCurrentTopic;
+      }
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+    
+    // after starting thread, hasCurrentTopic(thread) returns true
+    assertTrue(this.collaboration.hasCurrentTopic(thread));
+    this.flagTestThreadHasCurrentTopic = false;
+    DistributedTestCase.join(thread, 30 * 1000, null);
+    
+    // after thread finishes, hasCurrentTopic(thread) returns false
+    assertTrue(!this.collaboration.hasCurrentTopic(thread));
+  }
+  
+  public void testIsCurrentTopic() throws Exception {
+    this.log.info("[testIsCurrentTopic]");
+    Object topic = "testIsCurrentTopic";
+    assertTrue(!this.collaboration.isCurrentTopic(topic));
+    this.collaboration.acquireUninterruptibly(topic);
+    try {
+      assertTrue(this.collaboration.isCurrentTopic(topic));
+    }
+    finally {
+      this.collaboration.release();
+    }
+    assertTrue(!this.collaboration.isCurrentTopic(topic));
+  }
+
+  protected final ThreadGroup group = 
+      new ThreadGroup("CollaborationJUnitTest Threads") {
+        public void uncaughtException(Thread t, Throwable e)
+        {
+          if (e instanceof VirtualMachineError) {
+            SystemFailure.setFailure((VirtualMachineError)e); // don't throw
+          }
+          String s = "Uncaught exception in thread " + t;
+          log.error(s, e);
+          fail(s);
+        }
+      };
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitTest.java
deleted file mode 100755
index 711500e..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitTest.java
+++ /dev/null
@@ -1,617 +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.locks;
-
-import static org.junit.Assert.*;
-
-import java.util.*;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.CancelCriterion;
-import com.gemstone.gemfire.LogWriter;
-import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.internal.logging.LocalLogWriter;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import com.gemstone.gemfire.internal.logging.InternalLogWriter;
-
-import dunit.DistributedTestCase;
-import dunit.DistributedTestCase.WaitCriterion;
-
-/**
- * Tests the Collaboration Lock used internally by dlock service.
- *
- * @author Kirk Lund
- * @since 4.1.1
- */
-@Category(IntegrationTest.class)
-@Ignore("Test is broken and was named CollaborationJUnitDisabledTest")
-public class CollaborationJUnitTest {
-
-  protected LogWriter log = new LocalLogWriter(InternalLogWriter.INFO_LEVEL);
-  protected Collaboration collaboration;
-  
-  @Before
-  public void setUp() throws Exception {
-    this.collaboration = new Collaboration(new CancelCriterion() {
-      @Override
-      public String cancelInProgress() {
-        return null;
-      }
-      @Override
-      public RuntimeException generateCancelledException(Throwable e) {
-        return null;
-      }
-    });
-  }
-  
-  @After
-  public void tearDown() throws Exception {
-    this.collaboration = null;
-  }
-  
-  protected volatile boolean flagTestBlocksUntilRelease = false;
-  protected volatile boolean threadBStartedTestBlocksUntilRelease = false;
-  
-  @Test
-  public void testBlocksUntilRelease() throws Exception {
-    this.log.info("[testBlocksUntilRelease]");
-    Thread threadA = new Thread(group, new Runnable() {
-      @Override
-      public void run() {
-        collaboration.acquireUninterruptibly("topicA");
-        try {
-          flagTestBlocksUntilRelease = true;
-          while(flagTestBlocksUntilRelease) {
-            try {
-              Thread.sleep(10);
-            }
-            catch (InterruptedException ignore) {fail("interrupted");}
-          }
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    
-    // thread one acquires
-    threadA.start();
-    WaitCriterion ev = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return CollaborationJUnitTest.this.flagTestBlocksUntilRelease;
-      }
-      @Override
-      public String description() {
-        return "waiting for thread";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
-    assertTrue(this.collaboration.hasCurrentTopic(threadA));
-    
-    // thread two blocks until one releeases
-    Thread threadB = new Thread(group, new Runnable() {
-      @Override
-      public void run() {
-        threadBStartedTestBlocksUntilRelease = true;
-        collaboration.acquireUninterruptibly("topicB");
-        try {
-          flagTestBlocksUntilRelease = true;
-          WaitCriterion ev2 = new WaitCriterion() {
-            @Override
-            public boolean done() {
-              return !flagTestBlocksUntilRelease;
-            }
-            @Override
-            public String description() {
-              return "waiting for release";
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev2, 20 * 1000, 200, true);
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    
-    // start up threadB
-    threadB.start();
-    ev = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return threadBStartedTestBlocksUntilRelease;
-      }
-      @Override
-      public String description() {
-        return "waiting for thread b";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
-    
-    // threadA holds topic and threadB is waiting...
-    assertTrue(this.collaboration.hasCurrentTopic(threadA));
-    assertFalse(this.collaboration.hasCurrentTopic(threadB));
-
-    // let threadA release so that threadB gets lock
-    this.flagTestBlocksUntilRelease = false;
-    DistributedTestCase.join(threadA, 30 * 1000, null);
-    
-    // make sure threadB is doing what it's supposed to do...
-    ev = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return flagTestBlocksUntilRelease;
-      }
-      @Override
-      public String description() {
-        return "threadB";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
-    // threadB must have lock now... let threadB release
-    assertTrue(this.collaboration.hasCurrentTopic(threadB));
-    this.flagTestBlocksUntilRelease = false;
-    DistributedTestCase.join(threadB, 30 * 1000, null);
-
-    // collaboration should be free now    
-    assertFalse(this.collaboration.hasCurrentTopic(threadA));
-    assertFalse(this.collaboration.hasCurrentTopic(threadB));
-    assertFalse(this.collaboration.hasCurrentTopic());
-  }
-  
-  protected volatile boolean threadAFlag_TestLateComerJoinsIn = false;
-  protected volatile boolean threadBFlag_TestLateComerJoinsIn = false;
-  protected volatile boolean threadCFlag_TestLateComerJoinsIn = true;
-  protected volatile boolean threadDFlag_TestLateComerJoinsIn = false;
-  
-  @Test
-  public void testLateComerJoinsIn() throws Exception {
-    this.log.info("[testLateComerJoinsIn]");
-    
-    final Object topicA = "topicA";
-    final Object topicB = "topicB";
-    
-    // threads one and two acquire
-    Thread threadA = new Thread(group, new Runnable() {
-      @Override
-      public void run() {
-        collaboration.acquireUninterruptibly(topicA);
-        try {
-          threadAFlag_TestLateComerJoinsIn = true;
-          WaitCriterion ev = new WaitCriterion() {
-            @Override
-            public boolean done() {
-              return !threadAFlag_TestLateComerJoinsIn;
-            }
-            @Override
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    threadA.start();
-    WaitCriterion ev = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return threadAFlag_TestLateComerJoinsIn;
-      }
-      @Override
-      public String description() {
-        return "wait for ThreadA";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 30 * 1000, 200, true);
-    assertTrue(this.collaboration.hasCurrentTopic(threadA));
-    assertTrue(this.collaboration.isCurrentTopic(topicA));
-    
-    Thread threadB = new Thread(group, new Runnable() {
-      @Override
-      public void run() {
-        collaboration.acquireUninterruptibly(topicA);
-        try {
-          threadBFlag_TestLateComerJoinsIn = true;
-          WaitCriterion ev2 = new WaitCriterion() {
-            @Override
-            public boolean done() {
-              return !threadBFlag_TestLateComerJoinsIn;
-            }
-            @Override
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev2, 60 * 1000, 200, true);
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    threadB.start();
-    ev = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return threadBFlag_TestLateComerJoinsIn;
-      }
-      @Override
-      public String description() {
-        return "";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-    assertTrue(this.collaboration.hasCurrentTopic(threadB));
-    
-    // thread three blocks for new topic
-    Thread threadC = new Thread(group, new Runnable() {
-      @Override
-      public void run() {
-        threadCFlag_TestLateComerJoinsIn = false;
-        collaboration.acquireUninterruptibly(topicB);
-        try {
-          threadCFlag_TestLateComerJoinsIn = true;
-          WaitCriterion ev2 = new WaitCriterion() {
-            @Override
-            public boolean done() {
-              return !threadCFlag_TestLateComerJoinsIn;
-            }
-            @Override
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev2, 60 * 1000, 200, true);
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    threadC.start();
-    ev = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return threadCFlag_TestLateComerJoinsIn;
-      }
-      @Override
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-    assertFalse(this.collaboration.hasCurrentTopic(threadC));
-    assertFalse(this.collaboration.isCurrentTopic(topicB));
-    
-    // thread four (lateComer) acquires current topic immediately
-    Thread threadD = new Thread(group, new Runnable() {
-      @Override
-      public void run() {
-        collaboration.acquireUninterruptibly(topicA);
-        try {
-          threadDFlag_TestLateComerJoinsIn = true;
-          while(threadDFlag_TestLateComerJoinsIn) {
-            try {
-              Thread.sleep(10);
-            }
-            catch (InterruptedException ignore) {fail("interrupted");}
-          }
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    threadD.start();
-    ev = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return threadDFlag_TestLateComerJoinsIn;
-      }
-      @Override
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-    assertTrue(this.collaboration.hasCurrentTopic(threadD));
-    
-    // release threadA
-    this.threadAFlag_TestLateComerJoinsIn = false;
-    DistributedTestCase.join(threadA, 30 * 1000, null);
-    assertFalse(this.collaboration.hasCurrentTopic(threadA));
-    assertTrue(this.collaboration.hasCurrentTopic(threadB));
-    assertFalse(this.collaboration.hasCurrentTopic(threadC));
-    assertTrue(this.collaboration.hasCurrentTopic(threadD));
-    assertTrue(this.collaboration.isCurrentTopic(topicA));
-    assertFalse(this.collaboration.isCurrentTopic(topicB));
-    
-    // release threadB
-    this.threadBFlag_TestLateComerJoinsIn = false;
-    DistributedTestCase.join(threadB, 30 * 1000, null);
-    assertFalse(this.collaboration.hasCurrentTopic(threadB));
-    assertFalse(this.collaboration.hasCurrentTopic(threadC));
-    assertTrue(this.collaboration.hasCurrentTopic(threadD));
-    assertTrue(this.collaboration.isCurrentTopic(topicA));
-    assertFalse(this.collaboration.isCurrentTopic(topicB));
-    
-    // release threadD
-    this.threadDFlag_TestLateComerJoinsIn = false;
-    DistributedTestCase.join(threadD, 30 * 1000, null);
-    ev = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return threadCFlag_TestLateComerJoinsIn;
-      }
-      @Override
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-    assertTrue(this.collaboration.hasCurrentTopic(threadC));
-    assertFalse(this.collaboration.hasCurrentTopic(threadD));
-    assertFalse(this.collaboration.isCurrentTopic(topicA));
-    assertTrue(this.collaboration.isCurrentTopic(topicB));
-    
-    // release threadC
-    this.threadCFlag_TestLateComerJoinsIn = false;
-    DistributedTestCase.join(threadC, 30 * 1000, null);
-    assertFalse(this.collaboration.hasCurrentTopic(threadC));
-    assertFalse(this.collaboration.isCurrentTopic(topicA));
-    assertFalse(this.collaboration.isCurrentTopic(topicB));
-  }
-  
-  protected List waitingList = Collections.synchronizedList(new ArrayList());
-  protected List fairnessList = Collections.synchronizedList(new ArrayList());
-  protected volatile boolean runTestFairnessStressfully = true;
-  
-  @Test
-  public void testFairnessStressfully() throws Exception {
-    this.log.info("[testFairnessStressfully]");
-    final int numThreads = 20;
-    Thread threads[] = new Thread[numThreads];
-    
-    Runnable run = new Runnable() {
-      public void run() {
-        boolean released = false;
-        try {
-          String uniqueTopic = Thread.currentThread().getName();
-          while(runTestFairnessStressfully) {
-            waitingList.add(uniqueTopic);
-            collaboration.acquireUninterruptibly(uniqueTopic);
-            try {
-              released = false;
-              fairnessList.add(uniqueTopic);
-              waitingList.remove(uniqueTopic);
-            }
-            finally {
-              // wait for the other threads to line up...
-              WaitCriterion ev = new WaitCriterion() {
-                @Override
-                public boolean done() {
-                  return !runTestFairnessStressfully || waitingList.size() >= numThreads - 1;
-                }
-                @Override
-                public String description() {
-                  return "other threads lining up";
-                }
-              };
-              DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-              collaboration.release();
-              released = true;
-            }
-          }
-        }
-        finally {
-          if (!released) {
-            collaboration.release();
-          }
-        }
-      }
-    };
-    
-    try {
-      // many threads loop: acquire and release with unique topic
-      for (int t = 0; t < threads.length; t++) {
-        threads[t] = new Thread(group, run, String.valueOf(t));
-        threads[t].start();
-      }
-
-      log.info("Started all threads... waiting for test to complete.");
-            
-      // wait for numThreads * 10
-      WaitCriterion ev = new WaitCriterion() {
-        @Override
-        public boolean done() {
-          return fairnessList.size() >= numThreads * 20;
-        }
-        @Override
-        public String description() {
-          return "waiting for numThreads * 10";
-        }
-      };
-      DistributedTestCase.waitForCriterion(ev, 5 * 60 * 1000, 200, true);
-    }
-    finally {
-      if (this.runTestFairnessStressfully) {
-        this.runTestFairnessStressfully = false;
-      }
-    }
-    
-    for (int t = 0; t < threads.length; t++) {
-      DistributedTestCase.join(threads[t], 30 * 1000, null);
-    }
-    
-    // assert that all topics are acquired in order
-    // count number of occurrences of each thread
-    int count[] = new int[numThreads];
-    for (int i = 0; i < count.length; i++) { // shouldn't be necessary
-      count[i] = 0;
-    }
-    synchronized(this.fairnessList) {
-      for (Iterator iter = this.fairnessList.iterator(); iter.hasNext();) {
-        int id = Integer.valueOf((String)iter.next()).intValue();
-        count[id] = count[id]+1;
-      }
-    }
-    
-    int totalLocks = 0;
-    int minLocks = Integer.MAX_VALUE;
-    int maxLocks = 0;
-    for (int i = 0; i < count.length; i++) {
-      int locks = count[i];
-      this.log.fine("testFairnessStressfully thread-" + i + " acquired topic " + 
-        locks + " times.");
-      if (locks < minLocks) minLocks = locks;
-      if (locks > maxLocks) maxLocks = locks;
-      totalLocks = totalLocks + locks;
-    }
-
-    this.log.info("[testFairnessStressfully] totalLocks=" + totalLocks + 
-                  " minLocks=" + minLocks +
-                  " maxLocks=" + maxLocks);
-
-    int expectedLocks = (totalLocks / numThreads) + 1;
-    
-    // NOTE: if you turn on fine logs, this deviation may be too small...
-    // slower machines may also fail depending on thread scheduling
-    int deviation = (int)(expectedLocks * 0.25);
-    int lowThreshold = expectedLocks - deviation;
-    int highThreshold = expectedLocks + deviation;
-
-    this.log.info("[testFairnessStressfully] deviation=" + deviation +
-                  " expectedLocks=" + expectedLocks + 
-                  " lowThreshold=" + lowThreshold +
-                  " highThreshold=" + highThreshold);
-                        
-    // if these assertions keep failing we'll have to rewrite the test
-    // to handle scheduling of the threads...
-                  
-    assertTrue("minLocks is less than lowThreshold",
-               minLocks >= lowThreshold);
-    assertTrue("maxLocks is greater than highThreshold",
-               maxLocks <= highThreshold);
-  }
-  
-  @Test
-  public void testHasCurrentTopic() throws Exception {
-    this.log.info("[testHasCurrentTopic]");
-    assertTrue(!this.collaboration.hasCurrentTopic());
-    this.collaboration.acquireUninterruptibly("testHasCurrentTopic");
-    try {
-      assertTrue(this.collaboration.hasCurrentTopic());
-    }
-    finally {
-      this.collaboration.release();
-    }
-    assertTrue(!this.collaboration.hasCurrentTopic());
-  }
-  
-  protected volatile boolean flagTestThreadHasCurrentTopic = false;
-
-  @Test
-  public void testThreadHasCurrentTopic() throws Exception {
-    this.log.info("[testThreadHasCurrentTopic]");
-    Thread thread = new Thread(group, new Runnable() {
-      @Override
-      public void run() {
-        collaboration.acquireUninterruptibly("testThreadHasCurrentTopic");
-        try {
-          flagTestThreadHasCurrentTopic = true;
-          WaitCriterion ev = new WaitCriterion() {
-            @Override
-            public boolean done() {
-              return !flagTestThreadHasCurrentTopic;
-            }
-            @Override
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    
-    // before starting thread, hasCurrentTopic(thread) returns false
-    assertTrue(!this.collaboration.hasCurrentTopic(thread));
-    thread.start();
-    WaitCriterion ev = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return flagTestThreadHasCurrentTopic;
-      }
-      @Override
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-    
-    // after starting thread, hasCurrentTopic(thread) returns true
-    assertTrue(this.collaboration.hasCurrentTopic(thread));
-    this.flagTestThreadHasCurrentTopic = false;
-    DistributedTestCase.join(thread, 30 * 1000, null);
-    
-    // after thread finishes, hasCurrentTopic(thread) returns false
-    assertTrue(!this.collaboration.hasCurrentTopic(thread));
-  }
-  
-  @Test
-  public void testIsCurrentTopic() throws Exception {
-    this.log.info("[testIsCurrentTopic]");
-    Object topic = "testIsCurrentTopic";
-    assertTrue(!this.collaboration.isCurrentTopic(topic));
-    this.collaboration.acquireUninterruptibly(topic);
-    try {
-      assertTrue(this.collaboration.isCurrentTopic(topic));
-    }
-    finally {
-      this.collaboration.release();
-    }
-    assertTrue(!this.collaboration.isCurrentTopic(topic));
-  }
-
-  protected final ThreadGroup group = 
-      new ThreadGroup("CollaborationJUnitTest Threads") {
-        @Override
-        public void uncaughtException(Thread t, Throwable e)
-        {
-          if (e instanceof VirtualMachineError) {
-            SystemFailure.setFailure((VirtualMachineError)e); // don't throw
-          }
-          String s = "Uncaught exception in thread " + t;
-          log.error(s, e);
-          fail(s);
-        }
-      };
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
index 5ec4af8..0ee9d4f 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
@@ -21,7 +21,6 @@ import java.util.Arrays;
 
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -32,15 +31,14 @@ import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.internal.cache.DiskRegionHelperFactory;
 import com.gemstone.gemfire.internal.cache.DiskRegionProperties;
 import com.gemstone.gemfire.internal.cache.DiskRegionTestingBase;
-import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Consolidated Disk Region Perftest. Overflow, Persist, OverflowWithPersist
  * modes are tested for Sync, AsyncWithBuffer and AsyncWithoutBufer writes.
  *  
  */
-@Category(PerformanceTest.class)
-@Ignore("Tests have no assertions")
+@Category(IntegrationTest.class)
 public class DiskRegionPerfJUnitPerformanceTest extends DiskRegionTestingBase
 {
   LogWriter log = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueStartStopJUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueStartStopJUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueStartStopJUnitDisabledTest.java
new file mode 100755
index 0000000..4cfc9ba
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueStartStopJUnitDisabledTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.ha;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.RegionQueue;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.internal.Assert;
+
+import junit.framework.TestCase;
+
+/**
+ * @author Mitul Bid
+ *
+ */
+public class HARegionQueueStartStopJUnitDisabledTest extends TestCase
+{
+
+  /**
+   * Creates the cache instance for the test
+   * 
+   * @return the cache instance
+   * @throws CacheException -
+   *           thrown if any exception occurs in cache creation
+   */
+  private Cache createCache() throws CacheException
+  {
+    return CacheFactory.create(DistributedSystem.connect(new Properties()));
+  }
+
+  /**
+   * Creates HA region-queue object
+   * 
+   * @return HA region-queue object
+   * @throws IOException
+   * @throws ClassNotFoundException
+   * @throws CacheException
+   * @throws InterruptedException
+   */
+  private RegionQueue createHARegionQueue(String name, Cache cache)
+      throws IOException, ClassNotFoundException, CacheException, InterruptedException
+  {
+    RegionQueue regionqueue =HARegionQueue.getHARegionQueueInstance(name, cache,HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+    return regionqueue;
+  }
+
+  public void testStartStop()
+  {
+    try {
+      boolean exceptionOccured = false;
+      Cache cache = createCache();
+      createHARegionQueue("test", cache);
+      Assert
+          .assertTrue(HARegionQueue.getDispatchedMessagesMapForTesting() != null);
+      HARegionQueue.stopHAServices();
+      try {
+        HARegionQueue.getDispatchedMessagesMapForTesting();
+      }
+      catch (NullPointerException e) {
+        exceptionOccured = true;
+      }
+      if (!exceptionOccured) {
+        fail("Expected exception to occur but did not occur");
+      }
+      HARegionQueue.startHAServices((GemFireCacheImpl)cache);
+      Assert
+          .assertTrue(HARegionQueue.getDispatchedMessagesMapForTesting() != null);
+      cache.close();
+      try {
+        HARegionQueue.getDispatchedMessagesMapForTesting();
+      }
+      catch (NullPointerException e) {
+        exceptionOccured = true;
+      }
+      if (!exceptionOccured) {
+        fail("Expected exception to occur but did not occur");
+      }
+      
+      cache = createCache();
+
+      try {
+        HARegionQueue.getDispatchedMessagesMapForTesting();
+      }
+      catch (NullPointerException e) {
+        exceptionOccured = true;
+      }
+      if (!exceptionOccured) {
+        fail("Expected exception to occur but did not occur");
+      }
+      
+    }
+    catch (Exception e) {
+      e.printStackTrace();
+      fail("Test failed due to " + e);
+    }
+
+  }
+  
+  
+}


[50/50] [abbrv] incubator-geode git commit: Merge branch 'release/1.0.0-incubating.M1'

Posted by ab...@apache.org.
Merge branch 'release/1.0.0-incubating.M1'


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

Branch: refs/heads/master
Commit: 008309dd6bd01d101071c6caa1786657e84953b3
Parents: 6902465 c263345
Author: Anthony Baker <ab...@apache.org>
Authored: Mon Feb 8 08:58:07 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Mon Feb 8 08:58:07 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |    9 +-
 COMPILING.txt                                   |    9 +
 DISCLAIMER                                      |    6 +
 KEYS                                            |  236 +
 LICENSE                                         |  279 +
 LICENSE.txt                                     |  202 -
 NOTICE                                          |  336 +-
 README.md                                       |   30 +-
 build.gradle                                    |  364 +-
 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                               |   46 +
 docker/README.md                                |   67 +
 docker/build-runtime-docker.sh                  |   32 +
 etc/eclipseFormatterProfile.xml                 |   11 +-
 gemfire-assembly/build.gradle                   |  245 +-
 gemfire-assembly/src/main/dist/DISCLAIMER       |    6 +
 gemfire-assembly/src/main/dist/LICENSE          |  346 +
 gemfire-assembly/src/main/dist/NOTICE           |  467 +
 gemfire-assembly/src/main/dist/bin/gfsh         |   16 +-
 .../src/main/dist/bin/gfsh-completion.bash      |   15 +
 gemfire-assembly/src/main/dist/bin/gfsh.bat     |   16 +-
 .../LocatorLauncherAssemblyJUnitTest.java       |  157 +
 .../management/internal/AgentUtilJUnitTest.java |  106 +
 .../LauncherLifecycleCommandsDUnitTest.java     | 1005 ++
 .../LauncherLifecycleCommandsJUnitTest.java     |  625 ++
 .../SharedConfigurationEndToEndDUnitTest.java   |  443 +
 gemfire-common/build.gradle                     |   20 +
 .../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                       |  219 +-
 .../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                     |   19 +-
 .../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    |   41 +-
 .../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 |   26 +-
 .../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    |   45 +
 .../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    |   51 +-
 .../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       |   23 +-
 .../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        |  171 +-
 .../gemfire/cache/AttributesMutator.java        |   37 +-
 .../java/com/gemstone/gemfire/cache/Cache.java  |   48 +-
 .../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    |   37 +-
 .../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  |   40 +-
 .../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 |   79 +
 .../gemstone/gemfire/cache/CustomExpiry.java    |   21 +-
 .../com/gemstone/gemfire/cache/DataPolicy.java  |   35 +-
 .../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  |   33 +-
 .../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         |   58 +
 .../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    |   22 +-
 .../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   |   37 +-
 .../cache/OperationAbortedException.java        |   21 +-
 .../gemfire/cache/PartitionAttributes.java      |   21 +-
 .../cache/PartitionAttributesFactory.java       |   26 +-
 .../gemfire/cache/PartitionResolver.java        |   20 +-
 .../PartitionedRegionDistributionException.java |   21 +-
 .../PartitionedRegionStorageException.java      |   21 +-
 .../java/com/gemstone/gemfire/cache/Region.java |   30 +-
 .../gemfire/cache/RegionAccessException.java    |   21 +-
 .../gemfire/cache/RegionAttributes.java         |   52 +-
 .../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   |   55 +-
 .../gemfire/cache/RegionMembershipListener.java |   21 +-
 .../cache/RegionReinitializedException.java     |   21 +-
 .../gemfire/cache/RegionRoleException.java      |   21 +-
 .../gemfire/cache/RegionRoleListener.java       |   21 +-
 .../gemstone/gemfire/cache/RegionService.java   |   25 +-
 .../gemstone/gemfire/cache/RegionShortcut.java  |   21 +-
 .../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     |   25 +-
 .../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    |   36 +-
 .../internal/AsyncEventQueueImpl.java           |   29 +-
 .../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        |   39 +-
 .../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       |   49 +-
 .../gemfire/cache/client/PoolManager.java       |   21 +-
 .../client/ServerConnectivityException.java     |   21 +-
 .../cache/client/ServerOperationException.java  |   21 +-
 .../ServerRefusedConnectionException.java       |   21 +-
 .../client/SubscriptionNotEnabledException.java |   21 +-
 .../client/doc-files/example-client-cache.xml   |   16 +
 .../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  |   55 +-
 .../client/internal/ClientPartitionAdvisor.java |   21 +-
 .../internal/ClientRegionFactoryImpl.java       |   23 +-
 .../cache/client/internal/ClientUpdater.java    |   21 +-
 .../client/internal/CloseConnectionOp.java      |   21 +-
 .../gemfire/cache/client/internal/CommitOp.java |   21 +-
 .../cache/client/internal/Connection.java       |   24 +-
 .../client/internal/ConnectionFactory.java      |   21 +-
 .../client/internal/ConnectionFactoryImpl.java  |   59 +-
 .../cache/client/internal/ConnectionImpl.java   |   88 +-
 .../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       |   45 +-
 .../internal/ExplicitConnectionSourceImpl.java  |  106 +-
 .../gemfire/cache/client/internal/GetAllOp.java |   21 +-
 .../client/internal/GetClientPRMetaDataOp.java  |   21 +-
 .../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 |   37 +-
 .../cache/client/internal/PrimaryAckOp.java     |   21 +-
 .../cache/client/internal/ProxyCache.java       |   25 +-
 .../client/internal/ProxyCacheCloseOp.java      |   21 +-
 .../cache/client/internal/ProxyRegion.java      |   21 +-
 .../gemfire/cache/client/internal/PutAllOp.java |   27 +-
 .../gemfire/cache/client/internal/PutOp.java    |   39 +-
 .../gemfire/cache/client/internal/QueryOp.java  |   21 +-
 .../client/internal/QueueConnectionImpl.java    |   24 +-
 .../cache/client/internal/QueueManager.java     |   21 +-
 .../cache/client/internal/QueueManagerImpl.java |   39 +-
 .../cache/client/internal/QueueState.java       |   16 +
 .../cache/client/internal/QueueStateImpl.java   |   30 +-
 .../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      |  202 +-
 .../internal/SingleHopClientExecutor.java       |   44 +-
 .../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/LocatorJoinMessage.java         |   96 -
 .../locator/wan/LocatorMembershipListener.java  |   21 +-
 .../locator/wan/RemoteLocatorJoinRequest.java   |   78 -
 .../locator/wan/RemoteLocatorJoinResponse.java  |   80 -
 .../locator/wan/RemoteLocatorPingRequest.java   |   47 -
 .../locator/wan/RemoteLocatorPingResponse.java  |   46 -
 .../locator/wan/RemoteLocatorRequest.java       |   57 -
 .../locator/wan/RemoteLocatorResponse.java      |   65 -
 .../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      |   24 +-
 .../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  |  108 +-
 .../gemstone/gemfire/cache/control/package.html |   16 +
 .../gemfire/cache/doc-files/example-cache.xml   |   16 +
 .../gemfire/cache/doc-files/example2-cache.xml  |   17 +
 .../gemfire/cache/doc-files/example3-cache.xml  |   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     |   53 +
 .../gemstone/gemfire/cache/hdfs/HDFSStore.java  |  343 +
 .../gemfire/cache/hdfs/HDFSStoreFactory.java    |  205 +
 .../gemfire/cache/hdfs/HDFSStoreMutator.java    |  197 +
 .../cache/hdfs/StoreExistsException.java        |   33 +
 .../cache/hdfs/internal/FailureTracker.java     |   97 +
 .../cache/hdfs/internal/FlushObserver.java      |   54 +
 .../hdfs/internal/HDFSBucketRegionQueue.java    | 1233 +++
 .../cache/hdfs/internal/HDFSEntriesSet.java     |  329 +
 .../cache/hdfs/internal/HDFSEventListener.java  |  180 +
 .../hdfs/internal/HDFSEventQueueFilter.java     |   74 +
 .../hdfs/internal/HDFSGatewayEventImpl.java     |  181 +
 .../hdfs/internal/HDFSIntegrationUtil.java      |  118 +
 .../HDFSParallelGatewaySenderQueue.java         |  473 +
 .../hdfs/internal/HDFSStoreConfigHolder.java    |  560 ++
 .../cache/hdfs/internal/HDFSStoreCreation.java  |  199 +
 .../hdfs/internal/HDFSStoreFactoryImpl.java     |   78 +
 .../cache/hdfs/internal/HDFSStoreImpl.java      |  639 ++
 .../hdfs/internal/HDFSStoreMutatorImpl.java     |  200 +
 .../HDFSWriteOnlyStoreEventListener.java        |  185 +
 .../hdfs/internal/HoplogListenerForRegion.java  |   73 +
 .../cache/hdfs/internal/PersistedEventImpl.java |  203 +
 .../hdfs/internal/QueuedPersistentEvent.java    |   27 +
 .../hdfs/internal/SignalledFlushObserver.java   |  123 +
 .../internal/SortedHDFSQueuePersistedEvent.java |   87 +
 .../internal/SortedHoplogPersistedEvent.java    |  115 +
 .../UnsortedHDFSQueuePersistedEvent.java        |   77 +
 .../internal/UnsortedHoplogPersistedEvent.java  |   93 +
 .../cache/hdfs/internal/cardinality/Bits.java   |   54 +
 .../cardinality/CardinalityMergeException.java  |   42 +
 .../hdfs/internal/cardinality/HyperLogLog.java  |  313 +
 .../hdfs/internal/cardinality/IBuilder.java     |   41 +
 .../hdfs/internal/cardinality/ICardinality.java |   87 +
 .../hdfs/internal/cardinality/MurmurHash.java   |  261 +
 .../hdfs/internal/cardinality/RegisterSet.java  |  136 +
 .../hdfs/internal/hoplog/AbstractHoplog.java    |  357 +
 .../hoplog/AbstractHoplogOrganizer.java         |  430 +
 .../cache/hdfs/internal/hoplog/BloomFilter.java |   36 +
 .../hoplog/CloseTmpHoplogsTimerTask.java        |  109 +
 .../hdfs/internal/hoplog/CompactionStatus.java  |   73 +
 .../cache/hdfs/internal/hoplog/FlushStatus.java |   73 +
 .../internal/hoplog/HDFSCompactionManager.java  |  330 +
 .../internal/hoplog/HDFSFlushQueueArgs.java     |   94 +
 .../internal/hoplog/HDFSFlushQueueFunction.java |  287 +
 .../hoplog/HDFSForceCompactionArgs.java         |  108 +
 .../hoplog/HDFSForceCompactionFunction.java     |  130 +
 .../HDFSForceCompactionResultCollector.java     |  132 +
 .../hoplog/HDFSLastCompactionTimeFunction.java  |   57 +
 .../internal/hoplog/HDFSRegionDirector.java     |  480 +
 .../hdfs/internal/hoplog/HDFSStoreDirector.java |   79 +
 .../hoplog/HDFSUnsortedHoplogOrganizer.java     |  448 +
 .../hdfs/internal/hoplog/HFileSortedOplog.java  |  853 ++
 .../hoplog/HdfsSortedOplogOrganizer.java        | 2007 ++++
 .../cache/hdfs/internal/hoplog/Hoplog.java      |  263 +
 .../hdfs/internal/hoplog/HoplogConfig.java      |   75 +
 .../hdfs/internal/hoplog/HoplogListener.java    |   47 +
 .../hdfs/internal/hoplog/HoplogOrganizer.java   |  123 +
 .../hdfs/internal/hoplog/HoplogSetIterator.java |  166 +
 .../hdfs/internal/hoplog/HoplogSetReader.java   |  114 +
 .../internal/hoplog/SequenceFileHoplog.java     |  396 +
 .../hoplog/mapred/AbstractGFRecordReader.java   |  106 +
 .../internal/hoplog/mapred/GFInputFormat.java   |   95 +
 .../internal/hoplog/mapred/GFOutputFormat.java  |   76 +
 .../mapreduce/AbstractGFRecordReader.java       |  140 +
 .../hoplog/mapreduce/GFInputFormat.java         |  124 +
 .../hdfs/internal/hoplog/mapreduce/GFKey.java   |   72 +
 .../hoplog/mapreduce/GFOutputFormat.java        |  199 +
 .../hoplog/mapreduce/HDFSSplitIterator.java     |  198 +
 .../internal/hoplog/mapreduce/HoplogUtil.java   |  464 +
 .../hoplog/mapreduce/RWSplitIterator.java       |   49 +
 .../hoplog/mapreduce/StreamSplitIterator.java   |   47 +
 .../org/apache/hadoop/io/SequenceFile.java      | 3726 +++++++
 .../operations/CloseCQOperationContext.java     |   21 +-
 .../operations/DestroyOperationContext.java     |   21 +-
 .../operations/ExecuteCQOperationContext.java   |   21 +-
 .../ExecuteFunctionOperationContext.java        |   21 +-
 .../GetDurableCQsOperationContext.java          |   21 +-
 .../cache/operations/GetOperationContext.java   |   28 +-
 .../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    |   91 +-
 .../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       |  124 +
 .../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       |  369 -
 .../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         |   46 +
 .../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   |   33 +-
 .../internal/AbstractGroupOrRangeJunction.java  |   47 +-
 .../cache/query/internal/AllGroupJunction.java  |   29 +-
 .../query/internal/AttributeDescriptor.java     |   22 +-
 .../gemfire/cache/query/internal/Bag.java       |  726 ++
 .../internal/CompiledAggregateFunction.java     |  185 +
 .../query/internal/CompiledBindArgument.java    |   37 +-
 .../query/internal/CompiledComparison.java      |   83 +-
 .../query/internal/CompiledConstruction.java    |   22 +-
 .../cache/query/internal/CompiledFunction.java  |   22 +-
 .../query/internal/CompiledGroupBySelect.java   |  522 +
 .../cache/query/internal/CompiledID.java        |   22 +-
 .../cache/query/internal/CompiledIn.java        |   74 +-
 .../query/internal/CompiledIndexOperation.java  |   22 +-
 .../query/internal/CompiledIteratorDef.java     |   21 +-
 .../cache/query/internal/CompiledJunction.java  |   33 +-
 .../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    |  845 +-
 .../query/internal/CompiledSortCriterion.java   |  353 +-
 .../query/internal/CompiledUnaryMinus.java      |   21 +-
 .../cache/query/internal/CompiledUndefined.java |   27 +-
 .../cache/query/internal/CompiledValue.java     |   26 +-
 .../query/internal/CompositeGroupJunction.java  |   43 +-
 .../gemfire/cache/query/internal/CqEntry.java   |   21 +-
 .../cache/query/internal/CqQueryVsdStats.java   |   21 +-
 .../cache/query/internal/CqStateImpl.java       |   22 +-
 .../internal/CumulativeNonDistinctResults.java  |  391 +
 .../cache/query/internal/DefaultQuery.java      |   77 +-
 .../query/internal/DefaultQueryService.java     |   32 +-
 .../cache/query/internal/ExecutionContext.java  |   62 +-
 .../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   |   35 +-
 .../cache/query/internal/LinkedStructSet.java   |   35 +-
 .../cache/query/internal/MapIndexable.java      |   16 +
 .../cache/query/internal/MethodDispatch.java    |   22 +-
 .../cache/query/internal/NWayMergeResults.java  |  546 ++
 .../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 |  237 +
 .../internal/OrderByComparatorUnmapped.java     |   90 +
 .../gemfire/cache/query/internal/Ordered.java   |   44 +
 .../cache/query/internal/OrganizedOperands.java |   21 +-
 .../cache/query/internal/PRQueryTraceInfo.java  |   22 +-
 .../gemfire/cache/query/internal/PathUtils.java |   85 +-
 .../gemfire/cache/query/internal/PlanInfo.java  |   22 +-
 .../cache/query/internal/ProxyQueryService.java |   21 +-
 .../gemfire/cache/query/internal/QCompiler.java |  160 +-
 .../gemfire/cache/query/internal/QRegion.java   |   21 +-
 .../gemfire/cache/query/internal/QScope.java    |   24 +-
 .../QueryExecutionCanceledException.java        |   21 +-
 .../query/internal/QueryExecutionContext.java   |   40 +-
 .../cache/query/internal/QueryExecutor.java     |   21 +-
 .../cache/query/internal/QueryMonitor.java      |   29 +-
 .../cache/query/internal/QueryObserver.java     |   29 +-
 .../query/internal/QueryObserverAdapter.java    |   27 +-
 .../query/internal/QueryObserverHolder.java     |   22 +-
 .../cache/query/internal/QueryUtils.java        |  173 +-
 .../cache/query/internal/RangeJunction.java     |   85 +-
 .../cache/query/internal/ResultsBag.java        |  692 +-
 .../ResultsCollectionCopyOnReadWrapper.java     |   21 +-
 ...ResultsCollectionPdxDeserializerWrapper.java |   21 +-
 .../internal/ResultsCollectionWrapper.java      |   34 +-
 .../cache/query/internal/ResultsSet.java        |   26 +-
 .../cache/query/internal/RuntimeIterator.java   |   25 +-
 .../query/internal/SelectResultsComparator.java |   21 +-
 .../cache/query/internal/SortedResultSet.java   |   30 +-
 .../cache/query/internal/SortedResultsBag.java  |  260 +
 .../cache/query/internal/SortedStructBag.java   |  369 +
 .../cache/query/internal/SortedStructSet.java   |  648 +-
 .../gemfire/cache/query/internal/StructBag.java |   52 +-
 .../cache/query/internal/StructFields.java      |   42 +
 .../cache/query/internal/StructImpl.java        |   25 +-
 .../gemfire/cache/query/internal/StructSet.java |   48 +-
 .../gemfire/cache/query/internal/Support.java   |   22 +-
 .../gemfire/cache/query/internal/Undefined.java |   26 +-
 .../internal/aggregate/AbstractAggregator.java  |   47 +
 .../cache/query/internal/aggregate/Avg.java     |   50 +
 .../query/internal/aggregate/AvgBucketNode.java |   49 +
 .../query/internal/aggregate/AvgDistinct.java   |   43 +
 .../aggregate/AvgDistinctPRQueryNode.java       |   35 +
 .../internal/aggregate/AvgPRQueryNode.java      |   47 +
 .../cache/query/internal/aggregate/Count.java   |   49 +
 .../query/internal/aggregate/CountDistinct.java |   33 +
 .../aggregate/CountDistinctPRQueryNode.java     |   44 +
 .../internal/aggregate/CountPRQueryNode.java    |   48 +
 .../internal/aggregate/DistinctAggregator.java  |   56 +
 .../cache/query/internal/aggregate/MaxMin.java  |   68 +
 .../cache/query/internal/aggregate/Sum.java     |   48 +
 .../query/internal/aggregate/SumDistinct.java   |   35 +
 .../aggregate/SumDistinctPRQueryNode.java       |   46 +
 .../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     |  189 +-
 .../query/internal/index/AbstractMapIndex.java  |   27 +-
 .../internal/index/CompactMapRangeIndex.java    |   28 +-
 .../query/internal/index/CompactRangeIndex.java |   38 +-
 .../query/internal/index/DummyQRegion.java      |   81 +-
 .../index/FunctionalIndexCreationHelper.java    |   22 +-
 .../cache/query/internal/index/HashIndex.java   |  206 +-
 .../query/internal/index/HashIndexSet.java      | 1090 +--
 .../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      |   60 +-
 .../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  |   54 +-
 .../query/internal/index/PartitionedIndex.java  |   37 +-
 .../query/internal/index/PrimaryKeyIndex.java   |   21 +-
 .../index/PrimaryKeyIndexCreationHelper.java    |   21 +-
 .../cache/query/internal/index/RangeIndex.java  |   31 +-
 .../cache/query/internal/index/package.html     |   16 +
 .../gemfire/cache/query/internal/package.html   |   16 +
 .../query/internal/parse/ASTAggregateFunc.java  |   66 +
 .../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/ASTCount.java    |   45 -
 .../cache/query/internal/parse/ASTDummy.java    |   39 +
 .../cache/query/internal/parse/ASTGroupBy.java  |   44 +
 .../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   |  133 +-
 .../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/OQLLexer.java    |  276 +-
 .../internal/parse/OQLLexerTokenTypes.java      |  174 +-
 .../query/internal/parse/OQLLexerTokenTypes.txt |  174 +-
 .../cache/query/internal/parse/OQLParser.java   | 1105 ++-
 .../cache/query/internal/parse/UtilParser.java  |   21 +-
 .../cache/query/internal/parse/fixantlr.sh      |   15 +
 .../gemfire/cache/query/internal/parse/oql.g    |  109 +-
 .../internal/types/CollectionTypeImpl.java      |   41 +-
 .../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   |   38 +-
 .../query/internal/utils/LimitIterator.java     |   63 +
 .../cache/query/internal/utils/PDXUtils.java    |   95 +
 .../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    |   53 +
 .../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    |   28 +-
 .../gemfire/cache/wan/GatewayReceiver.java      |   20 +-
 .../cache/wan/GatewayReceiverFactory.java       |   20 +-
 .../gemfire/cache/wan/GatewaySender.java        |   22 +-
 .../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   |   24 +-
 .../distributed/ClientSocketFactory.java        |   21 +-
 .../distributed/DistributedLockService.java     |   21 +-
 .../gemfire/distributed/DistributedMember.java  |   21 +-
 .../gemfire/distributed/DistributedSystem.java  |  359 +-
 .../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    |   54 +-
 .../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     |  204 +-
 .../TXManagerCancelledException.java            |   23 +-
 .../internal/AbstractDistributionConfig.java    |  139 +-
 .../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   |  225 +-
 .../gemfire/distributed/internal/DSClock.java   |   18 +-
 .../internal/DirectReplyProcessor.java          |   21 +-
 .../distributed/internal/DistributedState.java  |   21 +-
 .../internal/DistributionAdvisee.java           |   21 +-
 .../internal/DistributionAdvisor.java           |   23 +-
 .../internal/DistributionChannel.java           |   25 +-
 .../internal/DistributionConfig.java            |  133 +-
 .../internal/DistributionConfigImpl.java        |  134 +-
 .../internal/DistributionConfigSnapshot.java    |   21 +-
 .../internal/DistributionException.java         |   21 +-
 .../internal/DistributionManager.java           |  590 +-
 .../internal/DistributionManagerConfig.java     |   93 -
 .../internal/DistributionMessage.java           |   71 +-
 .../internal/DistributionMessageObserver.java   |   23 +-
 .../distributed/internal/DistributionStats.java |  594 +-
 .../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     |  267 +-
 .../distributed/internal/InternalLocator.java   |  200 +-
 .../internal/LocatorLoadSnapshot.java           |   20 +-
 .../distributed/internal/LocatorStats.java      |   21 +-
 .../internal/LonerDistributionManager.java      |  325 +-
 .../gemfire/distributed/internal/MQueue.java    |   16 +
 .../internal/MembershipListener.java            |   24 +-
 .../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     |   23 +-
 .../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  |   30 +-
 .../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           |   22 +-
 .../distributed/internal/ShutdownMessage.java   |   23 +-
 .../gemfire/distributed/internal/Sizeable.java  |   16 +
 .../distributed/internal/SizeableRunnable.java  |   21 +-
 .../distributed/internal/StartupMessage.java    |   98 +-
 .../internal/StartupMessageData.java            |   45 +-
 .../internal/StartupMessageReplyProcessor.java  |   21 +-
 .../distributed/internal/StartupOperation.java  |   27 +-
 .../internal/StartupResponseMessage.java        |   76 +-
 .../StartupResponseWithVersionMessage.java      |   56 +-
 .../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          |  142 +-
 .../internal/direct/DirectChannelListener.java  |   38 +
 .../internal/direct/MissingStubException.java   |   28 -
 .../internal/direct/ShunnedMemberException.java |   34 +
 .../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            |   28 +-
 .../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          |   37 +-
 .../membership/InternalDistributedMember.java   |  386 +-
 .../internal/membership/InternalRole.java       |   21 +-
 .../internal/membership/MemberAttributes.java   |  172 +-
 .../internal/membership/MemberFactory.java      |   54 +-
 .../internal/membership/MemberServices.java     |   49 +-
 .../internal/membership/MembershipManager.java  |   94 +-
 .../internal/membership/MembershipTestHook.java |   21 +-
 .../internal/membership/NetMember.java          |   84 +-
 .../internal/membership/NetView.java            |  615 +-
 .../internal/membership/QuorumChecker.java      |   33 +-
 .../internal/membership/gms/GMSMember.java      |  458 +
 .../membership/gms/GMSMemberFactory.java        |  133 +
 .../internal/membership/gms/GMSUtil.java        |  159 +
 .../internal/membership/gms/NetLocator.java     |   32 +
 .../internal/membership/gms/ServiceConfig.java  |  187 +
 .../internal/membership/gms/Services.java       |  387 +
 .../internal/membership/gms/SuspectMember.java  |   58 +
 .../membership/gms/auth/GMSAuthenticator.java   |  235 +
 .../membership/gms/fd/GMSHealthMonitor.java     | 1333 +++
 .../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    |   81 +
 .../membership/gms/interfaces/Service.java      |   83 +
 .../gms/locator/FindCoordinatorRequest.java     |  148 +
 .../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 | 2246 +++++
 .../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           |  121 +
 .../membership/gms/messenger/GMSPingPonger.java |   65 +
 .../gms/messenger/GMSQuorumChecker.java         |  274 +
 .../membership/gms/messenger/JGAddress.java     |  221 +
 .../gms/messenger/JGroupsMessenger.java         | 1085 +++
 .../membership/gms/messenger/StatRecorder.java  |  162 +
 .../membership/gms/messenger/Transport.java     |  168 +
 .../gms/mgr/GMSMembershipManager.java           | 2647 +++++
 .../membership/gms/mgr/LocalViewMessage.java    |   85 +
 .../internal/membership/gms/package.html        |   57 +
 .../membership/jgroup/GFJGBasicAdapter.java     |  546 --
 .../membership/jgroup/GFJGPeerAdapter.java      |  484 -
 .../membership/jgroup/JGroupMember.java         |  251 -
 .../membership/jgroup/JGroupMemberFactory.java  |  119 -
 .../jgroup/JGroupMembershipManager.java         | 4212 --------
 .../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  |   46 +-
 .../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           |  123 +-
 .../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        |   31 +-
 .../internal/AbstractStatisticsFactory.java     |   21 +-
 .../gemfire/internal/ArchiveSplitter.java       |   21 +-
 .../com/gemstone/gemfire/internal/Assert.java   |   21 +-
 .../gemfire/internal/AvailablePort.java         |   54 +-
 .../com/gemstone/gemfire/internal/Banner.java   |   52 +-
 .../gemfire/internal/ByteArrayDataInput.java    |   21 +-
 .../internal/ByteBufferOutputStream.java        |   21 +-
 .../gemfire/internal/ByteBufferWriter.java      |   35 +
 .../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 |  117 +-
 .../internal/DSFIDNotFoundException.java        |   21 +-
 .../internal/DataSerializableFixedID.java       |   96 +-
 .../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  |  416 +-
 .../gemfire/internal/HistogramStats.java        |   21 +-
 .../gemfire/internal/HostStatHelper.java        |   21 +-
 .../gemfire/internal/HostStatSampler.java       |   25 +-
 .../InsufficientDiskSpaceException.java         |   23 +-
 .../internal/InternalDataSerializer.java        |  318 +-
 .../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 |   66 +-
 .../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    |   26 +-
 .../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 |   42 +
 .../gemfire/internal/SerializationVersions.java |   21 +-
 .../com/gemstone/gemfire/internal/SetUtils.java |   20 +-
 .../gemfire/internal/SharedLibrary.java         |   43 +-
 .../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  |   38 +-
 .../gemfire/internal/SystemFailureTestHook.java |   21 +-
 .../gemstone/gemfire/internal/SystemTimer.java  |   25 +-
 .../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  |   63 +-
 .../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      |   32 +-
 .../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      |   60 +-
 .../internal/admin/remote/RemoteObjectName.java |   21 +-
 .../admin/remote/RemoteRegionAttributes.java    |   68 +-
 .../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   |   68 +
 .../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        |  122 +-
 .../internal/cache/AbstractCacheServer.java     |  407 +
 .../cache/AbstractDiskLRURegionEntry.java       |   21 +-
 .../internal/cache/AbstractDiskRegion.java      |   54 +-
 .../internal/cache/AbstractDiskRegionEntry.java |   44 +-
 .../internal/cache/AbstractLRURegionEntry.java  |   21 +-
 .../internal/cache/AbstractLRURegionMap.java    |   53 +-
 .../cache/AbstractOplogDiskRegionEntry.java     |   29 +-
 .../gemfire/internal/cache/AbstractRegion.java  |  275 +-
 .../internal/cache/AbstractRegionEntry.java     |  632 +-
 .../internal/cache/AbstractRegionMap.java       |  758 +-
 .../internal/cache/AbstractUpdateOperation.java |   30 +-
 .../gemfire/internal/cache/AcceptHelper.java    |   21 +-
 .../cache/AddCacheServerProfileMessage.java     |   21 +-
 .../gemfire/internal/cache/BackupLock.java      |   21 +-
 .../internal/cache/BridgeEventConflator.java    |  158 -
 .../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   |   79 +-
 .../gemfire/internal/cache/BucketDump.java      |   21 +-
 .../internal/cache/BucketNotFoundException.java |   21 +-
 .../cache/BucketPersistenceAdvisor.java         |   21 +-
 .../gemfire/internal/cache/BucketRegion.java    |  356 +-
 .../internal/cache/BucketRegionEvictior.java    |   21 +-
 .../internal/cache/BucketRegionQueue.java       |  150 +-
 .../internal/cache/BucketServerLocation.java    |   20 +-
 .../internal/cache/BucketServerLocation66.java  |   20 +-
 .../cache/BytesAndBitsForCompactor.java         |   38 +-
 .../internal/cache/CacheClientStatus.java       |   21 +-
 .../gemfire/internal/cache/CacheConfig.java     |   43 +-
 .../cache/CacheDistributionAdvisee.java         |   21 +-
 .../cache/CacheDistributionAdvisor.java         |   49 +-
 .../internal/cache/CacheLifecycleListener.java  |   21 +-
 .../gemfire/internal/cache/CacheObserver.java   |   26 +-
 .../internal/cache/CacheObserverAdapter.java    |   25 +-
 .../internal/cache/CacheObserverHolder.java     |   21 +-
 .../gemfire/internal/cache/CachePerfStats.java  |  112 +-
 .../internal/cache/CacheServerAdvisor.java      |  173 +
 .../gemfire/internal/cache/CacheServerImpl.java |  821 ++
 .../internal/cache/CacheServerLauncher.java     |  183 +-
 .../gemfire/internal/cache/CacheService.java    |   42 +
 .../internal/cache/CacheStatisticsImpl.java     |   21 +-
 .../internal/cache/CachedDeserializable.java    |   21 +-
 .../cache/CachedDeserializableFactory.java      |   28 +-
 .../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        |   94 +-
 .../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   |   41 +-
 .../internal/cache/CustomEntryExpiryTask.java   |   16 +
 .../cache/CustomEvictionAttributesImpl.java     |   36 +
 .../internal/cache/DataLocationException.java   |   21 +-
 .../internal/cache/DestroyOperation.java        |   37 +-
 .../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       |  770 +-
 .../gemstone/gemfire/internal/cache/DiskId.java |   21 +-
 .../gemfire/internal/cache/DiskInitFile.java    |   99 +-
 .../gemfire/internal/cache/DiskRegion.java      |   51 +-
 .../gemfire/internal/cache/DiskRegionStats.java |   43 +-
 .../internal/cache/DiskStoreAttributes.java     |   21 +-
 .../gemfire/internal/cache/DiskStoreBackup.java |   21 +-
 .../internal/cache/DiskStoreFactoryImpl.java    |   21 +-
 .../gemfire/internal/cache/DiskStoreImpl.java   |   84 +-
 .../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     |  374 +
 .../cache/DistTXAdjunctCommitMessage.java       |   51 +
 .../internal/cache/DistTXCommitMessage.java     |  500 +
 .../cache/DistTXCoordinatorInterface.java       |   73 +
 .../internal/cache/DistTXPrecommitMessage.java  |  548 ++
 .../internal/cache/DistTXRollbackMessage.java   |  507 +
 .../gemfire/internal/cache/DistTXState.java     |  750 ++
 .../cache/DistTXStateOnCoordinator.java         |  416 +
 .../internal/cache/DistTXStateProxyImpl.java    |   48 +
 .../DistTXStateProxyImplOnCoordinator.java      | 1068 ++
 .../cache/DistTXStateProxyImplOnDatanode.java   |  128 +
 .../cache/DistributedCacheOperation.java        |  202 +-
 .../cache/DistributedClearOperation.java        |   27 +-
 .../cache/DistributedPutAllOperation.java       |  115 +-
 .../internal/cache/DistributedRegion.java       |  147 +-
 ...stributedRegionFunctionStreamingMessage.java |   26 +-
 .../cache/DistributedRemoveAllOperation.java    |   72 +-
 .../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  | 1170 ++-
 .../gemfire/internal/cache/EntryExpiryTask.java |   48 +-
 .../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    |   51 +-
 .../internal/cache/EvictionAttributesImpl.java  |   29 +-
 .../gemfire/internal/cache/EvictorService.java  |  285 +
 .../internal/cache/ExpirationScheduler.java     |   57 +-
 .../gemfire/internal/cache/ExpiryTask.java      |  107 +-
 .../internal/cache/ExportDiskRegion.java        |   16 +
 .../gemfire/internal/cache/FilterProfile.java   |   32 +-
 .../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        |  582 +-
 .../internal/cache/GemfireCacheHelper.java      |   23 +-
 .../gemfire/internal/cache/GridAdvisor.java     |   27 +-
 .../gemfire/internal/cache/HARegion.java        |   37 +-
 .../internal/cache/HDFSLRURegionMap.java        |  112 +
 .../gemfire/internal/cache/HDFSRegionMap.java   |   33 +
 .../internal/cache/HDFSRegionMapDelegate.java   |  541 ++
 .../internal/cache/HDFSRegionMapImpl.java       |   75 +
 .../internal/cache/HasCachePerfStats.java       |   16 +
 .../gemfire/internal/cache/ImageState.java      |   21 +-
 .../cache/InMemoryPersistentMemberView.java     |   21 +-
 .../internal/cache/IncomingGatewayStatus.java   |   21 +-
 .../internal/cache/InitialImageFlowControl.java |   25 +-
 .../internal/cache/InitialImageOperation.java   |   31 +-
 .../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   |   26 +-
 .../internal/cache/InternalCacheEvent.java      |   21 +-
 .../internal/cache/InternalDataView.java        |   31 +-
 .../internal/cache/InternalRegionArguments.java |   39 +-
 .../internal/cache/InvalidateOperation.java     |   23 +-
 .../InvalidatePartitionedRegionMessage.java     |   21 +-
 .../cache/InvalidateRegionOperation.java        |   21 +-
 .../cache/JtaAfterCompletionMessage.java        |   21 +-
 .../cache/JtaBeforeCompletionMessage.java       |   21 +-
 .../gemfire/internal/cache/KeyInfo.java         |   63 +-
 .../internal/cache/KeyWithRegionContext.java    |   21 +-
 .../gemfire/internal/cache/ListOfDeltas.java    |   31 +-
 .../internal/cache/LoaderHelperFactory.java     |   21 +-
 .../internal/cache/LoaderHelperImpl.java        |   23 +-
 .../gemfire/internal/cache/LocalDataSet.java    |   21 +-
 .../gemfire/internal/cache/LocalRegion.java     | 1149 ++-
 .../internal/cache/LocalRegionDataView.java     |   31 +-
 .../cache/MemberFunctionStreamingMessage.java   |   35 +-
 .../cache/MinimumSystemRequirements.java        |   25 +-
 .../cache/NetSearchExpirationCalculator.java    |   21 +-
 .../gemstone/gemfire/internal/cache/Node.java   |   20 +-
 .../internal/cache/NonLocalRegionEntry.java     |   61 +-
 .../cache/NonLocalRegionEntryWithStats.java     |   23 +-
 .../internal/cache/OffHeapRegionEntry.java      |   42 +
 .../cache/OfflineCompactionDiskRegion.java      |   25 +-
 .../gemstone/gemfire/internal/cache/OpType.java |   21 +-
 .../gemstone/gemfire/internal/cache/Oplog.java  |  293 +-
 .../gemfire/internal/cache/OplogSet.java        |   26 +-
 .../internal/cache/OrderedTombstoneMap.java     |   21 +-
 .../gemfire/internal/cache/OverflowOplog.java   |  148 +-
 .../internal/cache/OverflowOplogSet.java        |   38 +-
 .../internal/cache/PRContainsValueFunction.java |   21 +-
 .../internal/cache/PRHARedundancyProvider.java  |   26 +-
 .../internal/cache/PRQueryProcessor.java        |  105 +-
 .../internal/cache/PRSystemPropertyGetter.java  |   21 +-
 .../internal/cache/PartitionAttributesImpl.java |  339 +-
 .../internal/cache/PartitionRegionConfig.java   |   21 +-
 .../cache/PartitionRegionConfigValidator.java   |   21 +-
 .../internal/cache/PartitionedRegion.java       |  930 +-
 .../PartitionedRegionBucketMgmtHelper.java      |   20 +-
 .../cache/PartitionedRegionDataStore.java       |  128 +-
 .../cache/PartitionedRegionDataView.java        |   31 +-
 .../cache/PartitionedRegionException.java       |   21 +-
 .../internal/cache/PartitionedRegionHelper.java |   22 +-
 .../cache/PartitionedRegionQueryEvaluator.java  |  747 +-
 .../internal/cache/PartitionedRegionStats.java  |   33 +-
 .../internal/cache/PartitionedRegionStatus.java |   21 +-
 .../gemfire/internal/cache/PeerTXStateStub.java |   41 +-
 .../internal/cache/PersistentOplogSet.java      |   34 +-
 .../internal/cache/PlaceHolderDiskRegion.java   |   31 +-
 .../gemfire/internal/cache/PoolFactoryImpl.java |  195 +-
 .../gemfire/internal/cache/PoolManagerImpl.java |   48 +-
 .../gemfire/internal/cache/PoolStats.java       |   21 +-
 .../cache/PreferBytesCachedDeserializable.java  |   21 +-
 .../internal/cache/PrimaryBucketException.java  |   21 +-
 .../cache/ProfileExchangeProcessor.java         |   21 +-
 .../internal/cache/ProxyBucketRegion.java       |   24 +-
 .../gemfire/internal/cache/ProxyRegionMap.java  |   98 +-
 .../cache/PutAllPartialResultException.java     |   21 +-
 .../gemfire/internal/cache/QueuedOperation.java |   27 +-
 .../internal/cache/RegionClearedException.java  |   21 +-
 .../gemfire/internal/cache/RegionEntry.java     |   89 +-
 .../internal/cache/RegionEntryContext.java      |   25 +-
 .../internal/cache/RegionEntryFactory.java      |   28 +-
 .../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    |   26 +-
 .../gemfire/internal/cache/RegionListener.java  |   46 +
 .../gemfire/internal/cache/RegionMap.java       |   40 +-
 .../internal/cache/RegionMapFactory.java        |   26 +-
 .../gemfire/internal/cache/RegionQueue.java     |   23 +-
 .../internal/cache/RegionQueueException.java    |   21 +-
 .../gemfire/internal/cache/RegionStatus.java    |   21 +-
 .../internal/cache/RegionTTLExpiryTask.java     |   23 +-
 .../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    |  127 +-
 .../internal/cache/RemoteFetchEntryMessage.java |   22 +-
 .../cache/RemoteFetchVersionMessage.java        |   22 +-
 .../internal/cache/RemoteGetMessage.java        |   50 +-
 .../internal/cache/RemoteInvalidateMessage.java |   29 +-
 .../cache/RemoteOperationException.java         |   21 +-
 .../internal/cache/RemoteOperationMessage.java  |   74 +-
 .../RemoteOperationMessageWithDirectReply.java  |   21 +-
 .../internal/cache/RemotePutAllMessage.java     |   45 +-
 .../internal/cache/RemotePutMessage.java        |  323 +-
 .../internal/cache/RemoteRegionOperation.java   |   20 +-
 .../internal/cache/RemoteRemoveAllMessage.java  |   36 +-
 .../gemfire/internal/cache/RoleEventImpl.java   |   21 +-
 .../cache/SearchLoadAndWriteProcessor.java      |   46 +-
 .../internal/cache/SendQueueOperation.java      |   21 +-
 .../internal/cache/SerializationHelper.java     |   16 +
 .../internal/cache/ServerPingMessage.java       |  136 +
 .../internal/cache/StateFlushOperation.java     |   30 +-
 .../cache/StoreAllCachedDeserializable.java     |   21 +-
 .../internal/cache/TXBucketRegionState.java     |   21 +-
 .../gemfire/internal/cache/TXCommitMessage.java |  134 +-
 .../gemfire/internal/cache/TXEntry.java         |   25 +-
 .../gemfire/internal/cache/TXEntryState.java    |  249 +-
 .../internal/cache/TXEntryStateFactory.java     |   23 +-
 .../internal/cache/TXEntryUserAttrState.java    |   21 +-
 .../gemfire/internal/cache/TXEvent.java         |   37 +-
 .../internal/cache/TXFarSideCMTracker.java      |   23 +-
 .../gemstone/gemfire/internal/cache/TXId.java   |   22 +-
 .../gemfire/internal/cache/TXLockRequest.java   |   21 +-
 .../gemfire/internal/cache/TXManagerImpl.java   |  208 +-
 .../gemfire/internal/cache/TXMessage.java       |   27 +-
 .../internal/cache/TXRegionLockRequestImpl.java |   21 +-
 .../gemfire/internal/cache/TXRegionState.java   |  209 +-
 .../internal/cache/TXRemoteCommitMessage.java   |   21 +-
 .../internal/cache/TXRemoteRollbackMessage.java |   21 +-
 .../internal/cache/TXReservationMgr.java        |   21 +-
 .../gemfire/internal/cache/TXRmtEvent.java      |   33 +-
 .../gemfire/internal/cache/TXState.java         |  175 +-
 .../internal/cache/TXStateInterface.java        |   59 +-
 .../gemfire/internal/cache/TXStateProxy.java    |   23 +-
 .../internal/cache/TXStateProxyImpl.java        |   93 +-
 .../gemfire/internal/cache/TXStateStub.java     |   76 +-
 .../cache/TXSynchronizationRunnable.java        |   21 +-
 .../cache/TestHeapThresholdObserver.java        |   21 +-
 .../cache/TimestampedEntryEventImpl.java        |   21 +-
 .../gemstone/gemfire/internal/cache/Token.java  |   27 +-
 .../internal/cache/TombstoneService.java        |   36 +-
 .../internal/cache/TransactionMessage.java      |   27 +-
 .../gemfire/internal/cache/TxEntryFactory.java  |   16 +
 .../internal/cache/UnsharedImageState.java      |   21 +-
 .../cache/UpdateAttributesProcessor.java        |   23 +-
 .../cache/UpdateEntryVersionOperation.java      |   24 +-
 .../gemfire/internal/cache/UpdateOperation.java |  160 +-
 .../cache/UserSpecifiedDiskStoreAttributes.java |   21 +-
 .../cache/UserSpecifiedRegionAttributes.java    |   75 +-
 .../internal/cache/VMCachedDeserializable.java  |   21 +-
 .../gemfire/internal/cache/VMLRURegionMap.java  |   21 +-
 .../gemfire/internal/cache/VMRegionMap.java     |   21 +-
 .../cache/VMStatsDiskLRURegionEntry.java        |   21 +-
 .../cache/VMStatsDiskLRURegionEntryHeap.java    |   25 +-
 .../VMStatsDiskLRURegionEntryHeapIntKey.java    |   33 +-
 .../VMStatsDiskLRURegionEntryHeapLongKey.java   |   33 +-
 .../VMStatsDiskLRURegionEntryHeapObjectKey.java |   33 +-
 ...VMStatsDiskLRURegionEntryHeapStringKey1.java |   33 +-
 ...VMStatsDiskLRURegionEntryHeapStringKey2.java |   33 +-
 .../VMStatsDiskLRURegionEntryHeapUUIDKey.java   |   33 +-
 .../cache/VMStatsDiskLRURegionEntryOffHeap.java |   69 +
 .../VMStatsDiskLRURegionEntryOffHeapIntKey.java |  394 +
 ...VMStatsDiskLRURegionEntryOffHeapLongKey.java |  394 +
 ...StatsDiskLRURegionEntryOffHeapObjectKey.java |  387 +
 ...tatsDiskLRURegionEntryOffHeapStringKey1.java |  456 +
 ...tatsDiskLRURegionEntryOffHeapStringKey2.java |  497 +
 ...VMStatsDiskLRURegionEntryOffHeapUUIDKey.java |  398 +
 .../internal/cache/VMStatsDiskRegionEntry.java  |   21 +-
 .../cache/VMStatsDiskRegionEntryHeap.java       |   25 +-
 .../cache/VMStatsDiskRegionEntryHeapIntKey.java |   31 +-
 .../VMStatsDiskRegionEntryHeapLongKey.java      |   31 +-
 .../VMStatsDiskRegionEntryHeapObjectKey.java    |   31 +-
 .../VMStatsDiskRegionEntryHeapStringKey1.java   |   31 +-
 .../VMStatsDiskRegionEntryHeapStringKey2.java   |   31 +-
 .../VMStatsDiskRegionEntryHeapUUIDKey.java      |   31 +-
 .../cache/VMStatsDiskRegionEntryOffHeap.java    |   69 +
 .../VMStatsDiskRegionEntryOffHeapIntKey.java    |  297 +
 .../VMStatsDiskRegionEntryOffHeapLongKey.java   |  297 +
 .../VMStatsDiskRegionEntryOffHeapObjectKey.java |  290 +
 ...VMStatsDiskRegionEntryOffHeapStringKey1.java |  359 +
 ...VMStatsDiskRegionEntryOffHeapStringKey2.java |  400 +
 .../VMStatsDiskRegionEntryOffHeapUUIDKey.java   |  301 +
 .../internal/cache/VMStatsLRURegionEntry.java   |   21 +-
 .../cache/VMStatsLRURegionEntryHeap.java        |   25 +-
 .../cache/VMStatsLRURegionEntryHeapIntKey.java  |   33 +-
 .../cache/VMStatsLRURegionEntryHeapLongKey.java |   33 +-
 .../VMStatsLRURegionEntryHeapObjectKey.java     |   33 +-
 .../VMStatsLRURegionEntryHeapStringKey1.java    |   33 +-
 .../VMStatsLRURegionEntryHeapStringKey2.java    |   33 +-
 .../cache/VMStatsLRURegionEntryHeapUUIDKey.java |   33 +-
 .../cache/VMStatsLRURegionEntryOffHeap.java     |   69 +
 .../VMStatsLRURegionEntryOffHeapIntKey.java     |  305 +
 .../VMStatsLRURegionEntryOffHeapLongKey.java    |  305 +
 .../VMStatsLRURegionEntryOffHeapObjectKey.java  |  298 +
 .../VMStatsLRURegionEntryOffHeapStringKey1.java |  367 +
 .../VMStatsLRURegionEntryOffHeapStringKey2.java |  408 +
 .../VMStatsLRURegionEntryOffHeapUUIDKey.java    |  309 +
 .../internal/cache/VMStatsRegionEntry.java      |   21 +-
 .../internal/cache/VMStatsRegionEntryHeap.java  |   25 +-
 .../cache/VMStatsRegionEntryHeapIntKey.java     |   31 +-
 .../cache/VMStatsRegionEntryHeapLongKey.java    |   31 +-
 .../cache/VMStatsRegionEntryHeapObjectKey.java  |   31 +-
 .../cache/VMStatsRegionEntryHeapStringKey1.java |   31 +-
 .../cache/VMStatsRegionEntryHeapStringKey2.java |   31 +-
 .../cache/VMStatsRegionEntryHeapUUIDKey.java    |   31 +-
 .../cache/VMStatsRegionEntryOffHeap.java        |   69 +
 .../cache/VMStatsRegionEntryOffHeapIntKey.java  |  224 +
 .../cache/VMStatsRegionEntryOffHeapLongKey.java |  224 +
 .../VMStatsRegionEntryOffHeapObjectKey.java     |  217 +
 .../VMStatsRegionEntryOffHeapStringKey1.java    |  286 +
 .../VMStatsRegionEntryOffHeapStringKey2.java    |  327 +
 .../cache/VMStatsRegionEntryOffHeapUUIDKey.java |  228 +
 .../cache/VMThinDiskLRURegionEntry.java         |   21 +-
 .../cache/VMThinDiskLRURegionEntryHeap.java     |   25 +-
 .../VMThinDiskLRURegionEntryHeapIntKey.java     |   33 +-
 .../VMThinDiskLRURegionEntryHeapLongKey.java    |   33 +-
 .../VMThinDiskLRURegionEntryHeapObjectKey.java  |   33 +-
 .../VMThinDiskLRURegionEntryHeapStringKey1.java |   33 +-
 .../VMThinDiskLRURegionEntryHeapStringKey2.java |   33 +-
 .../VMThinDiskLRURegionEntryHeapUUIDKey.java    |   33 +-
 .../cache/VMThinDiskLRURegionEntryOffHeap.java  |   69 +
 .../VMThinDiskLRURegionEntryOffHeapIntKey.java  |  329 +
 .../VMThinDiskLRURegionEntryOffHeapLongKey.java |  329 +
 ...MThinDiskLRURegionEntryOffHeapObjectKey.java |  322 +
 ...ThinDiskLRURegionEntryOffHeapStringKey1.java |  391 +
 ...ThinDiskLRURegionEntryOffHeapStringKey2.java |  432 +
 .../VMThinDiskLRURegionEntryOffHeapUUIDKey.java |  333 +
 .../internal/cache/VMThinDiskRegionEntry.java   |   21 +-
 .../cache/VMThinDiskRegionEntryHeap.java        |   25 +-
 .../cache/VMThinDiskRegionEntryHeapIntKey.java  |   31 +-
 .../cache/VMThinDiskRegionEntryHeapLongKey.java |   31 +-
 .../VMThinDiskRegionEntryHeapObjectKey.java     |   31 +-
 .../VMThinDiskRegionEntryHeapStringKey1.java    |   31 +-
 .../VMThinDiskRegionEntryHeapStringKey2.java    |   31 +-
 .../cache/VMThinDiskRegionEntryHeapUUIDKey.java |   31 +-
 .../cache/VMThinDiskRegionEntryOffHeap.java     |   69 +
 .../VMThinDiskRegionEntryOffHeapIntKey.java     |  231 +
 .../VMThinDiskRegionEntryOffHeapLongKey.java    |  231 +
 .../VMThinDiskRegionEntryOffHeapObjectKey.java  |  224 +
 .../VMThinDiskRegionEntryOffHeapStringKey1.java |  293 +
 .../VMThinDiskRegionEntryOffHeapStringKey2.java |  334 +
 .../VMThinDiskRegionEntryOffHeapUUIDKey.java    |  235 +
 .../internal/cache/VMThinLRURegionEntry.java    |   21 +-
 .../cache/VMThinLRURegionEntryHeap.java         |   25 +-
 .../cache/VMThinLRURegionEntryHeapIntKey.java   |   33 +-
 .../cache/VMThinLRURegionEntryHeapLongKey.java  |   33 +-
 .../VMThinLRURegionEntryHeapObjectKey.java      |   33 +-
 .../VMThinLRURegionEntryHeapStringKey1.java     |   33 +-
 .../VMThinLRURegionEntryHeapStringKey2.java     |   33 +-
 .../cache/VMThinLRURegionEntryHeapUUIDKey.java  |   33 +-
 .../cache/VMThinLRURegionEntryOffHeap.java      |   69 +
 .../VMThinLRURegionEntryOffHeapIntKey.java      |  240 +
 .../VMThinLRURegionEntryOffHeapLongKey.java     |  240 +
 .../VMThinLRURegionEntryOffHeapObjectKey.java   |  233 +
 .../VMThinLRURegionEntryOffHeapStringKey1.java  |  302 +
 .../VMThinLRURegionEntryOffHeapStringKey2.java  |  343 +
 .../VMThinLRURegionEntryOffHeapUUIDKey.java     |  244 +
 .../internal/cache/VMThinRegionEntry.java       |   21 +-
 .../internal/cache/VMThinRegionEntryHeap.java   |   25 +-
 .../cache/VMThinRegionEntryHeapIntKey.java      |   31 +-
 .../cache/VMThinRegionEntryHeapLongKey.java     |   31 +-
 .../cache/VMThinRegionEntryHeapObjectKey.java   |   31 +-
 .../cache/VMThinRegionEntryHeapStringKey1.java  |   31 +-
 .../cache/VMThinRegionEntryHeapStringKey2.java  |   31 +-
 .../cache/VMThinRegionEntryHeapUUIDKey.java     |   31 +-
 .../cache/VMThinRegionEntryOffHeap.java         |   71 +
 .../cache/VMThinRegionEntryOffHeapIntKey.java   |  158 +
 .../cache/VMThinRegionEntryOffHeapLongKey.java  |  158 +
 .../VMThinRegionEntryOffHeapObjectKey.java      |  151 +
 .../VMThinRegionEntryOffHeapStringKey1.java     |  220 +
 .../VMThinRegionEntryOffHeapStringKey2.java     |  261 +
 .../cache/VMThinRegionEntryOffHeapUUIDKey.java  |  162 +
 .../internal/cache/ValidatingDiskRegion.java    |   60 +-
 .../internal/cache/ValueByteWrapper.java        |   21 +-
 .../internal/cache/VersionTimestamp.java        |   21 +-
 .../cache/VersionedStatsDiskLRURegionEntry.java |   21 +-
 .../VersionedStatsDiskLRURegionEntryHeap.java   |   25 +-
 ...sionedStatsDiskLRURegionEntryHeapIntKey.java |   33 +-
 ...ionedStatsDiskLRURegionEntryHeapLongKey.java |   33 +-
 ...nedStatsDiskLRURegionEntryHeapObjectKey.java |   33 +-
 ...edStatsDiskLRURegionEntryHeapStringKey1.java |   33 +-
 ...edStatsDiskLRURegionEntryHeapStringKey2.java |   33 +-
 ...ionedStatsDiskLRURegionEntryHeapUUIDKey.java |   33 +-
 ...VersionedStatsDiskLRURegionEntryOffHeap.java |   70 +
 ...nedStatsDiskLRURegionEntryOffHeapIntKey.java |  479 +
 ...edStatsDiskLRURegionEntryOffHeapLongKey.java |  479 +
 ...StatsDiskLRURegionEntryOffHeapObjectKey.java |  472 +
 ...tatsDiskLRURegionEntryOffHeapStringKey1.java |  541 ++
 ...tatsDiskLRURegionEntryOffHeapStringKey2.java |  582 ++
 ...edStatsDiskLRURegionEntryOffHeapUUIDKey.java |  483 +
 .../cache/VersionedStatsDiskRegionEntry.java    |   21 +-
 .../VersionedStatsDiskRegionEntryHeap.java      |   25 +-
 ...VersionedStatsDiskRegionEntryHeapIntKey.java |   31 +-
 ...ersionedStatsDiskRegionEntryHeapLongKey.java |   31 +-
 ...sionedStatsDiskRegionEntryHeapObjectKey.java |   31 +-
 ...ionedStatsDiskRegionEntryHeapStringKey1.java |   31 +-
 ...ionedStatsDiskRegionEntryHeapStringKey2.java |   31 +-
 ...ersionedStatsDiskRegionEntryHeapUUIDKey.java |   31 +-
 .../VersionedStatsDiskRegionEntryOffHeap.java   |   70 +
 ...sionedStatsDiskRegionEntryOffHeapIntKey.java |  382 +
 ...ionedStatsDiskRegionEntryOffHeapLongKey.java |  382 +
 ...nedStatsDiskRegionEntryOffHeapObjectKey.java |  375 +
 ...edStatsDiskRegionEntryOffHeapStringKey1.java |  444 +
 ...edStatsDiskRegionEntryOffHeapStringKey2.java |  485 +
 ...ionedStatsDiskRegionEntryOffHeapUUIDKey.java |  386 +
 .../cache/VersionedStatsLRURegionEntry.java     |   21 +-
 .../cache/VersionedStatsLRURegionEntryHeap.java |   25 +-
 .../VersionedStatsLRURegionEntryHeapIntKey.java |   33 +-
 ...VersionedStatsLRURegionEntryHeapLongKey.java |   33 +-
 ...rsionedStatsLRURegionEntryHeapObjectKey.java |   33 +-
 ...sionedStatsLRURegionEntryHeapStringKey1.java |   33 +-
 ...sionedStatsLRURegionEntryHeapStringKey2.java |   33 +-
 ...VersionedStatsLRURegionEntryHeapUUIDKey.java |   33 +-
 .../VersionedStatsLRURegionEntryOffHeap.java    |   70 +
 ...rsionedStatsLRURegionEntryOffHeapIntKey.java |  390 +
 ...sionedStatsLRURegionEntryOffHeapLongKey.java |  390 +
 ...onedStatsLRURegionEntryOffHeapObjectKey.java |  383 +
 ...nedStatsLRURegionEntryOffHeapStringKey1.java |  452 +
 ...nedStatsLRURegionEntryOffHeapStringKey2.java |  493 +
 ...sionedStatsLRURegionEntryOffHeapUUIDKey.java |  394 +
 .../cache/VersionedStatsRegionEntry.java        |   21 +-
 .../cache/VersionedStatsRegionEntryHeap.java    |   25 +-
 .../VersionedStatsRegionEntryHeapIntKey.java    |   31 +-
 .../VersionedStatsRegionEntryHeapLongKey.java   |   31 +-
 .../VersionedStatsRegionEntryHeapObjectKey.java |   31 +-
 ...VersionedStatsRegionEntryHeapStringKey1.java |   31 +-
 ...VersionedStatsRegionEntryHeapStringKey2.java |   31 +-
 .../VersionedStatsRegionEntryHeapUUIDKey.java   |   31 +-
 .../cache/VersionedStatsRegionEntryOffHeap.java |   69 +
 .../VersionedStatsRegionEntryOffHeapIntKey.java |  309 +
 ...VersionedStatsRegionEntryOffHeapLongKey.java |  309 +
 ...rsionedStatsRegionEntryOffHeapObjectKey.java |  302 +
 ...sionedStatsRegionEntryOffHeapStringKey1.java |  371 +
 ...sionedStatsRegionEntryOffHeapStringKey2.java |  412 +
 ...VersionedStatsRegionEntryOffHeapUUIDKey.java |  313 +
 .../cache/VersionedThinDiskLRURegionEntry.java  |   21 +-
 .../VersionedThinDiskLRURegionEntryHeap.java    |   25 +-
 ...rsionedThinDiskLRURegionEntryHeapIntKey.java |   33 +-
 ...sionedThinDiskLRURegionEntryHeapLongKey.java |   33 +-
 ...onedThinDiskLRURegionEntryHeapObjectKey.java |   33 +-
 ...nedThinDiskLRURegionEntryHeapStringKey1.java |   33 +-
 ...nedThinDiskLRURegionEntryHeapStringKey2.java |   33 +-
 ...sionedThinDiskLRURegionEntryHeapUUIDKey.java |   33 +-
 .../VersionedThinDiskLRURegionEntryOffHeap.java |   70 +
 ...onedThinDiskLRURegionEntryOffHeapIntKey.java |  414 +
 ...nedThinDiskLRURegionEntryOffHeapLongKey.java |  414 +
 ...dThinDiskLRURegionEntryOffHeapObjectKey.java |  407 +
 ...ThinDiskLRURegionEntryOffHeapStringKey1.java |  476 +
 ...ThinDiskLRURegionEntryOffHeapStringKey2.java |  517 +
 ...nedThinDiskLRURegionEntryOffHeapUUIDKey.java |  418 +
 .../cache/VersionedThinDiskRegionEntry.java     |   21 +-
 .../cache/VersionedThinDiskRegionEntryHeap.java |   25 +-
 .../VersionedThinDiskRegionEntryHeapIntKey.java |   31 +-
 ...VersionedThinDiskRegionEntryHeapLongKey.java |   31 +-
 ...rsionedThinDiskRegionEntryHeapObjectKey.java |   31 +-
 ...sionedThinDiskRegionEntryHeapStringKey1.java |   31 +-
 ...sionedThinDiskRegionEntryHeapStringKey2.java |   31 +-
 ...VersionedThinDiskRegionEntryHeapUUIDKey.java |   31 +-
 .../VersionedThinDiskRegionEntryOffHeap.java    |   70 +
 ...rsionedThinDiskRegionEntryOffHeapIntKey.java |  316 +
 ...sionedThinDiskRegionEntryOffHeapLongKey.java |  316 +
 ...onedThinDiskRegionEntryOffHeapObjectKey.java |  309 +
 ...nedThinDiskRegionEntryOffHeapStringKey1.java |  378 +
 ...nedThinDiskRegionEntryOffHeapStringKey2.java |  419 +
 ...sionedThinDiskRegionEntryOffHeapUUIDKey.java |  320 +
 .../cache/VersionedThinLRURegionEntry.java      |   21 +-
 .../cache/VersionedThinLRURegionEntryHeap.java  |   25 +-
 .../VersionedThinLRURegionEntryHeapIntKey.java  |   33 +-
 .../VersionedThinLRURegionEntryHeapLongKey.java |   33 +-
 ...ersionedThinLRURegionEntryHeapObjectKey.java |   33 +-
 ...rsionedThinLRURegionEntryHeapStringKey1.java |   33 +-
 ...rsionedThinLRURegionEntryHeapStringKey2.java |   33 +-
 .../VersionedThinLRURegionEntryHeapUUIDKey.java |   33 +-
 .../VersionedThinLRURegionEntryOffHeap.java     |   69 +
 ...ersionedThinLRURegionEntryOffHeapIntKey.java |  325 +
 ...rsionedThinLRURegionEntryOffHeapLongKey.java |  325 +
 ...ionedThinLRURegionEntryOffHeapObjectKey.java |  318 +
 ...onedThinLRURegionEntryOffHeapStringKey1.java |  387 +
 ...onedThinLRURegionEntryOffHeapStringKey2.java |  428 +
 ...rsionedThinLRURegionEntryOffHeapUUIDKey.java |  329 +
 .../cache/VersionedThinRegionEntry.java         |   21 +-
 .../cache/VersionedThinRegionEntryHeap.java     |   25 +-
 .../VersionedThinRegionEntryHeapIntKey.java     |   31 +-
 .../VersionedThinRegionEntryHeapLongKey.java    |   31 +-
 .../VersionedThinRegionEntryHeapObjectKey.java  |   31 +-
 .../VersionedThinRegionEntryHeapStringKey1.java |   31 +-
 .../VersionedThinRegionEntryHeapStringKey2.java |   31 +-
 .../VersionedThinRegionEntryHeapUUIDKey.java    |   31 +-
 .../cache/VersionedThinRegionEntryOffHeap.java  |   69 +
 .../VersionedThinRegionEntryOffHeapIntKey.java  |  243 +
 .../VersionedThinRegionEntryOffHeapLongKey.java |  243 +
 ...ersionedThinRegionEntryOffHeapObjectKey.java |  236 +
 ...rsionedThinRegionEntryOffHeapStringKey1.java |  305 +
 ...rsionedThinRegionEntryOffHeapStringKey2.java |  346 +
 .../VersionedThinRegionEntryOffHeapUUIDKey.java |  247 +
 .../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        |  881 ++
 .../cache/control/InternalResourceManager.java  | 1664 +---
 .../internal/cache/control/MemoryEvent.java     |  122 +-
 .../internal/cache/control/MemoryEventImpl.java |  187 -
 .../internal/cache/control/MemoryEventType.java |  352 -
 .../cache/control/MemoryThresholds.java         |  282 +
 .../cache/control/OffHeapMemoryMonitor.java     |  562 ++
 .../control/PartitionRebalanceDetailsImpl.java  |   29 +-
 .../cache/control/PartitionRebalanceEvent.java  |   24 -
 .../control/PartitionRebalanceEventImpl.java    |   51 -
 .../control/PartitionRebalanceEventType.java    |   29 -
 .../cache/control/RebalanceOperationImpl.java   |  178 +-
 .../cache/control/RebalanceResultsImpl.java     |   37 +-
 .../internal/cache/control/RegionFilter.java    |   21 +-
 .../internal/cache/control/ResourceAdvisor.java |  366 +-
 .../internal/cache/control/ResourceEvent.java   |   31 +-
 .../cache/control/ResourceListener.java         |   23 +-
 .../cache/control/ResourceManagerStats.java     |  140 +-
 .../internal/cache/control/ResourceMonitor.java |   54 +
 .../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      |   27 +-
 .../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        |   23 +-
 .../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   |   29 +-
 .../execute/MemberFunctionResultSender.java     |   20 +-
 .../execute/MemberFunctionResultWaiter.java     |   21 +-
 .../cache/execute/MemberMappedArgument.java     |   20 +-
 .../execute/MultiRegionFunctionContext.java     |   21 +-
 .../execute/MultiRegionFunctionContextImpl.java |   21 +-
 .../execute/MultiRegionFunctionExecutor.java    |   37 +-
 .../MultiRegionFunctionResultWaiter.java        |   21 +-
 .../internal/cache/execute/NoResult.java        |   20 +-
 .../PartitionedRegionFunctionExecutor.java      |   29 +-
 .../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        |   39 +-
 .../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 |  224 +-
 .../cache/lru/HeapLRUCapacityController.java    |   80 +-
 .../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     |   39 +-
 .../internal/cache/lru/NewLIFOClockHand.java    |   99 +-
 .../internal/cache/lru/NewLRUClockHand.java     |   35 +-
 .../internal/cache/lru/OffHeapEvictor.java      |   97 +
 .../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       |   44 +-
 .../DestroyRegionOnDataStoreMessage.java        |  118 +
 .../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  |   39 +-
 .../cache/partitioned/FetchEntryMessage.java    |   20 +-
 .../cache/partitioned/FetchKeysMessage.java     |   21 +-
 .../FetchPartitionDetailsMessage.java           |   21 +-
 .../cache/partitioned/FlushMessage.java         |   21 +-
 .../internal/cache/partitioned/GetMessage.java  |   85 +-
 .../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    |   27 +-
 .../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            |   25 +-
 .../partitioned/PartitionMemberInfoImpl.java    |   21 +-
 .../cache/partitioned/PartitionMessage.java     |   94 +-
 .../PartitionMessageWithDirectReply.java        |   21 +-
 .../partitioned/PartitionRegionInfoImpl.java    |   21 +-
 ...rtitionedRegionFunctionStreamingMessage.java |   21 +-
 .../partitioned/PartitionedRegionObserver.java  |   21 +-
 .../PartitionedRegionObserverAdapter.java       |   21 +-
 .../PartitionedRegionObserverHolder.java        |   23 +-
 .../PartitionedRegionRebalanceOp.java           |  187 +-
 .../partitioned/PrimaryRequestMessage.java      |   21 +-
 .../cache/partitioned/PutAllPRMessage.java      |  120 +-
 .../internal/cache/partitioned/PutMessage.java  |  254 +-
 .../cache/partitioned/QueryMessage.java         |   72 +-
 .../cache/partitioned/RecoveryRunnable.java     |   23 +-
 .../RedundancyAlreadyMetException.java          |   21 +-
 .../cache/partitioned/RedundancyLogger.java     |   21 +-
 .../cache/partitioned/RegionAdvisor.java        |   39 +-
 .../partitioned/RemoteFetchKeysMessage.java     |   21 +-
 .../cache/partitioned/RemoteSizeMessage.java    |   21 +-
 .../cache/partitioned/RemoveAllPRMessage.java   |   54 +-
 .../cache/partitioned/RemoveBucketMessage.java  |   21 +-
 .../cache/partitioned/RemoveIndexesMessage.java |   21 +-
 .../internal/cache/partitioned/SizeMessage.java |   59 +-
 .../cache/partitioned/SizedBasedLoadProbe.java  |   21 +-
 .../StreamingPartitionOperation.java            |   24 +-
 .../partitioned/rebalance/BucketOperator.java   |   67 +-
 .../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       |  167 +
 .../rebalance/PartitionedRegionLoadModel.java   |   98 +-
 .../rebalance/PercentageMoveDirector.java       |   21 +-
 .../rebalance/RebalanceDirector.java            |   21 +-
 .../rebalance/RebalanceDirectorAdapter.java     |   21 +-
 .../rebalance/RemoveOverRedundancy.java         |   21 +-
 .../rebalance/SatisfyRedundancy.java            |   28 +-
 .../rebalance/SatisfyRedundancyFPR.java         |   22 +-
 .../rebalance/SimulatedBucketOperator.java      |   32 +-
 .../cache/persistence/BackupInspector.java      |   21 +-
 .../cache/persistence/BackupManager.java        |   23 +-
 .../cache/persistence/BytesAndBits.java         |   21 +-
 .../cache/persistence/CanonicalIdHolder.java    |   21 +-
 .../CreatePersistentRegionProcessor.java        |   21 +-
 .../cache/persistence/DiskExceptionHandler.java |   21 +-
 .../persistence/DiskInitFileInterpreter.java    |   23 +-
 .../cache/persistence/DiskInitFileParser.java   |   60 +-
 .../cache/persistence/DiskRecoveryStore.java    |   23 +-
 .../cache/persistence/DiskRegionView.java       |   25 +-
 .../cache/persistence/DiskStoreFilter.java      |   23 +-
 .../internal/cache/persistence/DiskStoreID.java |   32 +-
 .../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     |   23 +-
 .../persistence/PersistenceObserverHolder.java  |   21 +-
 .../cache/persistence/PersistentMemberID.java   |   21 +-
 .../persistence/PersistentMemberManager.java    |   23 +-
 .../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      |   33 +-
 .../cache/persistence/soplog/Compactor.java     |  165 -
 .../soplog/CompositeSerializedComparator.java   |   48 -
 .../persistence/soplog/CursorIterator.java      |   82 +
 .../soplog/DelegatingSerializedComparator.java  |   21 +-
 .../soplog/HFileStoreStatistics.java            |  205 +
 .../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  |  246 -
 .../persistence/soplog/SortedOplogSet.java      |  109 -
 .../persistence/soplog/SortedOplogSetImpl.java  |  771 --
 .../soplog/SortedOplogStatistics.java           |   51 +-
 .../cache/persistence/soplog/SortedReader.java  |   37 +-
 .../persistence/soplog/TrackedReference.java    |   72 +-
 .../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 |   32 +-
 .../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        |  141 +-
 .../cache/tier/sockets/BaseCommand.java         |   77 +-
 .../cache/tier/sockets/BaseCommandQuery.java    |   18 +-
 .../cache/tier/sockets/CacheClientNotifier.java |  108 +-
 .../tier/sockets/CacheClientNotifierStats.java  |   21 +-
 .../cache/tier/sockets/CacheClientProxy.java    |   72 +-
 .../tier/sockets/CacheClientProxyStats.java     |   21 +-
 .../cache/tier/sockets/CacheClientUpdater.java  |   66 +-
 .../cache/tier/sockets/CacheServerHelper.java   |   21 +-
 .../cache/tier/sockets/CacheServerStats.java    |   21 +-
 .../cache/tier/sockets/ChunkedMessage.java      |   32 +-
 .../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   |   37 +-
 .../tier/sockets/ClientTombstoneMessage.java    |   21 +-
 .../cache/tier/sockets/ClientUpdateMessage.java |   21 +-
 .../tier/sockets/ClientUpdateMessageImpl.java   |   66 +-
 .../cache/tier/sockets/ClientUserAuths.java     |   21 +-
 .../cache/tier/sockets/CommandInitializer.java  |   39 +-
 .../cache/tier/sockets/ConnectionListener.java  |   21 +-
 .../tier/sockets/ConnectionListenerAdapter.java |   21 +-
 .../cache/tier/sockets/HAEventWrapper.java      |   21 +-
 .../internal/cache/tier/sockets/HandShake.java  |   46 +-
 .../tier/sockets/InterestResultPolicyImpl.java  |   21 +-
 .../internal/cache/tier/sockets/Message.java    |  141 +-
 .../cache/tier/sockets/MessageStats.java        |   21 +-
 .../cache/tier/sockets/ObjectPartList.java      |   33 +-
 .../cache/tier/sockets/ObjectPartList651.java   |   21 +-
 .../internal/cache/tier/sockets/Part.java       |  129 +-
 .../RemoveClientFromBlacklistMessage.java       |   27 +-
 .../tier/sockets/SerializedObjectPartList.java  |   21 +-
 .../cache/tier/sockets/ServerConnection.java    |  109 +-
 .../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     |   27 +-
 .../cache/tier/sockets/command/ContainsKey.java |   23 +-
 .../tier/sockets/command/ContainsKey66.java     |   23 +-
 .../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   |   29 +-
 .../tier/sockets/command/ExecuteFunction65.java |   28 +-
 .../tier/sockets/command/ExecuteFunction66.java |   29 +-
 .../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 |   66 +-
 .../cache/tier/sockets/command/Get70.java       |  175 +-
 .../cache/tier/sockets/command/GetAll.java      |   21 +-
 .../cache/tier/sockets/command/GetAll651.java   |   21 +-
 .../cache/tier/sockets/command/GetAll70.java    |  104 +-
 .../cache/tier/sockets/command/GetAllForRI.java |   21 +-
 .../sockets/command/GetAllWithCallback.java     |  109 +-
 .../command/GetClientPRMetadataCommand.java     |   22 +-
 .../command/GetClientPRMetadataCommand66.java   |   22 +-
 .../GetClientPartitionAttributesCommand.java    |   22 +-
 .../GetClientPartitionAttributesCommand66.java  |   22 +-
 .../cache/tier/sockets/command/GetEntry70.java  |   29 +-
 .../tier/sockets/command/GetEntryCommand.java   |   21 +-
 .../sockets/command/GetFunctionAttribute.java   |   21 +-
 .../tier/sockets/command/GetPDXEnumById.java    |   23 +-
 .../tier/sockets/command/GetPDXIdForEnum.java   |   23 +-
 .../tier/sockets/command/GetPDXIdForType.java   |   23 +-
 .../tier/sockets/command/GetPDXTypeById.java    |   23 +-
 .../tier/sockets/command/GetPdxEnums70.java     |   23 +-
 .../tier/sockets/command/GetPdxTypes70.java     |   23 +-
 .../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     |   32 +-
 .../tier/sockets/command/RequestEventValue.java |   20 +-
 .../tier/sockets/command/RollbackCommand.java   |   21 +-
 .../cache/tier/sockets/command/Size.java        |   23 +-
 .../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   |   23 +-
 .../internal/cache/tx/ClientTXStateStub.java    |   21 +-
 .../cache/tx/DistClientTXStateStub.java         |  140 +
 .../internal/cache/tx/DistTxEntryEvent.java     |  288 +
 .../internal/cache/tx/DistTxKeyInfo.java        |   54 +
 .../cache/tx/DistributedTXRegionStub.java       |   23 +-
 .../cache/tx/PartitionedTXRegionStub.java       |   33 +-
 .../gemfire/internal/cache/tx/TXRegionStub.java |   23 +-
 .../cache/tx/TransactionalOperation.java        |   32 +-
 .../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     |   29 +-
 .../cache/versions/VMRegionVersionVector.java   |   21 +-
 .../internal/cache/versions/VMVersionTag.java   |   21 +-
 .../internal/cache/versions/VersionHolder.java  |   21 +-
 .../internal/cache/versions/VersionSource.java  |   23 +-
 .../internal/cache/versions/VersionStamp.java   |   21 +-
 .../internal/cache/versions/VersionTag.java     |   30 +-
 .../internal/cache/vmotion/VMotionObserver.java |   20 +-
 .../cache/vmotion/VMotionObserverAdapter.java   |   20 +-
 .../cache/vmotion/VMotionObserverHolder.java    |   20 +-
 .../cache/wan/AbstractGatewaySender.java        |  234 +-
 .../AbstractGatewaySenderEventProcessor.java    |   59 +-
 .../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         |   62 +-
 .../cache/wan/GatewaySenderAttributes.java      |   34 +-
 .../GatewaySenderConfigurationException.java    |   21 +-
 .../wan/GatewaySenderEventCallbackArgument.java |   20 +-
 .../GatewaySenderEventCallbackDispatcher.java   |   22 +-
 .../cache/wan/GatewaySenderEventDispatcher.java |   21 +-
 .../cache/wan/GatewaySenderEventImpl.java       |  613 +-
 .../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 |   38 +-
 .../ConcurrentParallelGatewaySenderQueue.java   |   39 +-
 .../ParallelGatewaySenderEventProcessor.java    |  104 +-
 .../parallel/ParallelGatewaySenderQueue.java    |  195 +-
 .../ParallelQueueBatchRemovalMessage.java       |   21 +-
 .../parallel/ParallelQueueRemovalMessage.java   |   21 +-
 .../cache/wan/parallel/RREventIDResolver.java   |   21 +-
 .../cache/wan/serial/BatchDestroyOperation.java |   29 +-
 ...urrentSerialGatewaySenderEventProcessor.java |   35 +-
 .../SerialGatewaySenderEventProcessor.java      |  155 +-
 .../wan/serial/SerialGatewaySenderQueue.java    |  198 +-
 .../serial/SerialSecondaryGatewayListener.java  |   26 +-
 .../internal/cache/wan/spi/WANFactory.java      |   16 +
 .../cache/xmlcache/AbstractXmlParser.java       |   21 +-
 .../cache/xmlcache/AsyncEventQueueCreation.java |   33 +-
 .../cache/xmlcache/BindingCreation.java         |   21 +-
 .../cache/xmlcache/BridgeServerCreation.java    |  213 -
 .../internal/cache/xmlcache/CacheCreation.java  |  191 +-
 .../cache/xmlcache/CacheServerCreation.java     |  257 +
 .../CacheTransactionManagerCreation.java        |   29 +-
 .../internal/cache/xmlcache/CacheXml.java       |   79 +-
 .../cache/xmlcache/CacheXmlGenerator.java       |  118 +-
 .../internal/cache/xmlcache/CacheXmlParser.java |  277 +-
 .../xmlcache/CacheXmlPropertyResolver.java      |   21 +-
 .../CacheXmlPropertyResolverHelper.java         |   21 +-
 .../cache/xmlcache/CacheXmlVersion.java         |   20 +-
 .../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 |   25 +-
 .../cache/xmlcache/PivotalEntityResolver.java   |   23 +-
 .../cache/xmlcache/PropertyResolver.java        |   21 +-
 .../xmlcache/RegionAttributesCreation.java      |  104 +-
 .../internal/cache/xmlcache/RegionCreation.java |   21 +-
 .../cache/xmlcache/ResourceManagerCreation.java |  116 +-
 .../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         |  127 +
 .../internal/concurrent/AtomicLong5.java        |   48 +
 .../gemfire/internal/concurrent/Atomics.java    |   21 +-
 .../concurrent/CompactConcurrentHashSet2.java   |   22 +-
 .../internal/concurrent/ConcurrentHashSet.java  |   21 +-
 .../gemfire/internal/concurrent/LI.java         |   21 +-
 .../internal/concurrent/MapCallback.java        |  269 +
 .../internal/concurrent/MapCallbackAdapter.java |  140 +
 .../gemfire/internal/concurrent/MapResult.java  |   43 +
 .../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 | 4043 ++++----
 .../internal/i18n/ParentLocalizedStrings.java   | 4598 +++++----
 .../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      |  384 +-
 .../gemfire/internal/lang/SystemUtils.java      |   95 +-
 .../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    |  157 +-
 .../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   |   85 +
 .../internal/logging/log4j/Configurator.java    |   94 +-
 .../internal/logging/log4j/FastLogger.java      |   66 +-
 .../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 |   25 +-
 .../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  |   77 +
 .../internal/offheap/ByteBufferMemoryChunk.java |   90 +
 .../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 |  126 +
 .../gemfire/internal/offheap/DataType.java      |  270 +
 .../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       |   64 +
 .../gemfire/internal/offheap/MemoryBlock.java   |   71 +
 .../internal/offheap/MemoryBlockNode.java       |  170 +
 .../gemfire/internal/offheap/MemoryChunk.java   |   47 +
 .../offheap/MemoryChunkWithRefCount.java        |   34 +
 .../internal/offheap/MemoryInspector.java       |   42 +
 .../internal/offheap/MemoryUsageListener.java   |   27 +
 .../offheap/OffHeapCachedDeserializable.java    |  142 +
 .../gemfire/internal/offheap/OffHeapHelper.java |  133 +
 .../internal/offheap/OffHeapMemoryStats.java    |   54 +
 .../offheap/OffHeapRegionEntryHelper.java       |  430 +
 .../internal/offheap/OffHeapStorage.java        |  447 +
 .../offheap/OutOfOffHeapMemoryListener.java     |   44 +
 .../internal/offheap/RefCountChangeInfo.java    |  130 +
 .../internal/offheap/ReferenceCountHelper.java  |  254 +
 .../gemfire/internal/offheap/Releasable.java    |   31 +
 .../offheap/SimpleMemoryAllocatorImpl.java      |  687 ++
 .../gemfire/internal/offheap/StoredObject.java  |   95 +
 .../internal/offheap/SyncChunkStack.java        |  130 +
 .../internal/offheap/UnsafeMemoryChunk.java     |  278 +
 .../offheap/annotations/OffHeapIdentifier.java  |   69 +
 .../internal/offheap/annotations/Released.java  |   49 +
 .../internal/offheap/annotations/Retained.java  |   51 +
 .../offheap/annotations/Unretained.java         |   51 +
 .../com/gemstone/gemfire/internal/package.html  |   16 +
 .../internal/process/AttachProcessUtils.java    |   21 +
 .../process/BlockingProcessStreamReader.java    |   73 +
 ...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 |   55 +-
 .../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    |   21 +
 .../process/NonBlockingProcessStreamReader.java |  101 +
 .../gemfire/internal/process/PidFile.java       |  169 +
 .../process/PidUnavailableException.java        |   21 +-
 .../internal/process/ProcessController.java     |   16 +
 .../process/ProcessControllerFactory.java       |  114 +-
 .../process/ProcessControllerParameters.java    |   16 +
 .../process/ProcessLauncherContext.java         |   21 +-
 .../internal/process/ProcessStreamReader.java   |  154 +-
 .../ProcessTerminatedAbnormallyException.java   |   20 +-
 .../gemfire/internal/process/ProcessType.java   |   16 +
 .../gemfire/internal/process/ProcessUtils.java  |   50 +-
 .../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        |  181 +
 .../internal/redis/ByteToCommandDecoder.java    |  189 +
 .../gemstone/gemfire/internal/redis/Coder.java  |  517 +
 .../gemfire/internal/redis/Command.java         |  150 +
 .../gemfire/internal/redis/DoubleWrapper.java   |   79 +
 .../internal/redis/ExecutionHandlerContext.java |  380 +
 .../gemfire/internal/redis/Executor.java        |   38 +
 .../gemfire/internal/redis/Extendable.java      |   33 +
 .../redis/RedisCommandParserException.java      |   45 +
 .../internal/redis/RedisCommandType.java        | 2345 +++++
 .../gemfire/internal/redis/RedisConstants.java  |  202 +
 .../gemfire/internal/redis/RedisDataType.java   |  118 +
 .../redis/RedisDataTypeMismatchException.java   |   39 +
 .../internal/redis/RegionCreationException.java |   39 +
 .../gemfire/internal/redis/RegionProvider.java  |  554 ++
 .../redis/executor/AbstractExecutor.java        |  147 +
 .../redis/executor/AbstractScanExecutor.java    |   47 +
 .../internal/redis/executor/AuthExecutor.java   |   54 +
 .../internal/redis/executor/DBSizeExecutor.java |   31 +
 .../internal/redis/executor/DelExecutor.java    |   55 +
 .../internal/redis/executor/EchoExecutor.java   |   40 +
 .../internal/redis/executor/ExistsExecutor.java |   49 +
 .../redis/executor/ExpirationExecutor.java      |   41 +
 .../redis/executor/ExpireAtExecutor.java        |   93 +
 .../internal/redis/executor/ExpireExecutor.java |   93 +
 .../redis/executor/FlushAllExecutor.java        |   49 +
 .../internal/redis/executor/KeysExecutor.java   |   70 +
 .../internal/redis/executor/ListQuery.java      |   53 +
 .../redis/executor/PExpireAtExecutor.java       |   32 +
 .../redis/executor/PExpireExecutor.java         |   32 +
 .../internal/redis/executor/PTTLExecutor.java   |   33 +
 .../redis/executor/PersistExecutor.java         |   52 +
 .../internal/redis/executor/PingExecutor.java   |   31 +
 .../internal/redis/executor/QuitExecutor.java   |   31 +
 .../internal/redis/executor/ScanExecutor.java   |  144 +
 .../redis/executor/ShutDownExecutor.java        |   28 +
 .../internal/redis/executor/SortedSetQuery.java |  204 +
 .../internal/redis/executor/TTLExecutor.java    |   77 +
 .../internal/redis/executor/TimeExecutor.java   |   51 +
 .../internal/redis/executor/TypeExecutor.java   |   48 +
 .../internal/redis/executor/UnkownExecutor.java |   31 +
 .../redis/executor/hash/HDelExecutor.java       |   67 +
 .../redis/executor/hash/HExistsExecutor.java    |   66 +
 .../redis/executor/hash/HGetAllExecutor.java    |   63 +
 .../redis/executor/hash/HGetExecutor.java       |   62 +
 .../redis/executor/hash/HIncrByExecutor.java    |  109 +
 .../executor/hash/HIncrByFloatExecutor.java     |   99 +
 .../redis/executor/hash/HKeysExecutor.java      |   63 +
 .../redis/executor/hash/HLenExecutor.java       |   57 +
 .../redis/executor/hash/HMGetExecutor.java      |   72 +
 .../redis/executor/hash/HMSetExecutor.java      |   62 +
 .../redis/executor/hash/HScanExecutor.java      |  163 +
 .../redis/executor/hash/HSetExecutor.java       |   78 +
 .../redis/executor/hash/HSetNXExecutor.java     |   33 +
 .../redis/executor/hash/HValsExecutor.java      |   62 +
 .../redis/executor/hash/HashExecutor.java       |   39 +
 .../internal/redis/executor/hll/Bits.java       |   65 +
 .../executor/hll/CardinalityMergeException.java |   42 +
 .../redis/executor/hll/HllExecutor.java         |   38 +
 .../redis/executor/hll/HyperLogLog.java         |  360 +
 .../redis/executor/hll/HyperLogLogPlus.java     | 1069 ++
 .../internal/redis/executor/hll/IBuilder.java   |   41 +
 .../redis/executor/hll/ICardinality.java        |   89 +
 .../internal/redis/executor/hll/MurmurHash.java |  234 +
 .../redis/executor/hll/PFAddExecutor.java       |   65 +
 .../redis/executor/hll/PFCountExecutor.java     |   68 +
 .../redis/executor/hll/PFMergeExecutor.java     |   72 +
 .../redis/executor/hll/RegisterSet.java         |  126 +
 .../internal/redis/executor/hll/Varint.java     |  241 +
 .../redis/executor/list/LIndexExecutor.java     |  118 +
 .../redis/executor/list/LInsertExecutor.java    |   29 +
 .../redis/executor/list/LLenExecutor.java       |   58 +
 .../redis/executor/list/LPopExecutor.java       |   34 +
 .../redis/executor/list/LPushExecutor.java      |   34 +
 .../redis/executor/list/LPushXExecutor.java     |   34 +
 .../redis/executor/list/LRangeExecutor.java     |  113 +
 .../redis/executor/list/LRemExecutor.java       |  116 +
 .../redis/executor/list/LSetExecutor.java       |  108 +
 .../redis/executor/list/LTrimExecutor.java      |  124 +
 .../redis/executor/list/ListExecutor.java       |  150 +
 .../redis/executor/list/PopExecutor.java        |  150 +
 .../redis/executor/list/PushExecutor.java       |   54 +
 .../redis/executor/list/PushXExecutor.java      |   59 +
 .../redis/executor/list/RPopExecutor.java       |   34 +
 .../redis/executor/list/RPushExecutor.java      |   34 +
 .../redis/executor/list/RPushXExecutor.java     |   34 +
 .../redis/executor/set/SAddExecutor.java        |   60 +
 .../redis/executor/set/SCardExecutor.java       |   55 +
 .../redis/executor/set/SDiffExecutor.java       |   46 +
 .../redis/executor/set/SDiffStoreExecutor.java  |   33 +
 .../redis/executor/set/SInterExecutor.java      |   49 +
 .../redis/executor/set/SInterStoreExecutor.java |   34 +
 .../redis/executor/set/SIsMemberExecutor.java   |   62 +
 .../redis/executor/set/SMembersExecutor.java    |   56 +
 .../redis/executor/set/SMoveExecutor.java       |   72 +
 .../redis/executor/set/SPopExecutor.java        |   61 +
 .../redis/executor/set/SRandMemberExecutor.java |   96 +
 .../redis/executor/set/SRemExecutor.java        |   63 +
 .../redis/executor/set/SScanExecutor.java       |  154 +
 .../redis/executor/set/SUnionExecutor.java      |   51 +
 .../redis/executor/set/SUnionStoreExecutor.java |   34 +
 .../redis/executor/set/SetExecutor.java         |   23 +
 .../redis/executor/set/SetOpExecutor.java       |  109 +
 .../executor/sortedset/SortedSetExecutor.java   |   41 +
 .../redis/executor/sortedset/ZAddExecutor.java  |   88 +
 .../redis/executor/sortedset/ZCardExecutor.java |   54 +
 .../executor/sortedset/ZCountExecutor.java      |  145 +
 .../executor/sortedset/ZIncrByExecutor.java     |   77 +
 .../executor/sortedset/ZLexCountExecutor.java   |  143 +
 .../executor/sortedset/ZRangeByLexExecutor.java |  209 +
 .../sortedset/ZRangeByScoreExecutor.java        |  209 +
 .../executor/sortedset/ZRangeExecutor.java      |  125 +
 .../redis/executor/sortedset/ZRankExecutor.java |   98 +
 .../redis/executor/sortedset/ZRemExecutor.java  |   64 +
 .../sortedset/ZRemRangeByLexExecutor.java       |  153 +
 .../sortedset/ZRemRangeByRankExecutor.java      |  121 +
 .../sortedset/ZRemRangeByScoreExecutor.java     |  143 +
 .../sortedset/ZRevRangeByScoreExecutor.java     |   33 +
 .../executor/sortedset/ZRevRangeExecutor.java   |   34 +
 .../executor/sortedset/ZRevRankExecutor.java    |   32 +
 .../redis/executor/sortedset/ZScanExecutor.java |  161 +
 .../executor/sortedset/ZScoreExecutor.java      |   59 +
 .../redis/executor/string/AppendExecutor.java   |   69 +
 .../redis/executor/string/BitCountExecutor.java |   97 +
 .../redis/executor/string/BitOpExecutor.java    |  153 +
 .../redis/executor/string/BitPosExecutor.java   |  134 +
 .../redis/executor/string/DecrByExecutor.java   |  110 +
 .../redis/executor/string/DecrExecutor.java     |   95 +
 .../redis/executor/string/GetBitExecutor.java   |   82 +
 .../redis/executor/string/GetExecutor.java      |   51 +
 .../redis/executor/string/GetRangeExecutor.java |   98 +
 .../redis/executor/string/GetSetExecutor.java   |   59 +
 .../redis/executor/string/IncrByExecutor.java   |  107 +
 .../executor/string/IncrByFloatExecutor.java    |  122 +
 .../redis/executor/string/IncrExecutor.java     |   91 +
 .../redis/executor/string/MGetExecutor.java     |   73 +
 .../redis/executor/string/MSetExecutor.java     |   64 +
 .../redis/executor/string/MSetNXExecutor.java   |   88 +
 .../redis/executor/string/PSetEXExecutor.java   |   34 +
 .../redis/executor/string/SetBitExecutor.java   |  106 +
 .../redis/executor/string/SetEXExecutor.java    |   88 +
 .../redis/executor/string/SetExecutor.java      |  154 +
 .../redis/executor/string/SetNXExecutor.java    |   60 +
 .../redis/executor/string/SetRangeExecutor.java |   96 +
 .../redis/executor/string/StringExecutor.java   |   45 +
 .../redis/executor/string/StrlenExecutor.java   |   56 +
 .../executor/transactions/DiscardExecutor.java  |   42 +
 .../executor/transactions/ExecExecutor.java     |   88 +
 .../executor/transactions/MultiExecutor.java    |   47 +
 .../transactions/TransactionExecutor.java       |   23 +
 .../executor/transactions/UnwatchExecutor.java  |   31 +
 .../executor/transactions/WatchExecutor.java    |   31 +
 .../redis/org/apache/hadoop/fs/GlobPattern.java |  164 +
 .../internal/security/AuthorizeRequest.java     |   24 +-
 .../internal/security/AuthorizeRequestPP.java   |   24 +-
 .../security/FilterPostAuthorization.java       |   21 +-
 .../security/FilterPreAuthorization.java        |   21 +-
 .../internal/security/ObjectWithAuthz.java      |   21 +-
 .../gemfire/internal/security/package.html      |   16 +
 .../internal/sequencelog/EntryLogger.java       |   30 +-
 .../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     |   65 +-
 .../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     |  775 +-
 .../gemfire/internal/tcp/ConnectExceptions.java |   21 +-
 .../gemfire/internal/tcp/Connection.java        |  325 +-
 .../internal/tcp/ConnectionException.java       |   21 +-
 .../gemfire/internal/tcp/ConnectionTable.java   |  220 +-
 .../gemfire/internal/tcp/DirectReplySender.java |   21 +-
 .../tcp/ImmutableByteBufferInputStream.java     |   30 +-
 .../internal/tcp/MemberShunnedException.java    |   28 +-
 .../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       |   71 +-
 .../gemfire/internal/tcp/MsgStreamerList.java   |   31 +-
 .../gemfire/internal/tcp/NIOMsgReader.java      |   21 +-
 .../gemfire/internal/tcp/OioMsgReader.java      |   21 +-
 .../internal/tcp/ReenteredConnectException.java |   21 +-
 .../gemfire/internal/tcp/ServerDelegate.java    |   26 +-
 .../com/gemstone/gemfire/internal/tcp/Stub.java |  155 -
 .../gemfire/internal/tcp/TCPConduit.java        |  304 +-
 .../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 -
 .../gfsh/command/CommandExecLoaderListener.java |   96 -
 .../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       |  145 +-
 .../gemfire/internal/util/BlobHelper.java       |   41 +-
 .../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        |   87 +
 .../gemfire/internal/util/SingletonValue.java   |  318 +
 .../internal/util/StackTraceCollector.java      |  169 +
 .../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           |  121 +-
 .../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    |   26 +-
 .../gemfire/management/DiskBackupStatus.java    |   20 +-
 .../gemfire/management/DiskMetrics.java         |   20 +-
 .../gemfire/management/DiskStoreMXBean.java     |   20 +-
 .../DistributedLockServiceMXBean.java           |   20 +-
 .../management/DistributedRegionMXBean.java     |   31 +-
 .../management/DistributedSystemMXBean.java     |   28 +-
 .../management/EvictionAttributesData.java      |   30 +-
 .../FixedPartitionAttributesData.java           |   31 +-
 .../management/GatewayReceiverMXBean.java       |   20 +-
 .../gemfire/management/GatewaySenderMXBean.java |   20 +-
 .../gemfire/management/GemFireProperties.java   |   39 +-
 .../gemfire/management/JMXNotificationType.java |   28 +-
 .../management/JMXNotificationUserData.java     |   28 +-
 .../gemstone/gemfire/management/JVMMetrics.java |   27 +-
 .../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        |   59 +-
 .../management/MembershipAttributesData.java    |   55 +-
 .../gemfire/management/NetworkMetrics.java      |   20 +-
 .../gemstone/gemfire/management/OSMetrics.java  |   28 +-
 .../management/PartitionAttributesData.java     |   79 +-
 .../management/PersistentMemberDetails.java     |   27 +-
 .../management/RegionAttributesData.java        |   48 +-
 .../gemfire/management/RegionMXBean.java        |   32 +-
 .../gemfire/management/ServerLoadData.java      |   27 +-
 .../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   |   20 +-
 .../gemstone/gemfire/management/cli/Result.java |   21 +-
 .../gemfire/management/cli/package.html         |   16 +
 .../gemfire/management/internal/AgentUtil.java  |  154 +
 .../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  |   26 +-
 .../internal/FederationComponent.java           |   20 +-
 .../management/internal/FilterChain.java        |   20 +-
 .../management/internal/FilterParam.java        |   22 +-
 .../management/internal/IdentityConverter.java  |   20 +-
 .../management/internal/JettyHelper.java        |   22 +-
 .../management/internal/JmxManagerAdvisee.java  |   27 +-
 .../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    |  394 +-
 .../internal/ManagementCacheListener.java       |   22 +-
 .../internal/ManagementConstants.java           |   20 +-
 .../management/internal/ManagementFunction.java |   20 +-
 .../internal/ManagementMembershipListener.java  |   24 +-
 .../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  |  176 +-
 .../gemfire/management/internal/SSLUtil.java    |   16 +
 .../management/internal/StringBasedFilter.java  |   20 +-
 .../internal/SystemManagementService.java       |   28 +-
 .../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 |   24 +-
 .../internal/beans/DistributedRegionMBean.java  |   24 +-
 .../internal/beans/DistributedSystemBridge.java |   49 +-
 .../internal/beans/DistributedSystemMBean.java  |   25 +-
 .../internal/beans/GatewayReceiverMBean.java    |   20 +-
 .../beans/GatewayReceiverMBeanBridge.java       |   24 +-
 .../internal/beans/GatewaySenderMBean.java      |   20 +-
 .../beans/GatewaySenderMBeanBridge.java         |   20 +-
 .../internal/beans/HDFSRegionBridge.java        |  174 +
 .../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         |   22 +-
 .../internal/beans/ManagementAdapter.java       |   72 +-
 .../internal/beans/ManagementListener.java      |   20 +-
 .../management/internal/beans/ManagerMBean.java |   20 +-
 .../internal/beans/ManagerMBeanBridge.java      |   20 +-
 .../management/internal/beans/MemberMBean.java  |   86 +-
 .../internal/beans/MemberMBeanBridge.java       |  211 +-
 .../internal/beans/MetricsCalculator.java       |   20 +-
 .../internal/beans/PartitionedRegionBridge.java |   36 +-
 .../internal/beans/QueryDataFunction.java       |   20 +-
 .../management/internal/beans/RegionMBean.java  |   25 +-
 .../internal/beans/RegionMBeanBridge.java       |   31 +-
 .../beans/RegionMBeanCompositeDataFactory.java  |   23 +-
 .../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  |   58 +-
 .../beans/stats/MemberLevelDiskMonitor.java     |   20 +-
 .../beans/stats/RegionClusterStatsMonitor.java  |   27 +-
 .../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        |   22 +-
 .../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       |   30 +-
 .../management/internal/cli/LogWrapper.java     |   24 +-
 .../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       |  178 +-
 .../internal/cli/commands/DataCommands.java     |   23 +-
 .../internal/cli/commands/DeployCommands.java   |   20 +-
 .../cli/commands/DiskStoreCommands.java         |  104 +-
 .../cli/commands/DurableClientCommands.java     |   20 +-
 ...ExportImportSharedConfigurationCommands.java |   21 +-
 .../internal/cli/commands/FunctionCommands.java |   20 +-
 .../internal/cli/commands/GfshHelpCommands.java |   20 +-
 .../internal/cli/commands/IndexCommands.java    |   20 +-
 .../cli/commands/LauncherLifecycleCommands.java |  356 +-
 .../internal/cli/commands/MemberCommands.java   |   26 +-
 .../cli/commands/MiscellaneousCommands.java     |   71 +-
 .../internal/cli/commands/PDXCommands.java      |   16 +
 .../internal/cli/commands/QueueCommands.java    |   24 +-
 .../internal/cli/commands/RegionCommands.java   |   20 +-
 .../internal/cli/commands/ShellCommands.java    |   42 +-
 .../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 +-
 .../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  |   27 +-
 .../internal/cli/domain/MemberResult.java       |   21 +-
 .../cli/domain/PartitionAttributesInfo.java     |   92 +-
 .../cli/domain/RegionAttributesInfo.java        |   55 +-
 .../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 +
 .../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          |   29 +-
 .../functions/CreateDefinedIndexesFunction.java |   16 +
 .../cli/functions/CreateDiskStoreFunction.java  |   21 +-
 .../cli/functions/CreateIndexFunction.java      |   21 +-
 .../cli/functions/DataCommandFunction.java      |   27 +-
 .../internal/cli/functions/DeployFunction.java  |   21 +-
 .../functions/DescribeDiskStoreFunction.java    |   18 +-
 .../functions/DescribeHDFSStoreFunction.java    |   87 +
 .../cli/functions/DestroyDiskStoreFunction.java |   21 +-
 .../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 |   26 +-
 .../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/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     |   38 +-
 .../cli/functions/RegionDestroyFunction.java    |   20 +-
 .../cli/functions/RegionFunctionArgs.java       |   67 +-
 .../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           |  137 +-
 .../internal/cli/json/GfJsonArray.java          |   20 +-
 .../internal/cli/json/GfJsonException.java      |   22 +-
 .../internal/cli/json/GfJsonObject.java         |   20 +-
 .../management/internal/cli/json/TypedJson.java |   22 +-
 .../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       |   29 +-
 .../internal/cli/result/TableBuilderHelper.java |   39 +-
 .../internal/cli/result/TabularResultData.java  |   20 +-
 .../management/internal/cli/shell/Gfsh.java     |   80 +-
 .../internal/cli/shell/GfshConfig.java          |   70 +-
 .../cli/shell/GfshExecutionStrategy.java        |   24 +-
 .../cli/shell/JMXConnectionException.java       |   20 +-
 .../cli/shell/JMXInvocationException.java       |   20 +-
 .../internal/cli/shell/JmxOperationInvoker.java |   30 +-
 .../internal/cli/shell/MultiCommandHelper.java  |   16 +
 .../internal/cli/shell/OperationInvoker.java    |   20 +-
 .../internal/cli/shell/jline/ANSIBuffer.java    |  433 +
 .../internal/cli/shell/jline/ANSIHandler.java   |   25 +-
 .../cli/shell/jline/CygwinMinttyTerminal.java   |  158 +-
 .../internal/cli/shell/jline/GfshHistory.java   |   44 +-
 .../shell/jline/GfshUnsupportedTerminal.java    |   22 +-
 .../cli/shell/unsafe/GfshSignalHandler.java     |   21 +-
 .../internal/cli/util/CLIConsoleBufferUtil.java |   29 +-
 .../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    |   47 +
 .../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       |   24 +-
 .../cli/util/RegionAttributesNames.java         |   24 +-
 .../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          |   25 +-
 .../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  |   18 +-
 .../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  |   31 +-
 .../controllers/ConfigCommandsController.java   |   27 +-
 .../web/controllers/DataCommandsController.java |   21 +-
 .../controllers/DeployCommandsController.java   |   21 +-
 .../DiskStoreCommandsController.java            |   21 +-
 .../DurableClientCommandsController.java        |   21 +-
 .../controllers/FunctionCommandsController.java |   21 +-
 .../controllers/IndexCommandsController.java    |   21 +-
 .../LauncherLifecycleCommandsController.java    |   21 +-
 .../controllers/MemberCommandsController.java   |   21 +-
 .../MiscellaneousCommandsController.java        |   21 +-
 .../web/controllers/PdxCommandsController.java  |   16 +
 .../controllers/QueueCommandsController.java    |   27 +-
 .../controllers/RegionCommandsController.java   |   21 +-
 .../controllers/ShellCommandsController.java    |  221 +-
 .../web/controllers/WanCommandsController.java  |   23 +-
 .../EnvironmentVariablesHandlerInterceptor.java |   21 +-
 .../support/MemberMXBeanAdapter.java            |   92 +-
 .../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     |   23 +-
 .../pdx/ReflectionBasedAutoSerializer.java      |   30 +-
 .../gemfire/pdx/WritablePdxInstance.java        |   21 +-
 .../pdx/internal/AutoSerializableManager.java   |   34 +-
 .../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   |   28 +-
 .../gemstone/gemfire/pdx/internal/EnumId.java   |   21 +-
 .../gemstone/gemfire/pdx/internal/EnumInfo.java |   23 +-
 .../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    |   29 +-
 .../gemfire/pdx/internal/PdxInstanceEnum.java   |   23 +-
 .../pdx/internal/PdxInstanceFactoryImpl.java    |   24 +-
 .../gemfire/pdx/internal/PdxInstanceImpl.java   |   41 +-
 .../pdx/internal/PdxInstanceInputStream.java    |   21 +-
 .../gemfire/pdx/internal/PdxOutputStream.java   |   30 +-
 .../gemfire/pdx/internal/PdxReaderImpl.java     |   31 +-
 .../gemfire/pdx/internal/PdxString.java         |   25 +-
 .../gemstone/gemfire/pdx/internal/PdxType.java  |   21 +-
 .../gemfire/pdx/internal/PdxUnreadData.java     |   24 +-
 .../gemfire/pdx/internal/PdxWriterImpl.java     |   24 +-
 .../pdx/internal/PeerTypeRegistration.java      |   38 +-
 .../pdx/internal/TrackingPdxReaderImpl.java     |   21 +-
 .../gemfire/pdx/internal/TypeRegistration.java  |   21 +-
 .../gemfire/pdx/internal/TypeRegistry.java      |   40 +-
 .../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      |   24 +-
 .../java/com/gemstone/gemfire/pdx/package.html  |   15 +
 .../com/gemstone/gemfire/ra/GFConnection.java   |   16 +
 .../gemfire/ra/GFConnectionFactory.java         |   16 +
 .../gemfire/redis/GemFireRedisServer.java       |  718 ++
 .../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 ---
 .../config/GemFireFileConfigurationMonitor.java |  145 -
 .../config/xml/GemFireXmlConfiguration.java     |  344 -
 .../xml/GemFireXmlConfigurationFactory.java     |   59 -
 .../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                 | 1519 +++
 .../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 -
 .../tools/gfsh/app/windowsbindings.properties   |   15 +
 .../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   |  188 +
 .../java/batterytest/greplogs/LogConsumer.java  |  290 +
 .../src/test/java/cacheRunner/Portfolio.java    |  129 +
 .../src/test/java/cacheRunner/Position.java     |   74 +
 .../src/test/java/com/company/app/Customer.java |   37 +
 .../src/test/java/com/company/app/DBLoader.java |   64 +
 .../com/company/app/OrdersCacheListener.java    |   52 +
 .../java/com/company/data/DatabaseLoader.java   |   43 +
 .../java/com/company/data/MyDeclarable.java     |   33 +
 .../src/test/java/com/company/data/MySizer.java |   40 +
 .../com/company/data/MyTransactionListener.java |   39 +
 .../src/test/java/com/examples/LinkNode.java    |   78 +
 .../src/test/java/com/examples/SuperClass.java  |   93 +
 .../src/test/java/com/examples/TestObject.java  |   53 +
 .../src/test/java/com/examples/ds/Address.java  |   23 +
 .../src/test/java/com/examples/ds/Company.java  |   43 +
 .../java/com/examples/ds/CompanySerializer.java |   67 +
 .../src/test/java/com/examples/ds/Employee.java |   53 +
 .../com/examples/ds/PutDataSerializables.java   |   55 +
 .../src/test/java/com/examples/ds/User.java     |   58 +
 .../com/examples/snapshot/MyDataSerializer.java |   63 +
 .../java/com/examples/snapshot/MyObject.java    |   69 +
 .../snapshot/MyObjectDataSerializable.java      |   45 +
 .../java/com/examples/snapshot/MyObjectPdx.java |   33 +
 .../snapshot/MyObjectPdxSerializable.java       |   42 +
 .../com/examples/snapshot/MyPdxSerializer.java  |   64 +
 .../java/com/gemstone/gemfire/AppObject.java    |   24 +
 .../test/java/com/gemstone/gemfire/BadTest.java |   42 +
 .../com/gemstone/gemfire/CopyJUnitTest.java     |  575 ++
 .../com/gemstone/gemfire/DeltaTestImpl.java     |  397 +
 .../gemfire/DiskInstantiatorsJUnitTest.java     |  183 +
 .../com/gemstone/gemfire/GemFireTestCase.java   |   92 +
 .../java/com/gemstone/gemfire/Invariant.java    |   32 +
 .../com/gemstone/gemfire/InvariantResult.java   |   28 +
 .../com/gemstone/gemfire/JUnitTestSetup.java    |  143 +
 .../gemfire/JtaNoninvolvementJUnitTest.java     |  180 +
 .../gemfire/LocalStatisticsJUnitTest.java       |   50 +
 .../com/gemstone/gemfire/LonerDMJUnitTest.java  |  196 +
 .../gemstone/gemfire/StatisticsTestCase.java    |  388 +
 .../gemfire/StatisticsTypeJUnitTest.java        |   77 +
 .../com/gemstone/gemfire/TXExpiryJUnitTest.java |  421 +
 .../java/com/gemstone/gemfire/TXJUnitTest.java  | 6825 +++++++++++++
 .../com/gemstone/gemfire/TXWriterJUnitTest.java |  241 +
 .../gemstone/gemfire/TXWriterOOMEJUnitTest.java |   78 +
 .../com/gemstone/gemfire/TXWriterTestCase.java  |  152 +
 .../gemstone/gemfire/TestDataSerializer.java    |  117 +
 .../com/gemstone/gemfire/TimingTestCase.java    |   77 +
 .../com/gemstone/gemfire/UnitTestDoclet.java    |  259 +
 .../gemstone/gemfire/admin/AdminTestHelper.java |   16 +
 ...ibutedSystemFactoryIntegrationJUnitTest.java |   76 -
 .../DistributedSystemFactoryJUnitTest.java      |  149 -
 .../BindDistributedSystemJUnitTest.java         |   97 +
 .../internal/CacheHealthEvaluatorJUnitTest.java |  208 +
 .../internal/DistributedSystemTestCase.java     |   67 +
 .../admin/internal/HealthEvaluatorTestCase.java |   76 +
 .../MemberHealthEvaluatorJUnitTest.java         |  102 +
 .../cache/AttributesFactoryJUnitTest.java       |   41 +-
 .../gemfire/cache/Bug36619JUnitTest.java        |   23 +-
 .../gemfire/cache/Bug42039JUnitTest.java        |   96 +
 .../gemfire/cache/Bug52289JUnitTest.java        |   89 +
 .../gemfire/cache/CacheListenerJUnitTest.java   |  336 +
 .../cache/CacheRegionClearStatsDUnitTest.java   |  247 +
 .../gemstone/gemfire/cache/ClientHelper.java    |   73 +
 .../cache/ClientServerTimeSyncDUnitTest.java    |  199 +
 .../cache/ConnectionPoolAndLoaderDUnitTest.java |  488 +
 .../cache/ConnectionPoolFactoryJUnitTest.java   |  424 +
 .../gemfire/cache/OperationJUnitTest.java       |   23 +-
 .../gemfire/cache/PoolManagerJUnitTest.java     |  135 +
 .../gemstone/gemfire/cache/ProxyJUnitTest.java  | 1169 +++
 .../gemfire/cache/RegionFactoryJUnitTest.java   | 1195 +++
 .../gemfire/cache/RoleExceptionJUnitTest.java   |   23 +-
 .../SerialAsyncEventQueueImplJUnitTest.java     |   62 +
 .../client/ClientCacheFactoryJUnitTest.java     |  343 +
 .../client/ClientRegionFactoryJUnitTest.java    |  532 +
 .../ClientServerRegisterInterestsDUnitTest.java |  257 +
 .../internal/AutoConnectionSourceDUnitTest.java |  596 ++
 .../AutoConnectionSourceImplJUnitTest.java      |  398 +
 .../CacheServerSSLConnectionDUnitTest.java      |  426 +
 .../internal/ConnectionPoolImplJUnitTest.java   |  260 +
 .../internal/LocatorLoadBalancingDUnitTest.java |  496 +
 .../cache/client/internal/LocatorTestBase.java  |  357 +
 .../internal/OpExecutorImplJUnitTest.java       |  182 +-
 .../client/internal/QueueManagerJUnitTest.java  |  674 ++
 .../internal/SSLNoClientAuthDUnitTest.java      |  280 +
 .../internal/ServerBlackListJUnitTest.java      |   82 +-
 .../locator/LocatorStatusResponseJUnitTest.java |   22 +-
 .../pooling/ConnectionManagerJUnitTest.java     |  878 ++
 .../SignalledFlushObserverJUnitTest.java        |   97 +
 .../SortedListForAsyncQueueJUnitTest.java       |  563 ++
 .../management/MXMemoryPoolListenerExample.java |  210 +
 .../management/MemoryThresholdsDUnitTest.java   | 2328 +++++
 .../MemoryThresholdsOffHeapDUnitTest.java       | 1827 ++++
 .../management/ResourceManagerDUnitTest.java    | 1841 ++++
 .../ExceptionHandlingJUnitTest.java             |  177 +
 .../mapInterface/MapFunctionalJUnitTest.java    |  170 +
 .../mapInterface/PutAllGlobalLockJUnitTest.java |  110 +
 .../PutOperationContextJUnitTest.java           |  264 +
 .../GetOperationContextImplJUnitTest.java       |  291 +
 .../PartitionRegionHelperDUnitTest.java         |  742 ++
 .../BaseLineAndCompareQueryPerfJUnitTest.java   |  537 +
 .../query/Bug32947ValueConstraintJUnitTest.java |  127 +
 .../gemfire/cache/query/BugJUnitTest.java       |  570 ++
 .../gemfire/cache/query/CacheUtils.java         |  398 +
 .../cache/query/PdxStringQueryJUnitTest.java    |  727 ++
 .../gemstone/gemfire/cache/query/PerfQuery.java |  330 +
 .../gemfire/cache/query/QueryJUnitTest.java     |  457 +
 .../cache/query/QueryServiceJUnitTest.java      |  252 +
 .../gemfire/cache/query/QueryTestUtils.java     | 1651 ++++
 .../cache/query/QueryTestUtilsJUnitTest.java    |  134 +
 .../gemfire/cache/query/RegionJUnitTest.java    |  226 +
 .../cache/query/TypedIteratorJUnitTest.java     |  133 +
 .../com/gemstone/gemfire/cache/query/Utils.java |   57 +
 .../query/cq/dunit/CqQueryTestListener.java     |  399 +
 .../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     |   34 +-
 .../gemfire/cache/query/data/PortfolioData.java |   21 +-
 .../gemfire/cache/query/data/PortfolioNoDS.java |  240 +
 .../gemfire/cache/query/data/PortfolioPdx.java  |   33 +-
 .../gemfire/cache/query/data/Position.java      |   21 +-
 .../gemfire/cache/query/data/PositionNoDS.java  |  130 +
 .../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    |   62 +
 .../query/dunit/CompactRangeIndexDUnitTest.java |  200 +
 .../cache/query/dunit/CqTimeTestListener.java   |  267 +
 .../cache/query/dunit/GroupByDUnitImpl.java     |  329 +
 .../dunit/GroupByPartitionedQueryDUnitTest.java |  111 +
 .../query/dunit/GroupByQueryDUnitTest.java      |  190 +
 .../cache/query/dunit/HashIndexDUnitTest.java   |  131 +
 .../cache/query/dunit/HelperTestCase.java       |  267 +
 .../dunit/NonDistinctOrderByDUnitImpl.java      |  267 +
 .../NonDistinctOrderByPartitionedDUnitTest.java |  155 +
 .../query/dunit/PdxStringQueryDUnitTest.java    | 1983 ++++
 .../dunit/QueryAPITestPartitionResolver.java    |   44 +
 .../cache/query/dunit/QueryAuthorization.java   |   70 +
 .../dunit/QueryDataInconsistencyDUnitTest.java  |  574 ++
 .../dunit/QueryIndexUsingXMLDUnitTest.java      |  989 ++
 .../QueryParamsAuthorizationDUnitTest.java      |  111 +
 .../QueryUsingFunctionContextDUnitTest.java     | 1046 ++
 .../query/dunit/QueryUsingPoolDUnitTest.java    | 2588 +++++
 .../cache/query/dunit/RemoteQueryDUnitTest.java | 1494 +++
 ...esourceManagerWithQueryMonitorDUnitTest.java | 1164 +++
 .../query/dunit/SelectStarQueryDUnitTest.java   | 1623 ++++
 .../cache/query/facets/lang/Address.java        |   60 +
 .../gemfire/cache/query/facets/lang/Course.java |   91 +
 .../cache/query/facets/lang/Department.java     |   96 +
 .../query/facets/lang/DerivedEmployee.java      |   29 +
 .../cache/query/facets/lang/Employee.java       |  140 +
 .../cache/query/facets/lang/Faculty.java        |  134 +
 .../cache/query/facets/lang/G_Student.java      |   66 +
 .../gemfire/cache/query/facets/lang/Person.java |   96 +
 .../cache/query/facets/lang/Student.java        |  103 +
 .../cache/query/facets/lang/UG_Student.java     |   66 +
 .../gemfire/cache/query/facets/lang/Utils.java  |  134 +
 .../ComparisonOperatorsJUnitTest.java           |  185 +
 .../query/functional/ConstantsJUnitTest.java    |  109 +
 .../query/functional/CountStarJUnitTest.java    |  689 ++
 .../CustomerOptimizationsJUnitTest.java         | 1386 +++
 .../DistinctAndNonDistinctQueryJUnitTest.java   |  154 +
 ...ctResultsWithDupValuesInRegionJUnitTest.java |  484 +
 .../query/functional/FunctionJUnitTest.java     |  315 +
 .../functional/GroupByPartitionedJUnitTest.java |   50 +
 .../functional/GroupByReplicatedJUnitTest.java  |   39 +
 .../cache/query/functional/GroupByTestImpl.java | 1010 ++
 .../query/functional/GroupByTestInterface.java  |   58 +
 .../query/functional/INOperatorJUnitTest.java   |  471 +
 .../functional/IUM6Bug32345ReJUnitTest.java     |  224 +
 .../cache/query/functional/IUMJUnitTest.java    |  237 +
 .../IUMRCompositeIteratorJUnitTest.java         |  251 +
 .../IUMRMultiIndexesMultiRegionJUnitTest.java   | 1274 +++
 .../IUMRShuffleIteratorsJUnitTest.java          |  489 +
 .../functional/IUMRSingleRegionJUnitTest.java   |  901 ++
 ...ependentOperandsInWhereClause2JUnitTest.java |  200 +
 .../IndexCreationDeadLockJUnitTest.java         |  310 +
 .../functional/IndexCreationJUnitTest.java      | 1088 +++
 .../IndexMaintenanceAsynchJUnitTest.java        |  179 +
 .../functional/IndexOnEntrySetJUnitTest.java    |  335 +
 .../functional/IndexOperatorJUnitTest.java      |  225 +
 .../IndexPrimaryKeyUsageJUnitTest.java          |  335 +
 .../IndexUsageInNestedQueryJUnitTest.java       |  152 +
 .../IndexUsageWithAliasAsProjAtrbt.java         |  177 +
 ...IndexUsageWithAliasAsProjAtrbtJUnitTest.java |  185 +
 .../IndexUseMultFrmSnglCondJUnitTest.java       |  274 +
 ...ndexWithSngleFrmAndMultCondQryJUnitTest.java | 1354 +++
 .../functional/IteratorTypeDefEmpJUnitTest.java |   87 +
 .../functional/IteratorTypeDefJUnitTest.java    |  163 +
 .../IteratorTypeDefaultTypesJUnitTest.java      |  426 +
 .../functional/IumMultConditionJUnitTest.java   |  231 +
 .../functional/JavaSerializationJUnitTest.java  |   68 +
 .../functional/LikePredicateJUnitTest.java      | 2387 +++++
 .../query/functional/LimitClauseJUnitTest.java  | 1850 ++++
 .../functional/LogicalOperatorsJUnitTest.java   |  201 +
 .../cache/query/functional/MiscJUnitTest.java   |  726 ++
 .../functional/MultiIndexCreationJUnitTest.java |  385 +
 .../MultiRegionIndexUsageJUnitTest.java         |  903 ++
 .../functional/MultipleRegionsJUnitTest.java    |  109 +
 .../NegativeNumberQueriesJUnitTest.java         |  127 +
 .../query/functional/NestedQueryJUnitTest.java  |  511 +
 .../NonDistinctOrderByPartitionedJUnitTest.java |  260 +
 .../NonDistinctOrderByReplicatedJUnitTest.java  |  411 +
 .../NonDistinctOrderByTestImplementation.java   | 1545 +++
 .../query/functional/NumericQueryJUnitTest.java |  337 +
 .../functional/OrderByPartitionedJUnitTest.java |  667 ++
 .../functional/OrderByReplicatedJUnitTest.java  |  836 ++
 .../functional/OrderByTestImplementation.java   | 1184 +++
 .../functional/ParameterBindingJUnitTest.java   |  174 +
 .../PdxGroupByPartitionedJUnitTest.java         |   40 +
 .../PdxGroupByReplicatedJUnitTest.java          |   37 +
 .../query/functional/PdxGroupByTestImpl.java    | 1025 ++
 .../query/functional/PdxOrderByJUnitTest.java   |  357 +
 .../functional/QRegionInterfaceJUnitTest.java   |  164 +
 .../QueryREUpdateInProgressJUnitTest.java       |  436 +
 .../functional/QueryUndefinedJUnitTest.java     |  286 +
 .../functional/ReservedKeywordsJUnitTest.java   |   82 +
 .../ResultsDataSerializabilityJUnitTest.java    |  144 +
 .../query/functional/SelectToDateJUnitTest.java |  263 +
 .../functional/StructMemberAccessJUnitTest.java |  287 +
 .../query/functional/StructSetOrResultsSet.java |  456 +
 .../query/functional/TestNewFunctionSSorRS.java |  129 +
 .../CompiledAggregateFunctionJUnitTest.java     |  200 +
 .../CompiledGroupBySelectJUnitTest.java         |  161 +
 .../CompiledJunctionInternalsJUnitTest.java     | 3481 +++++++
 .../internal/CopyOnReadQueryJUnitTest.java      |  418 +
 .../internal/ExecutionContextJUnitTest.java     |  406 +
 .../query/internal/IndexManagerJUnitTest.java   |  182 +
 .../internal/NWayMergeResultsJUnitTest.java     |  551 ++
 .../internal/OrderByComparatorJUnitTest.java    |  231 +-
 .../internal/ProjectionAttributeJUnitTest.java  |  241 +
 .../query/internal/QCompilerJUnitTest.java      |  451 +
 ...ueryFromClauseCanonicalizationJUnitTest.java |  264 +
 .../QueryObjectSerializationJUnitTest.java      |   23 +-
 .../QueryObserverCallbackJUnitTest.java         |  424 +
 .../query/internal/QueryTraceJUnitTest.java     |  449 +
 .../query/internal/QueryUtilsJUnitTest.java     |  308 +
 .../query/internal/ResultsBagJUnitTest.java     |   23 +-
 .../ResultsBagLimitBehaviourJUnitTest.java      |   35 +-
 .../ResultsCollectionWrapperLimitJUnitTest.java |   23 +-
 .../SelectResultsComparatorJUnitTest.java       |   23 +-
 .../StructBagLimitBehaviourJUnitTest.java       |   31 +-
 .../query/internal/StructSetJUnitTest.java      |   23 +-
 .../internal/aggregate/AggregatorJUnitTest.java |  215 +
 ...syncIndexUpdaterThreadShutdownJUnitTest.java |  123 +
 .../index/AsynchIndexMaintenanceJUnitTest.java  |  261 +
 .../CompactRangeIndexIndexMapJUnitTest.java     |  187 +
 .../index/CompactRangeIndexJUnitTest.java       |  481 +
 ...rrentIndexInitOnOverflowRegionDUnitTest.java |  470 +
 ...ndexOperationsOnOverflowRegionDUnitTest.java |  727 ++
 ...pdateWithInplaceObjectModFalseDUnitTest.java |  713 ++
 ...ConcurrentIndexUpdateWithoutWLDUnitTest.java |  799 ++
 .../index/CopyOnReadIndexDUnitTest.java         |  647 ++
 .../index/CopyOnReadIndexJUnitTest.java         |  457 +
 .../DeclarativeIndexCreationJUnitTest.java      |  123 +
 .../internal/index/HashIndexJUnitTest.java      | 1592 +++
 .../internal/index/HashIndexSetJUnitTest.java   |  504 +
 .../index/IndexCreationInternalsJUnitTest.java  |  215 +
 .../internal/index/IndexElemArrayJUnitTest.java |  260 +-
 .../internal/index/IndexHintJUnitTest.java      |  506 +
 .../query/internal/index/IndexJUnitTest.java    |   97 +
 .../index/IndexMaintainceJUnitTest.java         |  514 +
 .../index/IndexMaintenanceJUnitTest.java        | 1439 +++
 .../index/IndexStatisticsJUnitTest.java         |  862 ++
 .../IndexTrackingQueryObserverDUnitTest.java    |  326 +
 .../IndexTrackingQueryObserverJUnitTest.java    |  172 +
 .../query/internal/index/IndexUseJUnitTest.java | 1890 ++++
 .../IndexedMergeEquiJoinScenariosJUnitTest.java |  604 ++
 ...itializeIndexEntryDestroyQueryDUnitTest.java |  453 +
 .../internal/index/MapIndexStoreJUnitTest.java  |  339 +
 .../MapRangeIndexMaintenanceJUnitTest.java      |  389 +
 .../index/MultiIndexCreationDUnitTest.java      |  196 +
 .../NewDeclarativeIndexCreationJUnitTest.java   |  173 +
 .../index/PdxCopyOnReadQueryJUnitTest.java      |   77 +
 ...gRegionCreationIndexUpdateTypeJUnitTest.java |   98 +
 .../PutAllWithIndexPerfDUnitDisabledTest.java   |  215 +
 .../internal/index/RangeIndexAPIJUnitTest.java  |  422 +
 .../PRBasicIndexCreationDUnitTest.java          | 1049 ++
 .../PRBasicIndexCreationDeadlockDUnitTest.java  |  235 +
 .../PRBasicMultiIndexCreationDUnitTest.java     | 1065 ++
 .../partitioned/PRBasicQueryDUnitTest.java      |  273 +
 .../PRBasicRemoveIndexDUnitTest.java            |  142 +
 .../PRColocatedEquiJoinDUnitTest.java           | 1720 ++++
 .../partitioned/PRIndexStatisticsJUnitTest.java |  704 ++
 .../partitioned/PRInvalidQueryDUnitTest.java    |  133 +
 .../partitioned/PRInvalidQueryJUnitTest.java    |  127 +
 .../partitioned/PRQueryCacheCloseDUnitTest.java |  339 +
 .../PRQueryCacheClosedJUnitTest.java            |  267 +
 .../query/partitioned/PRQueryDUnitHelper.java   | 3085 ++++++
 .../query/partitioned/PRQueryDUnitTest.java     | 1227 +++
 .../query/partitioned/PRQueryJUnitTest.java     |  188 +
 .../partitioned/PRQueryNumThreadsJUnitTest.java |  165 +
 .../query/partitioned/PRQueryPerfDUnitTest.java |  506 +
 .../PRQueryRegionCloseDUnitTest.java            |  209 +
 .../PRQueryRegionClosedJUnitTest.java           |  263 +
 .../PRQueryRegionDestroyedDUnitTest.java        |  223 +
 .../PRQueryRegionDestroyedJUnitTest.java        |  260 +
 .../PRQueryRemoteNodeExceptionDUnitTest.java    |  786 ++
 .../gemfire/cache/query/transaction/Person.java |   57 +
 .../query/transaction/QueryAndJtaJUnitTest.java |  480 +
 .../internal/ConnectionCountProbeJUnitTest.java |   23 +-
 .../cache/snapshot/CacheSnapshotJUnitTest.java  |  128 +
 .../snapshot/ParallelSnapshotDUnitTest.java     |  199 +
 .../gemfire/cache/snapshot/RegionGenerator.java |  125 +
 .../cache/snapshot/RegionSnapshotJUnitTest.java |  294 +
 .../snapshot/SnapshotByteArrayDUnitTest.java    |  138 +
 .../cache/snapshot/SnapshotDUnitTest.java       |  225 +
 .../snapshot/SnapshotPerformanceDUnitTest.java  |  171 +
 .../cache/snapshot/SnapshotTestCase.java        |   92 +
 .../cache/snapshot/WanSnapshotJUnitTest.java    |   98 +
 .../cache/util/PasswordUtilJUnitTest.java       |   23 +-
 .../gemfire/cache30/Bug34387DUnitTest.java      |  165 +
 .../gemfire/cache30/Bug34948DUnitTest.java      |  136 +
 .../gemfire/cache30/Bug35214DUnitTest.java      |  197 +
 .../gemfire/cache30/Bug38013DUnitTest.java      |  130 +
 .../gemfire/cache30/Bug38741DUnitTest.java      |  412 +
 .../cache30/Bug40255JUnitDisabledTest.java      |  139 +
 .../cache30/Bug40662JUnitDisabledTest.java      |   92 +
 .../gemfire/cache30/Bug44418JUnitTest.java      |  173 +
 .../gemfire/cache30/CacheCloseDUnitTest.java    |  100 +
 .../gemfire/cache30/CacheListenerTestCase.java  |  353 +
 .../gemfire/cache30/CacheLoaderTestCase.java    |  322 +
 .../gemfire/cache30/CacheLogRollDUnitTest.java  |  471 +
 .../gemfire/cache30/CacheMapTxnDUnitTest.java   |  528 +
 ...cheRegionsReliablityStatsCheckDUnitTest.java |  136 +
 .../cache30/CacheSerializableRunnable.java      |  112 +
 .../cache30/CacheStatisticsDUnitTest.java       |  507 +
 .../gemstone/gemfire/cache30/CacheTestCase.java |  639 ++
 .../gemfire/cache30/CacheWriterTestCase.java    |  507 +
 .../cache30/CacheXMLPartitionResolver.java      |   78 +
 .../gemfire/cache30/CacheXml30DUnitTest.java    |  818 ++
 .../gemfire/cache30/CacheXml40DUnitTest.java    |  150 +
 .../gemfire/cache30/CacheXml41DUnitTest.java    |  620 ++
 .../gemfire/cache30/CacheXml45DUnitTest.java    |  399 +
 .../gemfire/cache30/CacheXml51DUnitTest.java    |  352 +
 .../gemfire/cache30/CacheXml55DUnitTest.java    |   53 +
 .../gemfire/cache30/CacheXml57DUnitTest.java    |  650 ++
 .../gemfire/cache30/CacheXml58DUnitTest.java    |  505 +
 .../gemfire/cache30/CacheXml60DUnitTest.java    |  334 +
 .../gemfire/cache30/CacheXml61DUnitTest.java    |  126 +
 .../gemfire/cache30/CacheXml65DUnitTest.java    | 1157 +++
 .../gemfire/cache30/CacheXml66DUnitTest.java    |  371 +
 .../gemfire/cache30/CacheXml70DUnitTest.java    |  289 +
 .../gemfire/cache30/CacheXml80DUnitTest.java    |  285 +
 .../gemfire/cache30/CacheXml81DUnitTest.java    |  159 +
 .../gemfire/cache30/CacheXml90DUnitTest.java    |  222 +
 .../gemfire/cache30/CacheXmlTestCase.java       |  152 +
 .../cache30/CachedAllEventsDUnitTest.java       |  103 +
 .../gemfire/cache30/CallbackArgDUnitTest.java   |  168 +
 .../cache30/CertifiableTestCacheListener.java   |  149 +
 .../cache30/ClearMultiVmCallBkDUnitTest.java    |  243 +
 .../gemfire/cache30/ClearMultiVmDUnitTest.java  |  459 +
 .../cache30/ClientMembershipDUnitTest.java      | 1660 ++++
 .../ClientMembershipSelectorDUnitTest.java      |   32 +
 .../ClientRegisterInterestDUnitTest.java        |  427 +
 ...ClientRegisterInterestSelectorDUnitTest.java |   32 +
 .../cache30/ClientServerCCEDUnitTest.java       |  628 ++
 .../gemfire/cache30/ClientServerTestCase.java   |  385 +
 .../ConcurrentLeaveDuringGIIDUnitTest.java      |  214 +
 ...ibutedNoAckAsyncOverflowRegionDUnitTest.java |   61 +
 ...iskDistributedNoAckAsyncRegionDUnitTest.java |   53 +
 .../DiskDistributedNoAckRegionTestCase.java     |   42 +
 ...ributedNoAckSyncOverflowRegionDUnitTest.java |   63 +
 .../gemfire/cache30/DiskRegionDUnitTest.java    | 1611 +++
 .../gemfire/cache30/DiskRegionTestImpl.java     |  238 +
 .../cache30/DistAckMapMethodsDUnitTest.java     |  693 ++
 ...istributedAckOverflowRegionCCEDUnitTest.java |   82 +
 ...tedAckOverflowRegionCCEOffHeapDUnitTest.java |   83 +
 ...tributedAckPersistentRegionCCEDUnitTest.java |  230 +
 ...dAckPersistentRegionCCEOffHeapDUnitTest.java |   86 +
 .../DistributedAckRegionCCEDUnitTest.java       |  693 ++
 ...DistributedAckRegionCCEOffHeapDUnitTest.java |   86 +
 ...istributedAckRegionCompressionDUnitTest.java |   73 +
 .../cache30/DistributedAckRegionDUnitTest.java  |  152 +
 .../DistributedAckRegionOffHeapDUnitTest.java   |   86 +
 .../DistributedMulticastRegionDUnitTest.java    |  213 +
 .../DistributedNoAckRegionCCEDUnitTest.java     |  584 ++
 ...stributedNoAckRegionCCEOffHeapDUnitTest.java |   86 +
 .../DistributedNoAckRegionDUnitTest.java        |  298 +
 .../DistributedNoAckRegionOffHeapDUnitTest.java |   86 +
 .../gemfire/cache30/DynamicRegionDUnitTest.java |  279 +
 .../gemfire/cache30/GlobalLockingDUnitTest.java |  379 +
 .../cache30/GlobalRegionCCEDUnitTest.java       |  245 +
 .../GlobalRegionCCEOffHeapDUnitTest.java        |   86 +
 .../gemfire/cache30/GlobalRegionDUnitTest.java  |  404 +
 .../cache30/GlobalRegionOffHeapDUnitTest.java   |   95 +
 .../cache30/LRUEvictionControllerDUnitTest.java |  488 +
 .../gemfire/cache30/LocalRegionDUnitTest.java   |  258 +
 .../MemLRUEvictionControllerDUnitTest.java      |  280 +
 .../gemfire/cache30/MultiVMRegionTestCase.java  | 9162 ++++++++++++++++++
 .../gemfire/cache30/MyGatewayEventFilter1.java  |   61 +
 .../gemfire/cache30/MyGatewayEventFilter2.java  |   66 +
 .../cache30/MyGatewayTransportFilter1.java      |   53 +
 .../cache30/MyGatewayTransportFilter2.java      |   55 +
 .../OffHeapLRUEvictionControllerDUnitTest.java  |   84 +
 .../PRBucketSynchronizationDUnitTest.java       |  302 +
 .../PartitionedRegionCompressionDUnitTest.java  |   70 +
 .../cache30/PartitionedRegionDUnitTest.java     |  539 ++
 ...tionedRegionMembershipListenerDUnitTest.java |  151 +
 .../PartitionedRegionOffHeapDUnitTest.java      |   85 +
 .../cache30/PreloadedRegionTestCase.java        |   97 +
 .../gemfire/cache30/ProxyDUnitTest.java         |  524 +
 .../cache30/PutAllCallBkRemoteVMDUnitTest.java  |  386 +
 .../cache30/PutAllCallBkSingleVMDUnitTest.java  |  315 +
 .../gemfire/cache30/PutAllMultiVmDUnitTest.java |  353 +
 .../gemfire/cache30/QueueMsgDUnitTest.java      |  293 +
 .../cache30/RRSynchronizationDUnitTest.java     |  265 +
 .../gemfire/cache30/ReconnectDUnitTest.java     | 1126 +++
 .../ReconnectedCacheServerDUnitTest.java        |   68 +
 .../cache30/RegionAttributesTestCase.java       |  266 +
 .../cache30/RegionExpirationDUnitTest.java      |  268 +
 .../RegionMembershipListenerDUnitTest.java      |  409 +
 .../RegionReliabilityDistAckDUnitTest.java      |   39 +
 .../RegionReliabilityDistNoAckDUnitTest.java    |   39 +
 .../RegionReliabilityGlobalDUnitTest.java       |   39 +
 .../RegionReliabilityListenerDUnitTest.java     |  197 +
 .../cache30/RegionReliabilityTestCase.java      | 1413 +++
 .../gemfire/cache30/RegionTestCase.java         | 4110 ++++++++
 .../gemfire/cache30/ReliabilityTestCase.java    |   59 +
 .../cache30/RemoveAllMultiVmDUnitTest.java      |  288 +
 .../gemfire/cache30/RequiredRolesDUnitTest.java |  425 +
 .../cache30/RolePerformanceDUnitTest.java       |  181 +
 .../gemfire/cache30/SearchAndLoadDUnitTest.java |  999 ++
 .../cache30/SlowRecDUnitDisabledTest.java       | 1446 +++
 .../gemfire/cache30/TXDistributedDUnitTest.java | 1522 +++
 .../gemfire/cache30/TXOrderDUnitTest.java       |  411 +
 .../cache30/TXRestrictionsDUnitTest.java        |   94 +
 .../gemfire/cache30/TestCacheCallback.java      |  103 +
 .../gemfire/cache30/TestCacheListener.java      |  222 +
 .../gemfire/cache30/TestCacheLoader.java        |   45 +
 .../gemfire/cache30/TestCacheWriter.java        |  131 +
 .../gemfire/cache30/TestDiskRegion.java         |  257 +
 .../gemstone/gemfire/cache30/TestHeapLRU.java   |   90 +
 .../gemfire/cache30/TestPdxSerializer.java      |   69 +
 .../cache30/TestTransactionListener.java        |   93 +
 .../gemfire/cache30/TestTransactionWriter.java  |   44 +
 .../AnalyzeSerializablesJUnitTest.java          |  325 +
 .../codeAnalysis/ClassAndMethodDetails.java     |  154 +
 .../gemfire/codeAnalysis/ClassAndMethods.java   |   50 +
 .../codeAnalysis/ClassAndVariableDetails.java   |  108 +
 .../gemfire/codeAnalysis/ClassAndVariables.java |   64 +
 .../codeAnalysis/CompiledClassUtils.java        |  434 +
 .../codeAnalysis/decode/CompiledAttribute.java  |   46 +
 .../codeAnalysis/decode/CompiledClass.java      |  276 +
 .../codeAnalysis/decode/CompiledCode.java       |   70 +
 .../codeAnalysis/decode/CompiledField.java      |  116 +
 .../codeAnalysis/decode/CompiledMethod.java     |  153 +
 .../gemfire/codeAnalysis/decode/cp/Cp.java      |   78 +
 .../gemfire/codeAnalysis/decode/cp/CpClass.java |   42 +
 .../codeAnalysis/decode/cp/CpDouble.java        |   40 +
 .../codeAnalysis/decode/cp/CpFieldref.java      |   33 +
 .../gemfire/codeAnalysis/decode/cp/CpFloat.java |   31 +
 .../codeAnalysis/decode/cp/CpInteger.java       |   31 +
 .../decode/cp/CpInterfaceMethodref.java         |   24 +
 .../codeAnalysis/decode/cp/CpInvokeDynamic.java |   33 +
 .../gemfire/codeAnalysis/decode/cp/CpLong.java  |   39 +
 .../codeAnalysis/decode/cp/CpMethodHandle.java  |   33 +
 .../codeAnalysis/decode/cp/CpMethodType.java    |   31 +
 .../codeAnalysis/decode/cp/CpMethodref.java     |   25 +
 .../codeAnalysis/decode/cp/CpNameAndType.java   |   27 +
 .../codeAnalysis/decode/cp/CpString.java        |   25 +
 .../gemfire/codeAnalysis/decode/cp/CpUtf8.java  |  133 +
 .../AbstractLauncherIntegrationJUnitTest.java   |   71 +
 .../distributed/AbstractLauncherJUnitTest.java  |  109 +-
 .../AbstractLauncherJUnitTestCase.java          |  255 +
 .../AbstractLauncherServiceStatusJUnitTest.java |   53 +-
 .../AbstractLocatorLauncherJUnitTestCase.java   |  106 +
 .../AbstractServerLauncherJUnitTestCase.java    |   94 +
 .../gemfire/distributed/AuthInitializer.java    |   45 +
 .../distributed/CommonLauncherTestSuite.java    |   45 -
 .../distributed/DistributedMemberDUnitTest.java |  419 +
 .../DistributedSystemConnectPerf.java           |  135 +
 .../distributed/DistributedSystemDUnitTest.java |  421 +
 .../DistributedSystemIntegrationJUnitTest.java  |   91 +
 .../distributed/DistributedSystemJUnitTest.java |   78 +
 .../distributed/DistributedTestSuite.java       |   36 +
 .../distributed/HostedLocatorsDUnitTest.java    |  195 +
 .../LauncherMemberMXBeanJUnitTest.java          |  153 +
 .../gemfire/distributed/LauncherTestSuite.java  |   48 +
 .../gemfire/distributed/LocatorDUnitTest.java   | 1904 ++++
 .../gemfire/distributed/LocatorJUnitTest.java   |  202 +
 .../LocatorLauncherIntegrationJUnitTest.java    |  248 +
 .../distributed/LocatorLauncherJUnitTest.java   |  241 +-
 .../LocatorLauncherLocalFileJUnitTest.java      |   52 +
 .../LocatorLauncherLocalJUnitTest.java          |  843 ++
 .../LocatorLauncherRemoteFileJUnitTest.java     |  219 +
 .../LocatorLauncherRemoteJUnitTest.java         | 1005 ++
 .../distributed/LocatorStateJUnitTest.java      |  208 +
 .../gemfire/distributed/MyAuthenticator.java    |   54 +
 .../gemfire/distributed/MyPrincipal.java        |   29 +
 .../gemfire/distributed/RoleDUnitTest.java      |  162 +
 .../ServerLauncherIntegrationJUnitTest.java     |  312 +
 .../distributed/ServerLauncherJUnitTest.java    |  266 +-
 .../ServerLauncherLocalFileJUnitTest.java       |   55 +
 .../ServerLauncherLocalJUnitTest.java           | 1076 ++
 .../ServerLauncherRemoteFileJUnitTest.java      |  223 +
 .../ServerLauncherRemoteJUnitTest.java          | 1428 +++
 .../ServerLauncherWithSpringJUnitTest.java      |   99 +
 .../distributed/SystemAdminDUnitTest.java       |  125 +
 .../AtomicLongWithTerminalStateJUnitTest.java   |   23 +-
 .../distributed/internal/Bug40751DUnitTest.java |  140 +
 .../ConsoleDistributionManagerDUnitTest.java    |  444 +
 .../distributed/internal/DateMessage.java       |   98 +
 .../internal/DistributionAdvisorDUnitTest.java  |   94 +
 .../internal/DistributionManagerDUnitTest.java  |  576 ++
 .../InternalDistributedSystemJUnitTest.java     |  831 ++
 .../gemfire/distributed/internal/LDM.java       |   71 +
 .../internal/LocatorLoadSnapshotJUnitTest.java  |  384 +
 .../internal/ProduceDateMessages.java           |   53 +
 .../internal/ProductUseLogDUnitTest.java        |  107 +
 .../internal/ProductUseLogJUnitTest.java        |  149 +-
 .../internal/ServerLocatorJUnitTest.java        |   22 +-
 .../internal/SharedConfigurationJUnitTest.java  |   23 +-
 .../internal/StartupMessageDataJUnitTest.java   |   70 +-
 .../deadlock/DeadlockDetectorJUnitTest.java     |   48 +-
 .../deadlock/DependencyGraphJUnitTest.java      |   23 +-
 .../GemFireDeadlockDetectorDUnitTest.java       |  241 +
 .../deadlock/UnsafeThreadLocalJUnitTest.java    |   23 +-
 .../locks/CollaborationJUnitDisabledTest.java   |  562 ++
 .../internal/locks/DLockGrantorHelper.java      |   98 +
 .../locks/DLockReentrantLockJUnitTest.java      |   84 +
 ...entrantReadWriteWriteShareLockJUnitTest.java |   50 +-
 .../membership/MembershipJUnitTest.java         |  423 +-
 .../internal/membership/NetViewJUnitTest.java   |  274 +
 .../membership/gms/GMSMemberJUnitTest.java      |  164 +
 .../membership/gms/MembershipManagerHelper.java |  172 +
 .../gms/auth/GMSAuthenticatorJUnitTest.java     |  316 +
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |  680 ++
 .../locator/GMSLocatorRecoveryJUnitTest.java    |  186 +
 .../gms/membership/GMSJoinLeaveJUnitTest.java   | 1095 +++
 .../gms/membership/StatRecorderJUnitTest.java   |  227 +
 .../messenger/GMSQuorumCheckerJUnitTest.java    |  361 +
 .../membership/gms/messenger/InterceptUDP.java  |  109 +
 .../messenger/JGroupsMessengerJUnitTest.java    |  836 ++
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |  437 +
 .../StreamingOperationManyDUnitTest.java        |  213 +
 .../StreamingOperationOneDUnitTest.java         |  173 +
 ...cpServerBackwardCompatDUnitDisabledTest.java |  250 +
 .../internal/tcpserver/TcpServerJUnitTest.java  |  221 +
 .../support/DistributedSystemAdapter.java       |   21 +-
 .../gemfire/disttx/CacheMapDistTXDUnitTest.java |   66 +
 .../gemfire/disttx/DistTXDebugDUnitTest.java    | 1018 ++
 .../disttx/DistTXDistributedTestSuite.java      |   42 +
 .../gemfire/disttx/DistTXExpiryJUnitTest.java   |   57 +
 .../gemfire/disttx/DistTXJUnitTest.java         |   96 +
 .../disttx/DistTXManagerImplJUnitTest.java      |   54 +
 .../gemfire/disttx/DistTXOrderDUnitTest.java    |   55 +
 .../disttx/DistTXPersistentDebugDUnitTest.java  |  131 +
 .../DistTXReleasesOffHeapOnCloseJUnitTest.java  |   53 +
 .../disttx/DistTXRestrictionsDUnitTest.java     |   46 +
 .../disttx/DistTXWithDeltaDUnitTest.java        |   38 +
 .../gemfire/disttx/DistTXWriterJUnitTest.java   |   56 +
 .../disttx/DistTXWriterOOMEJUnitTest.java       |   57 +
 .../disttx/DistributedTransactionDUnitTest.java | 2201 +++++
 .../gemfire/disttx/PRDistTXDUnitTest.java       |   65 +
 .../gemfire/disttx/PRDistTXJUnitTest.java       |   92 +
 .../disttx/PRDistTXWithVersionsDUnitTest.java   |   66 +
 ...entPartitionedRegionWithDistTXDUnitTest.java |   46 +
 .../internal/AbstractConfigJUnitTest.java       |  114 +
 .../gemfire/internal/ArrayEqualsJUnitTest.java  |  193 +
 .../gemfire/internal/AvailablePortHelper.java   |  133 +
 .../internal/AvailablePortJUnitTest.java        |   27 +-
 ...wardCompatibilitySerializationDUnitTest.java |  300 +
 .../gemfire/internal/Bug49856JUnitTest.java     |   85 +
 .../gemfire/internal/Bug51616JUnitTest.java     |   61 +
 .../gemfire/internal/ByteArrayData.java         |   21 +-
 .../gemstone/gemfire/internal/ClassBuilder.java |   21 +-
 .../ClassNotFoundExceptionDUnitTest.java        |  243 +
 .../internal/ClassPathLoaderJUnitTest.java      |   23 +-
 .../internal/CopyOnWriteHashSetJUnitTest.java   |   23 +-
 .../internal/DataSerializableJUnitTest.java     | 3671 +++++++
 .../gemfire/internal/FileUtilJUnitTest.java     |   44 +-
 .../internal/GemFireStatSamplerJUnitTest.java   |  618 ++
 .../GemFireVersionIntegrationJUnitTest.java     |   54 +
 .../internal/GemFireVersionJUnitTest.java       |   23 +-
 .../internal/HeapDataOutputStreamJUnitTest.java |  133 +-
 .../gemfire/internal/InlineKeyJUnitTest.java    |  173 +
 .../gemfire/internal/JSSESocketJUnitTest.java   |  248 +
 .../internal/JarClassLoaderJUnitTest.java       |   81 +-
 .../gemfire/internal/JarDeployerDUnitTest.java  |  748 ++
 .../com/gemstone/gemfire/internal/JavaExec.java |   69 +
 .../gemfire/internal/LineWrapUnitJUnitTest.java |   23 +-
 .../gemstone/gemfire/internal/LongBuffer.java   |   96 +
 .../gemfire/internal/NanoTimer2JUnitTest.java   |   79 -
 .../gemfire/internal/NanoTimerJUnitTest.java    |  134 +
 .../gemfire/internal/ObjIdMapJUnitTest.java     |   23 +-
 .../internal/OneTaskOnlyDecoratorJUnitTest.java |   23 +-
 .../internal/PdxDeleteFieldDUnitTest.java       |  214 +
 .../internal/PdxDeleteFieldJUnitTest.java       |  214 +
 .../gemfire/internal/PdxRenameDUnitTest.java    |  218 +
 .../gemfire/internal/PdxRenameJUnitTest.java    |  164 +
 .../PutAllOperationContextJUnitTest.java        |   23 +-
 .../internal/SSLConfigIntegrationJUnitTest.java |   51 +
 .../gemfire/internal/SSLConfigJUnitTest.java    | 1233 +++
 ...hreadPoolExecutorWithKeepAliveJUnitTest.java |  342 +
 .../internal/SimpleStatSamplerJUnitTest.java    |  358 +
 .../gemfire/internal/SocketCloserJUnitTest.java |  196 +
 .../internal/SocketCloserWithWaitJUnitTest.java |   38 +
 .../StatArchiveWriterReaderJUnitTest.java       | 1735 ++++
 .../gemfire/internal/StatSamplerJUnitTest.java  |  361 +
 .../gemfire/internal/StatSamplerTestCase.java   |  185 +
 .../internal/UniqueIdGeneratorJUnitTest.java    |   23 +-
 .../internal/cache/AbstractRegionJUnitTest.java |   23 +-
 .../gemfire/internal/cache/BackupDUnitTest.java |  742 ++
 .../gemfire/internal/cache/BackupJUnitTest.java |  435 +
 .../internal/cache/Bug33359DUnitTest.java       |  157 +
 .../internal/cache/Bug33726DUnitTest.java       |  160 +
 .../internal/cache/Bug33726JUnitTest.java       |  134 +
 .../Bug34179TooManyFilesOpenJUnitTest.java      |  130 +
 .../internal/cache/Bug34583JUnitTest.java       |  101 +
 .../internal/cache/Bug37241DUnitTest.java       |  235 +
 .../internal/cache/Bug37244JUnitTest.java       |  282 +
 .../internal/cache/Bug37377DUnitTest.java       |  381 +
 .../internal/cache/Bug37500JUnitTest.java       |  184 +
 .../internal/cache/Bug39079DUnitTest.java       |  413 +
 .../internal/cache/Bug40299DUnitTest.java       |  304 +
 .../internal/cache/Bug40632DUnitTest.java       |  149 +
 .../internal/cache/Bug41091DUnitTest.java       |  173 +
 .../internal/cache/Bug41733DUnitTest.java       |  222 +
 .../internal/cache/Bug41957DUnitTest.java       |  151 +
 .../internal/cache/Bug42010StatsDUnitTest.java  |  531 +
 .../internal/cache/Bug42055DUnitTest.java       |   89 +
 .../internal/cache/Bug45164DUnitTest.java       |  104 +
 .../internal/cache/Bug45934DUnitTest.java       |  135 +
 .../internal/cache/Bug47667DUnitTest.java       |   88 +
 .../internal/cache/Bug48182JUnitTest.java       |  200 +
 .../internal/cache/CacheAdvisorDUnitTest.java   |  264 +
 .../cache/CacheLifecycleListenerJUnitTest.java  |  324 +
 .../internal/cache/CacheServiceJUnitTest.java   |   59 +
 .../cache/ChunkValueWrapperJUnitTest.java       |  188 +
 .../internal/cache/ClearDAckDUnitTest.java      |  282 +
 .../internal/cache/ClearGlobalDUnitTest.java    |  212 +
 ...ssagesRegionCreationAndDestroyJUnitTest.java |  173 +
 .../cache/ClientServerGetAllDUnitTest.java      |  801 ++
 ...ServerInvalidAndDestroyedEntryDUnitTest.java |  515 +
 .../ClientServerTransactionCCEDUnitTest.java    |  107 +
 .../cache/ClientServerTransactionDUnitTest.java | 3358 +++++++
 .../cache/ComplexDiskRegionJUnitTest.java       |  169 +
 .../ConcurrentDestroySubRegionDUnitTest.java    |  173 +
 ...entFlushingAndRegionOperationsJUnitTest.java |  641 ++
 .../cache/ConcurrentMapLocalJUnitTest.java      |  105 +
 .../cache/ConcurrentMapOpsDUnitTest.java        | 1236 +++
 .../ConcurrentRegionOperationsJUnitTest.java    |  926 ++
 ...rentRollingAndRegionOperationsJUnitTest.java |  997 ++
 .../internal/cache/ConflationJUnitTest.java     |  421 +
 .../cache/ConnectDisconnectDUnitTest.java       |  185 +
 .../cache/CustomerIDPartitionResolver.java      |   86 +
 .../internal/cache/DeltaFaultInDUnitTest.java   |  146 +
 .../cache/DeltaPropagationDUnitTest.java        | 1514 +++
 .../cache/DeltaPropagationStatsDUnitTest.java   |  624 ++
 .../internal/cache/DeltaSizingDUnitTest.java    |  265 +
 .../gemfire/internal/cache/DiskIFJUnitTest.java |  796 ++
 .../gemfire/internal/cache/DiskIdJUnitTest.java |   23 +-
 .../internal/cache/DiskInitFileJUnitTest.java   |   23 +-
 .../cache/DiskOfflineCompactionJUnitTest.java   |  826 ++
 .../internal/cache/DiskOldAPIsJUnitTest.java    |  437 +
 ...iskRandomOperationsAndRecoveryJUnitTest.java |  715 ++
 .../cache/DiskRegByteArrayDUnitTest.java        |  268 +
 .../cache/DiskRegCacheXmlJUnitTest.java         |  299 +
 .../DiskRegCachexmlGeneratorJUnitTest.java      |  302 +
 .../internal/cache/DiskRegCbkChkJUnitTest.java  |  118 +
 .../DiskRegOplogSwtchingAndRollerJUnitTest.java | 1021 ++
 .../cache/DiskRegRecoveryJUnitTest.java         | 1495 +++
 .../cache/DiskRegionAsyncRecoveryJUnitTest.java |  538 +
 ...RegionChangingRegionAttributesJUnitTest.java |  143 +
 .../cache/DiskRegionClearJUnitTest.java         |  289 +
 .../internal/cache/DiskRegionHelperFactory.java |  266 +
 .../DiskRegionIllegalArguementsJUnitTest.java   |  281 +
 ...iskRegionIllegalCacheXMLvaluesJUnitTest.java |  146 +
 .../internal/cache/DiskRegionJUnitTest.java     | 3189 ++++++
 .../internal/cache/DiskRegionProperties.java    |  222 +
 .../internal/cache/DiskRegionTestingBase.java   |  407 +
 .../cache/DiskStoreFactoryJUnitTest.java        |  450 +
 .../cache/DiskWriteAttributesJUnitTest.java     |   23 +-
 ...DistrbutedRegionProfileOffHeapDUnitTest.java |  255 +
 .../cache/DistributedCacheTestCase.java         |  468 +
 .../cache/EnumListenerEventJUnitTest.java       |   23 +-
 .../internal/cache/EventTrackerDUnitTest.java   |  420 +
 .../cache/EvictionDUnitDisabledTest.java        |  240 +
 .../cache/EvictionObjectSizerDUnitTest.java     |  361 +
 .../internal/cache/EvictionStatsDUnitTest.java  |  556 ++
 .../internal/cache/EvictionTestBase.java        |  588 ++
 .../internal/cache/FaultingInJUnitTest.java     |  236 +
 .../cache/FixedPRSinglehopDUnitTest.java        |  891 ++
 .../internal/cache/GIIDeltaDUnitTest.java       | 2625 +++++
 .../internal/cache/GIIFlowControlDUnitTest.java |  452 +
 .../internal/cache/GridAdvisorDUnitTest.java    | 1067 ++
 .../internal/cache/HABug36773DUnitTest.java     |  374 +
 .../HAOverflowMemObjectSizerDUnitTest.java      |  305 +
 .../cache/IncrementalBackupDUnitTest.java       | 1106 +++
 .../cache/InterruptClientServerDUnitTest.java   |  252 +
 .../internal/cache/InterruptDiskJUnitTest.java  |  135 +
 ...InterruptsConserveSocketsFalseDUnitTest.java |   37 +
 .../internal/cache/InterruptsDUnitTest.java     |  214 +
 .../internal/cache/IteratorDUnitTest.java       |  105 +
 .../LIFOEvictionAlgoEnabledRegionJUnitTest.java |  343 +
 ...victionAlgoMemoryEnabledRegionJUnitTest.java |  436 +
 .../internal/cache/MapClearGIIDUnitTest.java    |  280 +
 .../internal/cache/MapInterface2JUnitTest.java  |  282 +
 .../internal/cache/MapInterfaceJUnitTest.java   |  301 +
 .../internal/cache/MockCacheService.java        |   24 +
 .../internal/cache/MockCacheServiceImpl.java    |   39 +
 .../MultipleOplogsRollingFeatureJUnitTest.java  |  257 +
 .../cache/NetSearchMessagingDUnitTest.java      |  440 +
 .../cache/OffHeapEvictionDUnitTest.java         |  136 +
 .../cache/OffHeapEvictionStatsDUnitTest.java    |   90 +
 .../gemfire/internal/cache/OffHeapTestUtil.java |   64 +
 .../cache/OfflineSnapshotJUnitTest.java         |  137 +
 .../gemfire/internal/cache/OldVLJUnitTest.java  |   23 +-
 .../cache/OldValueImporterTestBase.java         |  181 +
 .../cache/OplogEntryIdMapJUnitTest.java         |   23 +-
 .../cache/OplogEntryIdSetJUnitTest.java         |   23 +-
 .../gemfire/internal/cache/OplogJUnitTest.java  | 4034 ++++++++
 .../internal/cache/OplogRVVJUnitTest.java       |   23 +-
 .../cache/OrderedTombstoneMapJUnitTest.java     |   23 +-
 .../cache/P2PDeltaPropagationDUnitTest.java     |  617 ++
 .../internal/cache/PRBadToDataDUnitTest.java    |  109 +
 .../cache/PRConcurrentMapOpsJUnitTest.java      |  231 +
 .../cache/PRDataStoreMemoryJUnitTest.java       |  154 +
 .../PRDataStoreMemoryOffHeapJUnitTest.java      |   51 +
 .../gemfire/internal/cache/PRTXJUnitTest.java   |  150 +
 .../cache/PartitionAttributesImplJUnitTest.java |  543 ++
 .../cache/PartitionListenerDUnitTest.java       |  204 +
 ...dRegionAPIConserveSocketsFalseDUnitTest.java |   45 +
 .../cache/PartitionedRegionAPIDUnitTest.java    | 1512 +++
 .../PartitionedRegionAsSubRegionDUnitTest.java  |  338 +
 ...gionBucketCreationDistributionDUnitTest.java | 1560 +++
 .../PartitionedRegionCacheCloseDUnitTest.java   |  295 +
 ...rtitionedRegionCacheLoaderForRootRegion.java |   62 +
 ...artitionedRegionCacheLoaderForSubRegion.java |   65 +
 ...rtitionedRegionCacheXMLExampleDUnitTest.java |  135 +
 .../PartitionedRegionCreationDUnitTest.java     |  926 ++
 .../PartitionedRegionCreationJUnitTest.java     |  548 ++
 .../cache/PartitionedRegionDUnitTestCase.java   |  536 +
 .../PartitionedRegionDataStoreJUnitTest.java    |  187 +
 ...rtitionedRegionDelayedRecoveryDUnitTest.java |  312 +
 .../PartitionedRegionDestroyDUnitTest.java      |  307 +
 .../PartitionedRegionEntryCountDUnitTest.java   |  149 +
 .../PartitionedRegionEvictionDUnitTest.java     | 1761 ++++
 .../cache/PartitionedRegionHADUnitTest.java     |  505 +
 ...onedRegionHAFailureAndRecoveryDUnitTest.java |  524 +
 .../cache/PartitionedRegionHelperJUnitTest.java |   23 +-
 .../PartitionedRegionInvalidateDUnitTest.java   |  213 +
 ...artitionedRegionLocalMaxMemoryDUnitTest.java |  318 +
 ...nedRegionLocalMaxMemoryOffHeapDUnitTest.java |   77 +
 .../PartitionedRegionMultipleDUnitTest.java     |  596 ++
 ...rtitionedRegionOffHeapEvictionDUnitTest.java |   98 +
 .../cache/PartitionedRegionPRIDDUnitTest.java   |  269 +
 .../cache/PartitionedRegionQueryDUnitTest.java  | 1158 +++
 ...artitionedRegionQueryEvaluatorJUnitTest.java |  307 +
 ...artitionedRegionRedundancyZoneDUnitTest.java |  167 +
 ...tionedRegionSerializableObjectJUnitTest.java |  178 +
 .../PartitionedRegionSingleHopDUnitTest.java    | 2388 +++++
 ...RegionSingleHopWithServerGroupDUnitTest.java | 1731 ++++
 ...onedRegionSingleNodeOperationsJUnitTest.java | 1542 +++
 .../cache/PartitionedRegionSizeDUnitTest.java   |  608 ++
 .../cache/PartitionedRegionStatsDUnitTest.java  |  647 ++
 .../cache/PartitionedRegionStatsJUnitTest.java  |  534 +
 .../cache/PartitionedRegionTestHelper.java      |  354 +
 .../PartitionedRegionTestUtilsDUnitTest.java    |  577 ++
 .../PartitionedRegionWithSameNameDUnitTest.java |  924 ++
 .../PersistentPartitionedRegionJUnitTest.java   |  225 +
 .../internal/cache/PutAllDAckDUnitTest.java     |  225 +
 .../internal/cache/PutAllGlobalDUnitTest.java   |  288 +
 .../cache/RegionEntryFlagsJUnitTest.java        |   97 +
 .../internal/cache/RegionListenerJUnitTest.java |   63 +
 .../cache/RemotePutReplyMessageJUnitTest.java   |   53 +
 .../cache/RemoteTransactionCCEDUnitTest.java    |   35 +
 .../cache/RemoteTransactionDUnitTest.java       | 4326 +++++++++
 .../internal/cache/RemoveAllDAckDUnitTest.java  |  168 +
 .../internal/cache/RemoveDAckDUnitTest.java     |  198 +
 .../internal/cache/RemoveGlobalDUnitTest.java   |  235 +
 .../internal/cache/RunCacheInOldGemfire.java    |  188 +
 .../cache/SimpleDiskRegionJUnitTest.java        |  391 +
 .../internal/cache/SizingFlagDUnitTest.java     | 1039 ++
 .../internal/cache/SnapshotTestUtil.java        |   47 +
 .../internal/cache/SystemFailureDUnitTest.java  |  878 ++
 .../internal/cache/TXManagerImplJUnitTest.java  |  334 +
 .../cache/TXReservationMgrJUnitTest.java        |  151 +
 .../gemfire/internal/cache/TestDelta.java       |   96 +
 .../internal/cache/TestHelperForHydraTests.java |   32 +
 .../internal/cache/TestNonSizerObject.java      |   63 +
 .../internal/cache/TestObjectSizerImpl.java     |   72 +
 .../gemfire/internal/cache/TestUtils.java       |   32 +
 .../cache/TombstoneCreationJUnitTest.java       |  231 +
 .../cache/TransactionsWithDeltaDUnitTest.java   |  376 +
 .../internal/cache/UnitTestValueHolder.java     |   43 +
 .../gemfire/internal/cache/UnzipUtil.java       |   84 +
 .../internal/cache/UpdateVersionJUnitTest.java  |  407 +
 .../gemfire/internal/cache/VLJUnitTest.java     |   23 +-
 .../cache/control/FilterByPathJUnitTest.java    |   23 +-
 .../cache/control/MemoryMonitorJUnitTest.java   |  762 ++
 .../control/MemoryMonitorOffHeapJUnitTest.java  |  338 +
 .../control/MemoryThresholdsJUnitTest.java      |  152 +
 .../control/RebalanceOperationDUnitTest.java    | 3270 +++++++
 .../control/TestMemoryThresholdListener.java    |  169 +
 ...skRegOverflowAsyncGetInMemPerfJUnitTest.java |  136 +
 ...iskRegOverflowAsyncJUnitPerformanceTest.java |  167 +
 ...lowSyncGetInMemPerfJUnitPerformanceTest.java |  129 +
 ...DiskRegOverflowSyncJUnitPerformanceTest.java |  169 +
 ...egionOverflowAsyncRollingOpLogJUnitTest.java |  226 +
 ...RegionOverflowSyncRollingOpLogJUnitTest.java |  224 +
 .../DiskRegionPerfJUnitPerformanceTest.java     |  568 ++
 .../DiskRegionPersistOnlySyncJUnitTest.java     |  234 +
 ...DiskRegionRollOpLogJUnitPerformanceTest.java |  638 ++
 ...ltiThreadedOplogPerJUnitPerformanceTest.java |  217 +
 .../cache/execute/Bug51193DUnitTest.java        |  246 +
 .../ClientServerFunctionExecutionDUnitTest.java |  952 ++
 .../execute/ColocationFailoverDUnitTest.java    |  527 +
 .../cache/execute/CustomResultCollector.java    |   50 +
 .../execute/CustomerIDPartitionResolver.java    |   82 +
 ...ributedRegionFunctionExecutionDUnitTest.java | 1652 ++++
 .../FunctionExecution_ExceptionDUnitTest.java   |  647 ++
 .../execute/FunctionServiceStatsDUnitTest.java  | 1372 +++
 .../cache/execute/LocalDataSetDUnitTest.java    |  389 +
 .../cache/execute/LocalDataSetFunction.java     |   90 +
 .../execute/LocalDataSetIndexingDUnitTest.java  |  304 +
 .../LocalFunctionExecutionDUnitTest.java        |  183 +
 .../MemberFunctionExecutionDUnitTest.java       |  678 ++
 .../MultiRegionFunctionExecutionDUnitTest.java  |  298 +
 .../execute/MyFunctionExecutionException.java   |   59 +
 .../cache/execute/MyTransactionFunction.java    |  517 +
 .../OnGroupsFunctionExecutionDUnitTest.java     | 1239 +++
 ...ntServerFunctionExecutionNoAckDUnitTest.java |  258 +
 ...tServerRegionFunctionExecutionDUnitTest.java | 1674 ++++
 ...egionFunctionExecutionFailoverDUnitTest.java |  580 ++
 ...onFunctionExecutionNoSingleHopDUnitTest.java | 1274 +++
 ...onExecutionSelectorNoSingleHopDUnitTest.java | 1232 +++
 ...gionFunctionExecutionSingleHopDUnitTest.java | 1222 +++
 .../cache/execute/PRClientServerTestBase.java   |  870 ++
 .../cache/execute/PRColocationDUnitTest.java    | 2735 ++++++
 .../execute/PRCustomPartitioningDUnitTest.java  |  574 ++
 .../execute/PRFunctionExecutionDUnitTest.java   | 3216 ++++++
 .../PRFunctionExecutionTimeOutDUnitTest.java    |  885 ++
 ...ctionExecutionWithResultSenderDUnitTest.java |  665 ++
 .../execute/PRPerformanceTestDUnitTest.java     |  424 +
 .../cache/execute/PRTransactionDUnitTest.java   |  766 ++
 .../PRTransactionWithVersionsDUnitTest.java     |   29 +
 .../internal/cache/execute/PerfFunction.java    |   69 +
 .../internal/cache/execute/PerfTxFunction.java  |   74 +
 .../cache/execute/PerformanceTestFunction.java  |   74 +
 .../execute/SingleHopGetAllPutAllDUnitTest.java |  204 +
 .../internal/cache/execute/TestFunction.java    |  150 +
 .../internal/cache/execute/data/CustId.java     |   72 +
 .../internal/cache/execute/data/Customer.java   |   77 +
 .../internal/cache/execute/data/Order.java      |   65 +
 .../internal/cache/execute/data/OrderId.java    |   82 +
 .../internal/cache/execute/data/Shipment.java   |   65 +
 .../internal/cache/execute/data/ShipmentId.java |   89 +
 .../SimpleExtensionPointJUnitTest.java          |   23 +-
 .../extension/mock/AbstractMockExtension.java   |   61 +
 .../mock/AbstractMockExtensionXmlGenerator.java |   41 +
 .../mock/AlterMockCacheExtensionFunction.java   |   90 +
 .../mock/AlterMockRegionExtensionFunction.java  |  104 +
 .../mock/CreateMockCacheExtensionFunction.java  |   86 +
 .../mock/CreateMockRegionExtensionFunction.java |  101 +
 .../mock/DestroyMockCacheExtensionFunction.java |   89 +
 .../DestroyMockRegionExtensionFunction.java     |   99 +
 .../extension/mock/MockCacheExtension.java      |   53 +
 .../mock/MockCacheExtensionXmlGenerator.java    |   57 +
 .../extension/mock/MockExtensionCommands.java   |  219 +
 .../extension/mock/MockExtensionXmlParser.java  |  102 +
 .../extension/mock/MockRegionExtension.java     |   47 +
 .../mock/MockRegionExtensionXmlGenerator.java   |   57 +
 ...gionFunctionFunctionInvocationException.java |   60 +
 .../functions/DistributedRegionFunction.java    |   98 +
 .../cache/functions/LocalDataSetFunction.java   |   89 +
 .../internal/cache/functions/TestFunction.java  | 1149 +++
 .../ha/BlockingHARQAddOperationJUnitTest.java   |  239 +
 .../cache/ha/BlockingHARQStatsJUnitTest.java    |   77 +
 .../cache/ha/BlockingHARegionJUnitTest.java     |  480 +
 .../ha/BlockingHARegionQueueJUnitTest.java      |  213 +
 .../cache/ha/Bug36853EventsExpiryDUnitTest.java |  307 +
 .../internal/cache/ha/Bug48571DUnitTest.java    |  258 +
 .../internal/cache/ha/Bug48879DUnitTest.java    |  221 +
 .../internal/cache/ha/ConflatableObject.java    |  221 +
 .../cache/ha/EventIdOptimizationDUnitTest.java  |  595 ++
 .../cache/ha/EventIdOptimizationJUnitTest.java  |   22 +-
 .../internal/cache/ha/FailoverDUnitTest.java    |  343 +
 .../internal/cache/ha/HABugInPutDUnitTest.java  |  187 +
 .../internal/cache/ha/HAClearDUnitTest.java     |  663 ++
 .../cache/ha/HAConflationDUnitTest.java         |  448 +
 .../internal/cache/ha/HADuplicateDUnitTest.java |  334 +
 .../cache/ha/HAEventIdPropagationDUnitTest.java |  877 ++
 .../internal/cache/ha/HAExpiryDUnitTest.java    |  285 +
 .../internal/cache/ha/HAGIIBugDUnitTest.java    |  416 +
 .../internal/cache/ha/HAGIIDUnitTest.java       |  469 +
 .../gemfire/internal/cache/ha/HAHelper.java     |   51 +
 .../cache/ha/HARQAddOperationJUnitTest.java     | 1204 +++
 .../cache/ha/HARQueueNewImplDUnitTest.java      | 1369 +++
 .../internal/cache/ha/HARegionDUnitTest.java    |  408 +
 .../internal/cache/ha/HARegionJUnitTest.java    |  226 +
 .../cache/ha/HARegionQueueDUnitTest.java        | 1152 +++
 .../cache/ha/HARegionQueueJUnitTest.java        | 2154 ++++
 ...HARegionQueueStartStopJUnitDisabledTest.java |  123 +
 .../ha/HARegionQueueStartStopJUnitTest.java     |  133 +
 .../cache/ha/HARegionQueueStatsJUnitTest.java   |  504 +
 .../cache/ha/HASlowReceiverDUnitTest.java       |  290 +
 .../ha/OperationsPropagationDUnitTest.java      |  503 +
 .../internal/cache/ha/PutAllDUnitTest.java      |  598 ++
 .../cache/ha/StatsBugDUnitDisabledTest.java     |  368 +
 .../cache/ha/TestBlockingHARegionQueue.java     |  129 +
 .../cache/ha/ThreadIdentifierJUnitTest.java     |   23 +-
 .../cache/locks/TXLockServiceDUnitTest.java     |  722 ++
 .../internal/cache/lru/LRUClockJUnitTest.java   |  519 +
 .../cache/partitioned/Bug39356DUnitTest.java    |  236 +
 .../cache/partitioned/Bug43684DUnitTest.java    |  341 +
 .../cache/partitioned/Bug47388DUnitTest.java    |  290 +
 .../cache/partitioned/Bug51400DUnitTest.java    |  206 +
 .../partitioned/ElidedPutAllDUnitTest.java      |  123 +
 .../OfflineMembersDetailsJUnitTest.java         |   59 +
 .../partitioned/PartitionResolverDUnitTest.java |  411 +
 .../PartitionedRegionLoadModelJUnitTest.java    | 1564 +++
 .../PartitionedRegionLoaderWriterDUnitTest.java |  232 +
 ...rtitionedRegionMetaDataCleanupDUnitTest.java |  191 +
 .../partitioned/PersistPRKRFDUnitTest.java      |  233 +
 ...tentColocatedPartitionedRegionDUnitTest.java | 1595 +++
 .../PersistentPartitionedRegionDUnitTest.java   | 2222 +++++
 ...tentPartitionedRegionOldConfigDUnitTest.java |   51 +
 .../PersistentPartitionedRegionTestBase.java    |  803 ++
 ...rtitionedRegionWithTransactionDUnitTest.java |  195 +
 .../PutPutReplyMessageJUnitTest.java            |   54 +
 .../cache/partitioned/ShutdownAllDUnitTest.java |  849 ++
 ...treamingPartitionOperationManyDUnitTest.java |  235 +
 ...StreamingPartitionOperationOneDUnitTest.java |  218 +
 .../fixed/CustomerFixedPartitionResolver.java   |  113 +
 .../fixed/FixedPartitioningDUnitTest.java       | 1698 ++++
 .../fixed/FixedPartitioningTestBase.java        | 1406 +++
 ...ngWithColocationAndPersistenceDUnitTest.java | 1439 +++
 .../cache/partitioned/fixed/MyDate1.java        |   72 +
 .../cache/partitioned/fixed/MyDate2.java        |   52 +
 .../cache/partitioned/fixed/MyDate3.java        |   72 +
 .../fixed/QuarterPartitionResolver.java         |  142 +
 .../SingleHopQuarterPartitionResolver.java      |  173 +
 .../persistence/BackupInspectorJUnitTest.java   |  236 +
 .../PersistentRVVRecoveryDUnitTest.java         | 1003 ++
 .../PersistentRecoveryOrderDUnitTest.java       | 1839 ++++
 ...rsistentRecoveryOrderOldConfigDUnitTest.java |   67 +
 .../PersistentReplicatedTestBase.java           |  235 +
 .../TemporaryResultSetFactoryJUnitTest.java     |  136 +
 .../GFSnapshotJUnitPerformanceTest.java         |  158 +
 .../internal/cache/tier/Bug40396DUnitTest.java  |  214 +
 .../tier/sockets/AcceptorImplJUnitTest.java     |  276 +
 ...mpatibilityHigherVersionClientDUnitTest.java |  258 +
 .../cache/tier/sockets/Bug36269DUnitTest.java   |  227 +
 .../cache/tier/sockets/Bug36457DUnitTest.java   |  215 +
 .../cache/tier/sockets/Bug36805DUnitTest.java   |  247 +
 .../cache/tier/sockets/Bug36829DUnitTest.java   |  175 +
 .../cache/tier/sockets/Bug36995DUnitTest.java   |  263 +
 .../cache/tier/sockets/Bug37210DUnitTest.java   |  293 +
 .../cache/tier/sockets/Bug37805DUnitTest.java   |  140 +
 .../CacheServerMaxConnectionsJUnitTest.java     |  229 +
 ...heServerSelectorMaxConnectionsJUnitTest.java |   35 +
 .../cache/tier/sockets/CacheServerTestUtil.java |  647 ++
 .../CacheServerTransactionsDUnitTest.java       |  853 ++
 ...acheServerTransactionsSelectorDUnitTest.java |   35 +
 .../tier/sockets/ClearPropagationDUnitTest.java |  491 +
 .../tier/sockets/ClientConflationDUnitTest.java |  536 +
 .../sockets/ClientHealthMonitorJUnitTest.java   |  265 +
 .../ClientHealthMonitorSelectorJUnitTest.java   |   35 +
 .../sockets/ClientInterestNotifyDUnitTest.java  |  647 ++
 .../tier/sockets/ClientServerMiscDUnitTest.java | 1392 +++
 .../ClientServerMiscSelectorDUnitTest.java      |   36 +
 .../cache/tier/sockets/ConflationDUnitTest.java |  916 ++
 .../tier/sockets/ConnectionProxyJUnitTest.java  |  858 ++
 .../DataSerializerPropogationDUnitTest.java     | 1389 +++
 .../cache/tier/sockets/DeltaEOFException.java   |  113 +
 .../DestroyEntryPropagationDUnitTest.java       |  514 +
 .../sockets/DurableClientBug39997DUnitTest.java |  123 +
 .../DurableClientQueueSizeDUnitTest.java        |  432 +
 .../DurableClientReconnectAutoDUnitTest.java    |   58 +
 .../DurableClientReconnectDUnitTest.java        |  759 ++
 .../sockets/DurableClientStatsDUnitTest.java    |  393 +
 .../sockets/DurableRegistrationDUnitTest.java   |  867 ++
 .../sockets/DurableResponseMatrixDUnitTest.java |  523 +
 .../sockets/EventIDVerificationDUnitTest.java   |  508 +
 .../EventIDVerificationInP2PDUnitTest.java      |  335 +
 .../cache/tier/sockets/FaultyDelta.java         |  162 +
 .../tier/sockets/FilterProfileJUnitTest.java    |  112 +
 .../ForceInvalidateEvictionDUnitTest.java       |  398 +
 ...ForceInvalidateOffHeapEvictionDUnitTest.java |   69 +
 .../cache/tier/sockets/HABug36738DUnitTest.java |  197 +
 .../cache/tier/sockets/HAInterestBaseTest.java  | 1015 ++
 .../sockets/HAInterestDistributedTestCase.java  |   31 +
 .../tier/sockets/HAInterestPart1DUnitTest.java  |  211 +
 .../tier/sockets/HAInterestPart2DUnitTest.java  |  374 +
 .../sockets/HAStartupAndFailoverDUnitTest.java  |  739 ++
 .../internal/cache/tier/sockets/HaHelper.java   |   33 +
 .../InstantiatorPropagationDUnitTest.java       | 1770 ++++
 .../tier/sockets/InterestListDUnitTest.java     | 1011 ++
 .../sockets/InterestListEndpointDUnitTest.java  |  501 +
 .../InterestListEndpointPRDUnitTest.java        |   42 +
 .../InterestListEndpointSelectorDUnitTest.java  |   35 +
 .../sockets/InterestListFailoverDUnitTest.java  |  318 +
 .../sockets/InterestListRecoveryDUnitTest.java  |  514 +
 .../sockets/InterestRegrListenerDUnitTest.java  |  467 +
 .../sockets/InterestResultPolicyDUnitTest.java  |  394 +
 .../sockets/NewRegionAttributesDUnitTest.java   |  493 +
 .../tier/sockets/ObjectPartListJUnitTest.java   |  130 +
 .../tier/sockets/RedundancyLevelJUnitTest.java  |  130 +
 .../sockets/RedundancyLevelPart1DUnitTest.java  |  534 +
 .../sockets/RedundancyLevelPart2DUnitTest.java  |  536 +
 .../sockets/RedundancyLevelPart3DUnitTest.java  |  248 +
 .../tier/sockets/RedundancyLevelTestBase.java   |  655 ++
 .../tier/sockets/RegionCloseDUnitTest.java      |  245 +
 ...erInterestBeforeRegionCreationDUnitTest.java |  279 +
 .../sockets/RegisterInterestKeysDUnitTest.java  |  247 +
 .../RegisterInterestKeysPRDUnitTest.java        |   41 +
 .../sockets/ReliableMessagingDUnitTest.java     |  436 +
 .../sockets/UnregisterInterestDUnitTest.java    |  338 +
 .../sockets/UpdatePropagationDUnitTest.java     |  605 ++
 .../sockets/UpdatePropagationPRDUnitTest.java   |   40 +
 .../VerifyEventIDGenerationInP2PDUnitTest.java  |  189 +
 ...UpdatesFromNonInterestEndPointDUnitTest.java |  257 +
 .../tier/sockets/command/CommitCommandTest.java |   55 +
 .../cache/versions/RVVExceptionJUnitTest.java   |   48 +
 .../versions/RegionVersionHolderJUnitTest.java  | 1894 ++++
 .../RegionVersionHolderRandomJUnitTest.java     |  191 +
 ...RegionVersionHolderSmallBitSetJUnitTest.java |   46 +
 .../versions/RegionVersionVectorJUnitTest.java  |  547 ++
 .../cache/wan/AsyncEventQueueTestBase.java      | 1666 ++++
 .../cache/wan/CompressionConstants.java         |   37 +
 .../cache/wan/CompressionInputStream.java       |  147 +
 .../cache/wan/CompressionOutputStream.java      |  123 +
 .../cache/wan/CustomAsyncEventListener.java     |   57 +
 .../gemfire/internal/cache/wan/Filter70.java    |   63 +
 .../cache/wan/MyAsyncEventListener.java         |   53 +
 .../cache/wan/MyAsyncEventListener2.java        |   98 +
 .../cache/wan/MyDistributedSystemListener.java  |   65 +
 .../cache/wan/MyGatewaySenderEventListener.java |   77 +
 .../wan/MyGatewaySenderEventListener2.java      |   77 +
 .../cache/wan/MyGatewayTransportFilter1.java    |   57 +
 .../cache/wan/MyGatewayTransportFilter2.java    |   56 +
 .../cache/wan/MyGatewayTransportFilter3.java    |   56 +
 .../cache/wan/MyGatewayTransportFilter4.java    |   56 +
 .../internal/cache/wan/QueueListener.java       |   79 +
 .../asyncqueue/AsyncEventListenerDUnitTest.java | 1920 ++++
 .../AsyncEventListenerOffHeapDUnitTest.java     |   33 +
 .../AsyncEventQueueStatsDUnitTest.java          |  320 +
 .../AsyncEventQueueValidationsJUnitTest.java    |   82 +
 .../ConcurrentAsyncEventQueueDUnitTest.java     |  336 +
 ...ncurrentAsyncEventQueueOffHeapDUnitTest.java |   32 +
 .../CommonParallelAsyncEventQueueDUnitTest.java |   59 +
 ...ParallelAsyncEventQueueOffHeapDUnitTest.java |   32 +
 .../xmlcache/AbstractXmlParserJUnitTest.java    |  168 +
 .../cache/xmlcache/CacheCreationJUnitTest.java  |  209 +
 .../cache/xmlcache/CacheXmlParserJUnitTest.java |  169 +
 .../xmlcache/CacheXmlVersionJUnitTest.java      |   75 +
 .../PivotalEntityResolverJUnitTest.java         |  145 +
 .../cache/xmlcache/RegionCreationJUnitTest.java |   57 +
 .../xmlcache/XmlGeneratorUtilsJUnitTest.java    |  250 +
 .../classpathloaderjunittest/DoesExist.java     |   16 +
 .../CompressionCacheConfigDUnitTest.java        |  191 +
 .../CompressionCacheListenerDUnitTest.java      |  360 +
 ...ompressionCacheListenerOffHeapDUnitTest.java |   80 +
 .../CompressionRegionConfigDUnitTest.java       |  530 +
 .../CompressionRegionFactoryDUnitTest.java      |  154 +
 .../CompressionRegionOperationsDUnitTest.java   |  523 +
 ...ressionRegionOperationsOffHeapDUnitTest.java |   72 +
 .../compression/CompressionStatsDUnitTest.java  |  682 ++
 .../compression/SnappyCompressorJUnitTest.java  |   39 +-
 .../datasource/AbstractPoolCacheJUnitTest.java  |  251 +
 .../internal/datasource/CleanUpJUnitTest.java   |  126 +
 .../ConnectionPoolCacheImplJUnitTest.java       |  213 +
 .../datasource/ConnectionPoolingJUnitTest.java  |  339 +
 .../datasource/DataSourceFactoryJUnitTest.java  |  115 +
 .../internal/datasource/RestartJUnitTest.java   |   82 +
 .../internal/i18n/BasicI18nJUnitTest.java       |  407 +
 .../io/CompositeOutputStreamJUnitTest.java      |  453 +
 .../gemfire/internal/jndi/ContextJUnitTest.java |  440 +
 .../internal/jta/BlockingTimeOutJUnitTest.java  |  311 +
 .../gemfire/internal/jta/CacheUtils.java        |  226 +
 .../internal/jta/DataSourceJTAJUnitTest.java    | 1024 ++
 .../internal/jta/ExceptionJUnitTest.java        |  126 +
 .../jta/GlobalTransactionJUnitTest.java         |  254 +
 .../gemstone/gemfire/internal/jta/JTAUtils.java |  386 +
 .../internal/jta/JtaIntegrationJUnitTest.java   |  134 +
 .../gemstone/gemfire/internal/jta/SyncImpl.java |   39 +
 .../internal/jta/TransactionImplJUnitTest.java  |  109 +
 .../jta/TransactionManagerImplJUnitTest.java    |  298 +
 .../jta/TransactionTimeOutJUnitTest.java        |  333 +
 .../jta/UserTransactionImplJUnitTest.java       |  130 +
 .../internal/jta/dunit/CommitThread.java        |  186 +
 .../internal/jta/dunit/ExceptionsDUnitTest.java |  318 +
 .../jta/dunit/IdleTimeOutDUnitTest.java         |  346 +
 .../jta/dunit/LoginTimeOutDUnitTest.java        |  345 +
 .../jta/dunit/MaxPoolSizeDUnitTest.java         |  312 +
 .../internal/jta/dunit/RollbackThread.java      |  186 +
 .../jta/dunit/TransactionTimeOutDUnitTest.java  |  492 +
 .../dunit/TxnManagerMultiThreadDUnitTest.java   |  521 +
 .../internal/jta/dunit/TxnTimeOutDUnitTest.java |  310 +
 .../internal/jta/functional/CacheJUnitTest.java | 1199 +++
 .../jta/functional/TestXACacheLoader.java       |  107 +
 .../internal/lang/ClassUtilsJUnitTest.java      |  165 +
 .../internal/lang/InOutParameterJUnitTest.java  |   74 +
 .../internal/lang/InitializerJUnitTest.java     |   75 +
 .../internal/lang/ObjectUtilsJUnitTest.java     |  189 +
 .../internal/lang/StringUtilsJUnitTest.java     |  351 +
 .../internal/lang/SystemUtilsJUnitTest.java     |  101 +
 .../internal/lang/ThreadUtilsJUnitTest.java     |  214 +
 .../DistributedSystemLogFileJUnitTest.java      | 1506 +++
 .../logging/LocatorLogFileJUnitTest.java        |  126 +
 .../logging/LogServiceIntegrationJUnitTest.java |  223 +
 .../LogServiceIntegrationTestSupport.java       |   40 +
 .../internal/logging/LogServiceJUnitTest.java   |  120 +
 .../LogWriterDisabledPerformanceTest.java       |   64 +
 .../logging/LogWriterImplJUnitTest.java         |   83 +
 .../logging/LogWriterPerformanceTest.java       |  121 +
 .../logging/LoggingIntegrationTestSuite.java    |   35 +
 .../logging/LoggingPerformanceTestCase.java     |  200 +
 .../internal/logging/LoggingUnitTestSuite.java  |   42 +
 .../logging/MergeLogFilesJUnitTest.java         |  248 +
 .../gemfire/internal/logging/NullLogWriter.java |  146 +
 .../internal/logging/SortLogFileJUnitTest.java  |  115 +
 .../internal/logging/TestLogWriterFactory.java  |  134 +
 .../logging/log4j/AlertAppenderJUnitTest.java   |  257 +
 .../logging/log4j/ConfigLocatorJUnitTest.java   |  222 +
 .../log4j/FastLoggerIntegrationJUnitTest.java   |  575 ++
 .../logging/log4j/FastLoggerJUnitTest.java      |  178 +
 .../FastLoggerWithDefaultConfigJUnitTest.java   |   90 +
 .../log4j/LocalizedMessageJUnitTest.java        |   64 +
 .../log4j/Log4J2DisabledPerformanceTest.java    |   67 +
 .../logging/log4j/Log4J2PerformanceTest.java    |  146 +
 .../log4j/Log4jIntegrationTestSuite.java        |   28 +
 .../logging/log4j/Log4jUnitTestSuite.java       |   32 +
 .../log4j/LogWriterAppenderJUnitTest.java       |  224 +
 .../LogWriterLoggerDisabledPerformanceTest.java |   66 +
 .../log4j/LogWriterLoggerPerformanceTest.java   |  145 +
 .../internal/net/SocketUtilsJUnitTest.java      |  130 +
 .../offheap/AbstractStoredObjectTestBase.java   |  203 +
 .../offheap/ByteArrayMemoryChunkJUnitTest.java  |   30 +
 .../offheap/ChunkWithHeapFormJUnitTest.java     |   64 +
 .../offheap/DataAsAddressJUnitTest.java         |  368 +
 .../internal/offheap/DataTypeJUnitTest.java     |  633 ++
 .../DirectByteBufferMemoryChunkJUnitTest.java   |   33 +
 .../offheap/FreeListOffHeapRegionJUnitTest.java |   46 +
 .../offheap/GemFireChunkFactoryJUnitTest.java   |  129 +
 .../internal/offheap/GemFireChunkJUnitTest.java |  921 ++
 .../offheap/GemFireChunkSliceJUnitTest.java     |   72 +
 .../HeapByteBufferMemoryChunkJUnitTest.java     |   33 +
 .../internal/offheap/InlineKeyJUnitTest.java    |  185 +
 .../offheap/LifecycleListenerJUnitTest.java     |  222 +
 .../offheap/MemoryChunkJUnitTestBase.java       |  276 +
 .../offheap/NullOffHeapMemoryStats.java         |  114 +
 .../offheap/NullOutOfOffHeapMemoryListener.java |   39 +
 .../internal/offheap/OffHeapIndexJUnitTest.java |   92 +
 .../internal/offheap/OffHeapRegionBase.java     |  594 ++
 .../offheap/OffHeapStorageJUnitTest.java        |  281 +
 .../offheap/OffHeapValidationJUnitTest.java     |  540 ++
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java |  115 +
 .../OldFreeListOffHeapRegionJUnitTest.java      |   47 +
 .../offheap/OutOfOffHeapMemoryDUnitTest.java    |  379 +
 .../offheap/RefCountChangeInfoJUnitTest.java    |  207 +
 ...mpleMemoryAllocatorFillPatternJUnitTest.java |  527 +
 .../offheap/SimpleMemoryAllocatorJUnitTest.java |  675 ++
 .../internal/offheap/StoredObjectTestSuite.java |   33 +
 .../TxReleasesOffHeapOnCloseJUnitTest.java      |   63 +
 .../offheap/UnsafeMemoryChunkJUnitTest.java     |   31 +
 .../BlockingProcessStreamReaderJUnitTest.java   |  480 +
 ...leProcessControllerIntegrationJUnitTest.java |  155 +
 .../LocalProcessControllerJUnitTest.java        |  119 +
 .../process/LocalProcessLauncherDUnitTest.java  |  153 +
 .../process/LocalProcessLauncherJUnitTest.java  |  182 +
 ...NonBlockingProcessStreamReaderJUnitTest.java |  411 +
 .../internal/process/PidFileJUnitTest.java      |  274 +
 .../ProcessControllerFactoryJUnitTest.java      |  159 +
 .../process/ProcessStreamReaderTestCase.java    |  238 +
 .../gemfire/internal/process/mbean/Process.java |   60 +
 .../internal/process/mbean/ProcessMBean.java    |   28 +
 ...tractSignalNotificationHandlerJUnitTest.java |  460 +
 .../internal/size/ObjectSizerJUnitTest.java     |   89 +
 .../internal/size/ObjectTraverserJUnitTest.java |  122 +
 .../internal/size/ObjectTraverserPerf.java      |  100 +
 .../size/SizeClassOnceObjectSizerJUnitTest.java |   70 +
 .../gemfire/internal/size/SizeTestUtil.java     |   33 +
 .../size/WellKnownClassSizerJUnitTest.java      |   62 +
 .../internal/statistics/DummyStatistics.java    |  209 +
 .../statistics/SampleCollectorJUnitTest.java    |  347 +
 .../statistics/StatMonitorHandlerJUnitTest.java |  256 +
 .../statistics/StatisticsDUnitTest.java         |  947 ++
 .../statistics/StatisticsMonitorJUnitTest.java  |  227 +
 .../internal/statistics/TestSampleHandler.java  |  177 +
 .../statistics/TestStatArchiveWriter.java       |   60 +
 .../statistics/TestStatisticsManager.java       |   42 +
 .../statistics/TestStatisticsSampler.java       |   59 +
 .../statistics/ValueMonitorJUnitTest.java       |  374 +
 .../internal/stats50/AtomicStatsJUnitTest.java  |  128 +
 .../internal/tcp/ConnectionJUnitTest.java       |   88 +
 .../util/AbortableTaskServiceJUnitTest.java     |   40 +-
 .../internal/util/ArrayUtilsJUnitTest.java      |   20 +-
 .../gemfire/internal/util/BytesJUnitTest.java   |   23 +-
 .../internal/util/CollectionUtilsJUnitTest.java |   20 +-
 .../internal/util/DelayedActionJUnitTest.java   |   23 +-
 .../gemfire/internal/util/IOUtilsJUnitTest.java |   23 +-
 .../gemfire/internal/util/SerializableImpl.java |   21 +-
 .../util/SerializableImplWithValue.java         |   21 +-
 .../gemstone/gemfire/internal/util/Sizeof.java  |  164 -
 .../gemfire/internal/util/Valuable.java         |   21 +-
 .../CompactConcurrentHashSetJUnitTest.java      |  190 +-
 .../ConcurrentHashMapIteratorJUnitTest.java     |   23 +-
 .../concurrent/CopyOnWriteHashMapJUnitTest.java |  505 +
 .../concurrent/ReentrantSemaphoreJUnitTest.java |   24 +-
 .../SemaphoreReadWriteLockJUnitTest.java        |   23 +-
 .../cm/ConcurrentHashMapJUnitTest.java          |  631 ++
 .../concurrent/cm/CountedMapLoopsJUnitTest.java |  225 +
 .../concurrent/cm/IntMapCheckJUnitTest.java     |  618 ++
 .../util/concurrent/cm/LoopHelpers.java         |  219 +
 .../util/concurrent/cm/MapCheckJUnitTest.java   |  631 ++
 .../util/concurrent/cm/MapLoopsJUnitTest.java   |  230 +
 .../util/concurrent/cm/RLJBarJUnitTest.java     |  197 +
 .../concurrent/cm/StringMapLoopsJUnitTest.java  |  240 +
 .../management/CacheManagementDUnitTest.java    |  953 ++
 .../management/ClientHealthStatsDUnitTest.java  |  444 +
 .../gemfire/management/CompositeStats.java      |  103 +
 .../gemfire/management/CompositeTestMBean.java  |   65 +
 .../gemfire/management/CompositeTestMXBean.java |   33 +
 .../management/CompositeTypeTestDUnitTest.java  |  180 +
 .../gemfire/management/CustomMBean.java         |   76 +
 .../gemfire/management/CustomMXBean.java        |   36 +
 .../management/DLockManagementDUnitTest.java    |  476 +
 .../DataBrowserJSONValidationJUnitTest.java     |  350 +
 .../management/DiskManagementDUnitTest.java     |  738 ++
 .../management/DistributedSystemDUnitTest.java  |  895 ++
 .../management/LocatorManagementDUnitTest.java  |  357 +
 .../gemstone/gemfire/management/MBeanUtil.java  |  549 ++
 .../gemfire/management/ManagementTestBase.java  |  757 ++
 .../MemberMBeanAttributesDUnitTest.java         |  274 +
 .../management/OffHeapManagementDUnitTest.java  |  977 ++
 .../gemfire/management/QueryDataDUnitTest.java  |  877 ++
 .../management/RegionManagementDUnitTest.java   | 1452 +++
 .../gemfire/management/TypedJsonJUnitTest.java  |  288 +
 ...ersalMembershipListenerAdapterDUnitTest.java | 2173 +++++
 .../stats/AsyncEventQueueStatsJUnitTest.java    |   66 +
 .../bean/stats/CacheServerStatsJUnitTest.java   |  167 +
 .../bean/stats/DiskStatsJUnitTest.java          |  122 +
 .../stats/DistributedSystemStatsDUnitTest.java  |  112 +
 .../stats/DistributedSystemStatsJUnitTest.java  |  122 +
 .../stats/GatewayReceiverStatsJUnitTest.java    |  207 +
 .../bean/stats/GatewaySenderStatsJUnitTest.java |  101 +
 .../bean/stats/MBeanStatsTestCase.java          |   86 +
 .../bean/stats/MemberLevelStatsJUnitTest.java   |  581 ++
 .../bean/stats/RegionStatsJUnitTest.java        |  271 +
 .../bean/stats/StatsRateJUnitTest.java          |  191 +
 .../internal/JettyHelperJUnitTest.java          |   80 +
 .../cli/ClasspathScanLoadHelperJUnitTest.java   |   92 +
 .../internal/cli/CliUtilDUnitTest.java          |  433 +
 .../internal/cli/CommandManagerJUnitTest.java   |  330 +
 .../cli/CommandSeparatorEscapeJUnitTest.java    |  138 +
 .../internal/cli/DataCommandJsonJUnitTest.java  |   61 +
 .../internal/cli/GfshParserJUnitTest.java       | 1154 +++
 .../management/internal/cli/HeadlessGfsh.java   |  374 +
 .../internal/cli/HeadlessGfshJUnitTest.java     |   86 +
 .../management/internal/cli/ResultHandler.java  |   23 +
 .../internal/cli/TableBuilderJUnitTest.java     |  314 +
 .../cli/annotations/CliArgumentJUnitTest.java   |  214 +
 .../AbstractCommandsSupportJUnitTest.java       |  404 +
 .../cli/commands/CliCommandTestBase.java        |  560 ++
 .../cli/commands/ConfigCommandsDUnitTest.java   |  497 +
 ...eateAlterDestroyRegionCommandsDUnitTest.java | 1148 +++
 .../cli/commands/DeployCommandsDUnitTest.java   |  480 +
 .../commands/DiskStoreCommandsDUnitTest.java    | 1154 +++
 .../commands/DiskStoreCommandsJUnitTest.java    |  405 +
 .../cli/commands/FunctionCommandsDUnitTest.java |  593 ++
 .../commands/GemfireDataCommandsDUnitTest.java  | 2087 ++++
 ...WithCacheLoaderDuringCacheMissDUnitTest.java |  371 +
 .../HTTPServiceSSLSupportJUnitTest.java         |  158 +
 .../cli/commands/IndexCommandsDUnitTest.java    |  817 ++
 .../cli/commands/IndexCommandsJUnitTest.java    |  208 +
 ...stAndDescribeDiskStoreCommandsDUnitTest.java |  197 +
 .../ListAndDescribeRegionDUnitTest.java         |  320 +
 .../cli/commands/ListIndexCommandDUnitTest.java |  672 ++
 .../cli/commands/MemberCommandsDUnitTest.java   |  287 +
 .../MiscellaneousCommandsDUnitTest.java         |  492 +
 ...laneousCommandsExportLogsPart1DUnitTest.java |  139 +
 ...laneousCommandsExportLogsPart2DUnitTest.java |  148 +
 ...laneousCommandsExportLogsPart3DUnitTest.java |  150 +
 ...laneousCommandsExportLogsPart4DUnitTest.java |  141 +
 .../cli/commands/QueueCommandsDUnitTest.java    |  390 +
 .../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 +
 .../RegionPathConverterJUnitTest.java           |   82 +
 .../internal/cli/domain/AbstractImpl.java       |   21 +
 .../management/internal/cli/domain/Impl1.java   |   21 +
 .../management/internal/cli/domain/Impl12.java  |   21 +
 .../internal/cli/domain/Interface1.java         |   21 +
 .../internal/cli/domain/Interface2.java         |   21 +
 .../management/internal/cli/domain/Stock.java   |   37 +
 .../management/internal/cli/dto/Car.java        |   75 +
 .../management/internal/cli/dto/Key1.java       |   67 +
 .../management/internal/cli/dto/Key2.java       |   64 +
 .../internal/cli/dto/ObjectWithCharAttr.java    |   60 +
 .../management/internal/cli/dto/Value1.java     |   97 +
 .../management/internal/cli/dto/Value2.java     |   90 +
 .../functions/DataCommandFunctionJUnitTest.java |  132 +
 .../DescribeDiskStoreFunctionJUnitTest.java     | 1676 ++++
 .../ListDiskStoresFunctionJUnitTest.java        |  328 +
 .../functions/ListIndexFunctionJUnitTest.java   |  435 +
 .../cli/parser/ParserUtilsJUnitTest.java        |   82 +
 .../preprocessor/PreprocessorJUnitTest.java     |  302 +
 .../PreprocessorUtilsJUnitTest.java             |  127 +
 .../cli/shell/GfshConfigInitFileJUnitTest.java  |  186 +
 .../shell/GfshExecutionStrategyJUnitTest.java   |  137 +
 .../cli/shell/GfshHistoryJUnitTest.java         |   89 +
 .../cli/shell/GfshInitFileJUnitTest.java        |  476 +
 .../SharedConfigurationDUnitTest.java           |  445 +
 .../configuration/ZipUtilsJUnitTest.java        |   96 +
 .../domain/CacheElementJUnitTest.java           |  144 +
 .../utils/XmlUtilsAddNewNodeJUnitTest.java      |  415 +
 .../configuration/utils/XmlUtilsJUnitTest.java  |  248 +
 .../internal/pulse/TestClientIdsDUnitTest.java  |  303 +
 .../internal/pulse/TestFunctionsDUnitTest.java  |  114 +
 .../internal/pulse/TestHeapDUnitTest.java       |  111 +
 .../internal/pulse/TestLocatorsDUnitTest.java   |   89 +
 .../pulse/TestSubscriptionsDUnitTest.java       |  305 +
 .../internal/security/JSONAuthCodeTest.java     |   18 +-
 .../security/JSONAuthorizationTest.java         |   18 +-
 .../security/ResourceOperationJUnit.java        |   18 +-
 .../ReadOpFileAccessControllerJUnitTest.java    |  201 +
 .../WanCommandsControllerJUnitTest.java         |  140 +
 .../gemfire/management/model/EmptyObject.java   |   24 +
 .../gemstone/gemfire/management/model/Item.java |   95 +
 .../gemfire/management/model/Order.java         |   88 +
 .../gemfire/management/model/SubOrder.java      |   30 +
 .../DomainObjectsAsValuesJUnitTest.java         |  129 +
 .../GemcachedBinaryClientJUnitTest.java         |  148 +
 .../GemcachedDevelopmentJUnitTest.java          |  265 +
 .../gemfire/memcached/IntegrationJUnitTest.java |   94 +
 .../gemfire/pdx/AutoSerializableJUnitTest.java  | 1401 +++
 .../gemfire/pdx/ByteSourceJUnitTest.java        |  752 ++
 .../ClientsWithVersioningRetryDUnitTest.java    |  511 +
 .../com/gemstone/gemfire/pdx/DSInsidePdx.java   |  109 +
 .../pdx/DistributedSystemIdDUnitTest.java       |  157 +
 .../com/gemstone/gemfire/pdx/DomainObject.java  |  106 +
 .../gemstone/gemfire/pdx/DomainObjectBad.java   |   25 +
 .../gemfire/pdx/DomainObjectClassLoadable.java  |   27 +
 .../gemfire/pdx/DomainObjectPdxAuto.java        |  192 +
 ...DomainObjectPdxAutoNoDefaultConstructor.java |   63 +
 .../java/com/gemstone/gemfire/pdx/Employee.java |   92 +
 .../pdx/JSONPdxClientServerDUnitTest.java       |  630 ++
 .../com/gemstone/gemfire/pdx/NestedPdx.java     |  114 +
 .../gemfire/pdx/NonDelegatingLoader.java        |   66 +
 .../OffHeapByteBufferByteSourceJUnitTest.java   |   52 +
 .../gemfire/pdx/OffHeapByteSourceJUnitTest.java |   65 +
 .../pdx/PDXAsyncEventQueueDUnitTest.java        |  155 +
 .../gemfire/pdx/PdxAttributesJUnitTest.java     |  250 +
 .../gemfire/pdx/PdxClientServerDUnitTest.java   |  798 ++
 .../pdx/PdxDeserializationDUnitTest.java        |  411 +
 .../pdx/PdxFormatterPutGetJUnitTest.java        |  208 +
 .../com/gemstone/gemfire/pdx/PdxInsideDS.java   |  107 +
 .../pdx/PdxInstanceFactoryJUnitTest.java        | 1210 +++
 .../gemfire/pdx/PdxInstanceJUnitTest.java       |  396 +
 .../gemfire/pdx/PdxSerializableDUnitTest.java   |  197 +
 .../gemfire/pdx/PdxSerializableJUnitTest.java   | 2171 +++++
 .../gemfire/pdx/PdxStringJUnitTest.java         |  167 +
 .../gemfire/pdx/PdxTypeExportDUnitTest.java     |  130 +
 .../gemfire/pdx/SeparateClassloaderPdx.java     |   43 +
 .../com/gemstone/gemfire/pdx/SimpleClass.java   |   83 +
 .../com/gemstone/gemfire/pdx/SimpleClass1.java  |  151 +
 .../com/gemstone/gemfire/pdx/SimpleClass2.java  |   32 +
 .../gemfire/pdx/TestObjectForPdxFormatter.java  | 1003 ++
 .../gemfire/pdx/VersionClassLoader.java         |  112 +
 .../gemstone/gemfire/redis/AuthJUnitTest.java   |  160 +
 .../gemfire/redis/ConcurrentStartTest.java      |   74 +
 .../gemstone/gemfire/redis/HashesJUnitTest.java |  191 +
 .../gemstone/gemfire/redis/ListsJUnitTest.java  |  254 +
 .../gemfire/redis/RedisDistDUnitTest.java       |  257 +
 .../gemstone/gemfire/redis/SetsJUnitTest.java   |  258 +
 .../gemfire/redis/SortedSetsJUnitTest.java      |  430 +
 .../gemfire/redis/StringsJunitTest.java         |  312 +
 .../web/controllers/AddFreeItemToOrders.java    |  153 +
 .../rest/internal/web/controllers/Customer.java |  109 +
 .../internal/web/controllers/DateTimeUtils.java |   40 +
 .../rest/internal/web/controllers/Gender.java   |   30 +
 .../internal/web/controllers/GetAllEntries.java |   68 +
 .../web/controllers/GetDeliveredOrders.java     |  106 +
 .../internal/web/controllers/GetRegions.java    |   76 +
 .../web/controllers/GetValueForKey.java         |   77 +
 .../rest/internal/web/controllers/Item.java     |  160 +
 .../rest/internal/web/controllers/Order.java    |  189 +
 .../rest/internal/web/controllers/Person.java   |  185 +
 .../web/controllers/PutKeyFunction.java         |   63 +
 .../web/controllers/RestAPITestBase.java        |  124 +
 .../internal/web/controllers/RestTestUtils.java |  110 +
 .../security/ClientAuthenticationDUnitTest.java |  967 ++
 .../ClientAuthenticationPart2DUnitTest.java     |   88 +
 .../security/ClientAuthorizationDUnitTest.java  |  798 ++
 .../security/ClientAuthorizationTestBase.java   | 1384 +++
 .../security/ClientMultiUserAuthzDUnitTest.java |  537 +
 .../DeltaClientAuthorizationDUnitTest.java      |  336 +
 .../DeltaClientPostAuthorizationDUnitTest.java  |  541 ++
 .../security/P2PAuthenticationDUnitTest.java    |  622 ++
 .../gemfire/security/SecurityTestUtil.java      | 1871 ++++
 .../gemfire/test/golden/ExecutableProcess.java  |   24 +
 .../gemfire/test/golden/FailOutputTestCase.java |   52 +
 .../golden/FailWithErrorInOutputJUnitTest.java  |   47 +
 .../FailWithExtraLineInOutputJUnitTest.java     |   76 +
 ...WithLineMissingFromEndOfOutputJUnitTest.java |   75 +
 ...hLineMissingFromMiddleOfOutputJUnitTest.java |   74 +
 .../FailWithLoggerErrorInOutputJUnitTest.java   |   46 +
 .../FailWithLoggerFatalInOutputJUnitTest.java   |   46 +
 .../FailWithLoggerWarnInOutputJUnitTest.java    |   46 +
 .../golden/FailWithProblemInOutputTestCase.java |   61 +
 .../golden/FailWithSevereInOutputJUnitTest.java |   47 +
 ...hTimeoutOfWaitForOutputToMatchJUnitTest.java |   68 +
 .../FailWithWarningInOutputJUnitTest.java       |   47 +
 .../gemfire/test/golden/GoldenComparator.java   |  142 +
 .../test/golden/GoldenStringComparator.java     |   39 +
 .../gemfire/test/golden/GoldenTestCase.java     |  157 +
 .../golden/GoldenTestFrameworkTestSuite.java    |   43 +
 .../gemfire/test/golden/PassJUnitTest.java      |   86 +
 .../golden/PassWithExpectedErrorJUnitTest.java  |   47 +
 .../golden/PassWithExpectedProblemTestCase.java |   91 +
 .../golden/PassWithExpectedSevereJUnitTest.java |   47 +
 .../PassWithExpectedWarningJUnitTest.java       |   47 +
 .../test/golden/RegexGoldenComparator.java      |   37 +
 .../test/golden/StringGoldenComparator.java     |   37 +
 .../gemfire/test/golden/log4j2-test.xml         |   18 +
 .../gemfire/test/process/MainLauncher.java      |   48 +
 .../test/process/MainLauncherJUnitTest.java     |  159 +
 .../gemfire/test/process/OutputFormatter.java   |   37 +
 .../test/process/ProcessOutputReader.java       |   89 +
 .../test/process/ProcessStreamReader.java       |   74 +
 .../process/ProcessTestFrameworkTestSuite.java  |   28 +
 .../gemfire/test/process/ProcessWrapper.java    |  464 +
 .../test/process/ProcessWrapperJUnitTest.java   |   72 +
 .../gemstone/gemfire/util/JSR166TestCase.java   |  479 +
 .../gemstone/gemfire/util/test/TestUtil.java    |   16 +
 .../com/gemstone/persistence/admin/Logger.java  |  278 +
 .../gemstone/persistence/logging/Formatter.java |   41 +
 .../gemstone/persistence/logging/Handler.java   |   98 +
 .../com/gemstone/persistence/logging/Level.java |  128 +
 .../gemstone/persistence/logging/LogRecord.java |  185 +
 .../gemstone/persistence/logging/Logger.java    |  566 ++
 .../persistence/logging/SimpleFormatter.java    |   77 +
 .../persistence/logging/StreamHandler.java      |   61 +
 .../test/java/com/gemstone/sequence/Arrow.java  |  124 +
 .../java/com/gemstone/sequence/Lifeline.java    |   98 +
 .../com/gemstone/sequence/LifelineState.java    |  114 +
 .../java/com/gemstone/sequence/LineMapper.java  |   36 +
 .../com/gemstone/sequence/SequenceDiagram.java  |  315 +
 .../com/gemstone/sequence/SequencePanel.java    |   83 +
 .../com/gemstone/sequence/StateColorMap.java    |   66 +
 .../java/com/gemstone/sequence/TimeAxis.java    |  122 +
 .../com/gemstone/sequence/ZoomingPanel.java     |  188 +
 .../sequence/gemfire/DefaultLineMapper.java     |   42 +
 .../gemfire/GemfireSequenceDisplay.java         |  343 +
 .../sequence/gemfire/HydraLineMapper.java       |  135 +
 .../sequence/gemfire/SelectGraphDialog.java     |  155 +
 .../com/main/MyDistributedSystemListener.java   |  115 +
 .../com/main/WANBootStrapping_Site1_Add.java    |  122 +
 .../com/main/WANBootStrapping_Site1_Remove.java |   75 +
 .../com/main/WANBootStrapping_Site2_Add.java    |  107 +
 .../com/main/WANBootStrapping_Site2_Remove.java |   73 +
 .../src/test/java/dunit/AsyncInvocation.java    |  217 +
 .../src/test/java/dunit/BounceResult.java       |   36 +
 gemfire-core/src/test/java/dunit/DUnitEnv.java  |   78 +
 .../test/java/dunit/DistributedTestCase.java    | 1436 +++
 gemfire-core/src/test/java/dunit/Host.java      |  208 +
 .../src/test/java/dunit/RMIException.java       |  170 +
 .../src/test/java/dunit/RemoteDUnitVMIF.java    |   36 +
 .../src/test/java/dunit/RepeatableRunnable.java |   29 +
 .../test/java/dunit/SerializableCallable.java   |   70 +
 .../test/java/dunit/SerializableRunnable.java   |   92 +
 gemfire-core/src/test/java/dunit/VM.java        | 1344 +++
 .../src/test/java/dunit/standalone/ChildVM.java |   82 +
 .../java/dunit/standalone/DUnitLauncher.java    |  463 +
 .../java/dunit/standalone/ProcessManager.java   |  261 +
 .../java/dunit/standalone/RemoteDUnitVM.java    |  144 +
 .../dunit/standalone/StandAloneDUnitEnv.java    |   75 +
 .../test/java/dunit/tests/BasicDUnitTest.java   |  132 +
 .../src/test/java/dunit/tests/TestFailure.java  |   50 +
 .../src/test/java/dunit/tests/VMDUnitTest.java  |  237 +
 gemfire-core/src/test/java/hydra/GsRandom.java  |  312 +
 .../test/java/hydra/HydraRuntimeException.java  |   33 +
 gemfire-core/src/test/java/hydra/Log.java       |  219 +
 .../src/test/java/hydra/LogVersionHelper.java   |   45 +
 .../src/test/java/hydra/MethExecutor.java       |  392 +
 .../src/test/java/hydra/MethExecutorResult.java |  186 +
 .../src/test/java/hydra/SchedulingOrder.java    |   36 +
 .../src/test/java/hydra/log/AnyLogWriter.java   |  555 ++
 .../java/hydra/log/CircularOutputStream.java    |  131 +
 .../java/org/xerial/snappy/SnappyUtils.java     |    9 -
 .../parReg/query/unittest/NewPortfolio.java     |  272 +
 .../java/parReg/query/unittest/Position.java    |  162 +
 .../src/test/java/perffmwk/Formatter.java       |  147 +
 .../java/security/AuthzCredentialGenerator.java |  465 +
 .../test/java/security/CredentialGenerator.java |  343 +
 .../security/DummyAuthzCredentialGenerator.java |  145 +
 .../java/security/DummyCredentialGenerator.java |   94 +
 .../security/LdapUserCredentialGenerator.java   |  160 +
 .../java/security/PKCSCredentialGenerator.java  |  112 +
 .../java/security/SSLCredentialGenerator.java   |  117 +
 .../UserPasswordWithExtraPropsAuthInit.java     |   77 +
 .../security/XmlAuthzCredentialGenerator.java   |  264 +
 .../templates/security/DummyAuthenticator.java  |   87 +
 .../templates/security/DummyAuthorization.java  |  118 +
 .../security/FunctionSecurityPrmsHolder.java    |   55 +
 .../security/LdapUserAuthenticator.java         |  117 +
 .../java/templates/security/PKCSAuthInit.java   |  133 +
 .../templates/security/PKCSAuthenticator.java   |  167 +
 .../java/templates/security/PKCSPrincipal.java  |   42 +
 .../security/UserPasswordAuthInit.java          |   84 +
 .../templates/security/UsernamePrincipal.java   |   46 +
 .../templates/security/XmlAuthorization.java    |  675 ++
 .../templates/security/XmlErrorHandler.java     |   82 +
 .../src/test/java/util/TestException.java       |   35 +
 ...gemstone.gemfire.internal.cache.CacheService |    1 +
 ...ne.gemfire.internal.cache.xmlcache.XmlParser |    5 +
 ...org.springframework.shell.core.CommandMarker |    8 +
 .../ClientCacheFactoryJUnitTest_single_pool.xml |   30 +
 .../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/cache/query/dunit/IndexCreation.xml |  131 +
 .../functional/index-creation-with-eviction.xml |   56 +
 .../index-creation-without-eviction.xml         |   67 +
 .../functional/index-recovery-overflow.xml      |   57 +
 .../query/internal/index/cachequeryindex.xml    |  125 +
 .../internal/index/cachequeryindexwitherror.xml |  134 +
 .../cache/query/partitioned/PRIndexCreation.xml |   44 +
 .../gemfire/cache30/attributesUnordered.xml     |   39 +
 .../com/gemstone/gemfire/cache30/badFloat.xml   |   30 +
 .../com/gemstone/gemfire/cache30/badInt.xml     |   33 +
 .../gemfire/cache30/badKeyConstraintClass.xml   |   31 +
 .../com/gemstone/gemfire/cache30/badScope.xml   |   30 +
 .../com/gemstone/gemfire/cache30/bug44710.xml   |   31 +
 .../gemfire/cache30/callbackNotDeclarable.xml   |   34 +
 .../gemfire/cache30/callbackWithException.xml   |   34 +
 .../com/gemstone/gemfire/cache30/coLocation.xml |   31 +
 .../gemstone/gemfire/cache30/coLocation3.xml    |   31 +
 .../com/gemstone/gemfire/cache30/ewtest.xml     |   68 +
 .../cache30/examples_3_0/example-cache.xml      |   87 +
 .../cache30/examples_4_0/example-cache.xml      |   94 +
 .../gemfire/cache30/loaderNotLoader.xml         |   33 +
 .../com/gemstone/gemfire/cache30/malformed.xml  |   29 +
 .../gemfire/cache30/namedAttributes.xml         |   59 +
 .../gemfire/cache30/partitionedRegion.xml       |   39 +
 .../gemfire/cache30/partitionedRegion51.xml     |   31 +
 .../gemstone/gemfire/cache30/sameRootRegion.xml |   37 +
 .../gemstone/gemfire/cache30/sameSubregion.xml  |   42 +
 .../gemfire/cache30/unknownNamedAttributes.xml  |   30 +
 .../gemfire/codeAnalysis/excludedClasses.txt    |  115 +
 .../gemstone/gemfire/codeAnalysis/openBugs.txt  |   23 +
 .../sanctionedDataSerializables.txt             | 2202 +++++
 .../codeAnalysis/sanctionedSerializables.txt    |  829 ++
 .../internal/SharedConfigurationJUnitTest.xml   |   17 +
 ...st_testWriteAfterSamplingBegins_expected.gfs |  Bin 0 -> 1970 bytes
 ...est_testWriteWhenSamplingBegins_expected.gfs |  Bin 0 -> 1933 bytes
 .../internal/cache/BackupJUnitTest.cache.xml    |   23 +
 .../internal/cache/DiskRegCacheXmlJUnitTest.xml |  233 +
 .../cache/PartitionRegionCacheExample1.xml      |   43 +
 .../cache/PartitionRegionCacheExample2.xml      |   42 +
 .../incorrect_bytes_threshold.xml               |   35 +
 .../faultyDiskXMLsForTesting/incorrect_dir.xml  |   35 +
 .../incorrect_dir_size.xml                      |   35 +
 .../incorrect_max_oplog_size.xml                |   35 +
 .../incorrect_roll_oplogs_value.xml             |   35 +
 .../incorrect_sync_value.xml                    |   35 +
 .../incorrect_time_interval.xml                 |   35 +
 .../mixed_diskstore_diskdir.xml                 |   38 +
 .../mixed_diskstore_diskwriteattrs.xml          |   38 +
 .../tier/sockets/RedundancyLevelJUnitTest.xml   |   38 +
 ...testDTDFallbackWithNonEnglishLocal.cache.xml |   23 +
 .../gemstone/gemfire/internal/jta/cachejta.xml  |  273 +
 .../domain/CacheElementJUnitTest.xml            |    7 +
 ...dNewNodeJUnitTest.testAddNewNodeNewNamed.xml |   25 +
 ...ewNodeJUnitTest.testAddNewNodeNewUnnamed.xml |   27 +
 ...itTest.testAddNewNodeNewUnnamedExtension.xml |   25 +
 ...NodeJUnitTest.testAddNewNodeReplaceNamed.xml |   22 +
 ...deJUnitTest.testAddNewNodeReplaceUnnamed.xml |   24 +
 ...st.testAddNewNodeReplaceUnnamedExtension.xml |   24 +
 ...sAddNewNodeJUnitTest.testDeleteNodeNamed.xml |   21 +
 ...ddNewNodeJUnitTest.testDeleteNodeUnnamed.xml |   23 +
 ...JUnitTest.testDeleteNodeUnnamedExtension.xml |   23 +
 .../utils/XmlUtilsAddNewNodeJUnitTest.xml       |   24 +
 ...Test.testBuildSchemaLocationMapAttribute.xml |   10 +
 ...testBuildSchemaLocationMapEmptyAttribute.xml |    8 +
 ...ationMapMapOfStringListOfStringAttribute.xml |   10 +
 ....testBuildSchemaLocationMapNullAttribute.xml |    7 +
 ...XmlUtilsJUnitTest.testQuerySingleElement.xml |   24 +
 .../gemstone/gemfire/pdx/jsonStrings/array.txt  |   22 +
 .../gemfire/pdx/jsonStrings/attachment.txt      |   11 +
 .../gemfire/pdx/jsonStrings/attachment2.txt     |   13 +
 .../gemstone/gemfire/pdx/jsonStrings/book.txt   |   17 +
 .../gemstone/gemfire/pdx/jsonStrings/image.txt  |   13 +
 .../gemstone/gemfire/pdx/jsonStrings/json1.txt  |   22 +
 .../gemstone/gemfire/pdx/jsonStrings/json10.txt |   20 +
 .../gemstone/gemfire/pdx/jsonStrings/json11.txt |   33 +
 .../gemstone/gemfire/pdx/jsonStrings/json12.txt |   32 +
 .../gemstone/gemfire/pdx/jsonStrings/json13.txt |   42 +
 .../gemstone/gemfire/pdx/jsonStrings/json14.txt |   15 +
 .../gemstone/gemfire/pdx/jsonStrings/json15.txt |    1 +
 .../gemstone/gemfire/pdx/jsonStrings/json16.txt |   31 +
 .../gemfire/pdx/jsonStrings/json16_2.txt        |   31 +
 .../gemstone/gemfire/pdx/jsonStrings/json17.txt |   27 +
 .../gemstone/gemfire/pdx/jsonStrings/json18.txt |   71 +
 .../gemstone/gemfire/pdx/jsonStrings/json19.txt |   18 +
 .../gemstone/gemfire/pdx/jsonStrings/json2.txt  |   11 +
 .../gemstone/gemfire/pdx/jsonStrings/json20.txt |   36 +
 .../gemstone/gemfire/pdx/jsonStrings/json21.txt |   36 +
 .../gemstone/gemfire/pdx/jsonStrings/json22.txt |   36 +
 .../gemstone/gemfire/pdx/jsonStrings/json23.txt |   23 +
 .../gemstone/gemfire/pdx/jsonStrings/json24.txt |   15 +
 .../gemstone/gemfire/pdx/jsonStrings/json25.txt |   33 +
 .../gemstone/gemfire/pdx/jsonStrings/json26.txt |   13 +
 .../gemstone/gemfire/pdx/jsonStrings/json27.txt |   25 +
 .../gemstone/gemfire/pdx/jsonStrings/json28.txt |   84 +
 .../gemstone/gemfire/pdx/jsonStrings/json29.txt |   11 +
 .../gemstone/gemfire/pdx/jsonStrings/json3.txt  |   26 +
 .../gemstone/gemfire/pdx/jsonStrings/json31.txt |    9 +
 .../gemstone/gemfire/pdx/jsonStrings/json4.txt  |   88 +
 .../gemstone/gemfire/pdx/jsonStrings/json5.txt  |   27 +
 .../gemstone/gemfire/pdx/jsonStrings/json6.txt  |   11 +
 .../gemstone/gemfire/pdx/jsonStrings/json7.txt  |   32 +
 .../gemstone/gemfire/pdx/jsonStrings/json8.txt  |   53 +
 .../gemstone/gemfire/pdx/jsonStrings/json9.txt  |   77 +
 .../gemfire/pdx/jsonStrings/jsonMongo.txt       |   10 +
 .../pdx/jsonStrings/jsonMongoSingleQuote.tx0    |   10 +
 .../gemfire/pdx/jsonStrings/jsonProductdb.txt   |   33 +
 .../gemfire/pdx/jsonStrings/json_google.txt     |   33 +
 .../gemfire/pdx/jsonStrings/jsoncustomer.txt    |   24 +
 .../gemfire/pdx/jsonStrings/jsonemptyobject.txo |   44 +
 .../gemfire/pdx/jsonStrings/jsonemptyobject.txt |   23 +
 .../gemfire/pdx/jsonStrings/jsonfacebook.txt    |   45 +
 .../gemfire/pdx/jsonStrings/jsonfeed.txt        |   33 +
 .../gemfire/pdx/jsonStrings/jsonfeed2.txt       |   47 +
 .../gemfire/pdx/jsonStrings/jsonflicker.txt     |   21 +
 .../gemfire/pdx/jsonStrings/jsoniphone.txt      |   78 +
 .../pdx/jsonStrings/jsonsolrwithcomment.tx0     |   29 +
 .../pdx/jsonStrings/jsonsolrwithcomment.txt     |   29 +
 .../gemfire/pdx/jsonStrings/jsontwitter1.txt    |  430 +
 .../gemfire/pdx/jsonStrings/jsontwitter2.txt    |  574 ++
 .../gemfire/pdx/jsonStrings/jsontwitter3.txt    |   70 +
 .../gemfire/pdx/jsonStrings/jsonutf.tx0         |    2 +
 .../gemfire/pdx/jsonStrings/jsonyahoo.txt       |   11 +
 .../gemfire/pdx/jsonStrings/jsonyoutube.txt     |   54 +
 .../gemfire/pdx/jsonStrings/linkden1.txt        |   16 +
 .../gemstone/gemfire/pdx/jsonStrings/odata.txt  |   16 +
 .../gemstone/gemfire/pdx/jsonStrings/odata2.txt |   18 +
 .../jsonStrings/unquoteJsonStrings/json1.txt    |   31 +
 .../gemfire/pdx/jsonStrings/weather.txt         |    1 +
 .../gemfire/test/golden/log4j2-test.xml         |   18 +
 .../src/test/resources/jta/cachejta.xml         |  273 +
 .../src/test/resources/lib/authz-dummy.xml      |  126 +
 .../src/test/resources/lib/authz-ldap.xml       |   85 +
 .../resources/lib/authz-multiUser-dummy.xml     |  106 +
 .../test/resources/lib/authz-multiUser-ldap.xml |   83 +
 .../test/resources/lib/keys/gemfire1.keystore   |  Bin 0 -> 1536 bytes
 .../test/resources/lib/keys/gemfire10.keystore  |  Bin 0 -> 1546 bytes
 .../test/resources/lib/keys/gemfire11.keystore  |  Bin 0 -> 1546 bytes
 .../test/resources/lib/keys/gemfire2.keystore   |  Bin 0 -> 1536 bytes
 .../test/resources/lib/keys/gemfire3.keystore   |  Bin 0 -> 1536 bytes
 .../test/resources/lib/keys/gemfire4.keystore   |  Bin 0 -> 1536 bytes
 .../test/resources/lib/keys/gemfire5.keystore   |  Bin 0 -> 1536 bytes
 .../test/resources/lib/keys/gemfire6.keystore   |  Bin 0 -> 1536 bytes
 .../test/resources/lib/keys/gemfire7.keystore   |  Bin 0 -> 1536 bytes
 .../test/resources/lib/keys/gemfire8.keystore   |  Bin 0 -> 1536 bytes
 .../test/resources/lib/keys/gemfire9.keystore   |  Bin 0 -> 1536 bytes
 .../resources/lib/keys/ibm/gemfire1.keystore    |  Bin 0 -> 1426 bytes
 .../resources/lib/keys/ibm/gemfire10.keystore   |  Bin 0 -> 1434 bytes
 .../resources/lib/keys/ibm/gemfire11.keystore   |  Bin 0 -> 1434 bytes
 .../resources/lib/keys/ibm/gemfire2.keystore    |  Bin 0 -> 1434 bytes
 .../resources/lib/keys/ibm/gemfire3.keystore    |  Bin 0 -> 1426 bytes
 .../resources/lib/keys/ibm/gemfire4.keystore    |  Bin 0 -> 1434 bytes
 .../resources/lib/keys/ibm/gemfire5.keystore    |  Bin 0 -> 1434 bytes
 .../resources/lib/keys/ibm/gemfire6.keystore    |  Bin 0 -> 1434 bytes
 .../resources/lib/keys/ibm/gemfire7.keystore    |  Bin 0 -> 1426 bytes
 .../resources/lib/keys/ibm/gemfire8.keystore    |  Bin 0 -> 1434 bytes
 .../resources/lib/keys/ibm/gemfire9.keystore    |  Bin 0 -> 1426 bytes
 .../test/resources/lib/keys/ibm/publickeyfile   |  Bin 0 -> 4535 bytes
 .../src/test/resources/lib/keys/publickeyfile   |  Bin 0 -> 4535 bytes
 .../resources/spring/spring-gemfire-context.xml |   42 +
 .../src/test/resources/ssl/trusted.keystore     |  Bin 0 -> 1176 bytes
 .../src/test/resources/ssl/untrusted.keystore   |  Bin 0 -> 1181 bytes
 .../resources/templates/security/authz5_5.dtd   |  105 +
 .../resources/templates/security/authz6_0.dtd   |  110 +
 gemfire-jgroups/build.gradle                    |   31 -
 .../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 | 1723 ----
 .../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 | 2607 -----
 .../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  |  719 --
 .../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   |  393 -
 .../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  | 2009 ----
 .../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 |  981 --
 .../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 |  898 --
 .../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       | 2729 ------
 .../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 -
 .../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/GFBasicAdapter.java       |   95 -
 .../org/jgroups/stack/GFBasicAdapterImpl.java   |  466 -
 .../org/jgroups/stack/GFPeerAdapter.java        |  168 -
 .../org/jgroups/stack/GFPeerAdapterImpl.java    |  321 -
 .../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        |  504 -
 .../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      |   47 -
 .../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       |  841 --
 .../gemstone/org/jgroups/util/GFLogWriter.java  |   55 -
 .../org/jgroups/util/GFStringIdImpl.java        |   68 -
 .../org/jgroups/util/GemFireTracer.java         |  468 -
 .../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  |  161 -
 .../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  |   25 +
 .../categories/DistributedTransactionsTest.java |   26 +
 .../test/junit/categories/IntegrationTest.java  |   25 +
 .../test/junit/categories/PerformanceTest.java  |   25 +
 .../gemfire/test/junit/categories/UnitTest.java |   25 +
 .../gemfire/test/junit/categories/WanTest.java  |   24 +
 .../test/junit/rules/ExpectedTimeout.java       |  180 +
 .../junit/rules/ExpectedTimeoutJUnitTest.java   |  204 +
 .../com/gemstone/junit/DistributedTest.java     |    9 -
 .../com/gemstone/junit/IntegrationTest.java     |    9 -
 .../com/gemstone/junit/PerformanceTest.java     |    9 -
 .../test/java/com/gemstone/junit/UnitTest.java  |    9 -
 .../test/java/com/gemstone/junit/WanTest.java   |    8 -
 gemfire-lucene/build.gradle                     |   46 +
 .../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 |   41 +
 ...serIntegrationJUnitTest.parseIndex.cache.xml |   41 +
 gemfire-rebalancer/build.gradle                 |   30 +
 .../gemfire/cache/util/AutoBalancer.java        |  554 ++
 .../util/AutoBalancerIntegrationJUnitTest.java  |  206 +
 .../cache/util/AutoBalancerJUnitTest.java       |  604 ++
 gemfire-site/.gitignore                         |    1 +
 gemfire-site/website/.gitignore                 |    1 +
 gemfire-site/website/README.md                  |   54 +
 gemfire-site/website/Rules                      |   69 +
 gemfire-site/website/build.sh                   |   18 +
 .../website/content/bootstrap/bootstrap.min.css |    9 +
 .../website/content/community/index.html        |  302 +
 .../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 | 1570 +++
 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         |  140 +
 .../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         |   66 +
 .../website/content/releases/index.html         |   82 +
 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        |  247 +
 gemfire-site/website/lib/default.rb             |   60 +
 gemfire-site/website/lib/helpers_.rb            |   16 +
 gemfire-site/website/lib/pandoc.template        |    4 +
 gemfire-site/website/nanoc.yaml                 |   94 +
 gemfire-site/website/run.sh                     |   18 +
 gemfire-site/website/utilities/map-markers.rb   |   75 +
 gemfire-site/website/utilities/markers.txt      |  440 +
 .../website/utilities/snapshot-apigee.rb        |   88 +
 gemfire-spark-connector/.gitignore              |    1 +
 gemfire-spark-connector/README.md               |   32 +
 gemfire-spark-connector/doc/10_demos.md         |   84 +
 gemfire-spark-connector/doc/1_building.md       |   36 +
 gemfire-spark-connector/doc/2_quick.md          |  178 +
 gemfire-spark-connector/doc/3_connecting.md     |   55 +
 gemfire-spark-connector/doc/4_loading.md        |  108 +
 gemfire-spark-connector/doc/5_rdd_join.md       |  237 +
 gemfire-spark-connector/doc/6_save_rdd.md       |   81 +
 gemfire-spark-connector/doc/7_save_dstream.md   |   68 +
 gemfire-spark-connector/doc/8_oql.md            |   58 +
 gemfire-spark-connector/doc/9_java_api.md       |  129 +
 .../connector/internal/RegionMetadata.java      |   93 +
 .../gemfirefunctions/QueryFunction.java         |   99 +
 .../RetrieveRegionFunction.java                 |  208 +
 .../RetrieveRegionMetadataFunction.java         |  118 +
 .../StructStreamingResultSender.java            |  219 +
 .../gemfire/spark/connector/Employee.java       |   54 +
 .../spark/connector/JavaApiIntegrationTest.java |  424 +
 .../gemfire/spark/connector/Portfolio.java      |  109 +
 .../gemfire/spark/connector/Position.java       |   73 +
 .../src/it/resources/test-regions.xml           |   49 +
 .../src/it/resources/test-retrieve-regions.xml  |   57 +
 .../spark/connector/BasicIntegrationTest.scala  |  598 ++
 .../RDDJoinRegionIntegrationTest.scala          |  300 +
 .../RetrieveRegionIntegrationTest.scala         |  253 +
 .../gemfire/spark/connector/package.scala       |   29 +
 .../connector/testkit/GemFireCluster.scala      |   47 +
 .../spark/connector/testkit/GemFireRunner.scala |  148 +
 .../spark/connector/testkit/IOUtils.scala       |   94 +
 .../spark/streaming/ManualClockHelper.scala     |   28 +
 .../spark/streaming/TestInputDStream.scala      |   44 +
 .../javaapi/GemFireJavaDStreamFunctions.java    |   86 +
 .../GemFireJavaPairDStreamFunctions.java        |   77 +
 .../javaapi/GemFireJavaPairRDDFunctions.java    |  238 +
 .../javaapi/GemFireJavaRDDFunctions.java        |  178 +
 .../javaapi/GemFireJavaSQLContextFunctions.java |   49 +
 .../GemFireJavaSparkContextFunctions.java       |   87 +
 .../connector/javaapi/GemFireJavaUtil.java      |  122 +
 .../spark/connector/GemFireConnection.scala     |   67 +
 .../spark/connector/GemFireConnectionConf.scala |   73 +
 .../connector/GemFireConnectionManager.scala    |   31 +
 .../connector/GemFireFunctionDeployer.scala     |   81 +
 .../connector/GemFireKryoRegistrator.scala      |   29 +
 .../connector/GemFirePairRDDFunctions.scala     |  140 +
 .../spark/connector/GemFireRDDFunctions.scala   |  120 +
 .../connector/GemFireSQLContextFunctions.scala  |   42 +
 .../GemFireSparkContextFunctions.scala          |   39 +
 .../internal/DefaultGemFireConnection.scala     |  164 +
 .../DefaultGemFireConnectionManager.scala       |   77 +
 .../connector/internal/LocatorHelper.scala      |  135 +
 .../StructStreamingResultCollector.scala        |  152 +
 .../connector/internal/oql/QueryParser.scala    |   58 +
 .../spark/connector/internal/oql/QueryRDD.scala |   83 +
 .../internal/oql/QueryResultCollector.scala     |   69 +
 .../connector/internal/oql/RDDConverter.scala   |   40 +
 .../connector/internal/oql/RowBuilder.scala     |   38 +
 .../connector/internal/oql/SchemaBuilder.scala  |   73 +
 .../internal/oql/UndefinedSerializer.scala      |   46 +
 .../connector/internal/rdd/GemFireJoinRDD.scala |   67 +
 .../internal/rdd/GemFireOuterJoinRDD.scala      |   69 +
 .../internal/rdd/GemFireRDDPartition.scala      |   36 +
 .../internal/rdd/GemFireRDDPartitioner.scala    |   59 +
 .../rdd/GemFireRDDPartitionerImpl.scala         |   89 +
 .../internal/rdd/GemFireRDDWriter.scala         |   82 +
 .../internal/rdd/GemFireRegionRDD.scala         |  138 +
 .../javaapi/GemFireJavaRegionRDD.scala          |   26 +
 .../spark/connector/javaapi/JavaAPIHelper.scala |   53 +
 .../gemfire/spark/connector/package.scala       |   69 +
 .../streaming/GemFireDStreamFunctions.scala     |   89 +
 .../spark/connector/streaming/package.scala     |   32 +
 .../gemfire/spark/connector/JavaAPITest.java    |  163 +
 .../connector/GemFireFunctionDeployerTest.scala |   58 +
 .../DefaultGemFireConnectionManagerTest.scala   |   82 +
 ...tStreamingResultSenderAndCollectorTest.scala |  254 +
 .../internal/oql/QueryParserTest.scala          |   83 +
 .../connector/ConnectorImplicitsTest.scala      |   50 +
 .../connector/GemFireConnectionConfTest.scala   |  100 +
 .../connector/GemFireDStreamFunctionsTest.scala |   79 +
 .../connector/GemFireRDDFunctionsTest.scala     |  139 +
 .../spark/connector/LocatorHelperTest.scala     |  168 +
 .../rdd/GemFireRDDPartitionerTest.scala         |  190 +
 .../connector/rdd/GemFireRegionRDDTest.scala    |  117 +
 .../basic-demos/src/main/java/demo/Emp.java     |   95 +
 .../src/main/java/demo/OQLJavaDemo.java         |   59 +
 .../src/main/java/demo/PairRDDSaveJavaDemo.java |   86 +
 .../src/main/java/demo/RDDSaveJavaDemo.java     |   85 +
 .../src/main/java/demo/RegionToRDDJavaDemo.java |   57 +
 .../src/main/scala/demo/NetworkWordCount.scala  |   75 +
 .../project/Dependencies.scala                  |   45 +
 .../project/GemFireSparkBuild.scala             |   76 +
 gemfire-spark-connector/project/Settings.scala  |   57 +
 .../project/build.properties                    |    1 +
 gemfire-spark-connector/project/plugins.sbt     |    8 +
 gemfire-spark-connector/scalastyle-config.xml   |  117 +
 gemfire-web-api/build.gradle                    |   67 +-
 .../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                        |   49 +-
 .../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       |   70 +-
 ...entVariablesHandlerInterceptorJUnitTest.java |   25 +-
 .../internal/web/domain/LinkIndexJUnitTest.java |   23 +-
 .../internal/web/domain/LinkJUnitTest.java      |   23 +-
 .../domain/QueryParameterSourceJUnitTest.java   |   23 +-
 .../web/http/ClientHttpRequestJUnitTest.java    |   23 +-
 ...ableObjectHttpMessageConverterJUnitTest.java |   23 +-
 .../RestHttpOperationInvokerJUnitTest.java      |   23 +-
 .../SimpleHttpOperationInvokerJUnitTest.java    |   23 +-
 .../web/util/ConvertUtilsJUnitTest.java         |   23 +-
 .../internal/web/util/UriUtilsJUnitTest.java    |   23 +-
 gradle.properties                               |   23 +-
 gradle/dependency-versions.properties           |   81 +
 gradle/rat.gradle                               |  173 +
 gradle/wrapper/gradle-wrapper.jar               |  Bin 51018 -> 53637 bytes
 gradle/wrapper/gradle-wrapper.properties        |    4 +-
 gradlew                                         |   12 +-
 settings.gradle                                 |   25 +-
 6086 files changed, 757272 insertions(+), 211197 deletions(-)
----------------------------------------------------------------------



[02/50] [abbrv] incubator-geode git commit: GEODE-627: Add source distribution

Posted by ab...@apache.org.
GEODE-627: Add source distribution

Add another distribution to create source archives, without gradle
wrapper binaries included.


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

Branch: refs/heads/master
Commit: bba7656013d4ca5b5f97de24f2957049273bef09
Parents: 91b4389
Author: Anthony Baker <ab...@pivotal.io>
Authored: Wed Dec 30 09:01:17 2015 -0600
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Thu Dec 31 06:21:34 2015 -0600

----------------------------------------------------------------------
 gemfire-assembly/build.gradle | 23 +++++++++++++++++++++++
 1 file changed, 23 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bba76560/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index 373d77a..ad466de 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -178,6 +178,29 @@ task gfshDepsJar (type: Jar, dependsOn: ':gemfire-core:classes') {
 }
 
 distributions {
+  src {
+    baseName = 'apache-geode-src'
+    contents {
+      from (rootDir) {
+        exclude 'gradlew'
+        exclude 'gradlew.bat'
+        exclude 'gradle/wrapper/gradle-wrapper.jar'
+        exclude 'gradle/wrapper/gradle-wrapper.properties'
+        exclude '.gradle'
+        exclude '**/build/**'
+        exclude '**/.project'
+        exclude '**/.classpath'
+        exclude '**/.settings/**'
+        exclude '**/build-eclipse/**'
+        exclude '.idea/**'
+        exclude '**/*.iml'
+        exclude '**/*.ipr'
+        exclude '**/*.iws'
+        exclude '**/tags'
+      }
+    }
+  }
+
   main {
     baseName = 'apache-geode' //TODO rootProject.name
     contents {


[15/50] [abbrv] incubator-geode git commit: Revert "GEODE-715: Move dunit.standalone under com.gemstone.gemfire.test"

Posted by ab...@apache.org.
Revert "GEODE-715: Move dunit.standalone under com.gemstone.gemfire.test"

This reverts commit c57a88c6f2437a8d74f51abbe2edc3355eb1806b.


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

Branch: refs/heads/master
Commit: dfba327e3060d20dee55d2ae97ed0f299e241cd5
Parents: a826281
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Jan 4 16:28:15 2016 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Mon Jan 4 16:28:15 2016 -0800

----------------------------------------------------------------------
 .../cache/PartitionedRegionDUnitTestCase.java   |   2 +-
 .../CompressionRegionConfigDUnitTest.java       |   2 +-
 .../test/dunit/standalone/BounceResult.java     |  36 --
 .../gemfire/test/dunit/standalone/ChildVM.java  |  81 ----
 .../test/dunit/standalone/DUnitLauncher.java    | 461 ------------------
 .../test/dunit/standalone/ProcessManager.java   | 259 -----------
 .../test/dunit/standalone/RemoteDUnitVM.java    | 142 ------
 .../test/dunit/standalone/RemoteDUnitVMIF.java  |  36 --
 .../dunit/standalone/StandAloneDUnitEnv.java    |  75 ---
 .../test/dunit/tests/BasicDUnitTest.java        | 132 ------
 .../gemfire/test/dunit/tests/TestFailure.java   |  50 --
 .../gemfire/test/dunit/tests/VMDUnitTest.java   | 237 ----------
 .../src/test/java/dunit/BounceResult.java       |  36 ++
 gemfire-core/src/test/java/dunit/DUnitEnv.java  |   2 -
 gemfire-core/src/test/java/dunit/Host.java      |   2 -
 .../src/test/java/dunit/RemoteDUnitVMIF.java    |  36 ++
 gemfire-core/src/test/java/dunit/VM.java        |   7 +-
 .../src/test/java/dunit/standalone/ChildVM.java |  82 ++++
 .../java/dunit/standalone/DUnitLauncher.java    | 463 +++++++++++++++++++
 .../java/dunit/standalone/ProcessManager.java   | 261 +++++++++++
 .../java/dunit/standalone/RemoteDUnitVM.java    | 144 ++++++
 .../dunit/standalone/StandAloneDUnitEnv.java    |  75 +++
 .../test/java/dunit/tests/BasicDUnitTest.java   | 132 ++++++
 .../src/test/java/dunit/tests/TestFailure.java  |  50 ++
 .../src/test/java/dunit/tests/VMDUnitTest.java  | 237 ++++++++++
 25 files changed, 1520 insertions(+), 1520 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
index 74f201d..a4d32f2 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
@@ -30,10 +30,10 @@ import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogWriterImpl;
 import com.gemstone.gemfire.internal.logging.PureLogWriter;
-import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 
 import dunit.Host;
 import dunit.SerializableRunnable;
+import dunit.standalone.DUnitLauncher;
 
 /**
  * This class is extended by some PartitionedRegion related DUnit test cases 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java
index 1fb22c6..af7d07f 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java
@@ -27,13 +27,13 @@ import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.compression.Compressor;
 import com.gemstone.gemfire.compression.SnappyCompressor;
-import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 
 import dunit.DistributedTestCase;
 import dunit.Host;
 import dunit.SerializableCallable;
 import dunit.SerializableRunnable;
 import dunit.VM;
+import dunit.standalone.DUnitLauncher;
 
 /**
  * Sanity checks on a number of basic cluster configurations with compression turned on.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java
deleted file mode 100644
index e117004..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java
+++ /dev/null
@@ -1,36 +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.test.dunit.standalone;
-
-public class BounceResult {
-  private final int newPid;
-  private final RemoteDUnitVMIF newClient;
-  
-  public BounceResult(int newPid, RemoteDUnitVMIF newClient) {
-    this.newPid = newPid;
-    this.newClient = newClient;
-  }
-
-  public int getNewPid() {
-    return newPid;
-  }
-
-  public RemoteDUnitVMIF getNewClient() {
-    return newClient;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java
deleted file mode 100644
index 49b53e8..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java
+++ /dev/null
@@ -1,81 +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.test.dunit.standalone;
-
-import hydra.HydraRuntimeException;
-import hydra.Log;
-
-import java.rmi.Naming;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.internal.OSProcess;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher.MasterRemote;
-
-/**
- * @author dsmith
- *
- */
-public class ChildVM {
-  
-  private static boolean stopMainLoop = false;
-  
-  /**
-   * tells the main() loop to exit
-   */
-  public static void stopVM() {
-    stopMainLoop = true;
-  }
-  
-  static {
-    createHydraLogWriter();
-  }
-  
-  private final static Logger logger = LogService.getLogger();
-  private static RemoteDUnitVM dunitVM;
-  
-  public static void main(String[] args) throws Throwable {
-    try {
-      int namingPort = Integer.getInteger(DUnitLauncher.RMI_PORT_PARAM).intValue();
-      int vmNum = Integer.getInteger(DUnitLauncher.VM_NUM_PARAM).intValue();
-      int pid = OSProcess.getId();
-      logger.info("VM" + vmNum + " is launching" + (pid > 0? " with PID " + pid : ""));
-      MasterRemote holder = (MasterRemote) Naming.lookup("//localhost:" + namingPort + "/" + DUnitLauncher.MASTER_PARAM);
-      DUnitLauncher.init(holder);
-      DUnitLauncher.locatorPort = holder.getLocatorPort();
-      dunitVM = new RemoteDUnitVM();
-      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 (!stopMainLoop) {
-        holder.ping();
-        Thread.sleep(1000);
-      }
-    } catch (Throwable t) {
-      t.printStackTrace();
-      System.exit(1);
-    }
-  }
-
-  private static void createHydraLogWriter() {
-    try {
-      Log.createLogWriter("dunit-childvm", "fine");
-    } catch (HydraRuntimeException ignore) {
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
deleted file mode 100644
index 1358722..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.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.test.dunit.standalone;
-
-import hydra.Log;
-import hydra.MethExecutorResult;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.net.InetAddress;
-import java.net.URISyntaxException;
-import java.nio.channels.FileChannel;
-import java.nio.charset.Charset;
-import java.rmi.AccessException;
-import java.rmi.AlreadyBoundException;
-import java.rmi.NotBoundException;
-import java.rmi.Remote;
-import java.rmi.RemoteException;
-import java.rmi.registry.LocateRegistry;
-import java.rmi.registry.Registry;
-import java.rmi.server.UnicastRemoteObject;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.core.LoggerContext;
-import org.apache.logging.log4j.core.appender.FileAppender;
-import org.apache.logging.log4j.core.config.LoggerConfig;
-import org.apache.logging.log4j.core.layout.PatternLayout;
-import org.junit.Assert;
-
-import batterytest.greplogs.ExpectedStrings;
-import batterytest.greplogs.LogConsumer;
-
-import com.gemstone.gemfire.distributed.Locator;
-import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.internal.logging.LogService;
-
-import dunit.DUnitEnv;
-import dunit.Host;
-import dunit.SerializableCallable;
-import dunit.VM;
-
-/**
- * A class to build a fake test configuration and launch some DUnit VMS.
- * 
- * For use within eclipse. This class completely skips hydra and just starts
- * some vms directly, creating a fake test configuration
- * 
- * Also, it's a good idea to set your working directory, because the test code
- * a lot of files that it leaves around.
- * 
- * @author dsmith
- *
- */
-public class DUnitLauncher {
-
-  /** change this to use a different log level in unit tests */
-  public static final String LOG_LEVEL = System.getProperty("logLevel", "info");
-  
-  static int locatorPort;
-
-  private static final int NUM_VMS = 4;
-  private static final int DEBUGGING_VM_NUM = -1;
-  private static final int LOCATOR_VM_NUM = -2;
-
-  static final long STARTUP_TIMEOUT = 30 * 1000;
-  private static final String SUSPECT_FILENAME = "dunit_suspect.log";
-  private static File DUNIT_SUSPECT_FILE;
-
-  public static final String DUNIT_DIR = "dunit";
-  public static final String WORKSPACE_DIR_PARAM = "WORKSPACE_DIR";
-  public static final boolean LOCATOR_LOG_TO_DISK = Boolean.getBoolean("locatorLogToDisk");
-
-  static final String MASTER_PARAM = "DUNIT_MASTER";
-  static final String RMI_PORT_PARAM = "gemfire.DUnitLauncher.RMI_PORT";
-  static final String VM_NUM_PARAM = "gemfire.DUnitLauncher.VM_NUM";
-
-  private static final String LAUNCHED_PROPERTY = "gemfire.DUnitLauncher.LAUNCHED";
-
-  private static Master master;
-
-  private DUnitLauncher() {
-  }
-  
-  private static boolean isHydra() {
-    try {
-      //TODO - this is hacky way to test for a hydra environment - see
-      //if there is registered test configuration object.
-      Class<?> clazz = Class.forName("hydra.TestConfig");
-      Method getInstance = clazz.getMethod("getInstance", new Class[0]);
-      getInstance.invoke(null);
-      return true;
-    } catch (Exception e) {
-      return false;
-    }
-  }
-  /**
-   * Launch DUnit. If the unit test was launched through
-   * the hydra framework, leave the test alone.
-   */
-  public static void launchIfNeeded() {
-    if(System.getProperties().contains(VM_NUM_PARAM)) {
-      //we're a dunit child vm, do nothing.
-      return;
-    }
-
-    if(!isHydra() &&!isLaunched()) {
-      try {
-        launch();
-      } catch (Exception e) {
-        throw new RuntimeException("Unable to launch dunit VMS", e);
-      }
-    }
-  }
-  
-  /**
-   * Test it see if the eclise dunit environment is launched.
-   */
-  public static boolean isLaunched() {
-    return Boolean.getBoolean(LAUNCHED_PROPERTY);
-  }
-  
-  public static String getLocatorString() {
-    return "localhost[" + locatorPort + "]";
-  }
-
-  
-  private static void launch() throws URISyntaxException, AlreadyBoundException, IOException, InterruptedException, NotBoundException  {
-//  initialize the log writer that hydra uses
-    Log.createLogWriter( "dunit-master", LOG_LEVEL );
-
-    DUNIT_SUSPECT_FILE = new File(SUSPECT_FILENAME);
-    DUNIT_SUSPECT_FILE.delete();
-    DUNIT_SUSPECT_FILE.deleteOnExit();
-    
-    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
-     
-    //create an RMI registry and add an object to share our tests config
-    int namingPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    Registry registry = LocateRegistry.createRegistry(namingPort);
-
-    final ProcessManager processManager = new ProcessManager(namingPort, registry);
-    master = new Master(registry, processManager);
-    registry.bind(MASTER_PARAM, master);
-
-    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();
-      }
-    });
-    
-    //Create a VM for the locator
-    processManager.launchVM(LOCATOR_VM_NUM);
-    
-    //Launch an initial set of VMs
-    for(int i=0; i < NUM_VMS; i++) {
-      processManager.launchVM(i);
-    }
-    
-    //wait for the VMS to start up
-    if(!processManager.waitForVMs(STARTUP_TIMEOUT)) {
-      throw new RuntimeException("VMs did not start up with 30 seconds");
-    }
-    
-    //populate the Host class with our stubs. The tests use this host class
-    DUnitHost host = new DUnitHost(InetAddress.getLocalHost().getCanonicalHostName(), processManager);
-    host.init(registry, NUM_VMS);
-
-    init(master);
-    
-    startLocator(registry);
-  }
-  
-  public static Properties getDistributedSystemProperties() {
-    Properties p = new Properties();
-    p.setProperty("locators", getLocatorString());
-    p.setProperty("mcast-port", "0");
-    p.setProperty("enable-cluster-configuration", "false");
-    p.setProperty("use-cluster-configuration", "false");
-    p.setProperty("log-level", LOG_LEVEL);
-    return p;
-  }
-
-  /**
-   * Add an appender to Log4j which sends all INFO+ messages to a separate file
-   * which will be used later to scan for suspect strings.  The pattern of the
-   * messages conforms to the original log format so that hydra will be able
-   * to parse them.
-   */
-  private static void addSuspectFileAppender(final String workspaceDir) {
-    final String suspectFilename = new File(workspaceDir, SUSPECT_FILENAME).getAbsolutePath();
-
-    final LoggerContext appenderContext = ((org.apache.logging.log4j.core.Logger)
-        LogManager.getLogger(LogService.BASE_LOGGER_NAME)).getContext();
-
-    final PatternLayout layout = PatternLayout.createLayout(
-        "[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} <%thread> tid=%tid] %message%n%throwable%n", null, null,
-        Charset.defaultCharset(), true, false, "", "");
-    
-    final FileAppender fileAppender = FileAppender.createAppender(suspectFilename, "true", "false",
-        DUnitLauncher.class.getName(), "true", "false", "false", "0", layout, null, null, null, appenderContext.getConfiguration());
-    fileAppender.start();
-
-    LoggerConfig loggerConfig = appenderContext.getConfiguration().getLoggerConfig(LogService.BASE_LOGGER_NAME);
-    loggerConfig.addAppender(fileAppender, Level.INFO, null);
-  }
-  
-  private static void startLocator(Registry registry) throws IOException, NotBoundException {
-    RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + LOCATOR_VM_NUM);
-    final File locatorLogFile =
-        LOCATOR_LOG_TO_DISK ? new File("locator-" + locatorPort + ".log") : new File(""); 
-    MethExecutorResult result = remote.executeMethodOnObject(new SerializableCallable() {
-      public Object call() throws IOException {
-        Properties p = getDistributedSystemProperties();
-        // I never want this locator to end up starting a jmx manager
-        // since it is part of the unit test framework
-        p.setProperty("jmx-manager", "false");
-        //Disable the shared configuration on this locator.
-        //Shared configuration tests create their own locator
-        p.setProperty("enable-cluster-configuration", "false");
-        //Tell the locator it's the first in the system for
-        //faster boot-up
-        
-        System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
-        try {
-          Locator.startLocatorAndDS(locatorPort, locatorLogFile, p);
-        } finally {
-          System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
-        }
-        
-        return null;
-      }
-    }, "call");
-    if(result.getException() != null) {
-      RuntimeException ex = new RuntimeException("Failed to start locator", result.getException());
-      ex.printStackTrace();
-      throw ex;
-    }
-  }
-
-  public static void init(MasterRemote master) {
-    DUnitEnv.set(new StandAloneDUnitEnv(master));
-    //fake out tests that are using a bunch of hydra stuff
-    String workspaceDir = System.getProperty(DUnitLauncher.WORKSPACE_DIR_PARAM) ;
-    workspaceDir = workspaceDir == null ? new File(".").getAbsolutePath() : workspaceDir;
-    
-    addSuspectFileAppender(workspaceDir);
-    
-    //Free off heap memory when disconnecting from the distributed system
-    System.setProperty("gemfire.free-off-heap-memory", "true");
-    
-    //indicate that this CM is controlled by the eclipse dunit.
-    System.setProperty(LAUNCHED_PROPERTY, "true");
-  }
-  
-  public static void closeAndCheckForSuspects() {
-    if (isLaunched()) {
-      final boolean skipLogMsgs = ExpectedStrings.skipLogMsgs("dunit");
-      final List<?> expectedStrings = ExpectedStrings.create("dunit");
-      final LogConsumer logConsumer = new LogConsumer(skipLogMsgs, expectedStrings, "log4j", 5);
-
-      final StringBuilder suspectStringBuilder = new StringBuilder();
-
-      BufferedReader buffReader = null;
-      FileChannel fileChannel = null;
-      try {
-        fileChannel = new FileOutputStream(DUNIT_SUSPECT_FILE, true).getChannel();
-        buffReader = new BufferedReader(new FileReader(DUNIT_SUSPECT_FILE));
-      } catch (FileNotFoundException e) {
-        System.err.println("Could not find the suspect string output file: " + e);
-        return;
-      }
-      try {
-        String line;
-        try {
-          while ((line = buffReader.readLine()) != null) {
-            final StringBuilder builder = logConsumer.consume(line);
-            if (builder != null) {
-              suspectStringBuilder.append(builder);
-            }
-          }
-        } catch (IOException e) {
-          System.err.println("Could not read the suspect string output file: " + e);
-        }
-        
-        try {
-          fileChannel.truncate(0);
-        } catch (IOException e) {
-          System.err.println("Could not truncate the suspect string output file: " + e);
-        }
-        
-      } finally {
-        try {
-          buffReader.close();
-          fileChannel.close();
-        } catch (IOException e) {
-          System.err.println("Could not close the suspect string output file: " + e);
-        }
-      }
-
-      if (suspectStringBuilder.length() != 0) {
-        System.err.println("Suspicious strings were written to the log during this run.\n"
-            + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n"
-            + suspectStringBuilder);
-        
-        Assert.fail("Suspicious strings were written to the log during this run.\n"
-            + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n"
-            + suspectStringBuilder);
-      }
-    }
-  }
-
-  public interface MasterRemote extends Remote {
-    public int getLocatorPort() throws RemoteException;
-    public void signalVMReady() throws RemoteException;
-    public void ping() throws RemoteException;
-    public BounceResult bounce(int pid) throws RemoteException;
-  }
-  
-  public static class Master extends UnicastRemoteObject implements MasterRemote {
-    private static final long serialVersionUID = 1178600200232603119L;
-    
-    private final Registry registry;
-    private final ProcessManager processManager;
-
-
-    public Master(Registry registry, ProcessManager processManager) throws RemoteException {
-      this.processManager = processManager;
-      this.registry = registry;
-    }
-
-    public int getLocatorPort()  throws RemoteException{
-      return locatorPort;
-    }
-
-    public synchronized void signalVMReady() {
-      processManager.signalVMReady();
-    }
-    
-    public void ping() {
-      //do nothing
-    }
-
-    @Override
-    public BounceResult bounce(int pid) {
-      processManager.bounce(pid);
-      
-      try {
-        if(!processManager.waitForVMs(STARTUP_TIMEOUT)) {
-          throw new RuntimeException("VMs did not start up with 30 seconds");
-        }
-        RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + pid);
-        return new BounceResult(pid, remote);
-      } catch (RemoteException | NotBoundException e) {
-        throw new RuntimeException("could not lookup name", e);
-      } catch (InterruptedException e) {
-        throw new RuntimeException("Failed waiting for VM", e);
-      }
-    }
-  }
-  
-  private static class DUnitHost extends Host {
-    private static final long serialVersionUID = -8034165624503666383L;
-    
-    private transient final VM debuggingVM;
-
-    private transient ProcessManager processManager;
-    
-    public DUnitHost(String hostName, ProcessManager processManager) throws RemoteException {
-      super(hostName);
-      this.debuggingVM = new VM(this, -1, new RemoteDUnitVM());
-      this.processManager = processManager;
-    }
-    
-    public void init(Registry registry, int numVMs) throws AccessException, RemoteException, NotBoundException, InterruptedException {
-      for(int i = 0; i < numVMs; i++) {
-        RemoteDUnitVMIF remote = processManager.getStub(i);
-        addVM(i, remote);
-      }
-      
-      addLocator(LOCATOR_VM_NUM, processManager.getStub(LOCATOR_VM_NUM));
-      
-      addHost(this);
-    }
-
-    @Override
-    public VM getVM(int n) {
-      
-      if(n == DEBUGGING_VM_NUM) {
-        //for ease of debugging, pass -1 to get the local VM
-        return debuggingVM;
-      }
-
-      int oldVMCount = getVMCount();
-      if(n >= oldVMCount) {
-        //If we don't have a VM with that number, dynamically create it.
-        try {
-          for(int i = oldVMCount; i <= n; i++) {
-            processManager.launchVM(i);
-          }
-          processManager.waitForVMs(STARTUP_TIMEOUT);
-
-          for(int i = oldVMCount; i <= n; i++) {
-            addVM(i, processManager.getStub(i));
-          }
-
-        } catch (IOException | InterruptedException | NotBoundException e) {
-          throw new RuntimeException("Could not dynamically launch vm + " + n, e);
-        }
-      }
-      
-      return super.getVM(n);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
deleted file mode 100644
index 7b053b6..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
+++ /dev/null
@@ -1,259 +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.test.dunit.standalone;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.PrintStream;
-import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-import java.rmi.AccessException;
-import java.rmi.NotBoundException;
-import java.rmi.RemoteException;
-import java.rmi.registry.Registry;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.commons.io.FileUtils;
-
-import com.gemstone.gemfire.internal.FileUtil;
-import com.gemstone.gemfire.internal.logging.LogService;
-
-/**
- * @author dsmith
- *
- */
-public class ProcessManager {
-  private int namingPort;
-  private Map<Integer, ProcessHolder> processes = new HashMap<Integer, ProcessHolder>();
-  private File log4jConfig;
-  private int pendingVMs;
-  private Registry registry;
-  private int debugPort = Integer.getInteger("dunit.debug.basePort", 0);
-  private int suspendVM = Integer.getInteger("dunit.debug.suspendVM", -100);
-
-  public ProcessManager(int namingPort, Registry registry) {
-    this.namingPort = namingPort;
-    this.registry = registry;
-  }
-  
-  public void launchVMs() throws IOException, NotBoundException {
-    log4jConfig = LogService.findLog4jConfigInCurrentDir();
-  }
-
-  public synchronized void launchVM(int vmNum) throws IOException {
-    if(processes.containsKey(vmNum)) {
-      throw new IllegalStateException("VM " + vmNum + " is already running.");
-    }
-    
-    String[] cmd = buildJavaCommand(vmNum, namingPort);
-    System.out.println("Executing " + Arrays.asList(cmd));
-    File workingDir = getVMDir(vmNum);
-    try {
-      FileUtil.delete(workingDir);
-    } catch(IOException e) {
-      //This delete is occasionally failing on some platforms, maybe due to a lingering
-      //process. Allow the process to be launched anyway.
-      System.err.println("Unable to delete " + workingDir + ". Currently contains " 
-                          + Arrays.asList(workingDir.list()));
-    }
-    workingDir.mkdirs();
-    if (log4jConfig != null) {
-      FileUtils.copyFileToDirectory(log4jConfig, workingDir);
-    }
-    
-    //TODO - delete directory contents, preferably with commons io FileUtils
-    Process process = Runtime.getRuntime().exec(cmd, null, workingDir);
-    pendingVMs++;
-    ProcessHolder holder = new ProcessHolder(process);
-    processes.put(vmNum, holder);
-    linkStreams(vmNum, holder, process.getErrorStream(), System.err);
-    linkStreams(vmNum, holder, process.getInputStream(), System.out);
-  }
-
-  public static File getVMDir(int vmNum) {
-    return new File(DUnitLauncher.DUNIT_DIR, "vm" + vmNum);
-  }
-  
-  public synchronized void killVMs() {
-    for(ProcessHolder process : processes.values()) {
-      if(process != null) {
-        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);
-    }
-    try {
-      ProcessHolder holder = processes.remove(vmNum);
-      holder.kill();
-      holder.getProcess().waitFor();
-      launchVM(vmNum);
-    } catch (InterruptedException | IOException e) {
-      throw new RuntimeException("Unable to restart VM " + vmNum, e);
-    }
-  }
-   
-  private void linkStreams(final int vmNum, final ProcessHolder holder, final InputStream in, final PrintStream out) {
-    Thread ioTransport = new Thread() {
-      public void run() {
-        BufferedReader reader = new BufferedReader(new InputStreamReader(in));
-        String vmName = (vmNum==-2)? "[locator]" : "[vm_"+vmNum+"]";
-        try {
-          String line = reader.readLine();
-          while(line != null) {
-            if (line.length() == 0) {
-              out.println();
-            } else {
-              out.print(vmName);
-              out.println(line);
-            }
-            line = reader.readLine();
-          }
-        } catch(Exception e) {
-          if(!holder.isKilled()) {
-            out.println("Error transporting IO from child process");
-            e.printStackTrace(out);
-          }
-        }
-      }
-    };
-
-    ioTransport.setDaemon(true);
-    ioTransport.start();
-  }
-
-  private String[] buildJavaCommand(int vmNum, int namingPort) {
-    String cmd = System.getProperty( "java.home" ) + File.separator + "bin" + File.separator + "java";
-    String classPath = System.getProperty("java.class.path");
-    //String tmpDir = System.getProperty("java.io.tmpdir");
-    String agent = getAgentString();
-
-    String jdkDebug = "";
-    if (debugPort > 0) {
-      jdkDebug += ",address=" + debugPort;
-      debugPort++;
-    }
-
-    String jdkSuspend = vmNum == suspendVM ? "y" : "n";
-
-    return new String[] {
-      cmd, "-classpath", classPath,
-      "-D" + DUnitLauncher.RMI_PORT_PARAM + "=" + namingPort,
-      "-D" + DUnitLauncher.VM_NUM_PARAM + "=" + vmNum,
-      "-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=" + jdkSuspend + jdkDebug,
-      "-XX:+HeapDumpOnOutOfMemoryError",
-      "-Xmx512m",
-      "-Dgemfire.DEFAULT_MAX_OPLOG_SIZE=10",
-      "-Dgemfire.disallowMcastDefaults=true",
-      "-ea",
-      agent,
-      "dunit.standalone.ChildVM"
-    };
-  }
-  
-  /**
-   * Get the java agent passed to this process and pass it to the child VMs.
-   * This was added to support jacoco code coverage reports
-   */
-  private String getAgentString() {
-    RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
-    if (runtimeBean != null) {
-      for(String arg: runtimeBean.getInputArguments()) {
-        if(arg.contains("-javaagent:")) {
-          //HACK for gradle bug  GRADLE-2859. Jacoco is passing a relative path
-          //That won't work when we pass this to dunit VMs in a different 
-          //directory
-          arg = arg.replace("-javaagent:..", "-javaagent:" + System.getProperty("user.dir") + File.separator + "..");
-          arg = arg.replace("destfile=..", "destfile=" + System.getProperty("user.dir") + File.separator + "..");
-          return arg;
-        }
-      }
-    }
-    
-    return "-DdummyArg=true";
-  }
-
-  synchronized void signalVMReady() {
-    pendingVMs--;
-    this.notifyAll();
-  }
-  
-  public synchronized boolean waitForVMs(long timeout) throws InterruptedException {
-    long end = System.currentTimeMillis() + timeout;
-    while(pendingVMs > 0) {
-      long remaining = end - System.currentTimeMillis();
-      if(remaining <= 0) {
-        return false;
-      }
-      this.wait(remaining);
-    }
-    
-    return true;
-  }
-  
-  private static class ProcessHolder {
-    private final Process process;
-    private volatile boolean killed = false;
-    
-    public ProcessHolder(Process process) {
-      this.process = process;
-    }
-
-    public void kill() {
-      this.killed = true;
-      process.destroy();
-      
-    }
-
-    public Process getProcess() {
-      return process;
-    }
-
-    public boolean isKilled() {
-      return killed;
-    }
-    
-    public boolean isAlive() {
-      return !killed && process.isAlive();
-    }
-  }
-
-  public RemoteDUnitVMIF getStub(int i) throws AccessException, RemoteException, NotBoundException, InterruptedException {
-    waitForVMs(DUnitLauncher.STARTUP_TIMEOUT);
-    return (RemoteDUnitVMIF) registry.lookup("vm" + i);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java
deleted file mode 100644
index 51c6177..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java
+++ /dev/null
@@ -1,142 +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.test.dunit.standalone;
-
-import java.rmi.RemoteException;
-import java.rmi.server.UnicastRemoteObject;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.internal.logging.LogService;
-
-import hydra.MethExecutor;
-import hydra.MethExecutorResult;
-
-/**
- * @author dsmith
- *
- */
-public class RemoteDUnitVM extends UnicastRemoteObject implements RemoteDUnitVMIF {
-  
-  private static final Logger logger = LogService.getLogger();
-  
-  public RemoteDUnitVM() throws RemoteException {
-    super();
-  }
-
-  /** 
-   * Called remotely by the master controller to cause the client to execute 
-   * the instance method on the object.  Does this synchronously (does not spawn
-   * a thread).  This method is used by the unit test framework, dunit.
-   *
-   * @param obj the object to execute the method on
-   * @param methodName the name of the method to execute
-   * @return the result of method execution
-   */ 
-   public MethExecutorResult executeMethodOnObject( Object obj, String methodName ) {
-     String name = obj.getClass().getName() + "." + methodName + 
-       " on object: " + obj;
-     logger.info("Received method: " + name);
-     long start = System.currentTimeMillis();
-     MethExecutorResult result = MethExecutor.executeObject( obj, methodName );
-     long delta = System.currentTimeMillis() - start;
-     logger.info( "Got result: " + result.toString().trim()  + " from " +
-               name + " (took " + delta + " ms)");
-     return result;
-   }
-
-   /**
-    * Executes a given instance method on a given object with the given
-    * arguments. 
-    */
-   public MethExecutorResult executeMethodOnObject(Object obj,
-                                                   String methodName,
-                                                   Object[] args) {
-     String name = obj.getClass().getName() + "." + methodName + 
-              (args != null ? " with " + args.length + " args": "") +
-       " on object: " + obj;
-     logger.info("Received method: " + name);
-     long start = System.currentTimeMillis();
-     MethExecutorResult result = 
-       MethExecutor.executeObject(obj, methodName, args);
-     long delta = System.currentTimeMillis() - start;
-     logger.info( "Got result: " + result.toString() + " from " + name + 
-               " (took " + delta + " ms)");
-     return result;
-   }
-
-  /** 
-   * Called remotely by the master controller to cause the client to execute 
-   * the method on the class.  Does this synchronously (does not spawn a thread).
-   * This method is used by the unit test framework, dunit.
-   *
-   * @param className the name of the class execute
-   * @param methodName the name of the method to execute
-   * @return the result of method execution
-   */ 
-   public MethExecutorResult executeMethodOnClass( String className, String methodName ) {
-     String name = className + "." + methodName;
-     logger.info("Received method: " +  name);
-     long start = System.currentTimeMillis();
-     MethExecutorResult result = MethExecutor.execute( className, methodName );
-     long delta = System.currentTimeMillis() - start;
-     logger.info( "Got result: " + result.toString() + " from " + name + 
-               " (took " + delta + " ms)");
-     
-     return result;
-   }
-
-   /**
-    * Executes a given static method in a given class with the given
-    * arguments. 
-    */
-   public MethExecutorResult executeMethodOnClass(String className,
-                                                  String methodName,
-                                                  Object[] args) {
-     String name = className + "." + methodName + 
-       (args != null ? " with " + args.length + " args": "");
-     logger.info("Received method: " + name);
-     long start = System.currentTimeMillis();
-     MethExecutorResult result = 
-       MethExecutor.execute(className, methodName, args);
-     long delta = System.currentTimeMillis() - start;
-     logger.info( "Got result: " + result.toString() + " from " + name +
-               " (took " + delta + " ms)");
-     return result;
-   }
-
-  public void executeTask(int tsid, int type, int index) throws RemoteException {
-    throw new UnsupportedOperationException();
-    
-  }
-  
-  public void runShutdownHook() throws RemoteException {
-    
-  }
-
-  public void notifyDynamicActionComplete(int actionId) throws RemoteException {
-    throw new UnsupportedOperationException();
-    
-  }
-
-  public void shutDownVM() throws RemoteException {
-    ChildVM.stopVM();
-  }
-
-  public void disconnectVM() throws RemoteException {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java
deleted file mode 100644
index 849e2f2..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java
+++ /dev/null
@@ -1,36 +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.test.dunit.standalone;
-
-import hydra.MethExecutorResult;
-
-import java.rmi.Remote;
-import java.rmi.RemoteException;
-
-public interface RemoteDUnitVMIF extends Remote {
-
-  MethExecutorResult executeMethodOnObject(Object o, String methodName) throws RemoteException;
-
-  MethExecutorResult executeMethodOnObject(Object o, String methodName,
-      Object[] args) throws RemoteException;
-
-  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/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java
deleted file mode 100644
index eef24fe..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java
+++ /dev/null
@@ -1,75 +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.test.dunit.standalone;
-
-import java.io.File;
-import java.rmi.RemoteException;
-import java.util.Properties;
-
-import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher.MasterRemote;
-
-import dunit.DUnitEnv;
-
-public class StandAloneDUnitEnv extends DUnitEnv {
-
-  private MasterRemote master;
-
-  public StandAloneDUnitEnv(MasterRemote master) {
-    this.master = master;
-  }
-
-  @Override
-  public String getLocatorString() {
-    return DUnitLauncher.getLocatorString();
-  }
-
-  @Override
-  public String getLocatorAddress() {
-    return "localhost";
-  }
-  
-  @Override
-  public int getLocatorPort() {
-    return DUnitLauncher.locatorPort;
-  }
-
-  @Override
-  public Properties getDistributedSystemProperties() {
-    return DUnitLauncher.getDistributedSystemProperties();
-  }
-
-  @Override
-  public int getPid() {
-    return Integer.getInteger(DUnitLauncher.VM_NUM_PARAM, -1).intValue();
-  }
-
-  @Override
-  public int getVMID() {
-    return getPid();
-  }
-
-  @Override
-  public BounceResult bounce(int pid) throws RemoteException {
-    return master.bounce(pid);
-  }
-
-  @Override
-  public File getWorkingDirectory(int pid) {
-    return ProcessManager.getVMDir(pid);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
deleted file mode 100644
index 76faf93..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
+++ /dev/null
@@ -1,132 +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.test.dunit.tests;
-
-import java.util.Properties;
-
-import dunit.AsyncInvocation;
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.RMIException;
-import dunit.VM;
-
-/**
- * This class tests the basic functionality of the distributed unit
- * test framework.
- */
-public class BasicDUnitTest extends DistributedTestCase {
-
-  public BasicDUnitTest(String name) {
-    super(name);
-  }
-
-  ////////  Test Methods
-
-  /**
-   * Tests how the Hydra framework handles an error
-   */
-  public void _testDontCatchRemoteException() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    vm.invoke(this.getClass(), "remoteThrowException");
-  }
-
-  public void testRemoteInvocationWithException() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(this.getClass(), "remoteThrowException");
-      fail("Should have thrown a BasicTestException");
-
-    } catch (RMIException ex) {
-      assertTrue(ex.getCause() instanceof BasicTestException);
-    }
-  } 
-
-  static class BasicTestException extends RuntimeException {
-    BasicTestException() {
-      this("Test exception.  Please ignore.");
-    }
-
-    BasicTestException(String s) {
-      super(s);
-    }
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   *
-   */
-  protected static void remoteThrowException() {
-    String s = "Test exception.  Please ignore.";
-    throw new BasicTestException(s);
-  }
-
-  public void _testRemoteInvocationBoolean() {
-
-  }
-
-  public void testRemoteInvokeAsync() throws InterruptedException {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    String name = this.getUniqueName();
-    String value = "Hello";
-
-    AsyncInvocation ai =
-      vm.invokeAsync(this.getClass(), "remoteBind", 
-                     new Object[] { name, value });
-    ai.join();
-    // TODO shouldn't we call fail() here?
-    if (ai.exceptionOccurred()) {
-      fail("remoteBind failed", ai.getException());
-    }
-
-    ai = vm.invokeAsync(this.getClass(), "remoteValidateBind",
-                        new Object[] {name, value });
-    ai.join();
-    if (ai.exceptionOccurred()) {
-      fail("remoteValidateBind failed", ai.getException());
-    }
-  }
-
-  private static Properties bindings = new Properties();
-  private static void remoteBind(String name, String s) {
-    new BasicDUnitTest("bogus").getSystem(); // forces connection
-    bindings.setProperty(name, s);
-  }
-
-  private static void remoteValidateBind(String name, String expected)
-  {
-    assertEquals(expected, bindings.getProperty(name));
-  }
-
-  public void testRemoteInvokeAsyncWithException() 
-    throws InterruptedException {
-
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-//    String name = this.getUniqueName();
-//    String value = "Hello";
-
-    AsyncInvocation ai =
-      vm.invokeAsync(this.getClass(), "remoteThrowException");
-    ai.join();
-    assertTrue(ai.exceptionOccurred());
-    Throwable ex = ai.getException();
-    assertTrue(ex instanceof BasicTestException);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java
deleted file mode 100644
index abbe229..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java
+++ /dev/null
@@ -1,50 +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.test.dunit.tests;
-
-import dunit.*;
-
-/**
- * The tests in this class always fail.  It is used when developing
- * DUnit to give us an idea of how test failure are logged, etc.
- *
- * @author David Whitlock
- *
- * @since 3.0
- */
-public class TestFailure extends DistributedTestCase {
-
-  public TestFailure(String name) {
-    super(name);
-  }
-
-  ////////  Test Methods
-
-  public void testFailure() {
-    assertTrue("Test Failure", false);
-  }
-
-  public void testError() {
-    String s = "Test Error";
-    throw new Error(s);
-  }
-
-  public void testHang() throws InterruptedException {
-    Thread.sleep(100000 * 1000);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java
deleted file mode 100644
index 3562f86..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java
+++ /dev/null
@@ -1,237 +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.test.dunit.tests;
-
-import dunit.*;
-
-import java.io.Serializable;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * This class tests the functionality of the {@link VM} class.
- */
-public class VMDUnitTest extends DistributedTestCase {
-
-  private static final boolean BOOLEAN_VALUE = true;
-  private static final byte BYTE_VALUE = (byte) 40;
-  private static final long LONG_VALUE = 42L;
-  private static final String STRING_VALUE = "BLAH BLAH BLAH";
-
-  public VMDUnitTest(String name) {
-    super(name);
-  }
-
-  ////////  Test Methods
-
-  public void notestInvokeNonExistentMethod() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(VMDUnitTest.class, "nonExistentMethod");
-      fail("Should have thrown an RMIException");
-
-    } catch (RMIException ex) {
-      String s = "Excepted a NoSuchMethodException, got a " +
-        ex.getCause();;
-      assertTrue(s, ex.getCause() instanceof NoSuchMethodException);
-    }
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   * @return
-   */
-  protected static byte remoteByteMethod() {
-    return BYTE_VALUE;
-  }
-
-  public void notestInvokeStaticBoolean() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(BOOLEAN_VALUE,
-                 vm.invokeBoolean(VMDUnitTest.class, "remoteBooleanMethod")); 
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   * @return
-   */
-  protected static boolean remoteBooleanMethod() {
-    return BOOLEAN_VALUE;
-  }
-
-  public void notestInvokeStaticBooleanNotBoolean() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invokeBoolean(VMDUnitTest.class, "remoteByteMethod");
-      fail("Should have thrown an IllegalArgumentException");
-
-    } catch (IllegalArgumentException ex) {
-      
-    }
-  }
-
-  public void notestInvokeStaticLong() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(LONG_VALUE,
-                 vm.invokeLong(VMDUnitTest.class, "remoteLongMethod")); 
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   * @return
-   */
-  protected static long remoteLongMethod() {
-    return LONG_VALUE;
-  }
-
-  public void notestInvokeStaticLongNotLong() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invokeLong(VMDUnitTest.class, "remoteByteMethod");
-      fail("Should have thrown an IllegalArgumentException");
-
-    } catch (IllegalArgumentException ex) {
-      
-    }
-  }
-
-  protected static class ClassWithLong implements Serializable {
-    public long getLong() {
-      return LONG_VALUE;
-    }
-  }
-
-  protected static class ClassWithByte implements Serializable {
-    public byte getByte() {
-      return BYTE_VALUE;
-    }
-  }
-
-  public void notestInvokeInstanceLong() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(LONG_VALUE,
-                 vm.invokeLong(new ClassWithLong(), "getLong"));
-  }
-
-  public void notestInvokeInstanceLongNotLong() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invokeLong(new ClassWithByte(), "getByte");
-      fail("Should have thrown an IllegalArgumentException");
-
-    } catch (IllegalArgumentException ex) {
-
-    }
-  }
-
-  protected static class InvokeRunnable
-    implements Serializable, Runnable {
-
-    public void run() {
-      throw new BasicDUnitTest.BasicTestException();
-    }
-  }
-
-  protected static class ClassWithString implements Serializable {
-    public String getString() {
-      return STRING_VALUE;
-    }
-  }
-
-  public void notestInvokeInstance() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(STRING_VALUE,
-                 vm.invoke(new ClassWithString(), "getString"));
-  }
-
-  public void notestInvokeRunnable() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(new InvokeRunnable());
-      fail("Should have thrown a BasicTestException");
-
-    } catch (RMIException ex) {
-      assertTrue(ex.getCause() instanceof BasicDUnitTest.BasicTestException);
-    }
-  }
-  
-  private static final AtomicInteger COUNTER = new AtomicInteger();
-  public static Integer getAndIncStaticCount() {
-    return new Integer(COUNTER.getAndIncrement());
-  }
-  public static Integer incrementStaticCount(Integer inc) {
-    return new Integer(COUNTER.addAndGet(inc.intValue()));
-  }
-  public static void incStaticCount() {
-    COUNTER.incrementAndGet();
-  }
-  public static class VMTestObject implements Serializable {
-    private static final long serialVersionUID = 1L;
-    private final AtomicInteger val;
-    public VMTestObject(int init) {
-      this.val = new AtomicInteger(init);
-    }
-    public Integer get() {
-      return new Integer(this.val.get());
-    }
-    public Integer incrementAndGet() {
-      return new Integer(this.val.incrementAndGet());
-    }
-    public void set(Integer newVal) {
-      this.val.set(newVal.intValue());
-    }
-  }
-  public void testReturnValue() throws Exception {
-    final Host host = Host.getHost(0);
-    final VM vm = host.getVM(0);
-    // Assert class static invocation works
-    AsyncInvocation a1 = vm.invokeAsync(getClass(), "getAndIncStaticCount");
-    a1.join();
-    assertEquals(new Integer(0), a1.getReturnValue());
-    // Assert class static invocation with args works
-    a1 = vm.invokeAsync(getClass(), "incrementStaticCount", new Object[] {new Integer(2)});
-    a1.join();
-    assertEquals(new Integer(3), a1.getReturnValue());
-    // Assert that previous values are not returned when invoking method w/ no return val
-    a1 = vm.invokeAsync(getClass(), "incStaticCount");
-    a1.join();
-    assertNull(a1.getReturnValue());
-    // Assert that previous null returns are over-written 
-    a1 = vm.invokeAsync(getClass(), "getAndIncStaticCount");
-    a1.join();
-    assertEquals(new Integer(4), a1.getReturnValue());
-
-    // Assert object method invocation works with zero arg method
-    final VMTestObject o = new VMTestObject(0);
-    a1 = vm.invokeAsync(o, "incrementAndGet", new Object[] {});
-    a1.join();
-    assertEquals(new Integer(1), a1.getReturnValue());
-    // Assert object method invocation works with no return
-    a1 = vm.invokeAsync(o, "set", new Object[] {new Integer(3)});
-    a1.join();
-    assertNull(a1.getReturnValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/dunit/BounceResult.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/BounceResult.java b/gemfire-core/src/test/java/dunit/BounceResult.java
new file mode 100644
index 0000000..d8ac186
--- /dev/null
+++ b/gemfire-core/src/test/java/dunit/BounceResult.java
@@ -0,0 +1,36 @@
+/*
+ * 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 dunit;
+
+public class BounceResult {
+  private final int newPid;
+  private final RemoteDUnitVMIF newClient;
+  
+  public BounceResult(int newPid, RemoteDUnitVMIF newClient) {
+    this.newPid = newPid;
+    this.newClient = newClient;
+  }
+
+  public int getNewPid() {
+    return newPid;
+  }
+
+  public RemoteDUnitVMIF getNewClient() {
+    return newClient;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/dunit/DUnitEnv.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/DUnitEnv.java b/gemfire-core/src/test/java/dunit/DUnitEnv.java
index 54fe67f..19c3635 100644
--- a/gemfire-core/src/test/java/dunit/DUnitEnv.java
+++ b/gemfire-core/src/test/java/dunit/DUnitEnv.java
@@ -23,8 +23,6 @@ import java.io.File;
 import java.rmi.RemoteException;
 import java.util.Properties;
 
-import com.gemstone.gemfire.test.dunit.standalone.BounceResult;
-
 
 /**
  * This class provides an abstraction over the environment

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/dunit/Host.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/Host.java b/gemfire-core/src/test/java/dunit/Host.java
index 0c69783..cc41316 100644
--- a/gemfire-core/src/test/java/dunit/Host.java
+++ b/gemfire-core/src/test/java/dunit/Host.java
@@ -18,8 +18,6 @@ package dunit;
 
 import java.util.*;
 
-import com.gemstone.gemfire.test.dunit.standalone.RemoteDUnitVMIF;
-
 /**
  * <P>This class represents a host on which a remote method may be
  * invoked.  It provides access to the VMs and GemFire systems that

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/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
new file mode 100644
index 0000000..5dffa47
--- /dev/null
+++ b/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java
@@ -0,0 +1,36 @@
+/*
+ * 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 dunit;
+
+import hydra.MethExecutorResult;
+
+import java.rmi.Remote;
+import java.rmi.RemoteException;
+
+public interface RemoteDUnitVMIF extends Remote {
+
+  MethExecutorResult executeMethodOnObject(Object o, String methodName) throws RemoteException;
+
+  MethExecutorResult executeMethodOnObject(Object o, String methodName,
+      Object[] args) throws RemoteException;
+
+  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/dfba327e/gemfire-core/src/test/java/dunit/VM.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/VM.java b/gemfire-core/src/test/java/dunit/VM.java
index f4cde93..3c0a7b7 100644
--- a/gemfire-core/src/test/java/dunit/VM.java
+++ b/gemfire-core/src/test/java/dunit/VM.java
@@ -16,6 +16,8 @@
  */
 package dunit;
 
+import hydra.MethExecutorResult;
+
 import java.io.File;
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -24,11 +26,6 @@ import java.util.concurrent.Callable;
 //import java.util.Iterator;
 //import java.util.Vector;
 
-import hydra.MethExecutorResult;
-
-import com.gemstone.gemfire.test.dunit.standalone.BounceResult;
-import com.gemstone.gemfire.test.dunit.standalone.RemoteDUnitVMIF;
-
 /**
  * This class represents a Java Virtual Machine that runs on a host.
  *

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/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
new file mode 100644
index 0000000..45a236a
--- /dev/null
+++ b/gemfire-core/src/test/java/dunit/standalone/ChildVM.java
@@ -0,0 +1,82 @@
+/*
+ * 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 dunit.standalone;
+
+import hydra.HydraRuntimeException;
+import hydra.Log;
+
+import java.rmi.Naming;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.internal.OSProcess;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+import dunit.standalone.DUnitLauncher.MasterRemote;
+
+/**
+ * @author dsmith
+ *
+ */
+public class ChildVM {
+  
+  private static boolean stopMainLoop = false;
+  
+  /**
+   * tells the main() loop to exit
+   */
+  public static void stopVM() {
+    stopMainLoop = true;
+  }
+  
+  static {
+    createHydraLogWriter();
+  }
+  
+  private final static Logger logger = LogService.getLogger();
+  private static RemoteDUnitVM dunitVM;
+  
+  public static void main(String[] args) throws Throwable {
+    try {
+      int namingPort = Integer.getInteger(DUnitLauncher.RMI_PORT_PARAM).intValue();
+      int vmNum = Integer.getInteger(DUnitLauncher.VM_NUM_PARAM).intValue();
+      int pid = OSProcess.getId();
+      logger.info("VM" + vmNum + " is launching" + (pid > 0? " with PID " + pid : ""));
+      MasterRemote holder = (MasterRemote) Naming.lookup("//localhost:" + namingPort + "/" + DUnitLauncher.MASTER_PARAM);
+      DUnitLauncher.init(holder);
+      DUnitLauncher.locatorPort = holder.getLocatorPort();
+      dunitVM = new RemoteDUnitVM();
+      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 (!stopMainLoop) {
+        holder.ping();
+        Thread.sleep(1000);
+      }
+    } catch (Throwable t) {
+      t.printStackTrace();
+      System.exit(1);
+    }
+  }
+
+  private static void createHydraLogWriter() {
+    try {
+      Log.createLogWriter("dunit-childvm", "fine");
+    } catch (HydraRuntimeException ignore) {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/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
new file mode 100644
index 0000000..72c33d6
--- /dev/null
+++ b/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
@@ -0,0 +1,463 @@
+/*
+ * 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 dunit.standalone;
+
+import hydra.Log;
+import hydra.MethExecutorResult;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.URISyntaxException;
+import java.nio.channels.FileChannel;
+import java.nio.charset.Charset;
+import java.rmi.AccessException;
+import java.rmi.AlreadyBoundException;
+import java.rmi.NotBoundException;
+import java.rmi.Remote;
+import java.rmi.RemoteException;
+import java.rmi.registry.LocateRegistry;
+import java.rmi.registry.Registry;
+import java.rmi.server.UnicastRemoteObject;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.appender.FileAppender;
+import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.apache.logging.log4j.core.layout.PatternLayout;
+import org.junit.Assert;
+
+import batterytest.greplogs.ExpectedStrings;
+import batterytest.greplogs.LogConsumer;
+
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+import dunit.BounceResult;
+import dunit.DUnitEnv;
+import dunit.Host;
+import dunit.RemoteDUnitVMIF;
+import dunit.SerializableCallable;
+import dunit.VM;
+
+/**
+ * A class to build a fake test configuration and launch some DUnit VMS.
+ * 
+ * For use within eclipse. This class completely skips hydra and just starts
+ * some vms directly, creating a fake test configuration
+ * 
+ * Also, it's a good idea to set your working directory, because the test code
+ * a lot of files that it leaves around.
+ * 
+ * @author dsmith
+ *
+ */
+public class DUnitLauncher {
+
+  /** change this to use a different log level in unit tests */
+  public static final String LOG_LEVEL = System.getProperty("logLevel", "info");
+  
+  static int locatorPort;
+
+  private static final int NUM_VMS = 4;
+  private static final int DEBUGGING_VM_NUM = -1;
+  private static final int LOCATOR_VM_NUM = -2;
+
+  static final long STARTUP_TIMEOUT = 30 * 1000;
+  private static final String SUSPECT_FILENAME = "dunit_suspect.log";
+  private static File DUNIT_SUSPECT_FILE;
+
+  public static final String DUNIT_DIR = "dunit";
+  public static final String WORKSPACE_DIR_PARAM = "WORKSPACE_DIR";
+  public static final boolean LOCATOR_LOG_TO_DISK = Boolean.getBoolean("locatorLogToDisk");
+
+  static final String MASTER_PARAM = "DUNIT_MASTER";
+  static final String RMI_PORT_PARAM = "gemfire.DUnitLauncher.RMI_PORT";
+  static final String VM_NUM_PARAM = "gemfire.DUnitLauncher.VM_NUM";
+
+  private static final String LAUNCHED_PROPERTY = "gemfire.DUnitLauncher.LAUNCHED";
+
+  private static Master master;
+
+  private DUnitLauncher() {
+  }
+  
+  private static boolean isHydra() {
+    try {
+      //TODO - this is hacky way to test for a hydra environment - see
+      //if there is registered test configuration object.
+      Class<?> clazz = Class.forName("hydra.TestConfig");
+      Method getInstance = clazz.getMethod("getInstance", new Class[0]);
+      getInstance.invoke(null);
+      return true;
+    } catch (Exception e) {
+      return false;
+    }
+  }
+  /**
+   * Launch DUnit. If the unit test was launched through
+   * the hydra framework, leave the test alone.
+   */
+  public static void launchIfNeeded() {
+    if(System.getProperties().contains(VM_NUM_PARAM)) {
+      //we're a dunit child vm, do nothing.
+      return;
+    }
+
+    if(!isHydra() &&!isLaunched()) {
+      try {
+        launch();
+      } catch (Exception e) {
+        throw new RuntimeException("Unable to launch dunit VMS", e);
+      }
+    }
+  }
+  
+  /**
+   * Test it see if the eclise dunit environment is launched.
+   */
+  public static boolean isLaunched() {
+    return Boolean.getBoolean(LAUNCHED_PROPERTY);
+  }
+  
+  public static String getLocatorString() {
+    return "localhost[" + locatorPort + "]";
+  }
+
+  
+  private static void launch() throws URISyntaxException, AlreadyBoundException, IOException, InterruptedException, NotBoundException  {
+//  initialize the log writer that hydra uses
+    Log.createLogWriter( "dunit-master", LOG_LEVEL );
+
+    DUNIT_SUSPECT_FILE = new File(SUSPECT_FILENAME);
+    DUNIT_SUSPECT_FILE.delete();
+    DUNIT_SUSPECT_FILE.deleteOnExit();
+    
+    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+     
+    //create an RMI registry and add an object to share our tests config
+    int namingPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    Registry registry = LocateRegistry.createRegistry(namingPort);
+
+    final ProcessManager processManager = new ProcessManager(namingPort, registry);
+    master = new Master(registry, processManager);
+    registry.bind(MASTER_PARAM, master);
+
+    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();
+      }
+    });
+    
+    //Create a VM for the locator
+    processManager.launchVM(LOCATOR_VM_NUM);
+    
+    //Launch an initial set of VMs
+    for(int i=0; i < NUM_VMS; i++) {
+      processManager.launchVM(i);
+    }
+    
+    //wait for the VMS to start up
+    if(!processManager.waitForVMs(STARTUP_TIMEOUT)) {
+      throw new RuntimeException("VMs did not start up with 30 seconds");
+    }
+    
+    //populate the Host class with our stubs. The tests use this host class
+    DUnitHost host = new DUnitHost(InetAddress.getLocalHost().getCanonicalHostName(), processManager);
+    host.init(registry, NUM_VMS);
+
+    init(master);
+    
+    startLocator(registry);
+  }
+  
+  public static Properties getDistributedSystemProperties() {
+    Properties p = new Properties();
+    p.setProperty("locators", getLocatorString());
+    p.setProperty("mcast-port", "0");
+    p.setProperty("enable-cluster-configuration", "false");
+    p.setProperty("use-cluster-configuration", "false");
+    p.setProperty("log-level", LOG_LEVEL);
+    return p;
+  }
+
+  /**
+   * Add an appender to Log4j which sends all INFO+ messages to a separate file
+   * which will be used later to scan for suspect strings.  The pattern of the
+   * messages conforms to the original log format so that hydra will be able
+   * to parse them.
+   */
+  private static void addSuspectFileAppender(final String workspaceDir) {
+    final String suspectFilename = new File(workspaceDir, SUSPECT_FILENAME).getAbsolutePath();
+
+    final LoggerContext appenderContext = ((org.apache.logging.log4j.core.Logger)
+        LogManager.getLogger(LogService.BASE_LOGGER_NAME)).getContext();
+
+    final PatternLayout layout = PatternLayout.createLayout(
+        "[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} <%thread> tid=%tid] %message%n%throwable%n", null, null,
+        Charset.defaultCharset(), true, false, "", "");
+    
+    final FileAppender fileAppender = FileAppender.createAppender(suspectFilename, "true", "false",
+        DUnitLauncher.class.getName(), "true", "false", "false", "0", layout, null, null, null, appenderContext.getConfiguration());
+    fileAppender.start();
+
+    LoggerConfig loggerConfig = appenderContext.getConfiguration().getLoggerConfig(LogService.BASE_LOGGER_NAME);
+    loggerConfig.addAppender(fileAppender, Level.INFO, null);
+  }
+  
+  private static void startLocator(Registry registry) throws IOException, NotBoundException {
+    RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + LOCATOR_VM_NUM);
+    final File locatorLogFile =
+        LOCATOR_LOG_TO_DISK ? new File("locator-" + locatorPort + ".log") : new File(""); 
+    MethExecutorResult result = remote.executeMethodOnObject(new SerializableCallable() {
+      public Object call() throws IOException {
+        Properties p = getDistributedSystemProperties();
+        // I never want this locator to end up starting a jmx manager
+        // since it is part of the unit test framework
+        p.setProperty("jmx-manager", "false");
+        //Disable the shared configuration on this locator.
+        //Shared configuration tests create their own locator
+        p.setProperty("enable-cluster-configuration", "false");
+        //Tell the locator it's the first in the system for
+        //faster boot-up
+        
+        System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
+        try {
+          Locator.startLocatorAndDS(locatorPort, locatorLogFile, p);
+        } finally {
+          System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
+        }
+        
+        return null;
+      }
+    }, "call");
+    if(result.getException() != null) {
+      RuntimeException ex = new RuntimeException("Failed to start locator", result.getException());
+      ex.printStackTrace();
+      throw ex;
+    }
+  }
+
+  public static void init(MasterRemote master) {
+    DUnitEnv.set(new StandAloneDUnitEnv(master));
+    //fake out tests that are using a bunch of hydra stuff
+    String workspaceDir = System.getProperty(DUnitLauncher.WORKSPACE_DIR_PARAM) ;
+    workspaceDir = workspaceDir == null ? new File(".").getAbsolutePath() : workspaceDir;
+    
+    addSuspectFileAppender(workspaceDir);
+    
+    //Free off heap memory when disconnecting from the distributed system
+    System.setProperty("gemfire.free-off-heap-memory", "true");
+    
+    //indicate that this CM is controlled by the eclipse dunit.
+    System.setProperty(LAUNCHED_PROPERTY, "true");
+  }
+  
+  public static void closeAndCheckForSuspects() {
+    if (isLaunched()) {
+      final boolean skipLogMsgs = ExpectedStrings.skipLogMsgs("dunit");
+      final List<?> expectedStrings = ExpectedStrings.create("dunit");
+      final LogConsumer logConsumer = new LogConsumer(skipLogMsgs, expectedStrings, "log4j", 5);
+
+      final StringBuilder suspectStringBuilder = new StringBuilder();
+
+      BufferedReader buffReader = null;
+      FileChannel fileChannel = null;
+      try {
+        fileChannel = new FileOutputStream(DUNIT_SUSPECT_FILE, true).getChannel();
+        buffReader = new BufferedReader(new FileReader(DUNIT_SUSPECT_FILE));
+      } catch (FileNotFoundException e) {
+        System.err.println("Could not find the suspect string output file: " + e);
+        return;
+      }
+      try {
+        String line;
+        try {
+          while ((line = buffReader.readLine()) != null) {
+            final StringBuilder builder = logConsumer.consume(line);
+            if (builder != null) {
+              suspectStringBuilder.append(builder);
+            }
+          }
+        } catch (IOException e) {
+          System.err.println("Could not read the suspect string output file: " + e);
+        }
+        
+        try {
+          fileChannel.truncate(0);
+        } catch (IOException e) {
+          System.err.println("Could not truncate the suspect string output file: " + e);
+        }
+        
+      } finally {
+        try {
+          buffReader.close();
+          fileChannel.close();
+        } catch (IOException e) {
+          System.err.println("Could not close the suspect string output file: " + e);
+        }
+      }
+
+      if (suspectStringBuilder.length() != 0) {
+        System.err.println("Suspicious strings were written to the log during this run.\n"
+            + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n"
+            + suspectStringBuilder);
+        
+        Assert.fail("Suspicious strings were written to the log during this run.\n"
+            + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n"
+            + suspectStringBuilder);
+      }
+    }
+  }
+
+  public interface MasterRemote extends Remote {
+    public int getLocatorPort() throws RemoteException;
+    public void signalVMReady() throws RemoteException;
+    public void ping() throws RemoteException;
+    public BounceResult bounce(int pid) throws RemoteException;
+  }
+  
+  public static class Master extends UnicastRemoteObject implements MasterRemote {
+    private static final long serialVersionUID = 1178600200232603119L;
+    
+    private final Registry registry;
+    private final ProcessManager processManager;
+
+
+    public Master(Registry registry, ProcessManager processManager) throws RemoteException {
+      this.processManager = processManager;
+      this.registry = registry;
+    }
+
+    public int getLocatorPort()  throws RemoteException{
+      return locatorPort;
+    }
+
+    public synchronized void signalVMReady() {
+      processManager.signalVMReady();
+    }
+    
+    public void ping() {
+      //do nothing
+    }
+
+    @Override
+    public BounceResult bounce(int pid) {
+      processManager.bounce(pid);
+      
+      try {
+        if(!processManager.waitForVMs(STARTUP_TIMEOUT)) {
+          throw new RuntimeException("VMs did not start up with 30 seconds");
+        }
+        RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + pid);
+        return new BounceResult(pid, remote);
+      } catch (RemoteException | NotBoundException e) {
+        throw new RuntimeException("could not lookup name", e);
+      } catch (InterruptedException e) {
+        throw new RuntimeException("Failed waiting for VM", e);
+      }
+    }
+  }
+  
+  private static class DUnitHost extends Host {
+    private static final long serialVersionUID = -8034165624503666383L;
+    
+    private transient final VM debuggingVM;
+
+    private transient ProcessManager processManager;
+    
+    public DUnitHost(String hostName, ProcessManager processManager) throws RemoteException {
+      super(hostName);
+      this.debuggingVM = new VM(this, -1, new RemoteDUnitVM());
+      this.processManager = processManager;
+    }
+    
+    public void init(Registry registry, int numVMs) throws AccessException, RemoteException, NotBoundException, InterruptedException {
+      for(int i = 0; i < numVMs; i++) {
+        RemoteDUnitVMIF remote = processManager.getStub(i);
+        addVM(i, remote);
+      }
+      
+      addLocator(LOCATOR_VM_NUM, processManager.getStub(LOCATOR_VM_NUM));
+      
+      addHost(this);
+    }
+
+    @Override
+    public VM getVM(int n) {
+      
+      if(n == DEBUGGING_VM_NUM) {
+        //for ease of debugging, pass -1 to get the local VM
+        return debuggingVM;
+      }
+
+      int oldVMCount = getVMCount();
+      if(n >= oldVMCount) {
+        //If we don't have a VM with that number, dynamically create it.
+        try {
+          for(int i = oldVMCount; i <= n; i++) {
+            processManager.launchVM(i);
+          }
+          processManager.waitForVMs(STARTUP_TIMEOUT);
+
+          for(int i = oldVMCount; i <= n; i++) {
+            addVM(i, processManager.getStub(i));
+          }
+
+        } catch (IOException | InterruptedException | NotBoundException e) {
+          throw new RuntimeException("Could not dynamically launch vm + " + n, e);
+        }
+      }
+      
+      return super.getVM(n);
+    }
+  }
+}



[17/50] [abbrv] incubator-geode git commit: Revert "GEODE-714: Modify all tests to use JUnit Categories"

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java
new file mode 100755
index 0000000..90679d9
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java
@@ -0,0 +1,1015 @@
+/*
+ * 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.tier.sockets;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.InterestResultPolicy;
+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.PoolManager;
+import com.gemstone.gemfire.cache.client.internal.Connection;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.cache.client.internal.RegisterInterestTracker;
+import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.tier.InterestType;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.VM;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Tests Interest Registration Functionality
+ */
+@SuppressWarnings({"deprecation", "rawtypes", "serial", "unchecked"})
+public class HAInterestBaseTest extends DistributedTestCase {
+  
+  protected static final int TIMEOUT_MILLIS = 60 * 1000;
+  protected static final int INTERVAL_MILLIS = 10;
+  
+  protected static final String REGION_NAME = "HAInterestBaseTest_region";
+  
+  protected static final String k1 = "k1";
+  protected static final String k2 = "k2";
+  protected static final String client_k1 = "client-k1";
+  protected static final String client_k2 = "client-k2";
+  protected static final String server_k1 = "server-k1";
+  protected static final String server_k2 = "server-k2";
+  protected static final String server_k1_updated = "server_k1_updated";
+
+  protected static Cache cache = null;
+  protected static PoolImpl pool = null;
+  protected static Connection conn = null;
+
+  protected static int PORT1;
+  protected static int PORT2;
+  protected static int PORT3;
+
+  protected static boolean isBeforeRegistrationCallbackCalled = false;
+  protected static boolean isBeforeInterestRecoveryCallbackCalled = false;
+  protected static boolean isAfterRegistrationCallbackCalled = false;
+
+  protected static Host host = null;
+  protected static VM server1 = null;
+  protected static VM server2 = null;
+  protected static VM server3 = null;
+  
+  protected volatile static boolean exceptionOccured = false;
+
+  public HAInterestBaseTest(String name) {
+    super(name);
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    host = Host.getHost(0);
+    server1 = host.getVM(0);
+    server2 = host.getVM(1);
+    server3 = host.getVM(2);
+    CacheServerTestUtil.disableShufflingOfEndpoints();
+    // start servers first
+    PORT1 = ((Integer) server1.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
+    PORT2 = ((Integer) server2.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
+    PORT3 = ((Integer) server3.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
+    exceptionOccured = false;
+    addExpectedException("java.net.ConnectException: Connection refused: connect");
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    // close the clients first
+    closeCache();
+
+    // then close the servers
+    server1.invoke(HAInterestBaseTest.class, "closeCache");
+    server2.invoke(HAInterestBaseTest.class, "closeCache");
+    server3.invoke(HAInterestBaseTest.class, "closeCache");
+    CacheServerTestUtil.resetDisableShufflingOfEndpointsFlag();
+  }
+
+  public static void closeCache() {
+    PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
+    PoolImpl.BEFORE_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = false;
+    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
+    PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
+    HAInterestBaseTest.isAfterRegistrationCallbackCalled = false;
+    HAInterestBaseTest.isBeforeInterestRecoveryCallbackCalled = false;
+    HAInterestBaseTest.isBeforeRegistrationCallbackCalled = false;
+    if (cache != null && !cache.isClosed()) {
+      cache.close();
+      cache.getDistributedSystem().disconnect();
+    }
+    cache = null;
+    pool = null;
+    conn = null;
+  }
+  
+  /**
+   * Return the current primary waiting for a primary to exist.
+   * 
+   * @since 5.7
+   */
+  public static VM getPrimaryVM() {
+    return getPrimaryVM(null);
+  }
+
+  /**
+   * Return the current primary waiting for a primary to exist and for it not to
+   * be the oldPrimary (if oldPrimary is NOT null).
+   * 
+   * @since 5.7
+   */
+  public static VM getPrimaryVM(final VM oldPrimary) {
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        int primaryPort = pool.getPrimaryPort();
+        if (primaryPort == -1) {
+          return false;
+        }
+        // we have a primary
+        VM currentPrimary = getServerVM(primaryPort);
+        if (currentPrimary != oldPrimary) {
+          return true;
+        }
+        return false;
+      }
+      @Override
+      public String description() {
+        return "waiting for primary";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    int primaryPort = pool.getPrimaryPort();
+    assertTrue(primaryPort != -1);
+    VM currentPrimary = getServerVM(primaryPort);
+    assertTrue(currentPrimary != oldPrimary);
+    return currentPrimary;
+  }
+
+  public static VM getBackupVM() {
+    return getBackupVM(null);
+  }
+
+  public static VM getBackupVM(VM stoppedBackup) {
+    VM currentPrimary = getPrimaryVM(null);
+    if (currentPrimary != server2 && server2 != stoppedBackup) {
+      return server2;
+    } else if (currentPrimary != server3 && server3 != stoppedBackup) {
+      return server3;
+    } else if (currentPrimary != server1 && server1 != stoppedBackup) {
+      return server1;
+    } else {
+      fail("expected currentPrimary " + currentPrimary + " to be " + server1 + ", or " + server2 + ", or " + server3);
+      return null;
+    }
+  }
+
+  /**
+   * Given a server vm (server1, server2, or server3) return its port.
+   * 
+   * @since 5.7
+   */
+  public static int getServerPort(VM vm) {
+    if (vm == server1) {
+      return PORT1;
+    } else if (vm == server2) {
+      return PORT2;
+    } else if (vm == server3) {
+      return PORT3;
+    } else {
+      fail("expected vm " + vm + " to be " + server1 + ", or " + server2 + ", or " + server3);
+      return -1;
+    }
+  }
+
+  /**
+   * Given a server port (PORT1, PORT2, or PORT3) return its vm.
+   * 
+   * @since 5.7
+   */
+  public static VM getServerVM(int port) {
+    if (port == PORT1) {
+      return server1;
+    } else if (port == PORT2) {
+      return server2;
+    } else if (port == PORT3) {
+      return server3;
+    } else {
+      fail("expected port " + port + " to be " + PORT1 + ", or " + PORT2 + ", or " + PORT3);
+      return null;
+    }
+  }
+
+  public static void verifyRefreshedEntriesFromServer() {
+    final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r1);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        Region.Entry re = r1.getEntry(k1);
+        if (re == null)
+          return false;
+        Object val = re.getValue();
+        return client_k1.equals(val);
+      }
+      @Override
+      public String description() {
+        return "waiting for client_k1 refresh from server";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        Region.Entry re = r1.getEntry(k2);
+        if (re == null)
+          return false;
+        Object val = re.getValue();
+        return client_k2.equals(val);
+      }
+      @Override
+      public String description() {
+        return "waiting for client_k2 refresh from server";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+  }
+
+  public static void verifyDeadAndLiveServers(final int expectedDeadServers, final int expectedLiveServers) {
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == expectedLiveServers;
+      }
+      @Override
+      public String description() {
+        return "waiting for pool.getConnectedServerCount() == expectedLiveServer";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+  }
+
+  public static void putK1andK2() {
+    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r1);
+    r1.put(k1, server_k1);
+    r1.put(k2, server_k2);
+  }
+
+  public static void setClientServerObserverForBeforeInterestRecoveryFailure() {
+    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
+      public void beforeInterestRecovery() {
+        synchronized (HAInterestBaseTest.class) {
+          Thread t = new Thread() {
+            public void run() {
+              getBackupVM().invoke(HAInterestBaseTest.class, "startServer");
+              getPrimaryVM().invoke(HAInterestBaseTest.class, "stopServer");
+            }
+          };
+          t.start();
+          try {
+            DistributedTestCase.join(t, 30 * 1000, getLogWriter());
+          } catch (Exception ignore) {
+            exceptionOccured = true;
+          }
+          HAInterestBaseTest.isBeforeInterestRecoveryCallbackCalled = true;
+          HAInterestBaseTest.class.notify();
+          PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
+        }
+      }
+    });
+  }
+
+  public static void setClientServerObserverForBeforeInterestRecovery() {
+    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
+      public void beforeInterestRecovery() {
+        synchronized (HAInterestBaseTest.class) {
+          Thread t = new Thread() {
+            public void run() {
+              Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+              assertNotNull(r1);
+              r1.put(k1, server_k1_updated);
+            }
+          };
+          t.start();
+
+          HAInterestBaseTest.isBeforeInterestRecoveryCallbackCalled = true;
+          HAInterestBaseTest.class.notify();
+          PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
+        }
+      }
+    });
+  }
+
+  public static void waitForBeforeInterestRecoveryCallBack() throws InterruptedException {
+    assertNotNull(cache);
+    synchronized (HAInterestBaseTest.class) {
+      while (!isBeforeInterestRecoveryCallbackCalled) {
+        HAInterestBaseTest.class.wait();
+      }
+    }
+  }
+
+  public static void setClientServerObserverForBeforeRegistration(final VM vm) {
+    PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = true;
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
+      public void beforeInterestRegistration() {
+        synchronized (HAInterestBaseTest.class) {
+          vm.invoke(HAInterestBaseTest.class, "startServer");
+          HAInterestBaseTest.isBeforeRegistrationCallbackCalled = true;
+          HAInterestBaseTest.class.notify();
+          PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
+        }
+      }
+    });
+  }
+
+  public static void waitForBeforeRegistrationCallback() throws InterruptedException {
+    assertNotNull(cache);
+    synchronized (HAInterestBaseTest.class) {
+      while (!isBeforeRegistrationCallbackCalled) {
+        HAInterestBaseTest.class.wait();
+      }
+    }
+  }
+
+  public static void setClientServerObserverForAfterRegistration(final VM vm) {
+    PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = true;
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
+      public void afterInterestRegistration() {
+        synchronized (HAInterestBaseTest.class) {
+          vm.invoke(HAInterestBaseTest.class, "startServer");
+          HAInterestBaseTest.isAfterRegistrationCallbackCalled = true;
+          HAInterestBaseTest.class.notify();
+          PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
+        }
+      }
+    });
+  }
+
+  public static void waitForAfterRegistrationCallback() throws InterruptedException {
+    assertNotNull(cache);
+    if (!isAfterRegistrationCallbackCalled) {
+      synchronized (HAInterestBaseTest.class) {
+        while (!isAfterRegistrationCallbackCalled) {
+          HAInterestBaseTest.class.wait();
+        }
+      }
+    }
+  }
+
+  public static void unSetClientServerObserverForRegistrationCallback() {
+    synchronized (HAInterestBaseTest.class) {
+      PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
+      PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
+      HAInterestBaseTest.isBeforeRegistrationCallbackCalled = false;
+      HAInterestBaseTest.isAfterRegistrationCallbackCalled = false;
+    }
+  }
+
+  public static void verifyDispatcherIsAlive() {
+    assertEquals("More than one BridgeServer", 1, cache.getCacheServers().size());
+    
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return cache.getCacheServers().size() == 1;
+      }
+      @Override
+      public String description() {
+        return "waiting for cache.getCacheServers().size() == 1";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    assertNotNull(bs.getAcceptor());
+    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
+    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
+
+    wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return ccn.getClientProxies().size() > 0;
+      }
+      @Override
+      public String description() {
+        return "waiting for ccn.getClientProxies().size() > 0";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    wc = new WaitCriterion() {
+      Iterator iter_prox;
+      CacheClientProxy proxy;
+
+      @Override
+      public boolean done() {
+        iter_prox = ccn.getClientProxies().iterator();
+        if (iter_prox.hasNext()) {
+          proxy = (CacheClientProxy) iter_prox.next();
+          return proxy._messageDispatcher.isAlive();
+        } else {
+          return false;
+        }
+      }
+
+      @Override
+      public String description() {
+        return "waiting for CacheClientProxy _messageDispatcher to be alive";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+  }
+
+  public static void verifyDispatcherIsNotAlive() {
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return cache.getCacheServers().size() == 1;
+      }
+      @Override
+      public String description() {
+        return "cache.getCacheServers().size() == 1";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    assertNotNull(bs.getAcceptor());
+    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
+    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
+    
+    wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return ccn.getClientProxies().size() > 0;
+      }
+      @Override
+      public String description() {
+        return "waiting for ccn.getClientProxies().size() > 0";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    Iterator iter_prox = ccn.getClientProxies().iterator();
+    if (iter_prox.hasNext()) {
+      CacheClientProxy proxy = (CacheClientProxy) iter_prox.next();
+      assertFalse("Dispatcher on secondary should not be alive", proxy._messageDispatcher.isAlive());
+    }
+  }
+
+  public static void createEntriesK1andK2OnServer() {
+    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r1);
+    if (!r1.containsKey(k1)) {
+      r1.create(k1, server_k1);
+    }
+    if (!r1.containsKey(k2)) {
+      r1.create(k2, server_k2);
+    }
+    assertEquals(r1.getEntry(k1).getValue(), server_k1);
+    assertEquals(r1.getEntry(k2).getValue(), server_k2);
+  }
+
+  public static void createEntriesK1andK2() {
+    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r1);
+    if (!r1.containsKey(k1)) {
+      r1.create(k1, client_k1);
+    }
+    if (!r1.containsKey(k2)) {
+      r1.create(k2, client_k2);
+    }
+    assertEquals(r1.getEntry(k1).getValue(), client_k1);
+    assertEquals(r1.getEntry(k2).getValue(), client_k2);
+  }
+
+  public static void createServerEntriesK1andK2() {
+    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r1);
+    if (!r1.containsKey(k1)) {
+      r1.create(k1, server_k1);
+    }
+    if (!r1.containsKey(k2)) {
+      r1.create(k2, server_k2);
+    }
+    assertEquals(r1.getEntry(k1).getValue(), server_k1);
+    assertEquals(r1.getEntry(k2).getValue(), server_k2);
+  }
+
+  public static void registerK1AndK2() {
+    Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+    r.registerInterest(list, InterestResultPolicy.KEYS_VALUES);
+  }
+
+  public static void reRegisterK1AndK2() {
+    Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+    r.registerInterest(list);
+  }
+
+  public static void startServer() throws IOException {
+    Cache c = CacheFactory.getAnyInstance();
+    assertEquals("More than one BridgeServer", 1, c.getCacheServers().size());
+    CacheServerImpl bs = (CacheServerImpl) c.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    bs.start();
+  }
+
+  public static void stopServer() {
+    assertEquals("More than one BridgeServer", 1, cache.getCacheServers().size());
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    bs.stop();
+  }
+
+  public static void stopPrimaryAndRegisterK1AndK2AndVerifyResponse() {
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == 3;
+      }
+      @Override
+      public String description() {
+        return "connected server count never became 3";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    // close primaryEP
+    getPrimaryVM().invoke(HAInterestBaseTest.class, "stopServer");
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
+    assertNotNull(serverKeys);
+    List resultKeys = (List) serverKeys.get(0);
+    assertEquals(2, resultKeys.size());
+    assertTrue(resultKeys.contains(k1));
+    assertTrue(resultKeys.contains(k2));
+  }
+
+  public static void stopPrimaryAndUnregisterRegisterK1() {
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == 3;
+      }
+      @Override
+      public String description() {
+        return "connected server count never became 3";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    // close primaryEP
+    getPrimaryVM().invoke(HAInterestBaseTest.class, "stopServer");
+    List list = new ArrayList();
+    list.add(k1);
+    srp.unregisterInterest(list, InterestType.KEY, false, false);
+  }
+
+  public static void stopBothPrimaryAndSecondaryAndRegisterK1AndK2AndVerifyResponse() {
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == 3;
+      }
+      @Override
+      public String description() {
+        return "connected server count never became 3";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    // close primaryEP
+    VM backup = getBackupVM();
+    getPrimaryVM().invoke(HAInterestBaseTest.class, "stopServer");
+    // close secondary
+    backup.invoke(HAInterestBaseTest.class, "stopServer");
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
+
+    assertNotNull(serverKeys);
+    List resultKeys = (List) serverKeys.get(0);
+    assertEquals(2, resultKeys.size());
+    assertTrue(resultKeys.contains(k1));
+    assertTrue(resultKeys.contains(k2));
+  }
+
+  /**
+   * returns the secondary that was stopped
+   */
+  public static VM stopSecondaryAndRegisterK1AndK2AndVerifyResponse() {
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == 3;
+      }
+      @Override
+      public String description() {
+        return "Never got three connected servers";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    // close secondary EP
+    VM result = getBackupVM();
+    result.invoke(HAInterestBaseTest.class, "stopServer");
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
+
+    assertNotNull(serverKeys);
+    List resultKeys = (List) serverKeys.get(0);
+    assertEquals(2, resultKeys.size());
+    assertTrue(resultKeys.contains(k1));
+    assertTrue(resultKeys.contains(k2));
+    return result;
+  }
+
+  /**
+   * returns the secondary that was stopped
+   */
+  public static VM stopSecondaryAndUNregisterK1() {
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == 3;
+      }
+      @Override
+      public String description() {
+        return "connected server count never became 3";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    // close secondary EP
+    VM result = getBackupVM();
+    result.invoke(HAInterestBaseTest.class, "stopServer");
+    List list = new ArrayList();
+    list.add(k1);
+    srp.unregisterInterest(list, InterestType.KEY, false, false);
+    return result;
+  }
+
+  public static void registerK1AndK2OnPrimaryAndSecondaryAndVerifyResponse() {
+    ServerLocation primary = pool.getPrimary();
+    ServerLocation secondary = (ServerLocation) pool.getRedundants().get(0);
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+
+    // Primary server
+    List serverKeys1 = srp.registerInterestOn(primary, list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
+    assertNotNull(serverKeys1);
+    // expect serverKeys in response from primary
+    List resultKeys = (List) serverKeys1.get(0);
+    assertEquals(2, resultKeys.size());
+    assertTrue(resultKeys.contains(k1));
+    assertTrue(resultKeys.contains(k2));
+
+    // Secondary server
+    List serverKeys2 = srp.registerInterestOn(secondary, list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
+    // if the list is null then it is empty
+    if (serverKeys2 != null) {
+      // no serverKeys in response from secondary
+      assertTrue(serverKeys2.isEmpty());
+    }
+  }
+
+  public static void verifyInterestRegistration() {
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return cache.getCacheServers().size() == 1;
+      }
+      @Override
+      public String description() {
+        return "waiting for cache.getCacheServers().size() == 1";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    assertNotNull(bs.getAcceptor());
+    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
+    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
+    
+    wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return ccn.getClientProxies().size() > 0;
+      }
+      @Override
+      public String description() {
+        return "waiting for ccn.getClientProxies().size() > 0";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    Iterator iter_prox = ccn.getClientProxies().iterator();
+
+    if (iter_prox.hasNext()) {
+      final CacheClientProxy ccp = (CacheClientProxy) iter_prox.next();
+      
+      wc = new WaitCriterion() {
+        @Override
+        public boolean done() {
+          Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
+              .getProfile(Region.SEPARATOR + REGION_NAME)
+              .getKeysOfInterestFor(ccp.getProxyID());
+          return keysMap != null && keysMap.size() == 2;
+        }
+        @Override
+        public String description() {
+          return "waiting for keys of interest to include 2 keys";
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+      Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex].getProfile(Region.SEPARATOR + REGION_NAME)
+          .getKeysOfInterestFor(ccp.getProxyID());
+      assertNotNull(keysMap);
+      assertEquals(2, keysMap.size());
+      assertTrue(keysMap.contains(k1));
+      assertTrue(keysMap.contains(k2));
+    }
+  }
+
+  public static void verifyInterestUNRegistration() {
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return cache.getCacheServers().size() == 1;
+      }
+      @Override
+      public String description() {
+        return "waiting for cache.getCacheServers().size() == 1";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    assertNotNull(bs.getAcceptor());
+    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
+    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
+    
+    wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return ccn.getClientProxies().size() > 0;
+      }
+      @Override
+      public String description() {
+        return "waiting for ccn.getClientProxies().size() > 0";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    Iterator iter_prox = ccn.getClientProxies().iterator();
+    if (iter_prox.hasNext()) {
+      final CacheClientProxy ccp = (CacheClientProxy) iter_prox.next();
+      
+      wc = new WaitCriterion() {
+        @Override
+        public boolean done() {
+          Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
+              .getProfile(Region.SEPARATOR + REGION_NAME)
+              .getKeysOfInterestFor(ccp.getProxyID());
+          return keysMap != null;
+        }
+        @Override
+        public String description() {
+          return "waiting for keys of interest to not be null";
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+      Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
+          .getProfile(Region.SEPARATOR + REGION_NAME)
+          .getKeysOfInterestFor(ccp.getProxyID());
+      assertNotNull(keysMap);
+      assertEquals(1, keysMap.size());
+      assertFalse(keysMap.contains(k1));
+      assertTrue(keysMap.contains(k2));
+    }
+  }
+
+  private void createCache(Properties props) throws Exception {
+    DistributedSystem ds = getSystem(props);
+    assertNotNull(ds);
+    ds.disconnect();
+    ds = getSystem(props);
+    cache = CacheFactory.create(ds);
+    assertNotNull(cache);
+  }
+
+  public static void createClientPoolCache(String testName, String host) throws Exception {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    new HAInterestBaseTest("temp").createCache(props);
+    CacheServerTestUtil.disableShufflingOfEndpoints();
+    PoolImpl p;
+    try {
+      p = (PoolImpl) PoolManager.createFactory()
+          .addServer(host, PORT1)
+          .addServer(host, PORT2)
+          .addServer(host, PORT3)
+          .setSubscriptionEnabled(true)
+          .setSubscriptionRedundancy(-1)
+          .setReadTimeout(1000)
+          .setPingInterval(1000)
+          // retryInterval should be more so that only registerInterste thread
+          // will initiate failover
+          // .setRetryInterval(20000)
+          .create("HAInterestBaseTestPool");
+    } finally {
+      CacheServerTestUtil.enableShufflingOfEndpoints();
+    }
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    factory.setConcurrencyChecksEnabled(true);
+    factory.setPoolName(p.getName());
+
+    cache.createRegion(REGION_NAME, factory.create());
+    pool = p;
+    conn = pool.acquireConnection();
+    assertNotNull(conn);
+  }
+
+  public static void createClientPoolCacheWithSmallRetryInterval(String testName, String host) throws Exception {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    new HAInterestBaseTest("temp").createCache(props);
+    CacheServerTestUtil.disableShufflingOfEndpoints();
+    PoolImpl p;
+    try {
+      p = (PoolImpl) PoolManager.createFactory()
+          .addServer(host, PORT1)
+          .addServer(host, PORT2)
+          .setSubscriptionEnabled(true)
+          .setSubscriptionRedundancy(-1)
+          .setReadTimeout(1000)
+          .setSocketBufferSize(32768)
+          .setMinConnections(6)
+          .setPingInterval(200)
+          // .setRetryInterval(200)
+          // retryAttempts 3
+          .create("HAInterestBaseTestPool");
+    } finally {
+      CacheServerTestUtil.enableShufflingOfEndpoints();
+    }
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    factory.setConcurrencyChecksEnabled(true);
+    factory.setPoolName(p.getName());
+
+    cache.createRegion(REGION_NAME, factory.create());
+
+    pool = p;
+    conn = pool.acquireConnection();
+    assertNotNull(conn);
+  }
+
+  public static void createClientPoolCacheConnectionToSingleServer(String testName, String hostName) throws Exception {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    new HAInterestBaseTest("temp").createCache(props);
+    PoolImpl p = (PoolImpl) PoolManager.createFactory()
+        .addServer(hostName, PORT1)
+        .setSubscriptionEnabled(true)
+        .setSubscriptionRedundancy(-1)
+        .setReadTimeout(1000)
+        // .setRetryInterval(20)
+        .create("HAInterestBaseTestPool");
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    factory.setConcurrencyChecksEnabled(true);
+    factory.setPoolName(p.getName());
+
+    cache.createRegion(REGION_NAME, factory.create());
+
+    pool = p;
+    conn = pool.acquireConnection();
+    assertNotNull(conn);
+  }
+
+  public static Integer createServerCache() throws Exception {
+    new HAInterestBaseTest("temp").createCache(new Properties());
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setEnableBridgeConflation(true);
+    factory.setMirrorType(MirrorType.KEYS_VALUES);
+    factory.setConcurrencyChecksEnabled(true);
+    cache.createRegion(REGION_NAME, factory.create());
+
+    CacheServer server = cache.addCacheServer();
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    server.setPort(port);
+    server.setMaximumTimeBetweenPings(180000);
+    // ensures updates to be sent instead of invalidations
+    server.setNotifyBySubscription(true);
+    server.start();
+    return new Integer(server.getPort());
+  }
+
+  public static Integer createServerCacheWithLocalRegion() throws Exception {
+    new HAInterestBaseTest("temp").createCache(new Properties());
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    factory.setConcurrencyChecksEnabled(true);
+    RegionAttributes attrs = factory.create();
+    cache.createRegion(REGION_NAME, attrs);
+
+    CacheServer server = cache.addCacheServer();
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    server.setPort(port);
+    // ensures updates to be sent instead of invalidations
+    server.setNotifyBySubscription(true);
+    server.setMaximumTimeBetweenPings(180000);
+    server.start();
+    return new Integer(server.getPort());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart1DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart1DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart1DUnitTest.java
index 27779a6..482fca9 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart1DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart1DUnitTest.java
@@ -19,7 +19,7 @@ package com.gemstone.gemfire.internal.cache.tier.sockets;
 import dunit.VM;
 
 @SuppressWarnings("serial")
-public class HAInterestPart1DUnitTest extends HAInterestTestCase {
+public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
 
   public HAInterestPart1DUnitTest(String name) {
     super(name);
@@ -31,14 +31,14 @@ public class HAInterestPart1DUnitTest extends HAInterestTestCase {
   public void testInterestRegistrationOnBothPrimaryAndSecondary() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
     // register K1 and K2
     registerK1AndK2();
-    server1.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
-    server2.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
-    server3.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server3.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
   }
 
   /**
@@ -48,9 +48,9 @@ public class HAInterestPart1DUnitTest extends HAInterestTestCase {
   public void testInterestRegistrationResponseOnBothPrimaryAndSecondary() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
     // register interest and verify response
     registerK1AndK2OnPrimaryAndSecondaryAndVerifyResponse();
   }
@@ -62,15 +62,15 @@ public class HAInterestPart1DUnitTest extends HAInterestTestCase {
   public void testRERegistrationWillNotCreateDuplicateKeysOnServerInterstMaps() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
     // register multiple times
     reRegisterK1AndK2();
 
-    server1.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
-    server2.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
-    server3.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server3.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
   }
 
   /**
@@ -81,9 +81,9 @@ public class HAInterestPart1DUnitTest extends HAInterestTestCase {
   public void testPrimaryFailureInRegisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
     // stop primary
     VM oldPrimary = getPrimaryVM();
     stopPrimaryAndRegisterK1AndK2AndVerifyResponse();
@@ -91,8 +91,8 @@ public class HAInterestPart1DUnitTest extends HAInterestTestCase {
     verifyDeadAndLiveServers(1, 2);
     // new primary
     VM newPrimary = getPrimaryVM(oldPrimary);
-    newPrimary.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
-    newPrimary.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    newPrimary.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
+    newPrimary.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
   }
 
   /**
@@ -102,17 +102,17 @@ public class HAInterestPart1DUnitTest extends HAInterestTestCase {
   public void testSecondaryFailureInRegisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
 
     VM primary = getPrimaryVM();
     stopSecondaryAndRegisterK1AndK2AndVerifyResponse();
 
     verifyDeadAndLiveServers(1, 2);
     // still primary
-    primary.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
-    primary.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    primary.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
+    primary.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
   }
 
   /**
@@ -124,17 +124,17 @@ public class HAInterestPart1DUnitTest extends HAInterestTestCase {
   public void testBothPrimaryAndSecondaryFailureInRegisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
     // stop server1 and server2
     VM oldPrimary = getPrimaryVM();
     stopBothPrimaryAndSecondaryAndRegisterK1AndK2AndVerifyResponse();
 
     verifyDeadAndLiveServers(2, 1);
     VM newPrimary = getPrimaryVM(oldPrimary);
-    newPrimary.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
-    newPrimary.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    newPrimary.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
+    newPrimary.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
   }
 
   /**
@@ -148,17 +148,17 @@ public class HAInterestPart1DUnitTest extends HAInterestTestCase {
   public void testProbablePrimaryFailureInRegisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
 
     VM oldPrimary = getPrimaryVM();
     stopPrimaryAndRegisterK1AndK2AndVerifyResponse();
 
     verifyDeadAndLiveServers(1, 2);
     VM newPrimary = getPrimaryVM(oldPrimary);
-    newPrimary.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
-    newPrimary.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    newPrimary.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
+    newPrimary.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
   }
 
   /**
@@ -172,40 +172,40 @@ public class HAInterestPart1DUnitTest extends HAInterestTestCase {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
     registerK1AndK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
 
-    server1.invoke(HAInterestTestCase.class, "stopServer");
-    server2.invoke(HAInterestTestCase.class, "stopServer");
-    server3.invoke(HAInterestTestCase.class, "stopServer");
+    server1.invoke(HAInterestBaseTest.class, "stopServer");
+    server2.invoke(HAInterestBaseTest.class, "stopServer");
+    server3.invoke(HAInterestBaseTest.class, "stopServer");
     // All servers are dead at this point , no primary in the system.
     verifyDeadAndLiveServers(3, 0);
 
     // now start one of the servers
-    server2.invoke(HAInterestTestCase.class, "startServer");
+    server2.invoke(HAInterestBaseTest.class, "startServer");
     verifyDeadAndLiveServers(2, 1);
     // verify that is it primary , and dispatcher is running
-    server2.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
+    server2.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
     // verify that interest is registered on this recovered EP
-    server2.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
 
     // now start one more server ; this should be now secondary
-    server1.invoke(HAInterestTestCase.class, "startServer");
+    server1.invoke(HAInterestBaseTest.class, "startServer");
     verifyDeadAndLiveServers(1, 2);
 
     // verify that is it secondary , dispatcher should not be runnig
-    server1.invoke(HAInterestTestCase.class, "verifyDispatcherIsNotAlive");
+    server1.invoke(HAInterestBaseTest.class, "verifyDispatcherIsNotAlive");
     // verify that interest is registered on this recovered EP as well
-    server1.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
 
     // now start one more server ; this should be now secondary
-    server3.invoke(HAInterestTestCase.class, "startServer");
+    server3.invoke(HAInterestBaseTest.class, "startServer");
     verifyDeadAndLiveServers(0, 3);
 
     // verify that is it secondary , dispatcher should not be runnig
-    server3.invoke(HAInterestTestCase.class, "verifyDispatcherIsNotAlive");
+    server3.invoke(HAInterestBaseTest.class, "verifyDispatcherIsNotAlive");
     // verify that interest is registered on this recovered EP as well
-    server3.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server3.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
index 31a2811..eaa1ca1 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
@@ -24,7 +24,7 @@ import dunit.DistributedTestCase;
 import dunit.VM;
 
 @SuppressWarnings({"rawtypes", "serial"})
-public class HAInterestPart2DUnitTest extends HAInterestTestCase {
+public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
 
   public HAInterestPart2DUnitTest(String name) {
     super(name);
@@ -37,9 +37,9 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
   public void testPrimaryFailureInUNregisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
 
     registerK1AndK2();
 
@@ -49,11 +49,11 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
     verifyDeadAndLiveServers(1, 2);
 
     VM newPrimary = getPrimaryVM(oldPrimary);
-    newPrimary.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
+    newPrimary.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
     // primary
-    newPrimary.invoke(HAInterestTestCase.class, "verifyInterestUNRegistration");
+    newPrimary.invoke(HAInterestBaseTest.class, "verifyInterestUNRegistration");
     // secondary
-    getBackupVM().invoke(HAInterestTestCase.class, "verifyInterestUNRegistration");
+    getBackupVM().invoke(HAInterestBaseTest.class, "verifyInterestUNRegistration");
   }
 
   /**
@@ -63,18 +63,18 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
   public void testSecondaryFailureInUNRegisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
     registerK1AndK2();
     VM stoppedBackup = stopSecondaryAndUNregisterK1();
     verifyDeadAndLiveServers(1, 2);
     // still primary
-    getPrimaryVM().invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
+    getPrimaryVM().invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
     // primary
-    getPrimaryVM().invoke(HAInterestTestCase.class, "verifyInterestUNRegistration");
+    getPrimaryVM().invoke(HAInterestBaseTest.class, "verifyInterestUNRegistration");
     // secondary
-    getBackupVM(stoppedBackup).invoke(HAInterestTestCase.class, "verifyInterestUNRegistration");
+    getBackupVM(stoppedBackup).invoke(HAInterestBaseTest.class, "verifyInterestUNRegistration");
   }
 
   /**
@@ -85,11 +85,11 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
   public void testDSMDetectsServerLiveJustBeforeInterestRegistration() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
     VM backup = getBackupVM();
-    backup.invoke(HAInterestTestCase.class, "stopServer");
+    backup.invoke(HAInterestBaseTest.class, "stopServer");
     verifyDeadAndLiveServers(1, 2);
     setClientServerObserverForBeforeRegistration(backup);
     try {
@@ -98,9 +98,9 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
     } finally {
       unSetClientServerObserverForRegistrationCallback();
     }
-    server1.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
-    server2.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
-    server3.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server3.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
   }
 
   /**
@@ -112,12 +112,12 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
 
     createEntriesK1andK2();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
 
     VM backup = getBackupVM();
-    backup.invoke(HAInterestTestCase.class, "stopServer");
+    backup.invoke(HAInterestBaseTest.class, "stopServer");
     verifyDeadAndLiveServers(1, 2);
 
     setClientServerObserverForAfterRegistration(backup);
@@ -128,9 +128,9 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
       unSetClientServerObserverForRegistrationCallback();
     }
 
-    server1.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
-    server2.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
-    server3.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server3.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
   }
 
   /**
@@ -143,16 +143,16 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
   public void testRefreshEntriesFromPrimaryWhenDSMDetectsServerLive() throws Exception {
     addExpectedException(ServerConnectivityException.class.getName());
     
-    PORT1 = ((Integer) server1.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    PORT1 = ((Integer) server1.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
     createClientPoolCacheConnectionToSingleServer(this.getName(), getServerHostName(server1.getHost()));
     registerK1AndK2();
     verifyRefreshedEntriesFromServer();
 
-    server1.invoke(HAInterestTestCase.class, "stopServer");
+    server1.invoke(HAInterestBaseTest.class, "stopServer");
     verifyDeadAndLiveServers(1, 0);
-    server1.invoke(HAInterestTestCase.class, "putK1andK2");
-    server1.invoke(HAInterestTestCase.class, "startServer");
+    server1.invoke(HAInterestBaseTest.class, "putK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "startServer");
     verifyDeadAndLiveServers(0, 1);
     final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
     assertNotNull(r1);
@@ -211,29 +211,29 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
    * refreshes registered entries from the server, because it is secondary
    */
   public void testGIIFromSecondaryWhenDSMDetectsServerLive() throws Exception {
-    server1.invoke(HAInterestTestCase.class, "closeCache");
-    server2.invoke(HAInterestTestCase.class, "closeCache");
-    server3.invoke(HAInterestTestCase.class, "closeCache");
+    server1.invoke(HAInterestBaseTest.class, "closeCache");
+    server2.invoke(HAInterestBaseTest.class, "closeCache");
+    server3.invoke(HAInterestBaseTest.class, "closeCache");
 
-    PORT1 = ((Integer) server1.invoke(HAInterestTestCase.class, "createServerCacheWithLocalRegion")).intValue();
-    PORT2 = ((Integer) server2.invoke(HAInterestTestCase.class, "createServerCacheWithLocalRegion")).intValue();
-    PORT3 = ((Integer) server3.invoke(HAInterestTestCase.class, "createServerCacheWithLocalRegion")).intValue();
+    PORT1 = ((Integer) server1.invoke(HAInterestBaseTest.class, "createServerCacheWithLocalRegion")).intValue();
+    PORT2 = ((Integer) server2.invoke(HAInterestBaseTest.class, "createServerCacheWithLocalRegion")).intValue();
+    PORT3 = ((Integer) server3.invoke(HAInterestBaseTest.class, "createServerCacheWithLocalRegion")).intValue();
 
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
 
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
 
     VM backup1 = getBackupVM();
     VM backup2 = getBackupVM(backup1);
-    backup1.invoke(HAInterestTestCase.class, "stopServer");
-    backup2.invoke(HAInterestTestCase.class, "stopServer");
+    backup1.invoke(HAInterestBaseTest.class, "stopServer");
+    backup2.invoke(HAInterestBaseTest.class, "stopServer");
     verifyDeadAndLiveServers(2, 1);
     registerK1AndK2();
     verifyRefreshedEntriesFromServer();
-    backup1.invoke(HAInterestTestCase.class, "putK1andK2");
-    backup1.invoke(HAInterestTestCase.class, "startServer");
+    backup1.invoke(HAInterestBaseTest.class, "putK1andK2");
+    backup1.invoke(HAInterestBaseTest.class, "startServer");
     verifyDeadAndLiveServers(1, 2);
     verifyRefreshedEntriesFromServer();
   }
@@ -246,19 +246,19 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
    * @throws Exception
    */
   public void testBug35945() throws Exception {
-    PORT1 = ((Integer) server1.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    PORT1 = ((Integer) server1.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
     createClientPoolCacheConnectionToSingleServer(this.getName(), getServerHostName(server1.getHost()));
     registerK1AndK2();
     verifyRefreshedEntriesFromServer();
 
-    server1.invoke(HAInterestTestCase.class, "stopServer");
+    server1.invoke(HAInterestBaseTest.class, "stopServer");
     verifyDeadAndLiveServers(1, 0);
     // put on stopped server
-    server1.invoke(HAInterestTestCase.class, "putK1andK2");
+    server1.invoke(HAInterestBaseTest.class, "putK1andK2");
     // spawn a thread to put on server , which will acquire a lock on entry
     setClientServerObserverForBeforeInterestRecovery();
-    server1.invoke(HAInterestTestCase.class, "startServer");
+    server1.invoke(HAInterestBaseTest.class, "startServer");
     verifyDeadAndLiveServers(0, 1);
     waitForBeforeInterestRecoveryCallBack();
     // verify updated value of k1 as a refreshEntriesFromServer
@@ -314,23 +314,23 @@ public class HAInterestPart2DUnitTest extends HAInterestTestCase {
   public void testInterestRecoveryFailure() throws Exception {
     addExpectedException("Server unreachable");
     
-    PORT1 = ((Integer) server1.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
-    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
-    PORT2 = ((Integer) server2.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
-    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    PORT1 = ((Integer) server1.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
+    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    PORT2 = ((Integer) server2.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
+    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
     createClientPoolCacheWithSmallRetryInterval(this.getName(), getServerHostName(server1.getHost()));
     registerK1AndK2();
     verifyRefreshedEntriesFromServer();
     VM backup = getBackupVM();
     VM primary = getPrimaryVM();
 
-    backup.invoke(HAInterestTestCase.class, "stopServer");
-    primary.invoke(HAInterestTestCase.class, "stopServer");
+    backup.invoke(HAInterestBaseTest.class, "stopServer");
+    primary.invoke(HAInterestBaseTest.class, "stopServer");
     verifyDeadAndLiveServers(2, 0);
 
-    primary.invoke(HAInterestTestCase.class, "putK1andK2");
+    primary.invoke(HAInterestBaseTest.class, "putK1andK2");
     setClientServerObserverForBeforeInterestRecoveryFailure();
-    primary.invoke(HAInterestTestCase.class, "startServer");
+    primary.invoke(HAInterestBaseTest.class, "startServer");
     waitForBeforeInterestRecoveryCallBack();
     if (exceptionOccured) {
       fail("The DSM could not ensure that server 1 is started & serevr 2 is stopped");


[19/50] [abbrv] incubator-geode git commit: GEODE-719: Add error logs while cache.xml processing

Posted by ab...@apache.org.
GEODE-719: Add error logs while cache.xml processing

While processing cache.xml, log an error before throwing an exception so
that the reason for cache close is clear by looking at the log.


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

Branch: refs/heads/master
Commit: 13a3adf696111861c9727d8e010b7b468670c6d4
Parents: cb7dbd0
Author: Swapnil Bawaskar <sb...@pivotal.io>
Authored: Thu Dec 31 13:00:01 2015 -0800
Committer: Swapnil Bawaskar <sb...@pivotal.io>
Committed: Tue Jan 5 10:35:48 2016 -0800

----------------------------------------------------------------------
 .../internal/cache/GemFireCacheImpl.java        | 12 ++++++----
 .../gemfire/cache30/CacheXml30DUnitTest.java    | 24 ++++++++++++++++++++
 .../gemfire/cache30/CacheXml41DUnitTest.java    |  7 ++++++
 .../gemfire/cache30/CacheXml60DUnitTest.java    | 14 ++++++++----
 .../gemfire/cache30/CacheXml81DUnitTest.java    |  4 ++++
 5 files changed, 53 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/13a3adf6/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index 506bd7a..9d77556 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -1359,12 +1359,16 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
             LocalizedStrings.GemFireCache_CACHE_INITIALIZED_USING__0__1, new Object[] {"generated description from old cache", cacheXmlDescription}));
       }
     } catch (IOException ex) {
-      throw new CacheXmlException(LocalizedStrings.GemFireCache_WHILE_OPENING_CACHE_XML_0_THE_FOLLOWING_ERROR_OCCURRED_1
-          .toLocalizedString(new Object[] { url.toString(), ex }));
+      String exceptionMsg = LocalizedStrings.GemFireCache_WHILE_OPENING_CACHE_XML_0_THE_FOLLOWING_ERROR_OCCURRED_1
+          .toLocalizedString(new Object[] { url.toString(), ex });
+      logger.error(exceptionMsg);
+      throw new CacheXmlException(exceptionMsg);
 
     } catch (CacheXmlException ex) {
-      CacheXmlException newEx = new CacheXmlException(LocalizedStrings.GemFireCache_WHILE_READING_CACHE_XML_0_1
-          .toLocalizedString(new Object[] { url, ex.getMessage() }));
+      String exceptionMsg = LocalizedStrings.GemFireCache_WHILE_READING_CACHE_XML_0_1
+          .toLocalizedString(new Object[] { url, ex.getMessage() });
+      logger.error(exceptionMsg);
+      CacheXmlException newEx = new CacheXmlException(exceptionMsg);
       newEx.setStackTrace(ex.getStackTrace());
       newEx.initCause(ex.getCause());
       throw newEx;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/13a3adf6/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml30DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml30DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml30DUnitTest.java
index b678884..adafa69 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml30DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml30DUnitTest.java
@@ -285,12 +285,15 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
   public void testMalformed() {
     setXmlFile(findFile("malformed.xml"));
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       getCache();
       fail("Should have thrown a CacheXmlException");
 
     } catch (CacheXmlException ex) {
       assertTrue(ex.getCause() instanceof SAXException);
+    } finally {
+      expectedException.remove();
     }
   }
 
@@ -300,6 +303,7 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
   public void testBadInt() {
     setXmlFile(findFile("badInt.xml"));
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       getCache();
       fail("Should have thrown a CacheXmlException");
@@ -310,6 +314,8 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
       assertTrue("Didn't expect cause:" + cause + " (a " +
                  cause.getClass().getName() + ")",
                  cause instanceof NumberFormatException);
+    } finally {
+      expectedException.remove();
     }
   }
 
@@ -319,12 +325,15 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
   public void testBadFloat() {
     setXmlFile(findFile("badFloat.xml"));
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       getCache();
       fail("Should have thrown a CacheXmlException");
 
     } catch (CacheXmlException ex) {
       assertTrue(ex.getCause() instanceof NumberFormatException);
+    } finally {
+      expectedException.remove();
     }
   }
 
@@ -335,12 +344,15 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
   public void testBadScope() {
     setXmlFile(findFile("badScope.xml"));
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       getCache();
       fail("Should have thrown a CacheXmlException");
 
     } catch (CacheXmlException ex) {
       assertTrue(ex.getCause() instanceof SAXException);
+    } finally {
+      expectedException.remove();
     }
   }
 
@@ -351,12 +363,15 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
   public void testBadKeyConstraintClass() {
     setXmlFile(findFile("badKeyConstraintClass.xml"));
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       getCache();
       fail("Should have thrown a CacheXmlException");
 
     } catch (CacheXmlException ex) {
       assertTrue(ex.getCause() instanceof ClassNotFoundException);
+    } finally {
+      expectedException.remove();
     }
   }
 
@@ -367,6 +382,7 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
   public void testCallbackNotDeclarable() {
     setXmlFile(findFile("callbackNotDeclarable.xml"));
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       getCache();
       fail("Should have thrown a CacheXmlException");
@@ -376,6 +392,8 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
       assertNull(/*"Didn't expect a cause of " + cause + " (a " +
                    cause.getClass().getName() + ")" + " from " + ex, */
                  cause);
+    } finally {
+      expectedException.remove();
     }
   }
 
@@ -386,6 +404,7 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
   public void testCallbackWithException() {
     setXmlFile(findFile("callbackWithException.xml"));
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       getCache();
       fail("Should have thrown a CacheXmlException");
@@ -394,6 +413,8 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
       if (!(ex.getCause() instanceof TestException)) {
         throw ex;
       }
+    } finally {
+      expectedException.remove();
     }
 
   }
@@ -405,6 +426,7 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
   public void testLoaderNotLoader() {
     setXmlFile(findFile("loaderNotLoader.xml"));
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       getCache();
       fail("Should have thrown a CacheXmlException");
@@ -412,6 +434,8 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
     } catch (CacheXmlException ex) {
       Throwable cause = ex.getCause();
       assertNull("Didn't expect a " + cause, cause);
+    } finally {
+      expectedException.remove();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/13a3adf6/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
index 0d9cdf0..5a41d31 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
@@ -22,6 +22,7 @@ import com.gemstone.gemfire.internal.cache.xmlcache.*;
 
 import java.io.*;
 
+import dunit.DistributedTestCase;
 import org.xml.sax.SAXException;
 
 /**
@@ -175,6 +176,7 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
 
     setXmlFile(findFile("sameRootRegion.xml"));
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       getCache();
       fail("Should have thrown a CacheXmlException");
@@ -188,6 +190,8 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
         fail("Expected a RegionExistsException, not a "
             + cause.getClass().getName(), cause);
       }
+    } finally {
+      expectedException.remove();
     }
   }
 
@@ -216,6 +220,7 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
 
     setXmlFile(findFile("sameSubregion.xml"));
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       getCache();
       fail("Should have thrown a CacheXmlException");
@@ -229,6 +234,8 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
         fail("Expected a RegionExistsException, not a "
             + cause.getClass().getName(), cause);
       }
+    } finally {
+      expectedException.remove();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/13a3adf6/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml60DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml60DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml60DUnitTest.java
index ddb35b7..8699a05 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml60DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml60DUnitTest.java
@@ -307,22 +307,28 @@ public class CacheXml60DUnitTest extends CacheXml58DUnitTest
     sc.registerInstantiator(NotDataSerializable.class, 15);
     closeCache();
     cc.setSerializerCreation(sc);
-    
+
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       testXml(cc);
       fail("Instantiator should not have registered due to bad class.");
+    } catch(Exception e) {
+    } finally {
+      expectedException.remove();
     }
-    catch(Exception e){}
     
     sc = new SerializerCreation();
     sc.registerSerializer(BadSerializer.class);
     closeCache();
     cc.setSerializerCreation(sc);
-    
+
+    ExpectedException expectedException1 = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       testXml(cc);
       fail("Serializer should not have registered due to bad class.");
+    } catch(Exception e){
+    } finally {
+      expectedException1.remove();
     }
-    catch(Exception e){}
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/13a3adf6/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml81DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml81DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml81DUnitTest.java
index 3c77326..831875a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml81DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml81DUnitTest.java
@@ -16,6 +16,7 @@
  */
 package com.gemstone.gemfire.cache30;
 
+import dunit.DistributedTestCase;
 import org.junit.Rule;
 import org.junit.Test;
 import org.xml.sax.Locator;
@@ -140,6 +141,7 @@ public class CacheXml81DUnitTest extends CacheXml80DUnitTest {
     assertEquals(0, extension.onCreateCounter.get());
     assertEquals(0, extension.getXmlGeneratorCounter.get());
 
+    ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {
       testXml(cache);
       fail("Excepted CacheXmlException");
@@ -149,6 +151,8 @@ public class CacheXml81DUnitTest extends CacheXml80DUnitTest {
         assertTrue(((SAXParseException) e.getCause()).getColumnNumber() > 0);
         assertEquals("Value is 'exception'.", e.getCause().getMessage());
       }
+    } finally {
+      expectedException.remove();
     }
   }
 


[27/50] [abbrv] incubator-geode git commit: GEODE-712: Move junit asserts out of finally block This closes #68

Posted by ab...@apache.org.
GEODE-712: Move junit asserts out of finally block
This closes #68


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

Branch: refs/heads/master
Commit: 7c281327a5d7e249755a4d87a7dc12d39615eb03
Parents: dfb3306
Author: Scott Jewell <sj...@pivotal.io>
Authored: Wed Jan 6 13:52:34 2016 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Jan 6 15:32:07 2016 -0800

----------------------------------------------------------------------
 .../gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7c281327/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 4dda498..6bc488d 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
@@ -217,13 +217,13 @@ public class SimpleMemoryAllocatorJUnitTest {
         assertFalse(listener.isClosed());
         assertFalse(stats2.isClosed());
         stats = stats2;
-      } finally {
         ma.close();
         assertTrue(listener.isClosed());
         assertFalse(stats.isClosed());
+      } finally {
         SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-        assertTrue(stats.isClosed());
       }
+      assertTrue(stats.isClosed());
     }
   }
   @Test


[38/50] [abbrv] incubator-geode git commit: GEODE-775: Add build support to easily sign release files

Posted by ab...@apache.org.
GEODE-775: Add build support to easily sign release files

Use the gradle signing plugin to sign jar artifacts and distributions.
This is only done for release versions (-SNAPSHOT).  Exclude KEYS and
signatures from the distributions.  Requires a user gradle file with
the following properties:

    signing.keyId=
    signing.password=
    signing.secretKeyRingFile=


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

Branch: refs/heads/master
Commit: 5e8eed518e97201e48754314a2bddb962dbda229
Parents: 010f5f6
Author: Anthony Baker <ab...@apache.org>
Authored: Tue Jan 12 08:42:06 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Tue Jan 12 13:15:37 2016 -0800

----------------------------------------------------------------------
 build.gradle                  | 12 ++++++++++++
 gemfire-assembly/build.gradle | 20 ++++++++++++++++++--
 2 files changed, 30 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5e8eed51/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 1f1077c..b9bd12f 100755
--- a/build.gradle
+++ b/build.gradle
@@ -37,6 +37,7 @@ dependencyVersions.keys().each{ k -> project.ext[k] = dependencyVersions[k]}
 
 allprojects {
   version = versionNumber + '.' + releaseType
+  ext.isReleaseVersion = !version.endsWith("SNAPSHOT")
 
   // We want to see all test results.  This is equivalatent to setting --continue
   // on the command line.
@@ -110,6 +111,17 @@ gradle.taskGraph.whenReady({ graph ->
 
 subprojects {
   apply plugin: 'java'
+  apply plugin: 'signing'
+
+  // set these properties in ~/.gradle/gradle.properties file: 
+  // signing.keyId
+  // signing.password
+  // signing.secretKeyRingFile
+  if (isReleaseVersion) {
+    signing {
+      sign configurations.archives
+    }
+  }
 
   // apply compiler options
   gradle.taskGraph.whenReady( { graph ->

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5e8eed51/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index 4c74513..54c33c6 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+import org.gradle.api.internal.artifacts.publish.ArchivePublishArtifact;
 
 buildscript {
     repositories {
@@ -30,11 +31,17 @@ apply plugin: 'maven-publish-auth'
 
 jar.enabled = false
 
-
 configurations {
   archives
 }
 
+// gradle doesn't automatically remove the jar artifact even though we disabled it
+// this causes the signing plugin to fail
+configurations.all {
+  artifacts.removeAll artifacts.findAll { it instanceof ArchivePublishArtifact && !it.archiveTask.enabled }
+}
+
+
 dependencies {
   provided project(':gemfire-core')
   
@@ -184,6 +191,13 @@ gradle.taskGraph.whenReady( { graph ->
     archive.doLast {
       ant.checksum file:"${archive.archivePath}", algorithm:"md5"
       ant.checksum file:"${archive.archivePath}", algorithm:"sha-256"
+
+      if (isReleaseVersion) {      
+        signing {
+          required { isReleaseVersion }
+          sign archive.archivePath
+        }
+      }
     }
   }
 })
@@ -193,6 +207,7 @@ distributions {
     baseName = 'apache-geode-src'
     contents {
       from (rootDir) {
+        exclude 'KEYS'
         exclude 'gradlew'
         exclude 'gradlew.bat'
         exclude 'gradle/wrapper/gradle-wrapper.jar'
@@ -216,6 +231,7 @@ distributions {
     baseName = 'apache-geode' //TODO rootProject.name
     contents {
       duplicatesStrategy 'exclude'
+      exclude '*.asc'
       
       into ('config') {
         from defaultCacheConfig
@@ -323,7 +339,7 @@ afterEvaluate {
   // the ones defined in root ASF pom
   publishing {
     repositories {
-      if (project.version.endsWith('-SNAPSHOT')) {
+      if (!isReleaseVersion) {
         // Apache Development Snapshot Repository
         maven {
           name "apache.snapshots.https"


[40/50] [abbrv] incubator-geode git commit: GEODE-778: Remove grgit plugin to avoid build failures

Posted by ab...@apache.org.
GEODE-778: Remove grgit plugin to avoid build failures

Applying the grgit plugin in a non-git working tree was causing
build failures.  Removing the plugin restores the correct
behavior.


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

Branch: refs/heads/master
Commit: 561d033be4e3628893862a11ef1fc463f0fab4d0
Parents: a48bac4
Author: Anthony Baker <ab...@apache.org>
Authored: Tue Jan 12 19:57:13 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Wed Jan 13 07:03:24 2016 -0800

----------------------------------------------------------------------
 build.gradle | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/561d033b/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index b9bd12f..edf567d 100755
--- a/build.gradle
+++ b/build.gradle
@@ -56,7 +56,6 @@ allprojects {
 
   group = "org.apache.geode"
 
-  apply plugin: 'org.ajoberstar.grgit'
   apply plugin: 'idea'
   apply plugin: 'eclipse'
   


[48/50] [abbrv] incubator-geode git commit: GEODE-815: Update copyright year in NOTICE

Posted by ab...@apache.org.
GEODE-815: Update copyright year in NOTICE


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

Branch: refs/heads/master
Commit: 2a2915af2ee77bbfb1fd873b472012e182a4c680
Parents: a6774c4
Author: Anthony Baker <ab...@apache.org>
Authored: Mon Jan 25 11:30:05 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Wed Jan 27 10:56:39 2016 -0800

----------------------------------------------------------------------
 NOTICE                                | 2 +-
 gemfire-assembly/src/main/dist/NOTICE | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2a2915af/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index e73e3db..3f5dd07 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,5 +1,5 @@
 Apache Geode (incubating) 
-Copyright 2015 The Apache Software Foundation.
+Copyright 2016 The Apache Software Foundation.
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2a2915af/gemfire-assembly/src/main/dist/NOTICE
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/NOTICE b/gemfire-assembly/src/main/dist/NOTICE
index e73e3db..3f5dd07 100644
--- a/gemfire-assembly/src/main/dist/NOTICE
+++ b/gemfire-assembly/src/main/dist/NOTICE
@@ -1,5 +1,5 @@
 Apache Geode (incubating) 
-Copyright 2015 The Apache Software Foundation.
+Copyright 2016 The Apache Software Foundation.
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).


[41/50] [abbrv] incubator-geode git commit: GEODE-777: Add sources and javadoc jars

Posted by ab...@apache.org.
GEODE-777: Add sources and javadoc jars

Generate sources and javadoc jars for each subproject.  This is
important for a maven workflow.


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

Branch: refs/heads/master
Commit: e2c35d7d400516560a35bd2a779e25c5bbf6ab0b
Parents: 561d033
Author: Anthony Baker <ab...@apache.org>
Authored: Tue Jan 12 17:07:05 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Wed Jan 13 10:07:59 2016 -0800

----------------------------------------------------------------------
 build.gradle                  | 12 ++++++++++++
 gemfire-assembly/build.gradle |  4 ++++
 2 files changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2c35d7d/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index edf567d..f42f786 100755
--- a/build.gradle
+++ b/build.gradle
@@ -309,8 +309,20 @@ subprojects {
     classifier 'test'
   }
 
+  task sourcesJar(type: Jar, dependsOn: classes) {
+    classifier = 'sources'
+    from sourceSets.main.allSource
+  }
+
+  task javadocJar(type: Jar, dependsOn: javadoc) {
+    description 'Assembles a jar archive of javadocs.'
+    from javadoc.destinationDir
+    classifier = 'javadoc'
+  }
+
   artifacts {
     testOutput jarTest
+    archives sourcesJar, javadocJar
   }
 
   sourceSets {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2c35d7d/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index 54c33c6..a3f26a9 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -233,6 +233,10 @@ distributions {
       duplicatesStrategy 'exclude'
       exclude '*.asc'
       
+      exclude '*.asc'
+      exclude '*-sources.jar'
+      exclude '*-javadoc.jar'
+
       into ('config') {
         from defaultCacheConfig
         from defaultDistributionConfig


[26/50] [abbrv] incubator-geode git commit: fixing CI failure

Posted by ab...@apache.org.
fixing CI failure

This test periodically failed to find 2 live servers due to an
assumption in the test that Pool.borrowConnection() would never find
the initial connection created by the pool in prefillConnections().
That caused it to only be connected to one of the three servers instead
of the expected two.  The fix is to explicitely ask for a connection
to the second server.


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

Branch: refs/heads/master
Commit: dfb3306c527673548858e77fb3b1049e3b1df186
Parents: 61a16b1
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Wed Jan 6 15:10:51 2016 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Wed Jan 6 15:10:51 2016 -0800

----------------------------------------------------------------------
 .../internal/cache/tier/sockets/CacheServerTestUtil.java |  2 ++
 .../tier/sockets/HAStartupAndFailoverDUnitTest.java      | 11 +++++++----
 2 files changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfb3306c/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java
index 18b7d39..123a275 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java
@@ -516,6 +516,7 @@ public class CacheServerTestUtil extends DistributedTestCase
    */
   public static void disableShufflingOfEndpoints()
   {
+    // TODO DISABLE_RANDOM doesn't seem to be used anywhere
     System.setProperty("gemfire.PoolImpl.DISABLE_RANDOM", "true");
     System.setProperty("gemfire.bridge.disableShufflingOfEndpoints", "true");
   }
@@ -525,6 +526,7 @@ public class CacheServerTestUtil extends DistributedTestCase
    */
   public static void enableShufflingOfEndpoints()
   {
+    // TODO DISABLE_RANDOM doesn't seem to be used anywhere
     System.setProperty("gemfire.PoolImpl.DISABLE_RANDOM", "false");
     System.setProperty("gemfire.bridge.disableShufflingOfEndpoints", "false");
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfb3306c/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
index 1d9f08d..9a00680 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
@@ -281,7 +281,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       verifyDeadAndLiveServers(2,1);
       server3.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsAlive");
     }
-
+    
     /**
      * Tests failover initialization by cache operation Threads on secondary
      */
@@ -652,10 +652,13 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
     cache.createRegion(REGION_NAME, attrs);
 
     pool = p;
-    conn = pool.acquireConnection();
-    assertNotNull(conn);
-
+    // since the default minConnections is 1 we currently have a connection to
+    // server1 (vm_0).  Now we create a connection to server2 (vm_1)
+    conn = pool.acquireConnection(new ServerLocation(Host.getHost(0).getHostName(), PORT2));
 
+    // assert that the conn is to server2 since the tests assume that this is so
+    assertNotNull(conn);
+    assertTrue(conn.getEndpoint().getLocation().getPort() == PORT2);
   }
 
   public static void createClientCacheWithIncorrectPrimary(String testName, String host) throws Exception


[30/50] [abbrv] incubator-geode git commit: GEODE-610: Add LICENSE and NOTICE for binary distribution

Posted by ab...@apache.org.
GEODE-610: Add LICENSE and NOTICE for binary distribution

Add a boilerplate NOTICE file for the binary distribution.  Add
the external dependencies in the LICENSE file. The LICENSE file
will still need the bundled source dependencies to be added.
Also, the components that are dual-licensed under ASL and some
other license may need to be added.


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

Branch: refs/heads/master
Commit: 7492dce89d48bb9299093906d9d105d3d71095a5
Parents: 5eae2b9
Author: Anthony Baker <ab...@pivotal.io>
Authored: Sat Jan 2 09:12:52 2016 -0600
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Wed Jan 6 17:37:43 2016 -0800

----------------------------------------------------------------------
 gemfire-assembly/src/main/dist/LICENSE | 275 ++++++++++++++++++++++++++++
 gemfire-assembly/src/main/dist/NOTICE  |   5 +
 2 files changed, 280 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7492dce8/gemfire-assembly/src/main/dist/LICENSE
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/LICENSE b/gemfire-assembly/src/main/dist/LICENSE
new file mode 100644
index 0000000..893fce3
--- /dev/null
+++ b/gemfire-assembly/src/main/dist/LICENSE
@@ -0,0 +1,275 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+===========================================================================
+
+Apache Geode includes a number of components and libraries with separate 
+copyright notices and license terms. Your use of those components are 
+subject to the terms and conditions of the following licenses. 
+
+
+---------------------------------------------------------------------------
+Antlr (http://www.antlr.org/) antlr:antlr:2.7.7
+---------------------------------------------------------------------------
+This product bundles Antlr which is available under a BSD license.  
+For details see http://www.antlr.org/license.html.
+
+---------------------------------------------------------------------------
+javax.activation (http://java.sun.com/javase/technologies/desktop/javabeans/jaf/index.jsp) javax.activation:activation:1.1.1
+---------------------------------------------------------------------------
+This product bundles javax.activation which is available under a CDDL license.  
+For details see https://glassfish.dev.java.net/public/CDDLv1.0.html.
+
+---------------------------------------------------------------------------
+javax.mail (http://www.oracle.com/) javax.mail:javax.mail-api:1.4.5
+---------------------------------------------------------------------------
+This product bundles javax.mail which is available under a CDDL or GPLv2+CE license.  
+For details see http://www.sun.com/cddl or https://glassfish.dev.java.net/public/CDDL+GPL.html.
+
+---------------------------------------------------------------------------
+javax.resource (https://glassfish.java.net/) javax.resource:javax.resource-api:1.7
+---------------------------------------------------------------------------
+This product bundles javax.resource which is available under a CDDL or GPLv2+CE license.  
+For details see https://glassfish.dev.java.net/public/CDDL+GPL.html.
+
+---------------------------------------------------------------------------
+javax.servlet (https://glassfish.java.net/) javax.servlet:javax.servlet-api:3.1.0
+---------------------------------------------------------------------------
+This product bundles javax.servlet which is available under a CDDL or GPLv2+CE license.  
+For details see https://glassfish.dev.java.net/public/CDDL+GPL.html.
+
+---------------------------------------------------------------------------
+javax.transaction (https://glassfish.java.net/) javax.transaction:javax.transaction-api:1.2
+---------------------------------------------------------------------------
+This product bundles javax.transaction which is available under a CDDL or GPLv2+CE license.  
+For details see https://glassfish.dev.java.net/public/CDDL+GPL.html.
+
+---------------------------------------------------------------------------
+JLine (http://jline.sourceforge.net) jline:jline:2.12
+---------------------------------------------------------------------------
+This product bundles JLine which is available under a BSD license.  
+For details see http://www.opensource.org/licenses/bsd-license.php.
+
+---------------------------------------------------------------------------
+Paranamer (https://github.com/paul-hammant/paranamer) com.thoughtworks.paranamer:paranamer:2.3
+---------------------------------------------------------------------------
+This product bundles Paranamer which is available under a BSD license.  
+For details see http://www.opensource.org/licenses/bsd-license.php.
+
+---------------------------------------------------------------------------
+scala-reflect (http://www.scala-lang.org/) org.scala-lang:scala-reflect:2.10.0
+---------------------------------------------------------------------------
+This product bundles scala-reflect which is available under a BSD-like license.  
+For details see http://www.scala-lang.org/downloads/license.html.
+
+---------------------------------------------------------------------------
+scala-library (http://www.scala-lang.org/) org.scala-lang:scala-library:2.10.0
+---------------------------------------------------------------------------
+This product bundles scala-library which is available under a BSD-like license.  
+For details see http://www.scala-lang.org/downloads/license.html.
+
+---------------------------------------------------------------------------
+SLF4J API (http://www.slf4j.org) org.slf4j:slf4j-api:jar:1.7.7
+---------------------------------------------------------------------------
+This product bundles SLF4J API which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7492dce8/gemfire-assembly/src/main/dist/NOTICE
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/NOTICE b/gemfire-assembly/src/main/dist/NOTICE
new file mode 100644
index 0000000..e73e3db
--- /dev/null
+++ b/gemfire-assembly/src/main/dist/NOTICE
@@ -0,0 +1,5 @@
+Apache Geode (incubating) 
+Copyright 2015 The Apache Software Foundation.
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).


[33/50] [abbrv] incubator-geode git commit: GEODE-610: Fix copy/paste errors in LICENSE text

Posted by ab...@apache.org.
GEODE-610: Fix copy/paste errors in LICENSE text

Thanks to Dave Barnes for pointing out these typos.


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

Branch: refs/heads/master
Commit: 3e5ade76b5cc7ad01074d2b623468ebc41df1d39
Parents: d4611ea
Author: Anthony Baker <ab...@pivotal.io>
Authored: Mon Jan 4 14:54:12 2016 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Wed Jan 6 17:40:03 2016 -0800

----------------------------------------------------------------------
 LICENSE                                | 6 +++---
 gemfire-assembly/src/main/dist/LICENSE | 6 +++---
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e5ade76/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index fd25ceb..e6dd69a 100644
--- a/LICENSE
+++ b/LICENSE
@@ -258,7 +258,7 @@ For details see http://www.opensource.org/licenses/mit-license.php.
 ---------------------------------------------------------------------------
 JSON (http://json.org)
 ---------------------------------------------------------------------------
-This product bundles JOpt Simple which is available under a BSD-like license.  
+This product bundles JSON which is available under a BSD-like license.  
 For details see http://json.org/license.html.
 
 ---------------------------------------------------------------------------
@@ -269,11 +269,11 @@ This product bundles reset.css which is available in the public domain.
 ---------------------------------------------------------------------------
 shred (https://github.com/pandastrike/shred)
 ---------------------------------------------------------------------------
-This product bundles JOpt Simple which is available under an MIT or ISC license.  
+This product bundles shred which is available under an MIT or ISC license.  
 For details see https://github.com/pandastrike/shred/blob/master/LICENSE.
 
 ---------------------------------------------------------------------------
 underscore (http://underscorejs.org)
 ---------------------------------------------------------------------------
-This product bundles JOpt Simple which is available under an MIT license.  
+This product bundles underscore which is available under an MIT license.  
 For details see http://www.opensource.org/licenses/mit-license.php.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e5ade76/gemfire-assembly/src/main/dist/LICENSE
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/LICENSE b/gemfire-assembly/src/main/dist/LICENSE
index aa83327..ce1cd84 100644
--- a/gemfire-assembly/src/main/dist/LICENSE
+++ b/gemfire-assembly/src/main/dist/LICENSE
@@ -301,7 +301,7 @@ For details see http://www.opensource.org/licenses/mit-license.php.
 ---------------------------------------------------------------------------
 JSON (http://json.org)
 ---------------------------------------------------------------------------
-This product bundles JOpt Simple which is available under a BSD-like license.  
+This product bundles JSON which is available under a BSD-like license.  
 For details see http://json.org/license.html.
 
 ---------------------------------------------------------------------------
@@ -330,7 +330,7 @@ For details see http://www.scala-lang.org/downloads/license.html.
 ---------------------------------------------------------------------------
 shred (https://github.com/pandastrike/shred)
 ---------------------------------------------------------------------------
-This product bundles JOpt Simple which is available under an MIT or ISC license.  
+This product bundles shred which is available under an MIT or ISC license.  
 For details see https://github.com/pandastrike/shred/blob/master/LICENSE.
 
 ---------------------------------------------------------------------------
@@ -342,5 +342,5 @@ For details see http://www.opensource.org/licenses/mit-license.php.
 ---------------------------------------------------------------------------
 underscore (http://underscorejs.org)
 ---------------------------------------------------------------------------
-This product bundles JOpt Simple which is available under an MIT license.  
+This product bundles underscore which is available under an MIT license.  
 For details see http://www.opensource.org/licenses/mit-license.php.


[42/50] [abbrv] incubator-geode git commit: GEODE-781: Update maven publishing to include signatures

Posted by ab...@apache.org.
GEODE-781: Update maven publishing to include signatures

The maven-publish plugin does not grok signature nor sign pom
files.  These are needed for publishing release versions on Maven
Central.  Switching over to the nexus plugin simplified signing,
generation of javadoc / sources jars, and publishing.

To publish to the local maven repository (~/.m2/repository):
  gradle install

To publish to the snapshot / release repositories:
  gradle uploadArtifacts

Signatures will only be created if the following properties are set
in ~/.gradle.properties:
  signing.keyId
  signing.secretKeyRingFile
  signing.password

To upload the artifacts to Maven, you will need to set the following
properties in ~/.gradle/gradle.properties:
  nexusUsername
  nexusPassword


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

Branch: refs/heads/master
Commit: e9bf2535fcd635bc183123138e1a8b28501f4374
Parents: e2c35d7
Author: Anthony Baker <ab...@apache.org>
Authored: Thu Jan 14 14:47:59 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Sat Jan 16 09:35:46 2016 -0800

----------------------------------------------------------------------
 build.gradle                          |  75 +++++++++++-------
 gemfire-assembly/build.gradle         | 119 +++++------------------------
 gemfire-core/build.gradle             |   4 -
 gemfire-web-api/build.gradle          |   4 -
 gemfire-web/build.gradle              |   4 -
 gradle/dependency-versions.properties |   3 -
 6 files changed, 68 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e9bf2535/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index f42f786..2322120 100755
--- a/build.gradle
+++ b/build.gradle
@@ -17,13 +17,14 @@
 
 buildscript {
   repositories {
-    maven {
-      url "https://plugins.gradle.org/m2/"
-    }
+    maven { url "https://plugins.gradle.org/m2/" }
   }
+
   dependencies {
     classpath "gradle.plugin.org.nosphere.apache:creadur-rat-gradle:0.2.0"
     classpath "org.ajoberstar:gradle-git:1.3.2"
+    classpath 'com.bmuschko:gradle-nexus-plugin:2.3.1'
+    classpath 'org.hibernate.build.gradle:gradle-maven-publish-auth:2.0.1'
   }
 }
 
@@ -110,17 +111,6 @@ gradle.taskGraph.whenReady({ graph ->
 
 subprojects {
   apply plugin: 'java'
-  apply plugin: 'signing'
-
-  // set these properties in ~/.gradle/gradle.properties file: 
-  // signing.keyId
-  // signing.password
-  // signing.secretKeyRingFile
-  if (isReleaseVersion) {
-    signing {
-      sign configurations.archives
-    }
-  }
 
   // apply compiler options
   gradle.taskGraph.whenReady( { graph ->
@@ -309,20 +299,8 @@ subprojects {
     classifier 'test'
   }
 
-  task sourcesJar(type: Jar, dependsOn: classes) {
-    classifier = 'sources'
-    from sourceSets.main.allSource
-  }
-
-  task javadocJar(type: Jar, dependsOn: javadoc) {
-    description 'Assembles a jar archive of javadocs.'
-    from javadoc.destinationDir
-    classifier = 'javadoc'
-  }
-
   artifacts {
     testOutput jarTest
-    archives sourcesJar, javadocJar
   }
 
   sourceSets {
@@ -495,6 +473,51 @@ subprojects {
     }
   })
 
+  // publishing configuration
+  apply plugin: 'com.bmuschko.nexus'
+  apply plugin: 'maven-publish-auth'
+  
+  extraArchive {
+    sources = true
+    javadoc = true
+    tests = false
+  }
+
+  nexus {
+    sign = true
+    repositoryUrl = 'https://repository.apache.org/service/local/staging/deploy/maven2'
+    snapshotRepositoryUrl = 'https://repository.apache.org/content/repositories/snapshots'
+  }
+  
+  modifyPom {
+    project {
+      name 'Apache Geode (incubating)'
+      description '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'
+      url 'http://geode.incubator.apache.org'
+
+      scm {
+        url 'https://git-wip-us.apache.org/repos/asf?p=incubator-geode.git;a=tree'
+        connection 'scm:https://git-wip-us.apache.org/repos/asf/incubator-geode.git'
+        developerConnection 'scm:https://git-wip-us.apache.org/repos/asf/incubator-geode.git'
+      }
+
+      licenses {
+        license {
+          name 'The Apache Software License, Version 2.0'
+          url 'http://www.apache.org/licenses/LICENSE-2.0.txt'
+        }
+      }
+
+      repositories {
+        repository {
+          id 'libs-release'
+          name 'Spring Maven libs-release Repository'
+          url 'http://repo.spring.io/libs-release'
+        }
+      }
+    }
+  }
+
   // Make precheckin task run all validation tests for checking in code.
   task precheckin (dependsOn: [ build, integrationTest, distributedTest ]) {
     description 'Run this task before checking in code to validate changes. This task combines the following tasks: build, integrationTest, and distributedTest'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e9bf2535/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index a3f26a9..5930f13 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -16,27 +16,27 @@
  */
 import org.gradle.api.internal.artifacts.publish.ArchivePublishArtifact;
 
-buildscript {
-    repositories {
-        mavenCentral()
-    }
-    dependencies {
-        classpath group: 'org.hibernate.build.gradle', name: 'gradle-maven-publish-auth', version: project.'gradle-maven-publish-auth.version'
-    }
-}
 
 apply plugin: 'distribution'
-apply plugin: 'maven-publish'
-apply plugin: 'maven-publish-auth'
 
 jar.enabled = false
+extraArchive {
+  sources = false
+  javadoc = false
+  tests = false
+}
 
-configurations {
-  archives
+// We are not publishing anything from this project into maven and using two distributions seems
+// to break the nexus plugin.  So we delete the nexus tasks but we have to replace the upload task
+// with an empty task of the same name...argh.
+afterEvaluate {
+  tasks.remove(install)
+  tasks.remove(uploadArchives)
+  task uploadArchives << { }
 }
 
-// gradle doesn't automatically remove the jar artifact even though we disabled it
-// this causes the signing plugin to fail
+// Gradle doesn't automatically remove the jar artifact even though we disabled it
+// this causes publishing to fail.  So we nuke all the disabled artifacts from all configurations.
 configurations.all {
   artifacts.removeAll artifacts.findAll { it instanceof ArchivePublishArtifact && !it.archiveTask.enabled }
 }
@@ -191,13 +191,6 @@ gradle.taskGraph.whenReady( { graph ->
     archive.doLast {
       ant.checksum file:"${archive.archivePath}", algorithm:"md5"
       ant.checksum file:"${archive.archivePath}", algorithm:"sha-256"
-
-      if (isReleaseVersion) {      
-        signing {
-          required { isReleaseVersion }
-          sign archive.archivePath
-        }
-      }
     }
   }
 })
@@ -263,12 +256,15 @@ distributions {
           include 'spring-data-gemfire-*'
         }
         from project(":gemfire-core").configurations.archives.allArtifacts.files
-
+        from project(":gemfire-core").webJar
+        from project(":gemfire-core").raJar
+        from project(":gemfire-core").jcaJar
+        
         from project(":gemfire-lucene").configurations.runtime
         from project(":gemfire-lucene").configurations.archives.allArtifacts.files
 
         // include this jar        
-        from project(":gemfire-web-api").jar.outputs.files.getFiles()
+        from project(":gemfire-web-api").jar
         
         // dependency jars
         from depsJar
@@ -287,79 +283,6 @@ distributions {
   }
 }
 
-
-// Repos to be added to POMs
-def springReleaseRepo = [ id:'spring-release', name:'Spring Maven RELEASE Repository', url:'https://repo.spring.io/release' ]
-def springMilestoneRepo = [ id:'spring-milestone', name:'Spring Maven MILESTONE Repository', url:'https://repo.spring.io/milestone' ]
-def springSnapshotRepo = [ id:'spring-snapshot', name:'Spring Maven SNAPSHOT Repository', url:'https://repo.spring.io/snapshot' ]
-def springLibsReleaseRepo = [ id:'libs-release', name:'Spring Maven libs-release Repository', url:'http://repo.spring.io/libs-release' ]
-def springExtReleaseLocalRepo = [ id:'ext-release-local', name:'Spring Maven ext-release-local Repository', url:'http://repo.spring.io/ext-release-local' ]
-
-def MavenRepos = [ springReleaseRepo, springSnapshotRepo, springLibsReleaseRepo, springExtReleaseLocalRepo ]
-
-// Jars to be published via Maven
-def commonJar = [publicationName:'commonJar', project:project(":gemfire-common").name]
-def coreJar = [publicationName:'coreJar', project:project(":gemfire-core").name]
-def jsonJar = [publicationName:'jsonJar', project:project(":gemfire-json").name]
-def joptsimpleJar = [publicationName:'joptsimpleJar', project:project(":gemfire-joptsimple").name]
-def luceneJar = [publicationName:'luceneJar', project:project(":gemfire-lucene").name]
-def MavenJars = [ coreJar, jsonJar, joptsimpleJar, commonJar, luceneJar ]
-
-afterEvaluate {
-  publishing {
-    publications {
-      MavenJars.each {
-        def publicationName = it.publicationName
-        def projectName = it.project
-        "$publicationName"(MavenPublication) {
-          artifactId projectName
-          artifact project(':' + projectName).jar
-          pom.withXml {
-            def repositoriesNode = asNode().appendNode('repositories')
-            MavenRepos.each {
-              def repositoryNode = repositoriesNode.appendNode('repository')
-              repositoryNode.appendNode('id', it.id)
-              repositoryNode.appendNode('name', it.name)
-              repositoryNode.appendNode('url', it.url)
-            }
-            def dependenciesNode = asNode().appendNode('dependencies')
-            //Iterate over the runtime dependencies
-            project(':' + projectName).configurations.runtime.allDependencies.each {
-              def dependencyNode = dependenciesNode.appendNode('dependency')
-              dependencyNode.appendNode('groupId', it.group)
-              dependencyNode.appendNode('artifactId', it.name)
-              dependencyNode.appendNode('version', it.version)
-            }
-          }
-        }
-      }
-    }
-  }
-
-  // The following is typically defined in a root ASF pom: org.apache:apache
-  // If anything breaks around publishing check that we're still synchronized
-  // with whatever Maven repo definitions are there. Pay special attention to
-  // the names of the repositories -- they need to be spelled exactly like
-  // the ones defined in root ASF pom
-  publishing {
-    repositories {
-      if (!isReleaseVersion) {
-        // Apache Development Snapshot Repository
-        maven {
-          name "apache.snapshots.https"
-          url "https://repository.apache.org/content/repositories/snapshots"
-        }
-      } else {
-        // Apache Release Distribution Repository
-        maven {
-          name "apache.releases.https"
-          url "https://repository.apache.org/service/local/staging/deploy/maven2"
-        }
-      }
-    }
-  } 
-}
-
 // Create a configuration closure to configure test targets with the install directory
 def dependOnInstalledProduct = {
   dependsOn installDist
@@ -371,9 +294,5 @@ def dependOnInstalledProduct = {
 test dependOnInstalledProduct
 distributedTest dependOnInstalledProduct
 
-artifacts {
-  archives depsJar, gfshDepsJar
-}
-
 // Make build final task to generate all test and product resources
 build.dependsOn installDist

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e9bf2535/gemfire-core/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-core/build.gradle b/gemfire-core/build.gradle
index 3e3c19b..041dc07 100755
--- a/gemfire-core/build.gradle
+++ b/gemfire-core/build.gradle
@@ -196,10 +196,6 @@ task jcaJar (type: Jar, dependsOn: raJar) {
   from raJar.archivePath
 }
 
-artifacts {
-  archives webJar, raJar, jcaJar
-}
-
 configurations {
   classesOutput {
     extendsFrom compile

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e9bf2535/gemfire-web-api/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-web-api/build.gradle b/gemfire-web-api/build.gradle
index 7c35ecd..940fddd 100755
--- a/gemfire-web-api/build.gradle
+++ b/gemfire-web-api/build.gradle
@@ -49,10 +49,6 @@ jar {
   from sourceSets.main.output
 }
 
-artifacts {
-  archives jar
-}
-
 war {
   classpath configurations.runtime
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e9bf2535/gemfire-web/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-web/build.gradle b/gemfire-web/build.gradle
index 1c926c0..596590b 100755
--- a/gemfire-web/build.gradle
+++ b/gemfire-web/build.gradle
@@ -50,10 +50,6 @@ idea.module.iml {
   }
 }
 
-artifacts {
-  archives jar
-}
-
 war {
   classpath configurations.runtime 
   classpath project(':gemfire-core').webJar.archivePath

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e9bf2535/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 0bbbccd..901bdd5 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -13,9 +13,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Buildscript Dependencies
-gradle-maven-publish-auth.version = 2.0.1
-
 # Dependency versions
 activation.version = 1.1.1
 annotations.version = 3.0.0


[36/50] [abbrv] incubator-geode git commit: Update release build version to 1.0.0-incubating.M1

Posted by ab...@apache.org.
Update release build version to 1.0.0-incubating.M1


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

Branch: refs/heads/master
Commit: d6d70306955037a407ab975c8c2004298289d4bb
Parents: a097fcf
Author: Anthony Baker <ab...@pivotal.io>
Authored: Mon Jan 11 14:13:04 2016 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Mon Jan 11 14:13:04 2016 -0800

----------------------------------------------------------------------
 build.gradle      | 3 ++-
 gradle.properties | 7 ++++++-
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d6d70306/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index f464dc3..1f1077c 100755
--- a/build.gradle
+++ b/build.gradle
@@ -36,7 +36,8 @@ dependencyVersions.load(new FileInputStream("${project.projectDir}/gradle/depend
 dependencyVersions.keys().each{ k -> project.ext[k] = dependencyVersions[k]}
 
 allprojects {
-  version = versionNumber + '-' + releaseType
+  version = versionNumber + '.' + releaseType
+
   // We want to see all test results.  This is equivalatent to setting --continue
   // on the command line.
   gradle.startParameter.continueOnFailure = true

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d6d70306/gradle.properties
----------------------------------------------------------------------
diff --git a/gradle.properties b/gradle.properties
index c6ca51f..3687ff0 100755
--- a/gradle.properties
+++ b/gradle.properties
@@ -12,8 +12,13 @@
 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 # See the License for the specific language governing permissions and
 # limitations under the License.
+
+# Set the release type using the following conventions:
+# M? - Milestone 
+# RC? - Release candidate
+# RELEASE - Final build
 versionNumber = 1.0.0-incubating
-releaseType = SNAPSHOT
+releaseType = M1
 
 org.gradle.daemon = true
 org.gradle.jvmargs = -Xmx2048m


[14/50] [abbrv] incubator-geode git commit: Revert "GEODE-715: Move dunit.standalone under com.gemstone.gemfire.test"

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/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
new file mode 100644
index 0000000..7fc762f
--- /dev/null
+++ b/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
@@ -0,0 +1,261 @@
+/*
+ * 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 dunit.standalone;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintStream;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.rmi.AccessException;
+import java.rmi.NotBoundException;
+import java.rmi.RemoteException;
+import java.rmi.registry.Registry;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.io.FileUtils;
+
+import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+import dunit.RemoteDUnitVMIF;
+
+/**
+ * @author dsmith
+ *
+ */
+public class ProcessManager {
+  private int namingPort;
+  private Map<Integer, ProcessHolder> processes = new HashMap<Integer, ProcessHolder>();
+  private File log4jConfig;
+  private int pendingVMs;
+  private Registry registry;
+  private int debugPort = Integer.getInteger("dunit.debug.basePort", 0);
+  private int suspendVM = Integer.getInteger("dunit.debug.suspendVM", -100);
+
+  public ProcessManager(int namingPort, Registry registry) {
+    this.namingPort = namingPort;
+    this.registry = registry;
+  }
+  
+  public void launchVMs() throws IOException, NotBoundException {
+    log4jConfig = LogService.findLog4jConfigInCurrentDir();
+  }
+
+  public synchronized void launchVM(int vmNum) throws IOException {
+    if(processes.containsKey(vmNum)) {
+      throw new IllegalStateException("VM " + vmNum + " is already running.");
+    }
+    
+    String[] cmd = buildJavaCommand(vmNum, namingPort);
+    System.out.println("Executing " + Arrays.asList(cmd));
+    File workingDir = getVMDir(vmNum);
+    try {
+      FileUtil.delete(workingDir);
+    } catch(IOException e) {
+      //This delete is occasionally failing on some platforms, maybe due to a lingering
+      //process. Allow the process to be launched anyway.
+      System.err.println("Unable to delete " + workingDir + ". Currently contains " 
+                          + Arrays.asList(workingDir.list()));
+    }
+    workingDir.mkdirs();
+    if (log4jConfig != null) {
+      FileUtils.copyFileToDirectory(log4jConfig, workingDir);
+    }
+    
+    //TODO - delete directory contents, preferably with commons io FileUtils
+    Process process = Runtime.getRuntime().exec(cmd, null, workingDir);
+    pendingVMs++;
+    ProcessHolder holder = new ProcessHolder(process);
+    processes.put(vmNum, holder);
+    linkStreams(vmNum, holder, process.getErrorStream(), System.err);
+    linkStreams(vmNum, holder, process.getInputStream(), System.out);
+  }
+
+  public static File getVMDir(int vmNum) {
+    return new File(DUnitLauncher.DUNIT_DIR, "vm" + vmNum);
+  }
+  
+  public synchronized void killVMs() {
+    for(ProcessHolder process : processes.values()) {
+      if(process != null) {
+        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);
+    }
+    try {
+      ProcessHolder holder = processes.remove(vmNum);
+      holder.kill();
+      holder.getProcess().waitFor();
+      launchVM(vmNum);
+    } catch (InterruptedException | IOException e) {
+      throw new RuntimeException("Unable to restart VM " + vmNum, e);
+    }
+  }
+   
+  private void linkStreams(final int vmNum, final ProcessHolder holder, final InputStream in, final PrintStream out) {
+    Thread ioTransport = new Thread() {
+      public void run() {
+        BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+        String vmName = (vmNum==-2)? "[locator]" : "[vm_"+vmNum+"]";
+        try {
+          String line = reader.readLine();
+          while(line != null) {
+            if (line.length() == 0) {
+              out.println();
+            } else {
+              out.print(vmName);
+              out.println(line);
+            }
+            line = reader.readLine();
+          }
+        } catch(Exception e) {
+          if(!holder.isKilled()) {
+            out.println("Error transporting IO from child process");
+            e.printStackTrace(out);
+          }
+        }
+      }
+    };
+
+    ioTransport.setDaemon(true);
+    ioTransport.start();
+  }
+
+  private String[] buildJavaCommand(int vmNum, int namingPort) {
+    String cmd = System.getProperty( "java.home" ) + File.separator + "bin" + File.separator + "java";
+    String classPath = System.getProperty("java.class.path");
+    //String tmpDir = System.getProperty("java.io.tmpdir");
+    String agent = getAgentString();
+
+    String jdkDebug = "";
+    if (debugPort > 0) {
+      jdkDebug += ",address=" + debugPort;
+      debugPort++;
+    }
+
+    String jdkSuspend = vmNum == suspendVM ? "y" : "n";
+
+    return new String[] {
+      cmd, "-classpath", classPath,
+      "-D" + DUnitLauncher.RMI_PORT_PARAM + "=" + namingPort,
+      "-D" + DUnitLauncher.VM_NUM_PARAM + "=" + vmNum,
+      "-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=" + jdkSuspend + jdkDebug,
+      "-XX:+HeapDumpOnOutOfMemoryError",
+      "-Xmx512m",
+      "-Dgemfire.DEFAULT_MAX_OPLOG_SIZE=10",
+      "-Dgemfire.disallowMcastDefaults=true",
+      "-ea",
+      agent,
+      "dunit.standalone.ChildVM"
+    };
+  }
+  
+  /**
+   * Get the java agent passed to this process and pass it to the child VMs.
+   * This was added to support jacoco code coverage reports
+   */
+  private String getAgentString() {
+    RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
+    if (runtimeBean != null) {
+      for(String arg: runtimeBean.getInputArguments()) {
+        if(arg.contains("-javaagent:")) {
+          //HACK for gradle bug  GRADLE-2859. Jacoco is passing a relative path
+          //That won't work when we pass this to dunit VMs in a different 
+          //directory
+          arg = arg.replace("-javaagent:..", "-javaagent:" + System.getProperty("user.dir") + File.separator + "..");
+          arg = arg.replace("destfile=..", "destfile=" + System.getProperty("user.dir") + File.separator + "..");
+          return arg;
+        }
+      }
+    }
+    
+    return "-DdummyArg=true";
+  }
+
+  synchronized void signalVMReady() {
+    pendingVMs--;
+    this.notifyAll();
+  }
+  
+  public synchronized boolean waitForVMs(long timeout) throws InterruptedException {
+    long end = System.currentTimeMillis() + timeout;
+    while(pendingVMs > 0) {
+      long remaining = end - System.currentTimeMillis();
+      if(remaining <= 0) {
+        return false;
+      }
+      this.wait(remaining);
+    }
+    
+    return true;
+  }
+  
+  private static class ProcessHolder {
+    private final Process process;
+    private volatile boolean killed = false;
+    
+    public ProcessHolder(Process process) {
+      this.process = process;
+    }
+
+    public void kill() {
+      this.killed = true;
+      process.destroy();
+      
+    }
+
+    public Process getProcess() {
+      return process;
+    }
+
+    public boolean isKilled() {
+      return killed;
+    }
+    
+    public boolean isAlive() {
+      return !killed && process.isAlive();
+    }
+  }
+
+  public RemoteDUnitVMIF getStub(int i) throws AccessException, RemoteException, NotBoundException, InterruptedException {
+    waitForVMs(DUnitLauncher.STARTUP_TIMEOUT);
+    return (RemoteDUnitVMIF) registry.lookup("vm" + i);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/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
new file mode 100644
index 0000000..742dc55
--- /dev/null
+++ b/gemfire-core/src/test/java/dunit/standalone/RemoteDUnitVM.java
@@ -0,0 +1,144 @@
+/*
+ * 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 dunit.standalone;
+
+import hydra.MethExecutor;
+import hydra.MethExecutorResult;
+
+import java.rmi.RemoteException;
+import java.rmi.server.UnicastRemoteObject;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.internal.logging.LogService;
+
+import dunit.RemoteDUnitVMIF;
+
+/**
+ * @author dsmith
+ *
+ */
+public class RemoteDUnitVM extends UnicastRemoteObject implements RemoteDUnitVMIF {
+  
+  private static final Logger logger = LogService.getLogger();
+  
+  public RemoteDUnitVM() throws RemoteException {
+    super();
+  }
+
+  /** 
+   * Called remotely by the master controller to cause the client to execute 
+   * the instance method on the object.  Does this synchronously (does not spawn
+   * a thread).  This method is used by the unit test framework, dunit.
+   *
+   * @param obj the object to execute the method on
+   * @param methodName the name of the method to execute
+   * @return the result of method execution
+   */ 
+   public MethExecutorResult executeMethodOnObject( Object obj, String methodName ) {
+     String name = obj.getClass().getName() + "." + methodName + 
+       " on object: " + obj;
+     logger.info("Received method: " + name);
+     long start = System.currentTimeMillis();
+     MethExecutorResult result = MethExecutor.executeObject( obj, methodName );
+     long delta = System.currentTimeMillis() - start;
+     logger.info( "Got result: " + result.toString().trim()  + " from " +
+               name + " (took " + delta + " ms)");
+     return result;
+   }
+
+   /**
+    * Executes a given instance method on a given object with the given
+    * arguments. 
+    */
+   public MethExecutorResult executeMethodOnObject(Object obj,
+                                                   String methodName,
+                                                   Object[] args) {
+     String name = obj.getClass().getName() + "." + methodName + 
+              (args != null ? " with " + args.length + " args": "") +
+       " on object: " + obj;
+     logger.info("Received method: " + name);
+     long start = System.currentTimeMillis();
+     MethExecutorResult result = 
+       MethExecutor.executeObject(obj, methodName, args);
+     long delta = System.currentTimeMillis() - start;
+     logger.info( "Got result: " + result.toString() + " from " + name + 
+               " (took " + delta + " ms)");
+     return result;
+   }
+
+  /** 
+   * Called remotely by the master controller to cause the client to execute 
+   * the method on the class.  Does this synchronously (does not spawn a thread).
+   * This method is used by the unit test framework, dunit.
+   *
+   * @param className the name of the class execute
+   * @param methodName the name of the method to execute
+   * @return the result of method execution
+   */ 
+   public MethExecutorResult executeMethodOnClass( String className, String methodName ) {
+     String name = className + "." + methodName;
+     logger.info("Received method: " +  name);
+     long start = System.currentTimeMillis();
+     MethExecutorResult result = MethExecutor.execute( className, methodName );
+     long delta = System.currentTimeMillis() - start;
+     logger.info( "Got result: " + result.toString() + " from " + name + 
+               " (took " + delta + " ms)");
+     
+     return result;
+   }
+
+   /**
+    * Executes a given static method in a given class with the given
+    * arguments. 
+    */
+   public MethExecutorResult executeMethodOnClass(String className,
+                                                  String methodName,
+                                                  Object[] args) {
+     String name = className + "." + methodName + 
+       (args != null ? " with " + args.length + " args": "");
+     logger.info("Received method: " + name);
+     long start = System.currentTimeMillis();
+     MethExecutorResult result = 
+       MethExecutor.execute(className, methodName, args);
+     long delta = System.currentTimeMillis() - start;
+     logger.info( "Got result: " + result.toString() + " from " + name +
+               " (took " + delta + " ms)");
+     return result;
+   }
+
+  public void executeTask(int tsid, int type, int index) throws RemoteException {
+    throw new UnsupportedOperationException();
+    
+  }
+  
+  public void runShutdownHook() throws RemoteException {
+    
+  }
+
+  public void notifyDynamicActionComplete(int actionId) throws RemoteException {
+    throw new UnsupportedOperationException();
+    
+  }
+
+  public void shutDownVM() throws RemoteException {
+    ChildVM.stopVM();
+  }
+
+  public void disconnectVM() throws RemoteException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/dunit/standalone/StandAloneDUnitEnv.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/standalone/StandAloneDUnitEnv.java b/gemfire-core/src/test/java/dunit/standalone/StandAloneDUnitEnv.java
new file mode 100644
index 0000000..085035d
--- /dev/null
+++ b/gemfire-core/src/test/java/dunit/standalone/StandAloneDUnitEnv.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 dunit.standalone;
+
+import java.io.File;
+import java.rmi.RemoteException;
+import java.util.Properties;
+
+import dunit.BounceResult;
+import dunit.DUnitEnv;
+import dunit.standalone.DUnitLauncher.MasterRemote;
+
+public class StandAloneDUnitEnv extends DUnitEnv {
+
+  private MasterRemote master;
+
+  public StandAloneDUnitEnv(MasterRemote master) {
+    this.master = master;
+  }
+
+  @Override
+  public String getLocatorString() {
+    return DUnitLauncher.getLocatorString();
+  }
+
+  @Override
+  public String getLocatorAddress() {
+    return "localhost";
+  }
+  
+  @Override
+  public int getLocatorPort() {
+    return DUnitLauncher.locatorPort;
+  }
+
+  @Override
+  public Properties getDistributedSystemProperties() {
+    return DUnitLauncher.getDistributedSystemProperties();
+  }
+
+  @Override
+  public int getPid() {
+    return Integer.getInteger(DUnitLauncher.VM_NUM_PARAM, -1).intValue();
+  }
+
+  @Override
+  public int getVMID() {
+    return getPid();
+  }
+
+  @Override
+  public BounceResult bounce(int pid) throws RemoteException {
+    return master.bounce(pid);
+  }
+
+  @Override
+  public File getWorkingDirectory(int pid) {
+    return ProcessManager.getVMDir(pid);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/dunit/tests/BasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/tests/BasicDUnitTest.java b/gemfire-core/src/test/java/dunit/tests/BasicDUnitTest.java
new file mode 100644
index 0000000..c284b3a
--- /dev/null
+++ b/gemfire-core/src/test/java/dunit/tests/BasicDUnitTest.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 dunit.tests;
+
+import java.util.Properties;
+
+import dunit.AsyncInvocation;
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.RMIException;
+import dunit.VM;
+
+/**
+ * This class tests the basic functionality of the distributed unit
+ * test framework.
+ */
+public class BasicDUnitTest extends DistributedTestCase {
+
+  public BasicDUnitTest(String name) {
+    super(name);
+  }
+
+  ////////  Test Methods
+
+  /**
+   * Tests how the Hydra framework handles an error
+   */
+  public void _testDontCatchRemoteException() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    vm.invoke(this.getClass(), "remoteThrowException");
+  }
+
+  public void testRemoteInvocationWithException() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invoke(this.getClass(), "remoteThrowException");
+      fail("Should have thrown a BasicTestException");
+
+    } catch (RMIException ex) {
+      assertTrue(ex.getCause() instanceof BasicTestException);
+    }
+  } 
+
+  static class BasicTestException extends RuntimeException {
+    BasicTestException() {
+      this("Test exception.  Please ignore.");
+    }
+
+    BasicTestException(String s) {
+      super(s);
+    }
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   *
+   */
+  protected static void remoteThrowException() {
+    String s = "Test exception.  Please ignore.";
+    throw new BasicTestException(s);
+  }
+
+  public void _testRemoteInvocationBoolean() {
+
+  }
+
+  public void testRemoteInvokeAsync() throws InterruptedException {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    String name = this.getUniqueName();
+    String value = "Hello";
+
+    AsyncInvocation ai =
+      vm.invokeAsync(this.getClass(), "remoteBind", 
+                     new Object[] { name, value });
+    ai.join();
+    // TODO shouldn't we call fail() here?
+    if (ai.exceptionOccurred()) {
+      fail("remoteBind failed", ai.getException());
+    }
+
+    ai = vm.invokeAsync(this.getClass(), "remoteValidateBind",
+                        new Object[] {name, value });
+    ai.join();
+    if (ai.exceptionOccurred()) {
+      fail("remoteValidateBind failed", ai.getException());
+    }
+  }
+
+  private static Properties bindings = new Properties();
+  private static void remoteBind(String name, String s) {
+    new BasicDUnitTest("bogus").getSystem(); // forces connection
+    bindings.setProperty(name, s);
+  }
+
+  private static void remoteValidateBind(String name, String expected)
+  {
+    assertEquals(expected, bindings.getProperty(name));
+  }
+
+  public void testRemoteInvokeAsyncWithException() 
+    throws InterruptedException {
+
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+//    String name = this.getUniqueName();
+//    String value = "Hello";
+
+    AsyncInvocation ai =
+      vm.invokeAsync(this.getClass(), "remoteThrowException");
+    ai.join();
+    assertTrue(ai.exceptionOccurred());
+    Throwable ex = ai.getException();
+    assertTrue(ex instanceof BasicTestException);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/dunit/tests/TestFailure.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/tests/TestFailure.java b/gemfire-core/src/test/java/dunit/tests/TestFailure.java
new file mode 100644
index 0000000..17a39fa
--- /dev/null
+++ b/gemfire-core/src/test/java/dunit/tests/TestFailure.java
@@ -0,0 +1,50 @@
+/*
+ * 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 dunit.tests;
+
+import dunit.*;
+
+/**
+ * The tests in this class always fail.  It is used when developing
+ * DUnit to give us an idea of how test failure are logged, etc.
+ *
+ * @author David Whitlock
+ *
+ * @since 3.0
+ */
+public class TestFailure extends DistributedTestCase {
+
+  public TestFailure(String name) {
+    super(name);
+  }
+
+  ////////  Test Methods
+
+  public void testFailure() {
+    assertTrue("Test Failure", false);
+  }
+
+  public void testError() {
+    String s = "Test Error";
+    throw new Error(s);
+  }
+
+  public void testHang() throws InterruptedException {
+    Thread.sleep(100000 * 1000);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfba327e/gemfire-core/src/test/java/dunit/tests/VMDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/tests/VMDUnitTest.java b/gemfire-core/src/test/java/dunit/tests/VMDUnitTest.java
new file mode 100644
index 0000000..27736dc
--- /dev/null
+++ b/gemfire-core/src/test/java/dunit/tests/VMDUnitTest.java
@@ -0,0 +1,237 @@
+/*
+ * 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 dunit.tests;
+
+import dunit.*;
+
+import java.io.Serializable;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * This class tests the functionality of the {@link VM} class.
+ */
+public class VMDUnitTest extends DistributedTestCase {
+
+  private static final boolean BOOLEAN_VALUE = true;
+  private static final byte BYTE_VALUE = (byte) 40;
+  private static final long LONG_VALUE = 42L;
+  private static final String STRING_VALUE = "BLAH BLAH BLAH";
+
+  public VMDUnitTest(String name) {
+    super(name);
+  }
+
+  ////////  Test Methods
+
+  public void notestInvokeNonExistentMethod() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invoke(VMDUnitTest.class, "nonExistentMethod");
+      fail("Should have thrown an RMIException");
+
+    } catch (RMIException ex) {
+      String s = "Excepted a NoSuchMethodException, got a " +
+        ex.getCause();;
+      assertTrue(s, ex.getCause() instanceof NoSuchMethodException);
+    }
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   * @return
+   */
+  protected static byte remoteByteMethod() {
+    return BYTE_VALUE;
+  }
+
+  public void notestInvokeStaticBoolean() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(BOOLEAN_VALUE,
+                 vm.invokeBoolean(VMDUnitTest.class, "remoteBooleanMethod")); 
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   * @return
+   */
+  protected static boolean remoteBooleanMethod() {
+    return BOOLEAN_VALUE;
+  }
+
+  public void notestInvokeStaticBooleanNotBoolean() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invokeBoolean(VMDUnitTest.class, "remoteByteMethod");
+      fail("Should have thrown an IllegalArgumentException");
+
+    } catch (IllegalArgumentException ex) {
+      
+    }
+  }
+
+  public void notestInvokeStaticLong() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(LONG_VALUE,
+                 vm.invokeLong(VMDUnitTest.class, "remoteLongMethod")); 
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   * @return
+   */
+  protected static long remoteLongMethod() {
+    return LONG_VALUE;
+  }
+
+  public void notestInvokeStaticLongNotLong() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invokeLong(VMDUnitTest.class, "remoteByteMethod");
+      fail("Should have thrown an IllegalArgumentException");
+
+    } catch (IllegalArgumentException ex) {
+      
+    }
+  }
+
+  protected static class ClassWithLong implements Serializable {
+    public long getLong() {
+      return LONG_VALUE;
+    }
+  }
+
+  protected static class ClassWithByte implements Serializable {
+    public byte getByte() {
+      return BYTE_VALUE;
+    }
+  }
+
+  public void notestInvokeInstanceLong() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(LONG_VALUE,
+                 vm.invokeLong(new ClassWithLong(), "getLong"));
+  }
+
+  public void notestInvokeInstanceLongNotLong() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invokeLong(new ClassWithByte(), "getByte");
+      fail("Should have thrown an IllegalArgumentException");
+
+    } catch (IllegalArgumentException ex) {
+
+    }
+  }
+
+  protected static class InvokeRunnable
+    implements Serializable, Runnable {
+
+    public void run() {
+      throw new BasicDUnitTest.BasicTestException();
+    }
+  }
+
+  protected static class ClassWithString implements Serializable {
+    public String getString() {
+      return STRING_VALUE;
+    }
+  }
+
+  public void notestInvokeInstance() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(STRING_VALUE,
+                 vm.invoke(new ClassWithString(), "getString"));
+  }
+
+  public void notestInvokeRunnable() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invoke(new InvokeRunnable());
+      fail("Should have thrown a BasicTestException");
+
+    } catch (RMIException ex) {
+      assertTrue(ex.getCause() instanceof BasicDUnitTest.BasicTestException);
+    }
+  }
+  
+  private static final AtomicInteger COUNTER = new AtomicInteger();
+  public static Integer getAndIncStaticCount() {
+    return new Integer(COUNTER.getAndIncrement());
+  }
+  public static Integer incrementStaticCount(Integer inc) {
+    return new Integer(COUNTER.addAndGet(inc.intValue()));
+  }
+  public static void incStaticCount() {
+    COUNTER.incrementAndGet();
+  }
+  public static class VMTestObject implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final AtomicInteger val;
+    public VMTestObject(int init) {
+      this.val = new AtomicInteger(init);
+    }
+    public Integer get() {
+      return new Integer(this.val.get());
+    }
+    public Integer incrementAndGet() {
+      return new Integer(this.val.incrementAndGet());
+    }
+    public void set(Integer newVal) {
+      this.val.set(newVal.intValue());
+    }
+  }
+  public void testReturnValue() throws Exception {
+    final Host host = Host.getHost(0);
+    final VM vm = host.getVM(0);
+    // Assert class static invocation works
+    AsyncInvocation a1 = vm.invokeAsync(getClass(), "getAndIncStaticCount");
+    a1.join();
+    assertEquals(new Integer(0), a1.getReturnValue());
+    // Assert class static invocation with args works
+    a1 = vm.invokeAsync(getClass(), "incrementStaticCount", new Object[] {new Integer(2)});
+    a1.join();
+    assertEquals(new Integer(3), a1.getReturnValue());
+    // Assert that previous values are not returned when invoking method w/ no return val
+    a1 = vm.invokeAsync(getClass(), "incStaticCount");
+    a1.join();
+    assertNull(a1.getReturnValue());
+    // Assert that previous null returns are over-written 
+    a1 = vm.invokeAsync(getClass(), "getAndIncStaticCount");
+    a1.join();
+    assertEquals(new Integer(4), a1.getReturnValue());
+
+    // Assert object method invocation works with zero arg method
+    final VMTestObject o = new VMTestObject(0);
+    a1 = vm.invokeAsync(o, "incrementAndGet", new Object[] {});
+    a1.join();
+    assertEquals(new Integer(1), a1.getReturnValue());
+    // Assert object method invocation works with no return
+    a1 = vm.invokeAsync(o, "set", new Object[] {new Integer(3)});
+    a1.join();
+    assertNull(a1.getReturnValue());
+  }
+}


[43/50] [abbrv] incubator-geode git commit: GEODE-781: Match repository id to allow Jenkins to upload Maven artifacts

Posted by ab...@apache.org.
GEODE-781: Match repository id to allow Jenkins to upload Maven artifacts


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

Branch: refs/heads/master
Commit: fa60ac7acabd3deb6206002685810ce4a7897674
Parents: e9bf253
Author: Anthony Baker <ab...@apache.org>
Authored: Fri Jan 15 11:07:11 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Sat Jan 16 09:35:57 2016 -0800

----------------------------------------------------------------------
 build.gradle | 13 +++++++++++--
 1 file changed, 11 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa60ac7a/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 2322120..6004226 100755
--- a/build.gradle
+++ b/build.gradle
@@ -475,7 +475,6 @@ subprojects {
 
   // publishing configuration
   apply plugin: 'com.bmuschko.nexus'
-  apply plugin: 'maven-publish-auth'
   
   extraArchive {
     sources = true
@@ -518,6 +517,17 @@ subprojects {
     }
   }
 
+  // Jenkins stores auth information in settings.xml.  We apply the maven-publish-auth plugin to read that
+  // from gradle.  However, we must match the repository id which the nexus plugin is not exposing.
+  apply plugin: 'maven-publish-auth'
+  afterEvaluate {
+    tasks.getByName('uploadArchives').repositories.mavenDeployer() {
+      if (project.nexus.snapshotRepositoryUrl) {
+        repository(id: 'apache.snapshots.https', url: project.nexus.snapshotRepositoryUrl)
+      }
+    }
+  }
+
   // Make precheckin task run all validation tests for checking in code.
   task precheckin (dependsOn: [ build, integrationTest, distributedTest ]) {
     description 'Run this task before checking in code to validate changes. This task combines the following tasks: build, integrationTest, and distributedTest'
@@ -535,4 +545,3 @@ subprojects {
   // Make sure clean task for rootProject runs last
   clean.finalizedBy rootProject.clean
 }
-


[37/50] [abbrv] incubator-geode git commit: GEODE-775: Generate md5 / sha-256 hashes for the distributions

Posted by ab...@apache.org.
GEODE-775: Generate md5 / sha-256 hashes for the distributions

The binary and source distributions created by the build need
accompanying hash files to verification.  This change uses the
doList hook to generate the hashes after the distribution is
created.


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

Branch: refs/heads/master
Commit: 010f5f673feba28991fbb7a0367e50cdc03839e7
Parents: d6d7030
Author: Anthony Baker <ab...@apache.org>
Authored: Tue Jan 12 07:41:35 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Tue Jan 12 08:45:48 2016 -0800

----------------------------------------------------------------------
 gemfire-assembly/build.gradle | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/010f5f67/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index ad466de..4c74513 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -177,6 +177,17 @@ task gfshDepsJar (type: Jar, dependsOn: ':gemfire-core:classes') {
   }
 }
 
+gradle.taskGraph.whenReady( { graph ->
+  tasks.withType(AbstractArchiveTask).findAll {
+    it.name.toLowerCase().contains("dist")
+  }.each { archive ->
+    archive.doLast {
+      ant.checksum file:"${archive.archivePath}", algorithm:"md5"
+      ant.checksum file:"${archive.archivePath}", algorithm:"sha-256"
+    }
+  }
+})
+
 distributions {
   src {
     baseName = 'apache-geode-src'


[07/50] [abbrv] incubator-geode git commit: deprecated the mcast-flow-control distributed-system property

Posted by ab...@apache.org.
deprecated the mcast-flow-control distributed-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/df6d7ff4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/df6d7ff4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/df6d7ff4

Branch: refs/heads/master
Commit: df6d7ff40138a640cea59fef8bc0f690692b36c1
Parents: 9bca880
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Jan 4 10:18:56 2016 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Jan 4 10:21:18 2016 -0800

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/distributed/DistributedSystem.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/df6d7ff4/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
index 159db86..c09d618 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
@@ -192,6 +192,7 @@ import com.gemstone.gemfire.security.GemFireSecurityException;
  *    ratio should be before sending a recharge.  The rechargeBlockMs
  *    tells the sender how long to wait for a recharge before explicitly
  *    requesting one.</dd>
+ *   <dd><U>Deprecated</U>: as of 9.0 GemFire does not include a flow-of-control protocol for multicast messaging.</dd>
  *   <dd><U>Default</U>: "1048576,0.25,5000"</dd>
  *   <dd><U>Allowed values</U>: 100000-maxInt, 0.1-0.5, 500-60000</dd>
  *   <dd><U>Since</U>: 5.0</dd>


[21/50] [abbrv] incubator-geode git commit: GEODE-430 CI failure

Posted by ab...@apache.org.
GEODE-430 CI failure

Race condition in the test.  I added a slight delay to let the client
finish refilling its pool of connections


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

Branch: refs/heads/master
Commit: 8b6d8f255e26c29f596f82b18c85ba1c5a4bb01f
Parents: a69a0dc
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Jan 5 16:11:39 2016 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Jan 5 16:12:50 2016 -0800

----------------------------------------------------------------------
 .../internal/cache/tier/sockets/UpdatePropagationDUnitTest.java    | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8b6d8f25/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
index f64b9b8..34ea3a6 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
@@ -372,6 +372,8 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase
         DistributedTestCase.waitForCriterion(ev, maxWaitTime, 200, true);
       }
     });
+    
+    pause(5000);
 
     //Do a put on Server1 via Connection object from client1.
     // Client1 should not receive updated value while client2 should receive


[32/50] [abbrv] incubator-geode git commit: GEODE-610: Add source dependencies to binary LICENSE file and fix typo

Posted by ab...@apache.org.
GEODE-610: Add source dependencies to binary LICENSE file and fix typo


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

Branch: refs/heads/master
Commit: d4611ea4673143a90b91618a13dce8dd0e0a3385
Parents: fdd0e47
Author: Anthony Baker <ab...@pivotal.io>
Authored: Sat Jan 2 10:24:56 2016 -0600
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Wed Jan 6 17:37:43 2016 -0800

----------------------------------------------------------------------
 LICENSE                                | 12 ++---
 gemfire-assembly/src/main/dist/LICENSE | 71 +++++++++++++++++++++++++++++
 2 files changed, 77 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d4611ea4/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index 25b301c..fd25ceb 100644
--- a/LICENSE
+++ b/LICENSE
@@ -232,12 +232,6 @@ This product bundles JOpt Simple which is available under an MIT license.
 For details see http://www.opensource.org/licenses/mit-license.php.
 
 ---------------------------------------------------------------------------
-JOpt Simple (http://pholser.github.io/jopt-simple/)
----------------------------------------------------------------------------
-This product bundles JOpt Simple which is available under a BSD-like license.  
-For details see http://json.org/license.html.
-
----------------------------------------------------------------------------
 jQuery (https://jquery.com)
 ---------------------------------------------------------------------------
 This product bundles jQuery which is available under an MIT license.  
@@ -262,6 +256,12 @@ This product bundles jQuery Wiggle which is available under an MIT license.
 For details see http://www.opensource.org/licenses/mit-license.php.
 
 ---------------------------------------------------------------------------
+JSON (http://json.org)
+---------------------------------------------------------------------------
+This product bundles JOpt Simple which is available under a BSD-like license.  
+For details see http://json.org/license.html.
+
+---------------------------------------------------------------------------
 reset.css (http://meyerweb.com/eric/tools/css/reset/)
 ---------------------------------------------------------------------------
 This product bundles reset.css which is available in the public domain.  

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d4611ea4/gemfire-assembly/src/main/dist/LICENSE
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/LICENSE b/gemfire-assembly/src/main/dist/LICENSE
index 893fce3..aa83327 100644
--- a/gemfire-assembly/src/main/dist/LICENSE
+++ b/gemfire-assembly/src/main/dist/LICENSE
@@ -215,6 +215,24 @@ This product bundles Antlr which is available under a BSD license.
 For details see http://www.antlr.org/license.html.
 
 ---------------------------------------------------------------------------
+Backbone (http://backbonejs.org)
+---------------------------------------------------------------------------
+This product bundles Backbone which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+Handlebars (http://handlebarsjs.com)
+---------------------------------------------------------------------------
+This product bundles Handlebars which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+highlight.js (https://highlightjs.org)
+---------------------------------------------------------------------------
+This product bundles highlight.js which is available under a BSD 3 clauses license.  
+For details see http://opensource.org/licenses/BSD-3-Clause.
+
+---------------------------------------------------------------------------
 javax.activation (http://java.sun.com/javase/technologies/desktop/javabeans/jaf/index.jsp) javax.activation:activation:1.1.1
 ---------------------------------------------------------------------------
 This product bundles javax.activation which is available under a CDDL license.  
@@ -251,12 +269,53 @@ This product bundles JLine which is available under a BSD license.
 For details see http://www.opensource.org/licenses/bsd-license.php.
 
 ---------------------------------------------------------------------------
+JOpt Simple (http://pholser.github.io/jopt-simple/)
+---------------------------------------------------------------------------
+This product bundles JOpt Simple which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+jQuery (https://jquery.com)
+---------------------------------------------------------------------------
+This product bundles jQuery which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+jQuery BBQ (http://benalman.com/projects/jquery-bbq-plugin/)
+---------------------------------------------------------------------------
+This product bundles jQuery BBQ which is available under an MIT and GPL dual license.  
+For details see http://benalman.com/about/license.
+
+---------------------------------------------------------------------------
+jQuery-slideto (https://github.com/Sleavely/jQuery-slideto)
+---------------------------------------------------------------------------
+This product bundles jQuery-slideto which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+jQuery Wiggle (https://github.com/wilhelm-murdoch/jQuery-Wiggle)
+---------------------------------------------------------------------------
+This product bundles jQuery Wiggle which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+JSON (http://json.org)
+---------------------------------------------------------------------------
+This product bundles JOpt Simple which is available under a BSD-like license.  
+For details see http://json.org/license.html.
+
+---------------------------------------------------------------------------
 Paranamer (https://github.com/paul-hammant/paranamer) com.thoughtworks.paranamer:paranamer:2.3
 ---------------------------------------------------------------------------
 This product bundles Paranamer which is available under a BSD license.  
 For details see http://www.opensource.org/licenses/bsd-license.php.
 
 ---------------------------------------------------------------------------
+reset.css (http://meyerweb.com/eric/tools/css/reset/)
+---------------------------------------------------------------------------
+This product bundles reset.css which is available in the public domain.  
+
+---------------------------------------------------------------------------
 scala-reflect (http://www.scala-lang.org/) org.scala-lang:scala-reflect:2.10.0
 ---------------------------------------------------------------------------
 This product bundles scala-reflect which is available under a BSD-like license.  
@@ -269,7 +328,19 @@ This product bundles scala-library which is available under a BSD-like license.
 For details see http://www.scala-lang.org/downloads/license.html.
 
 ---------------------------------------------------------------------------
+shred (https://github.com/pandastrike/shred)
+---------------------------------------------------------------------------
+This product bundles JOpt Simple which is available under an MIT or ISC license.  
+For details see https://github.com/pandastrike/shred/blob/master/LICENSE.
+
+---------------------------------------------------------------------------
 SLF4J API (http://www.slf4j.org) org.slf4j:slf4j-api:jar:1.7.7
 ---------------------------------------------------------------------------
 This product bundles SLF4J API which is available under an MIT license.  
 For details see http://www.opensource.org/licenses/mit-license.php.
+
+---------------------------------------------------------------------------
+underscore (http://underscorejs.org)
+---------------------------------------------------------------------------
+This product bundles JOpt Simple which is available under an MIT license.  
+For details see http://www.opensource.org/licenses/mit-license.php.


[49/50] [abbrv] incubator-geode git commit: GEODE-610: Add NOTICEs from dependent projects

Posted by ab...@apache.org.
GEODE-610: Add NOTICEs from dependent projects

For the binary distribution, each Apache-licensed jar that is
distributed with a NOTICE file must be included in the NOTICE
file that we include with the distribution.  I manually extracted
the NOTICE files from the dependent jars and merged the text.


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

Branch: refs/heads/master
Commit: c26334542fa9e90daff5f58467d6e174b3e0f3e2
Parents: 2a2915a
Author: Anthony Baker <ab...@apache.org>
Authored: Mon Jan 25 11:33:19 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Wed Jan 27 10:56:53 2016 -0800

----------------------------------------------------------------------
 gemfire-assembly/src/main/dist/NOTICE | 462 +++++++++++++++++++++++++++++
 1 file changed, 462 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c2633454/gemfire-assembly/src/main/dist/NOTICE
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/NOTICE b/gemfire-assembly/src/main/dist/NOTICE
index 3f5dd07..b429715 100644
--- a/gemfire-assembly/src/main/dist/NOTICE
+++ b/gemfire-assembly/src/main/dist/NOTICE
@@ -3,3 +3,465 @@ Copyright 2016 The Apache Software Foundation.
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
+
+
+The following NOTICEs pertain to software distributed with this project.
+
+Apache Commons FileUpload
+Copyright 2002-2014 The Apache Software Foundation
+
+Apache Commons IO
+Copyright 2002-2012 The Apache Software Foundation
+
+Apache Commons Lang
+Copyright 2001-2010 The Apache Software Foundation
+
+// ------------------------------------------------------------------
+// NOTICE file corresponding to the section 4d of The Apache License,
+// Version 2.0, in this case for Commons Logging
+// ------------------------------------------------------------------
+
+Commons Logging
+Copyright 2001-2007 The Apache Software Foundation
+
+This product includes/uses software(s) developed by 'an unknown organization'
+  - Unnamed - avalon-framework:avalon-framework:jar:4.1.3
+  - Unnamed - log4j:log4j:jar:1.2.12
+  - Unnamed - logkit:logkit:jar:1.0.1
+
+HBase
+Copyright 2015 The Apache Software Foundation
+
+# Jackson JSON processor
+
+Jackson is a high-performance, Free/Open Source JSON processing library.
+It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+been in development since 2007.
+It is currently developed by a community of developers, as well as supported
+commercially by FasterXML.com.
+
+## Licensing
+
+Jackson core and extension components may licensed under different licenses.
+To find the details that apply to this artifact see the accompanying LICENSE file.
+For more information, including possible other licensing options, contact
+FasterXML.com (http://fasterxml.com).
+
+## Credits
+
+A list of contributors may be found from CREDITS file, which is included
+in some artifacts (usually source distributions); but is always available
+from the source code management (SCM) system project uses.
+
+Java ClassMate library was originally written by Tatu Saloranta (tatu.saloranta@iki.fi)
+
+Other developers who have contributed code are:
+
+* Brian Langel
+
+Apache Log4j API
+Copyright 1999-2015 Apache Software Foundation
+
+Apache Log4j Core
+Copyright 1999-2012 Apache Software Foundation
+
+ResolverUtil.java
+Copyright 2005-2006 Tim Fennell
+
+Apache Log4j Commons Logging Bridge
+Copyright 1999-2015 Apache Software Foundation
+
+Apache Log4j JUL Adapter
+Copyright 1999-2015 Apache Software Foundation
+
+Apache Log4j SLF4J Binding
+Copyright 1999-2015 Apache Software Foundation
+
+Apache Lucene
+Copyright 2014 The Apache Software Foundation
+
+Includes software from other Apache Software Foundation projects,
+including, but not limited to:
+ - Apache Ant
+ - Apache Jakarta Regexp
+ - Apache Commons
+ - Apache Xerces
+
+ICU4J, (under analysis/icu) is licensed under an MIT styles license
+and Copyright (c) 1995-2008 International Business Machines Corporation and others
+
+Some data files (under analysis/icu/src/data) are derived from Unicode data such
+as the Unicode Character Database. See http://unicode.org/copyright.html for more
+details.
+
+Brics Automaton (under core/src/java/org/apache/lucene/util/automaton) is 
+BSD-licensed, created by Anders Møller. See http://www.brics.dk/automaton/
+
+The levenshtein automata tables (under core/src/java/org/apache/lucene/util/automaton) were
+automatically generated with the moman/finenight FSA library, created by
+Jean-Philippe Barrette-LaPierre. This library is available under an MIT license,
+see http://sites.google.com/site/rrettesite/moman and 
+http://bitbucket.org/jpbarrette/moman/overview/
+
+The class org.apache.lucene.util.WeakIdentityMap was derived from
+the Apache CXF project and is Apache License 2.0.
+
+The Google Code Prettify is Apache License 2.0.
+See http://code.google.com/p/google-code-prettify/
+
+JUnit (junit-4.10) is licensed under the Common Public License v. 1.0
+See http://junit.sourceforge.net/cpl-v10.html
+
+This product includes code (JaspellTernarySearchTrie) from Java Spelling Checkin
+g Package (jaspell): http://jaspell.sourceforge.net/
+License: The BSD License (http://www.opensource.org/licenses/bsd-license.php)
+
+The snowball stemmers in
+  analysis/common/src/java/net/sf/snowball
+were developed by Martin Porter and Richard Boulton.
+The snowball stopword lists in
+  analysis/common/src/resources/org/apache/lucene/analysis/snowball
+were developed by Martin Porter and Richard Boulton.
+The full snowball package is available from
+  http://snowball.tartarus.org/
+
+The KStem stemmer in
+  analysis/common/src/org/apache/lucene/analysis/en
+was developed by Bob Krovetz and Sergio Guzman-Lara (CIIR-UMass Amherst)
+under the BSD-license.
+
+The Arabic,Persian,Romanian,Bulgarian, and Hindi analyzers (common) come with a default
+stopword list that is BSD-licensed created by Jacques Savoy.  These files reside in:
+analysis/common/src/resources/org/apache/lucene/analysis/ar/stopwords.txt,
+analysis/common/src/resources/org/apache/lucene/analysis/fa/stopwords.txt,
+analysis/common/src/resources/org/apache/lucene/analysis/ro/stopwords.txt,
+analysis/common/src/resources/org/apache/lucene/analysis/bg/stopwords.txt,
+analysis/common/src/resources/org/apache/lucene/analysis/hi/stopwords.txt
+See http://members.unine.ch/jacques.savoy/clef/index.html.
+
+The German,Spanish,Finnish,French,Hungarian,Italian,Portuguese,Russian and Swedish light stemmers
+(common) are based on BSD-licensed reference implementations created by Jacques Savoy and
+Ljiljana Dolamic. These files reside in:
+analysis/common/src/java/org/apache/lucene/analysis/de/GermanLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/de/GermanMinimalStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/es/SpanishLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/fi/FinnishLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchMinimalStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/hu/HungarianLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/it/ItalianLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/pt/PortugueseLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/ru/RussianLightStemmer.java
+analysis/common/src/java/org/apache/lucene/analysis/sv/SwedishLightStemmer.java
+
+The Stempel analyzer (stempel) includes BSD-licensed software developed 
+by the Egothor project http://egothor.sf.net/, created by Leo Galambos, Martin Kvapil,
+and Edmond Nolan.
+
+The Polish analyzer (stempel) comes with a default
+stopword list that is BSD-licensed created by the Carrot2 project. The file resides
+in stempel/src/resources/org/apache/lucene/analysis/pl/stopwords.txt.
+See http://project.carrot2.org/license.html.
+
+The SmartChineseAnalyzer source code (smartcn) was
+provided by Xiaoping Gao and copyright 2009 by www.imdict.net.
+
+WordBreakTestUnicode_*.java (under modules/analysis/common/src/test/) 
+is derived from Unicode data such as the Unicode Character Database. 
+See http://unicode.org/copyright.html for more details.
+
+The Morfologik analyzer (morfologik) includes BSD-licensed software
+developed by Dawid Weiss and Marcin Miłkowski (http://morfologik.blogspot.com/).
+
+Morfologik uses data from Polish ispell/myspell dictionary
+(http://www.sjp.pl/slownik/en/) licenced on the terms of (inter alia)
+LGPL and Creative Commons ShareAlike.
+
+Morfologic includes data from BSD-licensed dictionary of Polish (SGJP)
+(http://sgjp.pl/morfeusz/)
+
+Servlet-api.jar and javax.servlet-*.jar are under the CDDL license, the original
+source code for this can be found at http://www.eclipse.org/jetty/downloads.php
+
+===========================================================================
+Kuromoji Japanese Morphological Analyzer - Apache Lucene Integration
+===========================================================================
+
+This software includes a binary and/or source version of data from
+
+  mecab-ipadic-2.7.0-20070801
+
+which can be obtained from
+
+  http://atilika.com/releases/mecab-ipadic/mecab-ipadic-2.7.0-20070801.tar.gz
+
+or
+
+  http://jaist.dl.sourceforge.net/project/mecab/mecab-ipadic/2.7.0-20070801/mecab-ipadic-2.7.0-20070801.tar.gz
+
+===========================================================================
+mecab-ipadic-2.7.0-20070801 Notice
+===========================================================================
+
+Nara Institute of Science and Technology (NAIST),
+the copyright holders, disclaims all warranties with regard to this
+software, including all implied warranties of merchantability and
+fitness, in no event shall NAIST be liable for
+any special, indirect or consequential damages or any damages
+whatsoever resulting from loss of use, data or profits, whether in an
+action of contract, negligence or other tortuous action, arising out
+of or in connection with the use or performance of this software.
+
+A large portion of the dictionary entries
+originate from ICOT Free Software.  The following conditions for ICOT
+Free Software applies to the current dictionary as well.
+
+Each User may also freely distribute the Program, whether in its
+original form or modified, to any third party or parties, PROVIDED
+that the provisions of Section 3 ("NO WARRANTY") will ALWAYS appear
+on, or be attached to, the Program, which is distributed substantially
+in the same form as set out herein and that such intended
+distribution, if actually made, will neither violate or otherwise
+contravene any of the laws and regulations of the countries having
+jurisdiction over the User or the intended distribution itself.
+
+NO WARRANTY
+
+The program was produced on an experimental basis in the course of the
+research and development conducted during the project and is provided
+to users as so produced on an experimental basis.  Accordingly, the
+program is provided without any warranty whatsoever, whether express,
+implied, statutory or otherwise.  The term "warranty" used herein
+includes, but is not limited to, any warranty of the quality,
+performance, merchantability and fitness for a particular purpose of
+the program and the nonexistence of any infringement or violation of
+any right of any third party.
+
+Each user of the program will agree and understand, and be deemed to
+have agreed and understood, that there is no warranty whatsoever for
+the program and, accordingly, the entire risk arising from or
+otherwise connected with the program is assumed by the user.
+
+Therefore, neither ICOT, the copyright holder, or any other
+organization that participated in or was otherwise related to the
+development of the program and their respective officials, directors,
+officers and other employees shall be held liable for any and all
+damages, including, without limitation, general, special, incidental
+and consequential damages, arising out of or otherwise in connection
+with the use or inability to use the program or any product, material
+or result produced or otherwise obtained by using the program,
+regardless of whether they have been advised of, or otherwise had
+knowledge of, the possibility of such damages at any time during the
+project or thereafter.  Each user will be deemed to have agreed to the
+foregoing by his or her commencement of use of the program.  The term
+"use" as used herein includes, but is not limited to, the use,
+modification, copying and distribution of the program and the
+production of secondary products from the program.
+
+In the case where the program, whether in its original form or
+modified, was distributed or delivered to or received by a user from
+any person, organization or entity other than ICOT, unless it makes or
+grants independently of ICOT any specific warranty to the user in
+writing, such person, organization or entity, will also be exempted
+from and not be held liable to the user for any such damages as noted
+above as far as the program is concerned.
+
+
+                            The Netty Project
+                            =================
+
+Please visit the Netty web site for more information:
+
+  * http://netty.io/
+
+Copyright 2011 The Netty Project
+
+The Netty Project 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.
+
+Also, please refer to each LICENSE.<component>.txt file, which is located in
+the 'license' directory of the distribution file, for the license terms of the
+components that this product depends on.
+
+-------------------------------------------------------------------------------
+This product contains the extensions to Java Collections Framework which has
+been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+  * LICENSE:
+    * license/LICENSE.jsr166y.txt (Public Domain)
+  * HOMEPAGE:
+    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+This product contains a modified version of Robert Harder's Public Domain
+Base64 Encoder and Decoder, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.base64.txt (Public Domain)
+  * HOMEPAGE:
+    * http://iharder.sourceforge.net/current/java/base64/
+
+This product contains a modified portion of 'Webbit', an event based  
+WebSocket and HTTP server, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.webbit.txt (BSD License)
+  * HOMEPAGE:
+    * https://github.com/joewalnes/webbit
+
+This product contains a modified portion of 'SLF4J', a simple logging
+facade for Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.slf4j.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.slf4j.org/
+
+
+This product contains a modified version of Roland Kuhn's ASL2
+AbstractNodeQueue, which is based on Dmitriy Vyukov's non-intrusive MPSC queue.
+It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.abstractnodequeue.txt (Public Domain)
+  * HOMEPAGE:
+    * https://github.com/akka/akka/blob/wip-2.2.3-for-scala-2.11/akka-actor/src/main/java/akka/dispatch/AbstractNodeQueue.java
+
+This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+ which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jctools.txt (ASL2 License)
+  * HOMEPAGE:
+    * https://github.com/JCTools/JCTools
+
+
+This product optionally depends on 'JZlib', a re-implementation of zlib in
+pure Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jzlib.txt (BSD style License)
+  * HOMEPAGE:
+    * http://www.jcraft.com/jzlib/
+
+This product optionally depends on 'Protocol Buffers', Google's data
+interchange format, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.protobuf.txt (New BSD License)
+  * HOMEPAGE:
+    * http://code.google.com/p/protobuf/
+
+This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+a temporary self-signed X.509 certificate when the JVM does not provide the
+equivalent functionality.  It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.bouncycastle.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.bouncycastle.org/
+
+This product optionally depends on 'Snappy', a compression library produced
+by Google Inc, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.snappy.txt (New BSD License)
+  * HOMEPAGE:
+    * http://code.google.com/p/snappy/
+
+This product optionally depends on 'JBoss Marshalling', an alternative Java
+serialization API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
+  * HOMEPAGE:
+    * http://www.jboss.org/jbossmarshalling
+
+This product optionally depends on 'Caliper', Google's micro-
+benchmarking framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.caliper.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://code.google.com/p/caliper/
+
+This product optionally depends on 'Apache Commons Logging', a logging
+framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-logging.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://commons.apache.org/logging/
+
+This product optionally depends on 'Apache Log4J', a logging framework, which
+can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.log4j.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://logging.apache.org/log4j/
+
+Spring Framework 3.2.12.RELEASE
+Copyright (c) 2002-2014 Pivotal, Inc.
+
+This product is licensed to you under the Apache License, Version 2.0
+(the "License"). You may not use this product except in compliance with
+the License.
+
+This product may include a number of subcomponents with separate
+copyright notices and license terms. Your use of the source code for
+these subcomponents is subject to the terms and conditions of the
+subcomponent's license, as noted in the license.txt file.
+
+   ======================================================================
+   == NOTICE file corresponding to section 4 d of the Apache License,  ==
+   == Version 2.0, for the Spring Framework distribution.              ==
+   ======================================================================
+
+   This product includes software developed by
+   the Apache Software Foundation (http://www.apache.org).
+
+   The end-user documentation included with a redistribution, if any,
+   must include the following acknowledgement:
+
+     "This product includes software developed by the Spring Framework
+      Project (http://www.springframework.org)."
+
+   Alternately, this acknowledgement may appear in the software itself,
+   if and wherever such third-party acknowledgements normally appear.
+
+   The names "Spring", "Spring Framework", and "Spring GemFire" must
+   not be used to endorse or promote products derived from this
+   software without prior written permission. For written permission,
+   please contact enquiries@springsource.com.
+
+   ======================================================================
+   == NOTICE file corresponding to section 4 d of the Apache License,  ==
+   == Version 2.0, for the Spring Framework distribution.              ==
+   ======================================================================
+
+   This product includes software developed by
+   the Apache Software Foundation (http://www.apache.org).
+
+   The end-user documentation included with a redistribution, if any,
+   must include the following acknowledgement:
+
+     "This product includes software developed by the Spring Framework
+      Project (http://www.springframework.org)."
+
+   Alternately, this acknowledgement may appear in the software itself,
+   if and wherever such third-party acknowledgements normally appear.
+
+   The names "Spring", "Spring Framework", and "Spring Shell" must
+   not be used to endorse or promote products derived from this
+   software without prior written permission. For written permission,
+   please contact enquiries@springsource.com.


[05/50] [abbrv] incubator-geode git commit: GEODE-714: Modify all tests to use JUnit Categories

Posted by ab...@apache.org.
GEODE-714: Modify all tests to use JUnit Categories

* Add DistributedTest @Category to DistributedTestCase

* Rename disabled tests and use @Ignore instead.

* Add PerformanceTest @Category to performance tests.

* Disable performance tests that perform no assertions.

* Modify build to check all tests for categories.

* Modify build to use **/*Test.class pattern for all testing tasks.


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

Branch: refs/heads/master
Commit: ca6148aa99faf8fcb278fd7a840f69f622e6fc3f
Parents: bba7656
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Dec 28 13:44:38 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Mon Jan 4 08:43:08 2016 -0800

----------------------------------------------------------------------
 build.gradle                                    |   21 +-
 .../test/java/com/gemstone/gemfire/BadTest.java |   42 -
 .../cache30/Bug40255JUnitDisabledTest.java      |  139 ---
 .../gemfire/cache30/Bug40255JUnitTest.java      |  143 +++
 .../cache30/Bug40662JUnitDisabledTest.java      |   92 --
 .../gemfire/cache30/Bug40662JUnitTest.java      |   93 ++
 .../locks/CollaborationJUnitDisabledTest.java   |  562 ----------
 .../internal/locks/CollaborationJUnitTest.java  |  617 +++++++++++
 .../DiskRegionPerfJUnitPerformanceTest.java     |    6 +-
 ...HARegionQueueStartStopJUnitDisabledTest.java |  123 ---
 .../cache/tier/sockets/HAInterestBaseTest.java  | 1015 -----------------
 .../tier/sockets/HAInterestPart1DUnitTest.java  |  102 +-
 .../tier/sockets/HAInterestPart2DUnitTest.java  |  116 +-
 .../cache/tier/sockets/HAInterestTestCase.java  | 1018 ++++++++++++++++++
 .../tier/sockets/command/CommitCommandTest.java |    6 +
 .../logging/LogWriterPerformanceTest.java       |    6 +
 .../log4j/Log4J2DisabledPerformanceTest.java    |    6 +
 .../logging/log4j/Log4J2PerformanceTest.java    |    9 +-
 .../LogWriterLoggerDisabledPerformanceTest.java |    6 +
 .../log4j/LogWriterLoggerPerformanceTest.java   |    9 +-
 .../test/java/dunit/DistributedTestCase.java    |    3 +
 ...IndexRepositoryImplJUnitPerformanceTest.java |  437 --------
 .../IndexRepositoryImplPerformanceTest.java     |  439 ++++++++
 23 files changed, 2473 insertions(+), 2537 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index f464dc3..4563590 100755
--- a/build.gradle
+++ b/build.gradle
@@ -345,7 +345,7 @@ subprojects {
   }
 
   test {
-    include '**/*JUnitTest.class'
+    include '**/*Test.class'
     useJUnit {
       includeCategories 'com.gemstone.gemfire.test.junit.categories.UnitTest'
       excludeCategories 'com.gemstone.gemfire.test.junit.categories.IntegrationTest'
@@ -364,10 +364,12 @@ subprojects {
   //This target does not run any tests. Rather, it validates that there are no
   //tests that are missing a category annotation
   task checkMissedTests(type: Test) {
-    include '**/*JUnitTest.class'
+    include '**/*Test.class'
     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'
+      excludeCategories 'com.gemstone.gemfire.test.junit.categories.PerformanceTest'
     }    
 
     beforeTest { descriptor ->
@@ -377,7 +379,7 @@ subprojects {
   }
 
   task integrationTest(type:Test) {
-    include '**/*JUnitTest.class'
+    include '**/*Test.class'
     useJUnit {
       excludeCategories 'com.gemstone.gemfire.test.junit.categories.UnitTest'
       includeCategories 'com.gemstone.gemfire.test.junit.categories.IntegrationTest'
@@ -392,17 +394,16 @@ subprojects {
   }
   
   task distributedTest(type:Test) {
-    include '**/*DUnitTest.class'
+    include '**/*Test.class'
     
     // 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'
+      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/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/BadTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/BadTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/BadTest.java
deleted file mode 100644
index 46dc799..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/BadTest.java
+++ /dev/null
@@ -1,42 +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;
-
-import junit.framework.*;
-
-/**
- * This test provides examples of a test failing and a test getting an
- * error.  We use it to test JUnit failure reporting.
- */
-public class BadTest extends TestCase {
-
-  public BadTest(String name) {
-    super(name);
-  }
-
-  ////////  Test Methods
-
-  public void testFailure() {
-    fail("I'm failing away...");
-  }
-
-  public void testError() {
-    String s = "I've failed";
-    throw new RuntimeException(s);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitDisabledTest.java
deleted file mode 100644
index 4fe8a49..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitDisabledTest.java
+++ /dev/null
@@ -1,139 +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.cache30;
-
-import java.io.File;
-import java.util.Properties;
-
-import junit.framework.TestCase;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-
-/**
- * @author Shobhit Agarwal
- *
- */
-public class Bug40255JUnitDisabledTest extends TestCase{
-
-  private static final String BUG_40255_XML = Bug40255JUnitDisabledTest.class.getResource("bug40255xmlparameterization.xml").getFile();
-  private static final String BUG_40255_PROPS = Bug40255JUnitDisabledTest.class.getResource("bug40255_gemfire.properties").getFile();
-
-  private static final String ATTR_PROPERTY_STRING = "region.disk.store";
-
-  private static final String ATTR_PROPERTY_VALUE = "teststore";
-
-  private static final String NESTED_ATTR_PROPERTY_STRING = "custom-nested.test";
-
-  private static final String NESTED_ATTR_PROPERTY_VALUE = "disk";
-
-  private static final String ELEMENT_PROPERTY_STRING = "custom-string.element";
-
-  private static final String ELEMENT_PROPERTY_VALUE = "example-string";
-
-  private static final String CONCAT_ELEMENT_PROPERTY_STRING = "concat.test";
-
-  private static final String CONCAT_ELEMENT_PROPERTY_VALUE = "-name";
-
-  private static final String ELEMENT_KEY_VALUE = "example-value";
-
-  DistributedSystem ds;
-  Cache cache;
-
-  @Override
-  public void setName(String name) {
-    super.setName(name);
-  }
-
-  public void testResolveReplacePropertyStringForLonerCache(){
-    Properties props = new Properties();
-    props.setProperty("mcast-port", "0");
-    props.setProperty("locators", "");
-    System.setProperty("gemfirePropertyFile", BUG_40255_PROPS);
-    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40255_XML);
-    System.setProperty(NESTED_ATTR_PROPERTY_STRING, NESTED_ATTR_PROPERTY_VALUE);
-    System.setProperty(ATTR_PROPERTY_STRING, ATTR_PROPERTY_VALUE);
-    System.setProperty(ELEMENT_PROPERTY_STRING, ELEMENT_PROPERTY_VALUE);
-    System.setProperty(CONCAT_ELEMENT_PROPERTY_STRING, CONCAT_ELEMENT_PROPERTY_VALUE);
-    
-    // create the directory where data is going to be stored
-    File dir = new File("persistData1");
-    dir.mkdir();
-
-    this.ds = DistributedSystem.connect(props);
-    this.cache = CacheFactory.create(this.ds);
-
-    Region exampleRegion = this.cache.getRegion("example-region");
-    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
-
-    //Check if disk store got same name as passed in system properties in setup().
-    assertEquals(attrs.getDiskStoreName(), System.getProperty(ATTR_PROPERTY_STRING));
-    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE));
-    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE), ELEMENT_KEY_VALUE);
-    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE));
-    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE), CONCAT_ELEMENT_PROPERTY_VALUE);
-  }
-
-  public void testResolveReplacePropertyStringForNonLonerCache(){
-    Properties props = new Properties();
-    props.setProperty("mcast-port", "10333");
-    props.setProperty("locators", "");
-    System.setProperty("gemfirePropertyFile", BUG_40255_PROPS);
-    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40255_XML);
-    System.setProperty(NESTED_ATTR_PROPERTY_STRING, NESTED_ATTR_PROPERTY_VALUE);
-    System.setProperty(ATTR_PROPERTY_STRING, ATTR_PROPERTY_VALUE);
-    System.setProperty(ELEMENT_PROPERTY_STRING, ELEMENT_PROPERTY_VALUE);
-    System.setProperty(CONCAT_ELEMENT_PROPERTY_STRING, CONCAT_ELEMENT_PROPERTY_VALUE);
-    
-    // create the directory where data is going to be stored
-    File dir = new File("persistData1");
-    dir.mkdir();
-
-    this.ds = DistributedSystem.connect(props);
-    this.cache = CacheFactory.create(this.ds);
-
-    Region exampleRegion = this.cache.getRegion("example-region");
-    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
-
-    //Check if disk store got same name as passed in system properties in setup().
-    assertEquals(attrs.getDiskStoreName(), System.getProperty(ATTR_PROPERTY_STRING));
-    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE));
-    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE), ELEMENT_KEY_VALUE);
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
-    if (this.cache != null) {
-      this.cache.close();
-      this.cache = null;
-    }
-    if (this.ds != null) {
-      this.ds.disconnect();
-      this.ds = null;
-    }
-  }
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitTest.java
new file mode 100644
index 0000000..92bfbe7
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40255JUnitTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.cache30;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * @author Shobhit Agarwal
+ *
+ */
+@Category(IntegrationTest.class)
+@Ignore("Test is broken and was named Bug40255JUnitDisabledTest")
+public class Bug40255JUnitTest {
+
+  private static final String BUG_40255_XML = Bug40255JUnitTest.class.getResource("bug40255xmlparameterization.xml").getFile();
+  private static final String BUG_40255_PROPS = Bug40255JUnitTest.class.getResource("bug40255_gemfire.properties").getFile();
+
+  private static final String ATTR_PROPERTY_STRING = "region.disk.store";
+
+  private static final String ATTR_PROPERTY_VALUE = "teststore";
+
+  private static final String NESTED_ATTR_PROPERTY_STRING = "custom-nested.test";
+
+  private static final String NESTED_ATTR_PROPERTY_VALUE = "disk";
+
+  private static final String ELEMENT_PROPERTY_STRING = "custom-string.element";
+
+  private static final String ELEMENT_PROPERTY_VALUE = "example-string";
+
+  private static final String CONCAT_ELEMENT_PROPERTY_STRING = "concat.test";
+
+  private static final String CONCAT_ELEMENT_PROPERTY_VALUE = "-name";
+
+  private static final String ELEMENT_KEY_VALUE = "example-value";
+
+  DistributedSystem ds;
+  Cache cache;
+
+  @Test
+  public void testResolveReplacePropertyStringForLonerCache(){
+    Properties props = new Properties();
+    props.setProperty("mcast-port", "0");
+    props.setProperty("locators", "");
+    System.setProperty("gemfirePropertyFile", BUG_40255_PROPS);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40255_XML);
+    System.setProperty(NESTED_ATTR_PROPERTY_STRING, NESTED_ATTR_PROPERTY_VALUE);
+    System.setProperty(ATTR_PROPERTY_STRING, ATTR_PROPERTY_VALUE);
+    System.setProperty(ELEMENT_PROPERTY_STRING, ELEMENT_PROPERTY_VALUE);
+    System.setProperty(CONCAT_ELEMENT_PROPERTY_STRING, CONCAT_ELEMENT_PROPERTY_VALUE);
+    
+    // create the directory where data is going to be stored
+    File dir = new File("persistData1");
+    dir.mkdir();
+
+    this.ds = DistributedSystem.connect(props);
+    this.cache = CacheFactory.create(this.ds);
+
+    Region exampleRegion = this.cache.getRegion("example-region");
+    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
+
+    //Check if disk store got same name as passed in system properties in setup().
+    assertEquals(attrs.getDiskStoreName(), System.getProperty(ATTR_PROPERTY_STRING));
+    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE));
+    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE), ELEMENT_KEY_VALUE);
+    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE));
+    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE), CONCAT_ELEMENT_PROPERTY_VALUE);
+  }
+
+  @Test
+  public void testResolveReplacePropertyStringForNonLonerCache(){
+    Properties props = new Properties();
+    props.setProperty("mcast-port", "10333");
+    props.setProperty("locators", "");
+    System.setProperty("gemfirePropertyFile", BUG_40255_PROPS);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40255_XML);
+    System.setProperty(NESTED_ATTR_PROPERTY_STRING, NESTED_ATTR_PROPERTY_VALUE);
+    System.setProperty(ATTR_PROPERTY_STRING, ATTR_PROPERTY_VALUE);
+    System.setProperty(ELEMENT_PROPERTY_STRING, ELEMENT_PROPERTY_VALUE);
+    System.setProperty(CONCAT_ELEMENT_PROPERTY_STRING, CONCAT_ELEMENT_PROPERTY_VALUE);
+    
+    // create the directory where data is going to be stored
+    File dir = new File("persistData1");
+    dir.mkdir();
+
+    this.ds = DistributedSystem.connect(props);
+    this.cache = CacheFactory.create(this.ds);
+
+    Region exampleRegion = this.cache.getRegion("example-region");
+    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
+
+    //Check if disk store got same name as passed in system properties in setup().
+    assertEquals(attrs.getDiskStoreName(), System.getProperty(ATTR_PROPERTY_STRING));
+    assertNotNull(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE));
+    assertEquals(exampleRegion.get(ELEMENT_PROPERTY_VALUE+CONCAT_ELEMENT_PROPERTY_VALUE), ELEMENT_KEY_VALUE);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (this.cache != null) {
+      this.cache.close();
+      this.cache = null;
+    }
+    if (this.ds != null) {
+      this.ds.disconnect();
+      this.ds = null;
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitDisabledTest.java
deleted file mode 100644
index 9455641..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitDisabledTest.java
+++ /dev/null
@@ -1,92 +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.cache30;
-
-import java.util.Properties;
-
-import junit.framework.TestCase;
-
-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.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-
-/**
- * Test for Bug no. 40662. To verify the default action being set in eviction
- * attributes by CacheXmlParser when cache.xml has eviction attributes with no
- * eviction action specified. which was being set to EvictionAction.NONE
- * 
- * @author shoagarwal
- * @since 6.6
- */
-public class Bug40662JUnitDisabledTest extends TestCase {
-
-  private static final String BUG_40662_XML = Bug40662JUnitDisabledTest.class.getResource("bug40662noevictionaction.xml").getFile();
-
-  DistributedSystem ds;
-  Cache cache;
-
-  @Override
-  public void setName(String name) {
-    super.setName(name);
-  }
-
-  /**
-   * Test for checking eviction action in eviction attributes if no evicition
-   * action is specified in cache.xml
-   */
-  public void testEvictionActionSetLocalDestroyPass() {
-    Region exampleRegion = this.cache.getRegion("example-region");
-    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
-    EvictionAttributes evicAttrs = attrs.getEvictionAttributes();
-
-    //Default eviction action is LOCAL_DESTROY always. 
-    assertEquals(EvictionAction.LOCAL_DESTROY, evicAttrs.getAction());
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
-    if (this.cache != null) {
-      this.cache.close();
-      this.cache = null;
-    }
-    if (this.ds != null) {
-      this.ds.disconnect();
-      this.ds = null;
-    }
-  }
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    Properties props = new Properties();
-    props.setProperty("mcast-port", "0");
-    props.setProperty("locators", "");
-    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40662_XML);
-    this.ds = DistributedSystem.connect(props);
-    this.cache = CacheFactory.create(this.ds);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitTest.java
new file mode 100644
index 0000000..0d62127
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug40662JUnitTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.cache30;
+
+import static org.junit.Assert.*;
+
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
+
+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.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Test for Bug no. 40662. To verify the default action being set in eviction
+ * attributes by CacheXmlParser when cache.xml has eviction attributes with no
+ * eviction action specified. which was being set to EvictionAction.NONE
+ * 
+ * @author shoagarwal
+ * @since 6.6
+ */
+@Category(IntegrationTest.class)
+@Ignore("Test is broken and was named Bug40662JUnitDisabledTest")
+public class Bug40662JUnitTest {
+
+  private static final String BUG_40662_XML = Bug40662JUnitTest.class.getResource("bug40662noevictionaction.xml").getFile();
+
+  DistributedSystem ds;
+  Cache cache;
+
+  /**
+   * Test for checking eviction action in eviction attributes if no evicition
+   * action is specified in cache.xml
+   */
+  public void testEvictionActionSetLocalDestroyPass() {
+    Region exampleRegion = this.cache.getRegion("example-region");
+    RegionAttributes<Object, Object> attrs = exampleRegion.getAttributes();
+    EvictionAttributes evicAttrs = attrs.getEvictionAttributes();
+
+    //Default eviction action is LOCAL_DESTROY always. 
+    assertEquals(EvictionAction.LOCAL_DESTROY, evicAttrs.getAction());
+  }
+
+  @After
+  protected void tearDown() throws Exception {
+    if (this.cache != null) {
+      this.cache.close();
+      this.cache = null;
+    }
+    if (this.ds != null) {
+      this.ds.disconnect();
+      this.ds = null;
+    }
+  }
+
+  @Before
+  protected void setUp() throws Exception {
+    Properties props = new Properties();
+    props.setProperty("mcast-port", "0");
+    props.setProperty("locators", "");
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, BUG_40662_XML);
+    this.ds = DistributedSystem.connect(props);
+    this.cache = CacheFactory.create(this.ds);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitDisabledTest.java
deleted file mode 100755
index 7b93a36..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitDisabledTest.java
+++ /dev/null
@@ -1,562 +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.locks;
-
-import junit.framework.*;
-import java.util.*;
-
-import com.gemstone.gemfire.CancelCriterion;
-import com.gemstone.gemfire.LogWriter;
-import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.internal.logging.LocalLogWriter;
-import com.gemstone.gemfire.internal.logging.InternalLogWriter;
-
-import dunit.DistributedTestCase;
-import dunit.DistributedTestCase.WaitCriterion;
-
-/**
- * Tests the Collaboration Lock used internally by dlock service.
- *
- * @author Kirk Lund
- * @since 4.1.1
- */
-public class CollaborationJUnitDisabledTest extends TestCase {
-
-  protected LogWriter log = new LocalLogWriter(InternalLogWriter.INFO_LEVEL);
-  protected Collaboration collaboration;
-  
-  public CollaborationJUnitDisabledTest(String name) {
-    super(name);
-  }
-  
-  public void setUp() throws Exception {
-    this.collaboration = new Collaboration(new CancelCriterion() {
-
-      public String cancelInProgress() {
-        // TODO Auto-generated method stub
-        return null;
-      }
-
-      public RuntimeException generateCancelledException(Throwable e) {
-        // TODO Auto-generated method stub
-        return null;
-      }
-      
-    });
-  }
-  
-  public void tearDown() throws Exception {
-    this.collaboration = null;
-  }
-  
-  protected volatile boolean flagTestBlocksUntilRelease = false;
-  protected volatile boolean threadBStartedTestBlocksUntilRelease = false;
-  public void testBlocksUntilRelease() throws Exception {
-    this.log.info("[testBlocksUntilRelease]");
-    Thread threadA = new Thread(group, new Runnable() {
-      public void run() {
-        collaboration.acquireUninterruptibly("topicA");
-        try {
-          flagTestBlocksUntilRelease = true;
-          while(flagTestBlocksUntilRelease) {
-            try {
-              Thread.sleep(10);
-            }
-            catch (InterruptedException ignore) {fail("interrupted");}
-          }
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    
-    // thread one acquires
-    threadA.start();
-    WaitCriterion ev = new WaitCriterion() {
-      public boolean done() {
-        return CollaborationJUnitDisabledTest.this.flagTestBlocksUntilRelease;
-      }
-      public String description() {
-        return "waiting for thread";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
-    assertTrue(this.collaboration.hasCurrentTopic(threadA));
-    
-    // thread two blocks until one releeases
-    Thread threadB = new Thread(group, new Runnable() {
-      public void run() {
-        threadBStartedTestBlocksUntilRelease = true;
-        collaboration.acquireUninterruptibly("topicB");
-        try {
-          flagTestBlocksUntilRelease = true;
-          WaitCriterion ev2 = new WaitCriterion() {
-            public boolean done() {
-              return !flagTestBlocksUntilRelease;
-            }
-            public String description() {
-              return "waiting for release";
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev2, 20 * 1000, 200, true);
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    
-    // start up threadB
-    threadB.start();
-    ev = new WaitCriterion() {
-      public boolean done() {
-        return threadBStartedTestBlocksUntilRelease;
-      }
-      public String description() {
-        return "waiting for thread b";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
-    
-    // threadA holds topic and threadB is waiting...
-    assertTrue(this.collaboration.hasCurrentTopic(threadA));
-    assertFalse(this.collaboration.hasCurrentTopic(threadB));
-
-    // let threadA release so that threadB gets lock
-    this.flagTestBlocksUntilRelease = false;
-    DistributedTestCase.join(threadA, 30 * 1000, null);
-    
-    // make sure threadB is doing what it's supposed to do...
-    ev = new WaitCriterion() {
-      public boolean done() {
-        return flagTestBlocksUntilRelease;
-      }
-      public String description() {
-        return "threadB";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
-    // threadB must have lock now... let threadB release
-    assertTrue(this.collaboration.hasCurrentTopic(threadB));
-    this.flagTestBlocksUntilRelease = false;
-    DistributedTestCase.join(threadB, 30 * 1000, null);
-
-    // collaboration should be free now    
-    assertFalse(this.collaboration.hasCurrentTopic(threadA));
-    assertFalse(this.collaboration.hasCurrentTopic(threadB));
-    assertFalse(this.collaboration.hasCurrentTopic());
-  }
-  
-  protected volatile boolean threadAFlag_TestLateComerJoinsIn = false;
-  protected volatile boolean threadBFlag_TestLateComerJoinsIn = false;
-  protected volatile boolean threadCFlag_TestLateComerJoinsIn = true;
-  protected volatile boolean threadDFlag_TestLateComerJoinsIn = false;
-  public void testLateComerJoinsIn() throws Exception {
-    this.log.info("[testLateComerJoinsIn]");
-    
-    final Object topicA = "topicA";
-    final Object topicB = "topicB";
-    
-    // threads one and two acquire
-    Thread threadA = new Thread(group, new Runnable() {
-      public void run() {
-        collaboration.acquireUninterruptibly(topicA);
-        try {
-          threadAFlag_TestLateComerJoinsIn = true;
-          WaitCriterion ev = new WaitCriterion() {
-            public boolean done() {
-              return !threadAFlag_TestLateComerJoinsIn;
-            }
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    threadA.start();
-    WaitCriterion ev = new WaitCriterion() {
-      public boolean done() {
-        return threadAFlag_TestLateComerJoinsIn;
-      }
-      public String description() {
-        return "wait for ThreadA";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 30 * 1000, 200, true);
-    assertTrue(this.collaboration.hasCurrentTopic(threadA));
-    assertTrue(this.collaboration.isCurrentTopic(topicA));
-    
-    Thread threadB = new Thread(group, new Runnable() {
-      public void run() {
-        collaboration.acquireUninterruptibly(topicA);
-        try {
-          threadBFlag_TestLateComerJoinsIn = true;
-          WaitCriterion ev2 = new WaitCriterion() {
-            public boolean done() {
-              return !threadBFlag_TestLateComerJoinsIn;
-            }
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev2, 60 * 1000, 200, true);
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    threadB.start();
-    ev = new WaitCriterion() {
-      public boolean done() {
-        return threadBFlag_TestLateComerJoinsIn;
-      }
-      public String description() {
-        return "";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-    assertTrue(this.collaboration.hasCurrentTopic(threadB));
-    
-    // thread three blocks for new topic
-    Thread threadC = new Thread(group, new Runnable() {
-      public void run() {
-        threadCFlag_TestLateComerJoinsIn = false;
-        collaboration.acquireUninterruptibly(topicB);
-        try {
-          threadCFlag_TestLateComerJoinsIn = true;
-          WaitCriterion ev2 = new WaitCriterion() {
-            public boolean done() {
-              return !threadCFlag_TestLateComerJoinsIn;
-            }
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev2, 60 * 1000, 200, true);
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    threadC.start();
-    ev = new WaitCriterion() {
-      public boolean done() {
-        return threadCFlag_TestLateComerJoinsIn;
-      }
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-    assertFalse(this.collaboration.hasCurrentTopic(threadC));
-    assertFalse(this.collaboration.isCurrentTopic(topicB));
-    
-    // thread four (lateComer) acquires current topic immediately
-    Thread threadD = new Thread(group, new Runnable() {
-      public void run() {
-        collaboration.acquireUninterruptibly(topicA);
-        try {
-          threadDFlag_TestLateComerJoinsIn = true;
-          while(threadDFlag_TestLateComerJoinsIn) {
-            try {
-              Thread.sleep(10);
-            }
-            catch (InterruptedException ignore) {fail("interrupted");}
-          }
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    threadD.start();
-    ev = new WaitCriterion() {
-      public boolean done() {
-        return threadDFlag_TestLateComerJoinsIn;
-      }
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-    assertTrue(this.collaboration.hasCurrentTopic(threadD));
-    
-    // release threadA
-    this.threadAFlag_TestLateComerJoinsIn = false;
-    DistributedTestCase.join(threadA, 30 * 1000, null);
-    assertFalse(this.collaboration.hasCurrentTopic(threadA));
-    assertTrue(this.collaboration.hasCurrentTopic(threadB));
-    assertFalse(this.collaboration.hasCurrentTopic(threadC));
-    assertTrue(this.collaboration.hasCurrentTopic(threadD));
-    assertTrue(this.collaboration.isCurrentTopic(topicA));
-    assertFalse(this.collaboration.isCurrentTopic(topicB));
-    
-    // release threadB
-    this.threadBFlag_TestLateComerJoinsIn = false;
-    DistributedTestCase.join(threadB, 30 * 1000, null);
-    assertFalse(this.collaboration.hasCurrentTopic(threadB));
-    assertFalse(this.collaboration.hasCurrentTopic(threadC));
-    assertTrue(this.collaboration.hasCurrentTopic(threadD));
-    assertTrue(this.collaboration.isCurrentTopic(topicA));
-    assertFalse(this.collaboration.isCurrentTopic(topicB));
-    
-    // release threadD
-    this.threadDFlag_TestLateComerJoinsIn = false;
-    DistributedTestCase.join(threadD, 30 * 1000, null);
-    ev = new WaitCriterion() {
-      public boolean done() {
-        return threadCFlag_TestLateComerJoinsIn;
-      }
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-    assertTrue(this.collaboration.hasCurrentTopic(threadC));
-    assertFalse(this.collaboration.hasCurrentTopic(threadD));
-    assertFalse(this.collaboration.isCurrentTopic(topicA));
-    assertTrue(this.collaboration.isCurrentTopic(topicB));
-    
-    // release threadC
-    this.threadCFlag_TestLateComerJoinsIn = false;
-    DistributedTestCase.join(threadC, 30 * 1000, null);
-    assertFalse(this.collaboration.hasCurrentTopic(threadC));
-    assertFalse(this.collaboration.isCurrentTopic(topicA));
-    assertFalse(this.collaboration.isCurrentTopic(topicB));
-  }
-  
-  protected List waitingList = Collections.synchronizedList(new ArrayList());
-  protected List fairnessList = Collections.synchronizedList(new ArrayList());
-  protected volatile boolean runTestFairnessStressfully = true;
-  public void testFairnessStressfully() throws Exception {
-    this.log.info("[testFairnessStressfully]");
-    final int numThreads = 20;
-    Thread threads[] = new Thread[numThreads];
-    
-    Runnable run = new Runnable() {
-      public void run() {
-        boolean released = false;
-        try {
-          String uniqueTopic = Thread.currentThread().getName();
-          while(runTestFairnessStressfully) {
-            waitingList.add(uniqueTopic);
-            collaboration.acquireUninterruptibly(uniqueTopic);
-            try {
-              released = false;
-              fairnessList.add(uniqueTopic);
-              waitingList.remove(uniqueTopic);
-            }
-            finally {
-              // wait for the other threads to line up...
-              WaitCriterion ev = new WaitCriterion() {
-                public boolean done() {
-                  return !runTestFairnessStressfully || waitingList.size() >= numThreads - 1;
-                }
-                public String description() {
-                  return "other threads lining up";
-                }
-              };
-              DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-              collaboration.release();
-              released = true;
-            }
-          }
-        }
-        finally {
-          if (!released) {
-            collaboration.release();
-          }
-        }
-      }
-    };
-    
-    try {
-      // many threads loop: acquire and release with unique topic
-      for (int t = 0; t < threads.length; t++) {
-        threads[t] = new Thread(group, run, String.valueOf(t));
-        threads[t].start();
-      }
-
-      log.info("Started all threads... waiting for test to complete.");
-            
-      // wait for numThreads * 10
-      WaitCriterion ev = new WaitCriterion() {
-        public boolean done() {
-          return fairnessList.size() >= numThreads * 20;
-        }
-        public String description() {
-          return "waiting for numThreads * 10";
-        }
-      };
-      DistributedTestCase.waitForCriterion(ev, 5 * 60 * 1000, 200, true);
-    }
-    finally {
-      if (this.runTestFairnessStressfully) {
-        this.runTestFairnessStressfully = false;
-      }
-    }
-    
-    for (int t = 0; t < threads.length; t++) {
-      DistributedTestCase.join(threads[t], 30 * 1000, null);
-    }
-    
-    // assert that all topics are acquired in order
-    // count number of occurrences of each thread
-    int count[] = new int[numThreads];
-    for (int i = 0; i < count.length; i++) { // shouldn't be necessary
-      count[i] = 0;
-    }
-    synchronized(this.fairnessList) {
-      for (Iterator iter = this.fairnessList.iterator(); iter.hasNext();) {
-        int id = Integer.valueOf((String)iter.next()).intValue();
-        count[id] = count[id]+1;
-      }
-    }
-    
-    int totalLocks = 0;
-    int minLocks = Integer.MAX_VALUE;
-    int maxLocks = 0;
-    for (int i = 0; i < count.length; i++) {
-      int locks = count[i];
-      this.log.fine("testFairnessStressfully thread-" + i + " acquired topic " + 
-        locks + " times.");
-      if (locks < minLocks) minLocks = locks;
-      if (locks > maxLocks) maxLocks = locks;
-      totalLocks = totalLocks + locks;
-    }
-
-    this.log.info("[testFairnessStressfully] totalLocks=" + totalLocks + 
-                  " minLocks=" + minLocks +
-                  " maxLocks=" + maxLocks);
-
-    int expectedLocks = (totalLocks / numThreads) + 1;
-    
-    // NOTE: if you turn on fine logs, this deviation may be too small...
-    // slower machines may also fail depending on thread scheduling
-    int deviation = (int)(expectedLocks * 0.25);
-    int lowThreshold = expectedLocks - deviation;
-    int highThreshold = expectedLocks + deviation;
-
-    this.log.info("[testFairnessStressfully] deviation=" + deviation +
-                  " expectedLocks=" + expectedLocks + 
-                  " lowThreshold=" + lowThreshold +
-                  " highThreshold=" + highThreshold);
-                        
-    // if these assertions keep failing we'll have to rewrite the test
-    // to handle scheduling of the threads...
-                  
-    assertTrue("minLocks is less than lowThreshold",
-               minLocks >= lowThreshold);
-    assertTrue("maxLocks is greater than highThreshold",
-               maxLocks <= highThreshold);
-  }
-  
-  public void testHasCurrentTopic() throws Exception {
-    this.log.info("[testHasCurrentTopic]");
-    assertTrue(!this.collaboration.hasCurrentTopic());
-    this.collaboration.acquireUninterruptibly("testHasCurrentTopic");
-    try {
-      assertTrue(this.collaboration.hasCurrentTopic());
-    }
-    finally {
-      this.collaboration.release();
-    }
-    assertTrue(!this.collaboration.hasCurrentTopic());
-  }
-  
-  protected volatile boolean flagTestThreadHasCurrentTopic = false;
-  public void testThreadHasCurrentTopic() throws Exception {
-    this.log.info("[testThreadHasCurrentTopic]");
-    Thread thread = new Thread(group, new Runnable() {
-      public void run() {
-        collaboration.acquireUninterruptibly("testThreadHasCurrentTopic");
-        try {
-          flagTestThreadHasCurrentTopic = true;
-          WaitCriterion ev = new WaitCriterion() {
-            public boolean done() {
-              return !flagTestThreadHasCurrentTopic;
-            }
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-        }
-        finally {
-          collaboration.release();
-        }
-      }
-    });
-    
-    // before starting thread, hasCurrentTopic(thread) returns false
-    assertTrue(!this.collaboration.hasCurrentTopic(thread));
-    thread.start();
-    WaitCriterion ev = new WaitCriterion() {
-      public boolean done() {
-        return flagTestThreadHasCurrentTopic;
-      }
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-    
-    // after starting thread, hasCurrentTopic(thread) returns true
-    assertTrue(this.collaboration.hasCurrentTopic(thread));
-    this.flagTestThreadHasCurrentTopic = false;
-    DistributedTestCase.join(thread, 30 * 1000, null);
-    
-    // after thread finishes, hasCurrentTopic(thread) returns false
-    assertTrue(!this.collaboration.hasCurrentTopic(thread));
-  }
-  
-  public void testIsCurrentTopic() throws Exception {
-    this.log.info("[testIsCurrentTopic]");
-    Object topic = "testIsCurrentTopic";
-    assertTrue(!this.collaboration.isCurrentTopic(topic));
-    this.collaboration.acquireUninterruptibly(topic);
-    try {
-      assertTrue(this.collaboration.isCurrentTopic(topic));
-    }
-    finally {
-      this.collaboration.release();
-    }
-    assertTrue(!this.collaboration.isCurrentTopic(topic));
-  }
-
-  protected final ThreadGroup group = 
-      new ThreadGroup("CollaborationJUnitTest Threads") {
-        public void uncaughtException(Thread t, Throwable e)
-        {
-          if (e instanceof VirtualMachineError) {
-            SystemFailure.setFailure((VirtualMachineError)e); // don't throw
-          }
-          String s = "Uncaught exception in thread " + t;
-          log.error(s, e);
-          fail(s);
-        }
-      };
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitTest.java
new file mode 100755
index 0000000..711500e
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/CollaborationJUnitTest.java
@@ -0,0 +1,617 @@
+/*
+ * 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.locks;
+
+import static org.junit.Assert.*;
+
+import java.util.*;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.internal.logging.LocalLogWriter;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.internal.logging.InternalLogWriter;
+
+import dunit.DistributedTestCase;
+import dunit.DistributedTestCase.WaitCriterion;
+
+/**
+ * Tests the Collaboration Lock used internally by dlock service.
+ *
+ * @author Kirk Lund
+ * @since 4.1.1
+ */
+@Category(IntegrationTest.class)
+@Ignore("Test is broken and was named CollaborationJUnitDisabledTest")
+public class CollaborationJUnitTest {
+
+  protected LogWriter log = new LocalLogWriter(InternalLogWriter.INFO_LEVEL);
+  protected Collaboration collaboration;
+  
+  @Before
+  public void setUp() throws Exception {
+    this.collaboration = new Collaboration(new CancelCriterion() {
+      @Override
+      public String cancelInProgress() {
+        return null;
+      }
+      @Override
+      public RuntimeException generateCancelledException(Throwable e) {
+        return null;
+      }
+    });
+  }
+  
+  @After
+  public void tearDown() throws Exception {
+    this.collaboration = null;
+  }
+  
+  protected volatile boolean flagTestBlocksUntilRelease = false;
+  protected volatile boolean threadBStartedTestBlocksUntilRelease = false;
+  
+  @Test
+  public void testBlocksUntilRelease() throws Exception {
+    this.log.info("[testBlocksUntilRelease]");
+    Thread threadA = new Thread(group, new Runnable() {
+      @Override
+      public void run() {
+        collaboration.acquireUninterruptibly("topicA");
+        try {
+          flagTestBlocksUntilRelease = true;
+          while(flagTestBlocksUntilRelease) {
+            try {
+              Thread.sleep(10);
+            }
+            catch (InterruptedException ignore) {fail("interrupted");}
+          }
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    
+    // thread one acquires
+    threadA.start();
+    WaitCriterion ev = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return CollaborationJUnitTest.this.flagTestBlocksUntilRelease;
+      }
+      @Override
+      public String description() {
+        return "waiting for thread";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
+    assertTrue(this.collaboration.hasCurrentTopic(threadA));
+    
+    // thread two blocks until one releeases
+    Thread threadB = new Thread(group, new Runnable() {
+      @Override
+      public void run() {
+        threadBStartedTestBlocksUntilRelease = true;
+        collaboration.acquireUninterruptibly("topicB");
+        try {
+          flagTestBlocksUntilRelease = true;
+          WaitCriterion ev2 = new WaitCriterion() {
+            @Override
+            public boolean done() {
+              return !flagTestBlocksUntilRelease;
+            }
+            @Override
+            public String description() {
+              return "waiting for release";
+            }
+          };
+          DistributedTestCase.waitForCriterion(ev2, 20 * 1000, 200, true);
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    
+    // start up threadB
+    threadB.start();
+    ev = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return threadBStartedTestBlocksUntilRelease;
+      }
+      @Override
+      public String description() {
+        return "waiting for thread b";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
+    
+    // threadA holds topic and threadB is waiting...
+    assertTrue(this.collaboration.hasCurrentTopic(threadA));
+    assertFalse(this.collaboration.hasCurrentTopic(threadB));
+
+    // let threadA release so that threadB gets lock
+    this.flagTestBlocksUntilRelease = false;
+    DistributedTestCase.join(threadA, 30 * 1000, null);
+    
+    // make sure threadB is doing what it's supposed to do...
+    ev = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return flagTestBlocksUntilRelease;
+      }
+      @Override
+      public String description() {
+        return "threadB";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 5 * 1000, 200, true);
+    // threadB must have lock now... let threadB release
+    assertTrue(this.collaboration.hasCurrentTopic(threadB));
+    this.flagTestBlocksUntilRelease = false;
+    DistributedTestCase.join(threadB, 30 * 1000, null);
+
+    // collaboration should be free now    
+    assertFalse(this.collaboration.hasCurrentTopic(threadA));
+    assertFalse(this.collaboration.hasCurrentTopic(threadB));
+    assertFalse(this.collaboration.hasCurrentTopic());
+  }
+  
+  protected volatile boolean threadAFlag_TestLateComerJoinsIn = false;
+  protected volatile boolean threadBFlag_TestLateComerJoinsIn = false;
+  protected volatile boolean threadCFlag_TestLateComerJoinsIn = true;
+  protected volatile boolean threadDFlag_TestLateComerJoinsIn = false;
+  
+  @Test
+  public void testLateComerJoinsIn() throws Exception {
+    this.log.info("[testLateComerJoinsIn]");
+    
+    final Object topicA = "topicA";
+    final Object topicB = "topicB";
+    
+    // threads one and two acquire
+    Thread threadA = new Thread(group, new Runnable() {
+      @Override
+      public void run() {
+        collaboration.acquireUninterruptibly(topicA);
+        try {
+          threadAFlag_TestLateComerJoinsIn = true;
+          WaitCriterion ev = new WaitCriterion() {
+            @Override
+            public boolean done() {
+              return !threadAFlag_TestLateComerJoinsIn;
+            }
+            @Override
+            public String description() {
+              return null;
+            }
+          };
+          DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    threadA.start();
+    WaitCriterion ev = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return threadAFlag_TestLateComerJoinsIn;
+      }
+      @Override
+      public String description() {
+        return "wait for ThreadA";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 30 * 1000, 200, true);
+    assertTrue(this.collaboration.hasCurrentTopic(threadA));
+    assertTrue(this.collaboration.isCurrentTopic(topicA));
+    
+    Thread threadB = new Thread(group, new Runnable() {
+      @Override
+      public void run() {
+        collaboration.acquireUninterruptibly(topicA);
+        try {
+          threadBFlag_TestLateComerJoinsIn = true;
+          WaitCriterion ev2 = new WaitCriterion() {
+            @Override
+            public boolean done() {
+              return !threadBFlag_TestLateComerJoinsIn;
+            }
+            @Override
+            public String description() {
+              return null;
+            }
+          };
+          DistributedTestCase.waitForCriterion(ev2, 60 * 1000, 200, true);
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    threadB.start();
+    ev = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return threadBFlag_TestLateComerJoinsIn;
+      }
+      @Override
+      public String description() {
+        return "";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+    assertTrue(this.collaboration.hasCurrentTopic(threadB));
+    
+    // thread three blocks for new topic
+    Thread threadC = new Thread(group, new Runnable() {
+      @Override
+      public void run() {
+        threadCFlag_TestLateComerJoinsIn = false;
+        collaboration.acquireUninterruptibly(topicB);
+        try {
+          threadCFlag_TestLateComerJoinsIn = true;
+          WaitCriterion ev2 = new WaitCriterion() {
+            @Override
+            public boolean done() {
+              return !threadCFlag_TestLateComerJoinsIn;
+            }
+            @Override
+            public String description() {
+              return null;
+            }
+          };
+          DistributedTestCase.waitForCriterion(ev2, 60 * 1000, 200, true);
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    threadC.start();
+    ev = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return threadCFlag_TestLateComerJoinsIn;
+      }
+      @Override
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+    assertFalse(this.collaboration.hasCurrentTopic(threadC));
+    assertFalse(this.collaboration.isCurrentTopic(topicB));
+    
+    // thread four (lateComer) acquires current topic immediately
+    Thread threadD = new Thread(group, new Runnable() {
+      @Override
+      public void run() {
+        collaboration.acquireUninterruptibly(topicA);
+        try {
+          threadDFlag_TestLateComerJoinsIn = true;
+          while(threadDFlag_TestLateComerJoinsIn) {
+            try {
+              Thread.sleep(10);
+            }
+            catch (InterruptedException ignore) {fail("interrupted");}
+          }
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    threadD.start();
+    ev = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return threadDFlag_TestLateComerJoinsIn;
+      }
+      @Override
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+    assertTrue(this.collaboration.hasCurrentTopic(threadD));
+    
+    // release threadA
+    this.threadAFlag_TestLateComerJoinsIn = false;
+    DistributedTestCase.join(threadA, 30 * 1000, null);
+    assertFalse(this.collaboration.hasCurrentTopic(threadA));
+    assertTrue(this.collaboration.hasCurrentTopic(threadB));
+    assertFalse(this.collaboration.hasCurrentTopic(threadC));
+    assertTrue(this.collaboration.hasCurrentTopic(threadD));
+    assertTrue(this.collaboration.isCurrentTopic(topicA));
+    assertFalse(this.collaboration.isCurrentTopic(topicB));
+    
+    // release threadB
+    this.threadBFlag_TestLateComerJoinsIn = false;
+    DistributedTestCase.join(threadB, 30 * 1000, null);
+    assertFalse(this.collaboration.hasCurrentTopic(threadB));
+    assertFalse(this.collaboration.hasCurrentTopic(threadC));
+    assertTrue(this.collaboration.hasCurrentTopic(threadD));
+    assertTrue(this.collaboration.isCurrentTopic(topicA));
+    assertFalse(this.collaboration.isCurrentTopic(topicB));
+    
+    // release threadD
+    this.threadDFlag_TestLateComerJoinsIn = false;
+    DistributedTestCase.join(threadD, 30 * 1000, null);
+    ev = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return threadCFlag_TestLateComerJoinsIn;
+      }
+      @Override
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+    assertTrue(this.collaboration.hasCurrentTopic(threadC));
+    assertFalse(this.collaboration.hasCurrentTopic(threadD));
+    assertFalse(this.collaboration.isCurrentTopic(topicA));
+    assertTrue(this.collaboration.isCurrentTopic(topicB));
+    
+    // release threadC
+    this.threadCFlag_TestLateComerJoinsIn = false;
+    DistributedTestCase.join(threadC, 30 * 1000, null);
+    assertFalse(this.collaboration.hasCurrentTopic(threadC));
+    assertFalse(this.collaboration.isCurrentTopic(topicA));
+    assertFalse(this.collaboration.isCurrentTopic(topicB));
+  }
+  
+  protected List waitingList = Collections.synchronizedList(new ArrayList());
+  protected List fairnessList = Collections.synchronizedList(new ArrayList());
+  protected volatile boolean runTestFairnessStressfully = true;
+  
+  @Test
+  public void testFairnessStressfully() throws Exception {
+    this.log.info("[testFairnessStressfully]");
+    final int numThreads = 20;
+    Thread threads[] = new Thread[numThreads];
+    
+    Runnable run = new Runnable() {
+      public void run() {
+        boolean released = false;
+        try {
+          String uniqueTopic = Thread.currentThread().getName();
+          while(runTestFairnessStressfully) {
+            waitingList.add(uniqueTopic);
+            collaboration.acquireUninterruptibly(uniqueTopic);
+            try {
+              released = false;
+              fairnessList.add(uniqueTopic);
+              waitingList.remove(uniqueTopic);
+            }
+            finally {
+              // wait for the other threads to line up...
+              WaitCriterion ev = new WaitCriterion() {
+                @Override
+                public boolean done() {
+                  return !runTestFairnessStressfully || waitingList.size() >= numThreads - 1;
+                }
+                @Override
+                public String description() {
+                  return "other threads lining up";
+                }
+              };
+              DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+              collaboration.release();
+              released = true;
+            }
+          }
+        }
+        finally {
+          if (!released) {
+            collaboration.release();
+          }
+        }
+      }
+    };
+    
+    try {
+      // many threads loop: acquire and release with unique topic
+      for (int t = 0; t < threads.length; t++) {
+        threads[t] = new Thread(group, run, String.valueOf(t));
+        threads[t].start();
+      }
+
+      log.info("Started all threads... waiting for test to complete.");
+            
+      // wait for numThreads * 10
+      WaitCriterion ev = new WaitCriterion() {
+        @Override
+        public boolean done() {
+          return fairnessList.size() >= numThreads * 20;
+        }
+        @Override
+        public String description() {
+          return "waiting for numThreads * 10";
+        }
+      };
+      DistributedTestCase.waitForCriterion(ev, 5 * 60 * 1000, 200, true);
+    }
+    finally {
+      if (this.runTestFairnessStressfully) {
+        this.runTestFairnessStressfully = false;
+      }
+    }
+    
+    for (int t = 0; t < threads.length; t++) {
+      DistributedTestCase.join(threads[t], 30 * 1000, null);
+    }
+    
+    // assert that all topics are acquired in order
+    // count number of occurrences of each thread
+    int count[] = new int[numThreads];
+    for (int i = 0; i < count.length; i++) { // shouldn't be necessary
+      count[i] = 0;
+    }
+    synchronized(this.fairnessList) {
+      for (Iterator iter = this.fairnessList.iterator(); iter.hasNext();) {
+        int id = Integer.valueOf((String)iter.next()).intValue();
+        count[id] = count[id]+1;
+      }
+    }
+    
+    int totalLocks = 0;
+    int minLocks = Integer.MAX_VALUE;
+    int maxLocks = 0;
+    for (int i = 0; i < count.length; i++) {
+      int locks = count[i];
+      this.log.fine("testFairnessStressfully thread-" + i + " acquired topic " + 
+        locks + " times.");
+      if (locks < minLocks) minLocks = locks;
+      if (locks > maxLocks) maxLocks = locks;
+      totalLocks = totalLocks + locks;
+    }
+
+    this.log.info("[testFairnessStressfully] totalLocks=" + totalLocks + 
+                  " minLocks=" + minLocks +
+                  " maxLocks=" + maxLocks);
+
+    int expectedLocks = (totalLocks / numThreads) + 1;
+    
+    // NOTE: if you turn on fine logs, this deviation may be too small...
+    // slower machines may also fail depending on thread scheduling
+    int deviation = (int)(expectedLocks * 0.25);
+    int lowThreshold = expectedLocks - deviation;
+    int highThreshold = expectedLocks + deviation;
+
+    this.log.info("[testFairnessStressfully] deviation=" + deviation +
+                  " expectedLocks=" + expectedLocks + 
+                  " lowThreshold=" + lowThreshold +
+                  " highThreshold=" + highThreshold);
+                        
+    // if these assertions keep failing we'll have to rewrite the test
+    // to handle scheduling of the threads...
+                  
+    assertTrue("minLocks is less than lowThreshold",
+               minLocks >= lowThreshold);
+    assertTrue("maxLocks is greater than highThreshold",
+               maxLocks <= highThreshold);
+  }
+  
+  @Test
+  public void testHasCurrentTopic() throws Exception {
+    this.log.info("[testHasCurrentTopic]");
+    assertTrue(!this.collaboration.hasCurrentTopic());
+    this.collaboration.acquireUninterruptibly("testHasCurrentTopic");
+    try {
+      assertTrue(this.collaboration.hasCurrentTopic());
+    }
+    finally {
+      this.collaboration.release();
+    }
+    assertTrue(!this.collaboration.hasCurrentTopic());
+  }
+  
+  protected volatile boolean flagTestThreadHasCurrentTopic = false;
+
+  @Test
+  public void testThreadHasCurrentTopic() throws Exception {
+    this.log.info("[testThreadHasCurrentTopic]");
+    Thread thread = new Thread(group, new Runnable() {
+      @Override
+      public void run() {
+        collaboration.acquireUninterruptibly("testThreadHasCurrentTopic");
+        try {
+          flagTestThreadHasCurrentTopic = true;
+          WaitCriterion ev = new WaitCriterion() {
+            @Override
+            public boolean done() {
+              return !flagTestThreadHasCurrentTopic;
+            }
+            @Override
+            public String description() {
+              return null;
+            }
+          };
+          DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+        }
+        finally {
+          collaboration.release();
+        }
+      }
+    });
+    
+    // before starting thread, hasCurrentTopic(thread) returns false
+    assertTrue(!this.collaboration.hasCurrentTopic(thread));
+    thread.start();
+    WaitCriterion ev = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return flagTestThreadHasCurrentTopic;
+      }
+      @Override
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+    
+    // after starting thread, hasCurrentTopic(thread) returns true
+    assertTrue(this.collaboration.hasCurrentTopic(thread));
+    this.flagTestThreadHasCurrentTopic = false;
+    DistributedTestCase.join(thread, 30 * 1000, null);
+    
+    // after thread finishes, hasCurrentTopic(thread) returns false
+    assertTrue(!this.collaboration.hasCurrentTopic(thread));
+  }
+  
+  @Test
+  public void testIsCurrentTopic() throws Exception {
+    this.log.info("[testIsCurrentTopic]");
+    Object topic = "testIsCurrentTopic";
+    assertTrue(!this.collaboration.isCurrentTopic(topic));
+    this.collaboration.acquireUninterruptibly(topic);
+    try {
+      assertTrue(this.collaboration.isCurrentTopic(topic));
+    }
+    finally {
+      this.collaboration.release();
+    }
+    assertTrue(!this.collaboration.isCurrentTopic(topic));
+  }
+
+  protected final ThreadGroup group = 
+      new ThreadGroup("CollaborationJUnitTest Threads") {
+        @Override
+        public void uncaughtException(Thread t, Throwable e)
+        {
+          if (e instanceof VirtualMachineError) {
+            SystemFailure.setFailure((VirtualMachineError)e); // don't throw
+          }
+          String s = "Uncaught exception in thread " + t;
+          log.error(s, e);
+          fail(s);
+        }
+      };
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
index 0ee9d4f..5ec4af8 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
@@ -21,6 +21,7 @@ import java.util.Arrays;
 
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -31,14 +32,15 @@ import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.internal.cache.DiskRegionHelperFactory;
 import com.gemstone.gemfire.internal.cache.DiskRegionProperties;
 import com.gemstone.gemfire.internal.cache.DiskRegionTestingBase;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 
 /**
  * Consolidated Disk Region Perftest. Overflow, Persist, OverflowWithPersist
  * modes are tested for Sync, AsyncWithBuffer and AsyncWithoutBufer writes.
  *  
  */
-@Category(IntegrationTest.class)
+@Category(PerformanceTest.class)
+@Ignore("Tests have no assertions")
 public class DiskRegionPerfJUnitPerformanceTest extends DiskRegionTestingBase
 {
   LogWriter log = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueStartStopJUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueStartStopJUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueStartStopJUnitDisabledTest.java
deleted file mode 100755
index 4cfc9ba..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueStartStopJUnitDisabledTest.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-/**
- * 
- */
-package com.gemstone.gemfire.internal.cache.ha;
-
-import java.io.IOException;
-import java.util.Properties;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.RegionQueue;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.internal.Assert;
-
-import junit.framework.TestCase;
-
-/**
- * @author Mitul Bid
- *
- */
-public class HARegionQueueStartStopJUnitDisabledTest extends TestCase
-{
-
-  /**
-   * Creates the cache instance for the test
-   * 
-   * @return the cache instance
-   * @throws CacheException -
-   *           thrown if any exception occurs in cache creation
-   */
-  private Cache createCache() throws CacheException
-  {
-    return CacheFactory.create(DistributedSystem.connect(new Properties()));
-  }
-
-  /**
-   * Creates HA region-queue object
-   * 
-   * @return HA region-queue object
-   * @throws IOException
-   * @throws ClassNotFoundException
-   * @throws CacheException
-   * @throws InterruptedException
-   */
-  private RegionQueue createHARegionQueue(String name, Cache cache)
-      throws IOException, ClassNotFoundException, CacheException, InterruptedException
-  {
-    RegionQueue regionqueue =HARegionQueue.getHARegionQueueInstance(name, cache,HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
-    return regionqueue;
-  }
-
-  public void testStartStop()
-  {
-    try {
-      boolean exceptionOccured = false;
-      Cache cache = createCache();
-      createHARegionQueue("test", cache);
-      Assert
-          .assertTrue(HARegionQueue.getDispatchedMessagesMapForTesting() != null);
-      HARegionQueue.stopHAServices();
-      try {
-        HARegionQueue.getDispatchedMessagesMapForTesting();
-      }
-      catch (NullPointerException e) {
-        exceptionOccured = true;
-      }
-      if (!exceptionOccured) {
-        fail("Expected exception to occur but did not occur");
-      }
-      HARegionQueue.startHAServices((GemFireCacheImpl)cache);
-      Assert
-          .assertTrue(HARegionQueue.getDispatchedMessagesMapForTesting() != null);
-      cache.close();
-      try {
-        HARegionQueue.getDispatchedMessagesMapForTesting();
-      }
-      catch (NullPointerException e) {
-        exceptionOccured = true;
-      }
-      if (!exceptionOccured) {
-        fail("Expected exception to occur but did not occur");
-      }
-      
-      cache = createCache();
-
-      try {
-        HARegionQueue.getDispatchedMessagesMapForTesting();
-      }
-      catch (NullPointerException e) {
-        exceptionOccured = true;
-      }
-      if (!exceptionOccured) {
-        fail("Expected exception to occur but did not occur");
-      }
-      
-    }
-    catch (Exception e) {
-      e.printStackTrace();
-      fail("Test failed due to " + e);
-    }
-
-  }
-  
-  
-}


[29/50] [abbrv] incubator-geode git commit: GEODE-610: Remove extension on LICENSE.txt

Posted by ab...@apache.org.
GEODE-610: Remove extension on LICENSE.txt


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

Branch: refs/heads/master
Commit: 5eae2b923dfbdd83e94b3e934e265589eb51baab
Parents: 27e9436
Author: Anthony Baker <ab...@pivotal.io>
Authored: Fri Jan 1 08:01:12 2016 -0600
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Wed Jan 6 17:37:43 2016 -0800

----------------------------------------------------------------------
 LICENSE     | 202 +++++++++++++++++++++++++++++++++++++++++++++++++++++++
 LICENSE.txt | 202 -------------------------------------------------------
 2 files changed, 202 insertions(+), 202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5eae2b92/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/LICENSE
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5eae2b92/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
deleted file mode 100644
index d645695..0000000
--- a/LICENSE.txt
+++ /dev/null
@@ -1,202 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.


[45/50] [abbrv] incubator-geode git commit: GEODE-781: Remove the maven-publish-auth plugin

Posted by ab...@apache.org.
GEODE-781: Remove the maven-publish-auth plugin

Read the settings.xml file directly when build is run by Jenkins.


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

Branch: refs/heads/master
Commit: e5a7b9aaa82d4c0a04e41febfd515056c4669001
Parents: f06a43f
Author: Anthony Baker <ab...@apache.org>
Authored: Sat Jan 16 09:14:14 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Sat Jan 16 09:36:14 2016 -0800

----------------------------------------------------------------------
 build.gradle | 23 +++++++++++++++++------
 1 file changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e5a7b9aa/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 90d4a60..85e5c35 100755
--- a/build.gradle
+++ b/build.gradle
@@ -24,7 +24,6 @@ buildscript {
     classpath "gradle.plugin.org.nosphere.apache:creadur-rat-gradle:0.2.0"
     classpath "org.ajoberstar:gradle-git:1.3.2"
     classpath 'com.bmuschko:gradle-nexus-plugin:2.3.1'
-    classpath 'org.hibernate.build.gradle:gradle-maven-publish-auth:2.0.1'
   }
 }
 
@@ -517,12 +516,24 @@ subprojects {
     }
   }
 
-  // Jenkins stores auth information in settings.xml.  We apply the maven-publish-auth plugin to read that
-  // from gradle.  However, we must match the repository id which the nexus plugin is not exposing.
-  apply plugin: 'maven-publish-auth'
+  // The nexus plugin reads authentication from ~/.gradle/gradle.properties but the
+  // jenkins server stores publishing credentials in ~/.m2/settings.xml (maven).
+  // We match on the expected snapshot repository id.
   afterEvaluate {
-    tasks.getByName('uploadArchives').doFirst {
-      repositories.each { it.snapshotRepository.id = 'apache.snapshots.https' }
+    if (!isReleaseVersion && System.env.USER == 'jenkins') {
+      def settingsXml = new File(System.getProperty('user.home'), '.m2/settings.xml')
+      def snapshotCreds = new XmlSlurper().parse(settingsXml).servers.server.find { server ->
+        server.id.text() == 'apache.snapshots.https'
+      }
+
+      if (snapshotCreds != null) {
+        tasks.uploadArchives.doFirst {
+          repositories().withType(MavenDeployer).each { repo ->
+            repo.snapshotRepository.authentication.userName = snapshotCreds.username.text()
+            repo.snapshotRepository.authentication.password = snapshotCreds.password.text()
+          }
+        }
+      }
     }
   }
 


[16/50] [abbrv] incubator-geode git commit: Revert "GEODE-714: Modify all tests to use JUnit Categories"

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestTestCase.java
deleted file mode 100755
index 481863c..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestTestCase.java
+++ /dev/null
@@ -1,1018 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.cache.tier.sockets;
-
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.InterestResultPolicy;
-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.PoolManager;
-import com.gemstone.gemfire.cache.client.internal.Connection;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.client.internal.RegisterInterestTracker;
-import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
-import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
-import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
-import com.gemstone.gemfire.internal.cache.CacheServerImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.tier.InterestType;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.VM;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-
-import org.junit.experimental.categories.Category;
-
-/**
- * Tests Interest Registration Functionality
- */
-@SuppressWarnings({"deprecation", "rawtypes", "serial", "unchecked"})
-public class HAInterestTestCase extends DistributedTestCase {
-  
-  protected static final int TIMEOUT_MILLIS = 60 * 1000;
-  protected static final int INTERVAL_MILLIS = 10;
-  
-  protected static final String REGION_NAME = "HAInterestBaseTest_region";
-  
-  protected static final String k1 = "k1";
-  protected static final String k2 = "k2";
-  protected static final String client_k1 = "client-k1";
-  protected static final String client_k2 = "client-k2";
-  protected static final String server_k1 = "server-k1";
-  protected static final String server_k2 = "server-k2";
-  protected static final String server_k1_updated = "server_k1_updated";
-
-  protected static Cache cache = null;
-  protected static PoolImpl pool = null;
-  protected static Connection conn = null;
-
-  protected static int PORT1;
-  protected static int PORT2;
-  protected static int PORT3;
-
-  protected static boolean isBeforeRegistrationCallbackCalled = false;
-  protected static boolean isBeforeInterestRecoveryCallbackCalled = false;
-  protected static boolean isAfterRegistrationCallbackCalled = false;
-
-  protected static Host host = null;
-  protected static VM server1 = null;
-  protected static VM server2 = null;
-  protected static VM server3 = null;
-  
-  protected volatile static boolean exceptionOccured = false;
-
-  public HAInterestTestCase(String name) {
-    super(name);
-  }
-
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-    host = Host.getHost(0);
-    server1 = host.getVM(0);
-    server2 = host.getVM(1);
-    server3 = host.getVM(2);
-    CacheServerTestUtil.disableShufflingOfEndpoints();
-    // start servers first
-    PORT1 = ((Integer) server1.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
-    PORT2 = ((Integer) server2.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
-    PORT3 = ((Integer) server3.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
-    exceptionOccured = false;
-    addExpectedException("java.net.ConnectException: Connection refused: connect");
-  }
-
-  @Override
-  public void tearDown2() throws Exception {
-    // close the clients first
-    closeCache();
-
-    // then close the servers
-    server1.invoke(HAInterestTestCase.class, "closeCache");
-    server2.invoke(HAInterestTestCase.class, "closeCache");
-    server3.invoke(HAInterestTestCase.class, "closeCache");
-    CacheServerTestUtil.resetDisableShufflingOfEndpointsFlag();
-  }
-
-  public static void closeCache() {
-    PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
-    PoolImpl.BEFORE_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = false;
-    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
-    PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
-    HAInterestTestCase.isAfterRegistrationCallbackCalled = false;
-    HAInterestTestCase.isBeforeInterestRecoveryCallbackCalled = false;
-    HAInterestTestCase.isBeforeRegistrationCallbackCalled = false;
-    if (cache != null && !cache.isClosed()) {
-      cache.close();
-      cache.getDistributedSystem().disconnect();
-    }
-    cache = null;
-    pool = null;
-    conn = null;
-  }
-  
-  /**
-   * Return the current primary waiting for a primary to exist.
-   * 
-   * @since 5.7
-   */
-  public static VM getPrimaryVM() {
-    return getPrimaryVM(null);
-  }
-
-  /**
-   * Return the current primary waiting for a primary to exist and for it not to
-   * be the oldPrimary (if oldPrimary is NOT null).
-   * 
-   * @since 5.7
-   */
-  public static VM getPrimaryVM(final VM oldPrimary) {
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        int primaryPort = pool.getPrimaryPort();
-        if (primaryPort == -1) {
-          return false;
-        }
-        // we have a primary
-        VM currentPrimary = getServerVM(primaryPort);
-        if (currentPrimary != oldPrimary) {
-          return true;
-        }
-        return false;
-      }
-      @Override
-      public String description() {
-        return "waiting for primary";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    int primaryPort = pool.getPrimaryPort();
-    assertTrue(primaryPort != -1);
-    VM currentPrimary = getServerVM(primaryPort);
-    assertTrue(currentPrimary != oldPrimary);
-    return currentPrimary;
-  }
-
-  public static VM getBackupVM() {
-    return getBackupVM(null);
-  }
-
-  public static VM getBackupVM(VM stoppedBackup) {
-    VM currentPrimary = getPrimaryVM(null);
-    if (currentPrimary != server2 && server2 != stoppedBackup) {
-      return server2;
-    } else if (currentPrimary != server3 && server3 != stoppedBackup) {
-      return server3;
-    } else if (currentPrimary != server1 && server1 != stoppedBackup) {
-      return server1;
-    } else {
-      fail("expected currentPrimary " + currentPrimary + " to be " + server1 + ", or " + server2 + ", or " + server3);
-      return null;
-    }
-  }
-
-  /**
-   * Given a server vm (server1, server2, or server3) return its port.
-   * 
-   * @since 5.7
-   */
-  public static int getServerPort(VM vm) {
-    if (vm == server1) {
-      return PORT1;
-    } else if (vm == server2) {
-      return PORT2;
-    } else if (vm == server3) {
-      return PORT3;
-    } else {
-      fail("expected vm " + vm + " to be " + server1 + ", or " + server2 + ", or " + server3);
-      return -1;
-    }
-  }
-
-  /**
-   * Given a server port (PORT1, PORT2, or PORT3) return its vm.
-   * 
-   * @since 5.7
-   */
-  public static VM getServerVM(int port) {
-    if (port == PORT1) {
-      return server1;
-    } else if (port == PORT2) {
-      return server2;
-    } else if (port == PORT3) {
-      return server3;
-    } else {
-      fail("expected port " + port + " to be " + PORT1 + ", or " + PORT2 + ", or " + PORT3);
-      return null;
-    }
-  }
-
-  public static void verifyRefreshedEntriesFromServer() {
-    final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r1);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        Region.Entry re = r1.getEntry(k1);
-        if (re == null)
-          return false;
-        Object val = re.getValue();
-        return client_k1.equals(val);
-      }
-      @Override
-      public String description() {
-        return "waiting for client_k1 refresh from server";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        Region.Entry re = r1.getEntry(k2);
-        if (re == null)
-          return false;
-        Object val = re.getValue();
-        return client_k2.equals(val);
-      }
-      @Override
-      public String description() {
-        return "waiting for client_k2 refresh from server";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-  }
-
-  public static void verifyDeadAndLiveServers(final int expectedDeadServers, final int expectedLiveServers) {
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == expectedLiveServers;
-      }
-      @Override
-      public String description() {
-        return "waiting for pool.getConnectedServerCount() == expectedLiveServer";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-  }
-
-  public static void putK1andK2() {
-    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r1);
-    r1.put(k1, server_k1);
-    r1.put(k2, server_k2);
-  }
-
-  public static void setClientServerObserverForBeforeInterestRecoveryFailure() {
-    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
-    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
-      public void beforeInterestRecovery() {
-        synchronized (HAInterestTestCase.class) {
-          Thread t = new Thread() {
-            public void run() {
-              getBackupVM().invoke(HAInterestTestCase.class, "startServer");
-              getPrimaryVM().invoke(HAInterestTestCase.class, "stopServer");
-            }
-          };
-          t.start();
-          try {
-            DistributedTestCase.join(t, 30 * 1000, getLogWriter());
-          } catch (Exception ignore) {
-            exceptionOccured = true;
-          }
-          HAInterestTestCase.isBeforeInterestRecoveryCallbackCalled = true;
-          HAInterestTestCase.class.notify();
-          PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
-        }
-      }
-    });
-  }
-
-  public static void setClientServerObserverForBeforeInterestRecovery() {
-    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
-    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
-      public void beforeInterestRecovery() {
-        synchronized (HAInterestTestCase.class) {
-          Thread t = new Thread() {
-            public void run() {
-              Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-              assertNotNull(r1);
-              r1.put(k1, server_k1_updated);
-            }
-          };
-          t.start();
-
-          HAInterestTestCase.isBeforeInterestRecoveryCallbackCalled = true;
-          HAInterestTestCase.class.notify();
-          PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
-        }
-      }
-    });
-  }
-
-  public static void waitForBeforeInterestRecoveryCallBack() throws InterruptedException {
-    assertNotNull(cache);
-    synchronized (HAInterestTestCase.class) {
-      while (!isBeforeInterestRecoveryCallbackCalled) {
-        HAInterestTestCase.class.wait();
-      }
-    }
-  }
-
-  public static void setClientServerObserverForBeforeRegistration(final VM vm) {
-    PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = true;
-    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
-      public void beforeInterestRegistration() {
-        synchronized (HAInterestTestCase.class) {
-          vm.invoke(HAInterestTestCase.class, "startServer");
-          HAInterestTestCase.isBeforeRegistrationCallbackCalled = true;
-          HAInterestTestCase.class.notify();
-          PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
-        }
-      }
-    });
-  }
-
-  public static void waitForBeforeRegistrationCallback() throws InterruptedException {
-    assertNotNull(cache);
-    synchronized (HAInterestTestCase.class) {
-      while (!isBeforeRegistrationCallbackCalled) {
-        HAInterestTestCase.class.wait();
-      }
-    }
-  }
-
-  public static void setClientServerObserverForAfterRegistration(final VM vm) {
-    PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = true;
-    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
-      public void afterInterestRegistration() {
-        synchronized (HAInterestTestCase.class) {
-          vm.invoke(HAInterestTestCase.class, "startServer");
-          HAInterestTestCase.isAfterRegistrationCallbackCalled = true;
-          HAInterestTestCase.class.notify();
-          PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
-        }
-      }
-    });
-  }
-
-  public static void waitForAfterRegistrationCallback() throws InterruptedException {
-    assertNotNull(cache);
-    if (!isAfterRegistrationCallbackCalled) {
-      synchronized (HAInterestTestCase.class) {
-        while (!isAfterRegistrationCallbackCalled) {
-          HAInterestTestCase.class.wait();
-        }
-      }
-    }
-  }
-
-  public static void unSetClientServerObserverForRegistrationCallback() {
-    synchronized (HAInterestTestCase.class) {
-      PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
-      PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
-      HAInterestTestCase.isBeforeRegistrationCallbackCalled = false;
-      HAInterestTestCase.isAfterRegistrationCallbackCalled = false;
-    }
-  }
-
-  public static void verifyDispatcherIsAlive() {
-    assertEquals("More than one BridgeServer", 1, cache.getCacheServers().size());
-    
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return cache.getCacheServers().size() == 1;
-      }
-      @Override
-      public String description() {
-        return "waiting for cache.getCacheServers().size() == 1";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    assertNotNull(bs.getAcceptor());
-    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
-    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
-
-    wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return ccn.getClientProxies().size() > 0;
-      }
-      @Override
-      public String description() {
-        return "waiting for ccn.getClientProxies().size() > 0";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    wc = new WaitCriterion() {
-      Iterator iter_prox;
-      CacheClientProxy proxy;
-
-      @Override
-      public boolean done() {
-        iter_prox = ccn.getClientProxies().iterator();
-        if (iter_prox.hasNext()) {
-          proxy = (CacheClientProxy) iter_prox.next();
-          return proxy._messageDispatcher.isAlive();
-        } else {
-          return false;
-        }
-      }
-
-      @Override
-      public String description() {
-        return "waiting for CacheClientProxy _messageDispatcher to be alive";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-  }
-
-  public static void verifyDispatcherIsNotAlive() {
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return cache.getCacheServers().size() == 1;
-      }
-      @Override
-      public String description() {
-        return "cache.getCacheServers().size() == 1";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    assertNotNull(bs.getAcceptor());
-    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
-    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
-    
-    wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return ccn.getClientProxies().size() > 0;
-      }
-      @Override
-      public String description() {
-        return "waiting for ccn.getClientProxies().size() > 0";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    Iterator iter_prox = ccn.getClientProxies().iterator();
-    if (iter_prox.hasNext()) {
-      CacheClientProxy proxy = (CacheClientProxy) iter_prox.next();
-      assertFalse("Dispatcher on secondary should not be alive", proxy._messageDispatcher.isAlive());
-    }
-  }
-
-  public static void createEntriesK1andK2OnServer() {
-    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r1);
-    if (!r1.containsKey(k1)) {
-      r1.create(k1, server_k1);
-    }
-    if (!r1.containsKey(k2)) {
-      r1.create(k2, server_k2);
-    }
-    assertEquals(r1.getEntry(k1).getValue(), server_k1);
-    assertEquals(r1.getEntry(k2).getValue(), server_k2);
-  }
-
-  public static void createEntriesK1andK2() {
-    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r1);
-    if (!r1.containsKey(k1)) {
-      r1.create(k1, client_k1);
-    }
-    if (!r1.containsKey(k2)) {
-      r1.create(k2, client_k2);
-    }
-    assertEquals(r1.getEntry(k1).getValue(), client_k1);
-    assertEquals(r1.getEntry(k2).getValue(), client_k2);
-  }
-
-  public static void createServerEntriesK1andK2() {
-    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r1);
-    if (!r1.containsKey(k1)) {
-      r1.create(k1, server_k1);
-    }
-    if (!r1.containsKey(k2)) {
-      r1.create(k2, server_k2);
-    }
-    assertEquals(r1.getEntry(k1).getValue(), server_k1);
-    assertEquals(r1.getEntry(k2).getValue(), server_k2);
-  }
-
-  public static void registerK1AndK2() {
-    Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-    r.registerInterest(list, InterestResultPolicy.KEYS_VALUES);
-  }
-
-  public static void reRegisterK1AndK2() {
-    Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-    r.registerInterest(list);
-  }
-
-  public static void startServer() throws IOException {
-    Cache c = CacheFactory.getAnyInstance();
-    assertEquals("More than one BridgeServer", 1, c.getCacheServers().size());
-    CacheServerImpl bs = (CacheServerImpl) c.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    bs.start();
-  }
-
-  public static void stopServer() {
-    assertEquals("More than one BridgeServer", 1, cache.getCacheServers().size());
-    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    bs.stop();
-  }
-
-  public static void stopPrimaryAndRegisterK1AndK2AndVerifyResponse() {
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == 3;
-      }
-      @Override
-      public String description() {
-        return "connected server count never became 3";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    // close primaryEP
-    getPrimaryVM().invoke(HAInterestTestCase.class, "stopServer");
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
-    assertNotNull(serverKeys);
-    List resultKeys = (List) serverKeys.get(0);
-    assertEquals(2, resultKeys.size());
-    assertTrue(resultKeys.contains(k1));
-    assertTrue(resultKeys.contains(k2));
-  }
-
-  public static void stopPrimaryAndUnregisterRegisterK1() {
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == 3;
-      }
-      @Override
-      public String description() {
-        return "connected server count never became 3";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    // close primaryEP
-    getPrimaryVM().invoke(HAInterestTestCase.class, "stopServer");
-    List list = new ArrayList();
-    list.add(k1);
-    srp.unregisterInterest(list, InterestType.KEY, false, false);
-  }
-
-  public static void stopBothPrimaryAndSecondaryAndRegisterK1AndK2AndVerifyResponse() {
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == 3;
-      }
-      @Override
-      public String description() {
-        return "connected server count never became 3";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    // close primaryEP
-    VM backup = getBackupVM();
-    getPrimaryVM().invoke(HAInterestTestCase.class, "stopServer");
-    // close secondary
-    backup.invoke(HAInterestTestCase.class, "stopServer");
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
-
-    assertNotNull(serverKeys);
-    List resultKeys = (List) serverKeys.get(0);
-    assertEquals(2, resultKeys.size());
-    assertTrue(resultKeys.contains(k1));
-    assertTrue(resultKeys.contains(k2));
-  }
-
-  /**
-   * returns the secondary that was stopped
-   */
-  public static VM stopSecondaryAndRegisterK1AndK2AndVerifyResponse() {
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == 3;
-      }
-      @Override
-      public String description() {
-        return "Never got three connected servers";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    // close secondary EP
-    VM result = getBackupVM();
-    result.invoke(HAInterestTestCase.class, "stopServer");
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
-
-    assertNotNull(serverKeys);
-    List resultKeys = (List) serverKeys.get(0);
-    assertEquals(2, resultKeys.size());
-    assertTrue(resultKeys.contains(k1));
-    assertTrue(resultKeys.contains(k2));
-    return result;
-  }
-
-  /**
-   * returns the secondary that was stopped
-   */
-  public static VM stopSecondaryAndUNregisterK1() {
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == 3;
-      }
-      @Override
-      public String description() {
-        return "connected server count never became 3";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    // close secondary EP
-    VM result = getBackupVM();
-    result.invoke(HAInterestTestCase.class, "stopServer");
-    List list = new ArrayList();
-    list.add(k1);
-    srp.unregisterInterest(list, InterestType.KEY, false, false);
-    return result;
-  }
-
-  public static void registerK1AndK2OnPrimaryAndSecondaryAndVerifyResponse() {
-    ServerLocation primary = pool.getPrimary();
-    ServerLocation secondary = (ServerLocation) pool.getRedundants().get(0);
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-
-    // Primary server
-    List serverKeys1 = srp.registerInterestOn(primary, list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
-    assertNotNull(serverKeys1);
-    // expect serverKeys in response from primary
-    List resultKeys = (List) serverKeys1.get(0);
-    assertEquals(2, resultKeys.size());
-    assertTrue(resultKeys.contains(k1));
-    assertTrue(resultKeys.contains(k2));
-
-    // Secondary server
-    List serverKeys2 = srp.registerInterestOn(secondary, list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
-    // if the list is null then it is empty
-    if (serverKeys2 != null) {
-      // no serverKeys in response from secondary
-      assertTrue(serverKeys2.isEmpty());
-    }
-  }
-
-  public static void verifyInterestRegistration() {
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return cache.getCacheServers().size() == 1;
-      }
-      @Override
-      public String description() {
-        return "waiting for cache.getCacheServers().size() == 1";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    assertNotNull(bs.getAcceptor());
-    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
-    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
-    
-    wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return ccn.getClientProxies().size() > 0;
-      }
-      @Override
-      public String description() {
-        return "waiting for ccn.getClientProxies().size() > 0";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    Iterator iter_prox = ccn.getClientProxies().iterator();
-
-    if (iter_prox.hasNext()) {
-      final CacheClientProxy ccp = (CacheClientProxy) iter_prox.next();
-      
-      wc = new WaitCriterion() {
-        @Override
-        public boolean done() {
-          Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
-              .getProfile(Region.SEPARATOR + REGION_NAME)
-              .getKeysOfInterestFor(ccp.getProxyID());
-          return keysMap != null && keysMap.size() == 2;
-        }
-        @Override
-        public String description() {
-          return "waiting for keys of interest to include 2 keys";
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-      Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex].getProfile(Region.SEPARATOR + REGION_NAME)
-          .getKeysOfInterestFor(ccp.getProxyID());
-      assertNotNull(keysMap);
-      assertEquals(2, keysMap.size());
-      assertTrue(keysMap.contains(k1));
-      assertTrue(keysMap.contains(k2));
-    }
-  }
-
-  public static void verifyInterestUNRegistration() {
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return cache.getCacheServers().size() == 1;
-      }
-      @Override
-      public String description() {
-        return "waiting for cache.getCacheServers().size() == 1";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    assertNotNull(bs.getAcceptor());
-    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
-    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
-    
-    wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return ccn.getClientProxies().size() > 0;
-      }
-      @Override
-      public String description() {
-        return "waiting for ccn.getClientProxies().size() > 0";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    Iterator iter_prox = ccn.getClientProxies().iterator();
-    if (iter_prox.hasNext()) {
-      final CacheClientProxy ccp = (CacheClientProxy) iter_prox.next();
-      
-      wc = new WaitCriterion() {
-        @Override
-        public boolean done() {
-          Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
-              .getProfile(Region.SEPARATOR + REGION_NAME)
-              .getKeysOfInterestFor(ccp.getProxyID());
-          return keysMap != null;
-        }
-        @Override
-        public String description() {
-          return "waiting for keys of interest to not be null";
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-      Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
-          .getProfile(Region.SEPARATOR + REGION_NAME)
-          .getKeysOfInterestFor(ccp.getProxyID());
-      assertNotNull(keysMap);
-      assertEquals(1, keysMap.size());
-      assertFalse(keysMap.contains(k1));
-      assertTrue(keysMap.contains(k2));
-    }
-  }
-
-  private void createCache(Properties props) throws Exception {
-    DistributedSystem ds = getSystem(props);
-    assertNotNull(ds);
-    ds.disconnect();
-    ds = getSystem(props);
-    cache = CacheFactory.create(ds);
-    assertNotNull(cache);
-  }
-
-  public static void createClientPoolCache(String testName, String host) throws Exception {
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new HAInterestTestCase("temp").createCache(props);
-    CacheServerTestUtil.disableShufflingOfEndpoints();
-    PoolImpl p;
-    try {
-      p = (PoolImpl) PoolManager.createFactory()
-          .addServer(host, PORT1)
-          .addServer(host, PORT2)
-          .addServer(host, PORT3)
-          .setSubscriptionEnabled(true)
-          .setSubscriptionRedundancy(-1)
-          .setReadTimeout(1000)
-          .setPingInterval(1000)
-          // retryInterval should be more so that only registerInterste thread
-          // will initiate failover
-          // .setRetryInterval(20000)
-          .create("HAInterestBaseTestPool");
-    } finally {
-      CacheServerTestUtil.enableShufflingOfEndpoints();
-    }
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    factory.setConcurrencyChecksEnabled(true);
-    factory.setPoolName(p.getName());
-
-    cache.createRegion(REGION_NAME, factory.create());
-    pool = p;
-    conn = pool.acquireConnection();
-    assertNotNull(conn);
-  }
-
-  public static void createClientPoolCacheWithSmallRetryInterval(String testName, String host) throws Exception {
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new HAInterestTestCase("temp").createCache(props);
-    CacheServerTestUtil.disableShufflingOfEndpoints();
-    PoolImpl p;
-    try {
-      p = (PoolImpl) PoolManager.createFactory()
-          .addServer(host, PORT1)
-          .addServer(host, PORT2)
-          .setSubscriptionEnabled(true)
-          .setSubscriptionRedundancy(-1)
-          .setReadTimeout(1000)
-          .setSocketBufferSize(32768)
-          .setMinConnections(6)
-          .setPingInterval(200)
-          // .setRetryInterval(200)
-          // retryAttempts 3
-          .create("HAInterestBaseTestPool");
-    } finally {
-      CacheServerTestUtil.enableShufflingOfEndpoints();
-    }
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    factory.setConcurrencyChecksEnabled(true);
-    factory.setPoolName(p.getName());
-
-    cache.createRegion(REGION_NAME, factory.create());
-
-    pool = p;
-    conn = pool.acquireConnection();
-    assertNotNull(conn);
-  }
-
-  public static void createClientPoolCacheConnectionToSingleServer(String testName, String hostName) throws Exception {
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new HAInterestTestCase("temp").createCache(props);
-    PoolImpl p = (PoolImpl) PoolManager.createFactory()
-        .addServer(hostName, PORT1)
-        .setSubscriptionEnabled(true)
-        .setSubscriptionRedundancy(-1)
-        .setReadTimeout(1000)
-        // .setRetryInterval(20)
-        .create("HAInterestBaseTestPool");
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    factory.setConcurrencyChecksEnabled(true);
-    factory.setPoolName(p.getName());
-
-    cache.createRegion(REGION_NAME, factory.create());
-
-    pool = p;
-    conn = pool.acquireConnection();
-    assertNotNull(conn);
-  }
-
-  public static Integer createServerCache() throws Exception {
-    new HAInterestTestCase("temp").createCache(new Properties());
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setEnableBridgeConflation(true);
-    factory.setMirrorType(MirrorType.KEYS_VALUES);
-    factory.setConcurrencyChecksEnabled(true);
-    cache.createRegion(REGION_NAME, factory.create());
-
-    CacheServer server = cache.addCacheServer();
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    server.setPort(port);
-    server.setMaximumTimeBetweenPings(180000);
-    // ensures updates to be sent instead of invalidations
-    server.setNotifyBySubscription(true);
-    server.start();
-    return new Integer(server.getPort());
-  }
-
-  public static Integer createServerCacheWithLocalRegion() throws Exception {
-    new HAInterestTestCase("temp").createCache(new Properties());
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    factory.setConcurrencyChecksEnabled(true);
-    RegionAttributes attrs = factory.create();
-    cache.createRegion(REGION_NAME, attrs);
-
-    CacheServer server = cache.addCacheServer();
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    server.setPort(port);
-    // ensures updates to be sent instead of invalidations
-    server.setNotifyBySubscription(true);
-    server.setMaximumTimeBetweenPings(180000);
-    server.start();
-    return new Integer(server.getPort());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
index b6bfe22..b12f55b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
@@ -22,18 +22,12 @@ import static org.mockito.Mockito.when;
 import java.io.IOException;
 
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- * Exposes GEODE-537: NPE in JTA AFTER_COMPLETION command processing
- */
-@Category(UnitTest.class)
 public class CommitCommandTest {
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogWriterPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogWriterPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogWriterPerformanceTest.java
index 1f72a6b..77d7995 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogWriterPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogWriterPerformanceTest.java
@@ -21,22 +21,16 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.Properties;
 
-import org.junit.Ignore;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.internal.util.IOUtils;
-import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 
 /**
  * Tests performance of logging when level is OFF.
  * 
  * @author Kirk Lund
  */
-@Category(PerformanceTest.class)
-@Ignore("Tests have no assertions")
 public class LogWriterPerformanceTest extends LoggingPerformanceTestCase {
 
   public LogWriterPerformanceTest(String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2DisabledPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2DisabledPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2DisabledPerformanceTest.java
index caedadc..f98868b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2DisabledPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2DisabledPerformanceTest.java
@@ -20,13 +20,7 @@ import java.io.IOException;
 
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
-import org.junit.Ignore;
-import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
-
-@Category(PerformanceTest.class)
-@Ignore("Tests have no assertions")
 public class Log4J2DisabledPerformanceTest extends Log4J2PerformanceTest {
 
   public Log4J2DisabledPerformanceTest(String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
index ddf106d..a002389 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
@@ -29,17 +29,12 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
 import org.apache.logging.log4j.util.PropertiesUtil;
-import org.junit.Ignore;
-import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.logging.LoggingPerformanceTestCase;
 import com.gemstone.gemfire.internal.util.IOUtils;
-import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 import com.gemstone.org.apache.logging.log4j.core.config.xml.GemFireXmlConfigurationFactory;
 
-@Category(PerformanceTest.class)
-@Ignore("Tests have no assertions")
 public class Log4J2PerformanceTest extends LoggingPerformanceTestCase {
 
   protected static final int DEFAULT_LOG_FILE_SIZE_LIMIT = Integer.MAX_VALUE;
@@ -49,11 +44,13 @@ public class Log4J2PerformanceTest extends LoggingPerformanceTestCase {
   protected static final String SYS_LOG_FILE_SIZE_LIMIT = "gemfire-log-file-size-limit";
   protected static final String SYS_LOG_FILE_COUNT_LIMIT = "gemfire-log-file-count-limit";
   
-  private static void init() { // was a static initializer
+  static {
     // set log4j.configurationFactory to be our optimized version
     final String factory = GemFireXmlConfigurationFactory.class.getName();
     System.setProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY, factory);
+    System.out.println("Set "+ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY+" to "+factory);
     final String factoryClass = PropertiesUtil.getProperties().getStringProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY);
+    System.out.println("KIRK: factoryClass is " + factoryClass);
   }
   
   private File config = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerDisabledPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerDisabledPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerDisabledPerformanceTest.java
index 4be34c7..f964208 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerDisabledPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerDisabledPerformanceTest.java
@@ -20,13 +20,7 @@ import java.io.IOException;
 
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
-import org.junit.Ignore;
-import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
-
-@Category(PerformanceTest.class)
-@Ignore("Tests have no assertions")
 public class LogWriterLoggerDisabledPerformanceTest extends LogWriterLoggerPerformanceTest {
 
   public LogWriterLoggerDisabledPerformanceTest(String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
index 926bc75..61b5131 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
@@ -28,17 +28,12 @@ import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
 import org.apache.logging.log4j.util.PropertiesUtil;
-import org.junit.Ignore;
-import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.logging.LoggingPerformanceTestCase;
 import com.gemstone.gemfire.internal.util.IOUtils;
-import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 import com.gemstone.org.apache.logging.log4j.core.config.xml.GemFireXmlConfigurationFactory;
 
-@Category(PerformanceTest.class)
-@Ignore("Tests have no assertions")
 public class LogWriterLoggerPerformanceTest extends LoggingPerformanceTestCase {
 
   protected static final int DEFAULT_LOG_FILE_SIZE_LIMIT = Integer.MAX_VALUE;
@@ -48,11 +43,13 @@ public class LogWriterLoggerPerformanceTest extends LoggingPerformanceTestCase {
   protected static final String SYS_LOG_FILE_SIZE_LIMIT = "gemfire-log-file-size-limit";
   protected static final String SYS_LOG_FILE_COUNT_LIMIT = "gemfire-log-file-count-limit";
   
-  private static void init() { // was a static initializer
+  static {
     // set log4j.configurationFactory to be our optimized version
     final String factory = GemFireXmlConfigurationFactory.class.getName();
     System.setProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY, factory);
+    System.out.println("Set "+ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY+" to "+factory);
     final String factoryClass = PropertiesUtil.getProperties().getStringProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY);
+    System.out.println("KIRK: factoryClass is " + factoryClass);
   }
   
   private File config = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/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 6fa560f..a3d4785 100755
--- a/gemfire-core/src/test/java/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
@@ -34,7 +34,6 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
-import org.junit.experimental.categories.Category;
 import org.springframework.data.gemfire.support.GemfireCache;
 
 import junit.framework.TestCase;
@@ -87,7 +86,6 @@ import com.gemstone.gemfire.internal.logging.LogWriterImpl;
 import com.gemstone.gemfire.internal.logging.ManagerLogWriter;
 import com.gemstone.gemfire.internal.logging.log4j.LogWriterLogger;
 import com.gemstone.gemfire.management.internal.cli.LogWrapper;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 import dunit.standalone.DUnitLauncher;
 
@@ -103,7 +101,6 @@ import dunit.standalone.DUnitLauncher;
  *
  * @author David Whitlock
  */
-@Category(DistributedTest.class)
 @SuppressWarnings("serial")
 public abstract class DistributedTestCase extends TestCase implements java.io.Serializable {
   private static final Logger logger = LogService.getLogger();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitPerformanceTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitPerformanceTest.java
new file mode 100644
index 0000000..ab2db78
--- /dev/null
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitPerformanceTest.java
@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.gemstone.gemfire.cache.lucene.internal.repository;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.RAMDirectory;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.DataSerializable;
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneQuery;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
+import com.gemstone.gemfire.cache.query.QueryException;
+import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
+
+/**
+ * Microbenchmark of the IndexRepository to compare an
+ * IndexRepository built on top of cache with a 
+ * stock lucene IndexWriter with a RAMDirectory.
+ */
+@Category(PerformanceTest.class)
+public class IndexRepositoryImplJUnitPerformanceTest {
+  
+  private static final int NUM_WORDS = 1000;
+  private static int[] COMMIT_INTERVAL = new int[] {100, 1000, 5000};
+  private static int NUM_ENTRIES = 500_000;
+  private static int NUM_QUERIES = 500_000;
+
+  private StandardAnalyzer analyzer = new StandardAnalyzer();
+  
+  @Test
+  public  void testIndexRepository() throws Exception {
+    
+
+    doTest("IndexRepository", new TestCallbacks() {
+
+      private Cache cache;
+      private IndexRepositoryImpl repo;
+      private IndexWriter writer;
+
+      @Override
+      public void addObject(String key, String text) throws Exception {
+        repo.create(key, new TestObject(text));
+      }
+
+      @Override
+      public void commit()  throws Exception {
+        repo.commit();
+      }
+
+      @Override
+      public void init() throws Exception {
+        cache = new CacheFactory().set("mcast-port", "0")
+            .set("log-level", "error")
+            .create();
+        Region<String, File> fileRegion = cache.<String, File>createRegionFactory(RegionShortcut.REPLICATE).create("files");
+        Region<ChunkKey, byte[]> chunkRegion = cache.<ChunkKey, byte[]>createRegionFactory(RegionShortcut.REPLICATE).create("chunks");
+
+        RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion);
+        
+        
+        IndexWriterConfig config = new IndexWriterConfig(analyzer);
+        writer = new IndexWriter(dir, config);
+        String[] indexedFields= new String[] {"text"};
+        HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(indexedFields);
+        repo = new IndexRepositoryImpl(fileRegion, writer, mapper);
+      }
+
+      @Override
+      public void cleanup() throws IOException {
+        writer.close();
+        cache.close();
+      }
+
+      @Override
+      public void waitForAsync() throws Exception {
+        //do nothing
+      }
+
+      @Override
+      public int query(Query query) throws IOException {
+        TopEntriesCollector collector = new TopEntriesCollector();
+        repo.query(query, 100, collector);
+        return collector.size();
+      }
+    });
+  }
+  
+  /**
+   * Test our full lucene index implementation
+   * @throws Exception
+   */
+  @Test
+  public void testLuceneIndex() throws Exception {
+    
+
+    doTest("LuceneIndex", new TestCallbacks() {
+
+      private Cache cache;
+      private Region<String, TestObject> region;
+      private LuceneService service;
+
+      @Override
+      public void addObject(String key, String text) throws Exception {
+        region.create(key, new TestObject(text));
+      }
+
+      @Override
+      public void commit()  throws Exception {
+        //NA
+      }
+
+      @Override
+      public void init() throws Exception {
+        cache = new CacheFactory().set("mcast-port", "0")
+            .set("log-level", "warning")
+            .create();
+        service = LuceneServiceProvider.get(cache);
+        service.createIndex("index", "/region", "text");
+        region = cache.<String, TestObject>createRegionFactory(RegionShortcut.PARTITION)
+            .setPartitionAttributes(new PartitionAttributesFactory<>().setTotalNumBuckets(1).create())
+            .create("region");
+      }
+
+      @Override
+      public void cleanup() throws IOException {
+        cache.close();
+      }
+      
+      @Override
+      public void waitForAsync() throws Exception {
+        AsyncEventQueue aeq = cache.getAsyncEventQueue(LuceneServiceImpl.getUniqueIndexName("index", "/region"));
+        
+        //We will be at most 10 ms off
+        while(aeq.size() > 0) {
+          Thread.sleep(10);
+        }
+      }
+
+      @Override
+      public int query(final Query query) throws Exception {
+        LuceneQuery<Object, Object> luceneQuery = service.createLuceneQueryFactory().create("index", "/region", new LuceneQueryProvider() {
+          
+          @Override
+          public Query getQuery(LuceneIndex index) throws QueryException {
+            return query;
+          }
+        });
+        
+        LuceneQueryResults<Object, Object> results = luceneQuery.search();
+        return results.size();
+      }
+    });
+  }
+  
+  @Test
+  public  void testLuceneWithRegionDirectory() throws Exception {
+    doTest("RegionDirectory", new TestCallbacks() {
+
+      private IndexWriter writer;
+      private SearcherManager searcherManager;
+
+      @Override
+      public void init() throws Exception {
+        RegionDirectory dir = new RegionDirectory(new ConcurrentHashMap<String, File>(), new ConcurrentHashMap<ChunkKey, byte[]>());
+        IndexWriterConfig config = new IndexWriterConfig(analyzer);
+        writer = new IndexWriter(dir, config);
+        searcherManager = new SearcherManager(writer, true, null);
+      }
+
+      @Override
+      public void addObject(String key, String text) throws Exception {
+        Document doc = new Document();
+        doc.add(new TextField("key", key, Store.YES));
+        doc.add(new TextField("text", text, Store.NO));
+        writer.addDocument(doc);
+      }
+
+      @Override
+      public void commit() throws Exception {
+        writer.commit();
+        searcherManager.maybeRefresh();
+      }
+
+      @Override
+      public void cleanup() throws Exception {
+        writer.close();
+      }
+      
+      @Override
+      public void waitForAsync() throws Exception {
+        //do nothing
+      }
+
+      @Override
+      public int query(Query query) throws Exception {
+        IndexSearcher searcher = searcherManager.acquire();
+        try {
+          return searcher.count(query);
+        } finally {
+          searcherManager.release(searcher);
+        }
+      }
+      
+    });
+    
+  }
+  
+  @Test
+  public  void testLucene() throws Exception {
+    doTest("Lucene", new TestCallbacks() {
+
+      private IndexWriter writer;
+      private SearcherManager searcherManager;
+
+      @Override
+      public void init() throws Exception {
+        RAMDirectory dir = new RAMDirectory();
+        IndexWriterConfig config = new IndexWriterConfig(analyzer);
+        writer = new IndexWriter(dir, config);
+        searcherManager = new SearcherManager(writer, true, null);
+      }
+
+      @Override
+      public void addObject(String key, String text) throws Exception {
+        Document doc = new Document();
+        doc.add(new TextField("key", key, Store.YES));
+        doc.add(new TextField("text", text, Store.NO));
+        writer.addDocument(doc);
+      }
+
+      @Override
+      public void commit() throws Exception {
+        writer.commit();
+        searcherManager.maybeRefresh();
+      }
+
+      @Override
+      public void cleanup() throws Exception {
+        writer.close();
+      }
+      
+      @Override
+      public void waitForAsync() throws Exception {
+        //do nothing
+      }
+
+      @Override
+      public int query(Query query) throws Exception {
+        IndexSearcher searcher = searcherManager.acquire();
+        try {
+          return searcher.count(query);
+        } finally {
+          searcherManager.release(searcher);
+        }
+      }
+      
+    });
+    
+  }
+  
+  private void doTest(String testName, TestCallbacks callbacks) throws Exception {
+
+    //Create some random words. We need to be careful
+    //to make sure we get NUM_WORDS distinct words here
+    Set<String> wordSet = new HashSet<String>();
+    Random rand = new Random();
+    while(wordSet.size() < NUM_WORDS) {
+      int length = rand.nextInt(12) + 3;
+      char[] text = new char[length];
+      for(int i = 0; i < length; i++) {
+        text[i] = (char) (rand.nextInt(26) + 97);
+      }
+      wordSet.add(new String(text));
+    }
+    List<String> words = new ArrayList<String>(wordSet.size());
+    words.addAll(wordSet);
+    
+    
+    
+    //warm up
+    writeRandomWords(callbacks, words, rand, NUM_ENTRIES / 10, NUM_QUERIES / 10, COMMIT_INTERVAL[0]);
+    
+    //Do the actual test
+    
+    for(int i = 0; i < COMMIT_INTERVAL.length; i++) {
+      Results results = writeRandomWords(callbacks, words, rand, NUM_ENTRIES, NUM_QUERIES / 10, COMMIT_INTERVAL[i]);
+    
+      System.out.println(testName + " writes(entries=" + NUM_ENTRIES + ", commit=" + COMMIT_INTERVAL[i] + "): " + TimeUnit.NANOSECONDS.toMillis(results.writeTime));
+      System.out.println(testName + " queries(entries=" + NUM_ENTRIES + ", commit=" + COMMIT_INTERVAL[i] + "): " + TimeUnit.NANOSECONDS.toMillis(results.queryTime));
+    }
+  }
+
+  private Results writeRandomWords(TestCallbacks callbacks, List<String> words,
+      Random rand, int numEntries, int numQueries, int commitInterval) throws Exception {
+    Results results  = new Results();
+    callbacks.init();
+    int[] counts = new int[words.size()];
+    long start = System.nanoTime();
+    try {
+      for(int i =0; i < numEntries; i++) {
+        int word1 = rand.nextInt(words.size());
+        int word2 = rand.nextInt(words.size());
+        counts[word1]++;
+        counts[word2]++;
+        String value = words.get(word1) + " " + words.get(word2);
+        callbacks.addObject("key" + i, value);
+
+        if(i % commitInterval == 0 && i != 0) {
+          callbacks.commit();
+        }
+      }
+      callbacks.commit();
+      callbacks.waitForAsync();
+      long end = System.nanoTime();
+      results.writeTime = end - start;
+      
+      
+      start = System.nanoTime();
+      for(int i=0; i < numQueries; i++) {
+        int wordIndex = rand.nextInt(words.size());
+        String word = words.get(wordIndex);
+        Query query = new TermQuery(new Term("text", word));
+        int size  = callbacks.query(query);
+//        int size  = callbacks.query(parser.parse(word));
+        //All of my tests sometimes seem to be missing a couple of words, including the stock lucene
+//        assertEquals("Error on query " + i + " word=" + word, counts[wordIndex], size);
+      }
+      end = System.nanoTime();
+      results.queryTime = end - start;
+      
+      return results;
+    } finally {
+      callbacks.cleanup();
+    }
+  }
+
+  private static class TestObject implements DataSerializable {
+    private String text;
+
+    public TestObject() {
+      
+    }
+    
+    public TestObject(String text) {
+      super();
+      this.text = text;
+    }
+
+    @Override
+    public void toData(DataOutput out) throws IOException {
+      DataSerializer.writeString(text, out);
+    }
+
+    @Override
+    public void fromData(DataInput in)
+        throws IOException, ClassNotFoundException {
+      text = DataSerializer.readString(in);
+    }
+
+    @Override
+    public String toString() {
+      return text;
+    }
+    
+    
+  }
+  
+  private interface TestCallbacks {
+    public void init() throws Exception;
+    public int query(Query query) throws Exception;
+    public void addObject(String key, String text)  throws Exception;
+    public void commit() throws Exception;
+    public void waitForAsync() throws Exception;
+    public void cleanup() throws Exception;
+  }
+  
+  private static class Results {
+    long writeTime;
+    long queryTime;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb7dbd0b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
deleted file mode 100644
index 74f3742..0000000
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
+++ /dev/null
@@ -1,439 +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.cache.lucene.internal.repository;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.SearcherManager;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.store.RAMDirectory;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.DataSerializable;
-import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.PartitionAttributesFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
-import com.gemstone.gemfire.cache.lucene.LuceneIndex;
-import com.gemstone.gemfire.cache.lucene.LuceneQuery;
-import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
-import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
-import com.gemstone.gemfire.cache.lucene.LuceneService;
-import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
-import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
-import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
-import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
-import com.gemstone.gemfire.cache.query.QueryException;
-import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
-
-/**
- * Microbenchmark of the IndexRepository to compare an
- * IndexRepository built on top of cache with a 
- * stock lucene IndexWriter with a RAMDirectory.
- */
-@Category(PerformanceTest.class)
-@Ignore("Tests have no assertions")
-public class IndexRepositoryImplPerformanceTest {
-  
-  private static final int NUM_WORDS = 1000;
-  private static int[] COMMIT_INTERVAL = new int[] {100, 1000, 5000};
-  private static int NUM_ENTRIES = 500_000;
-  private static int NUM_QUERIES = 500_000;
-
-  private StandardAnalyzer analyzer = new StandardAnalyzer();
-  
-  @Test
-  public  void testIndexRepository() throws Exception {
-    
-
-    doTest("IndexRepository", new TestCallbacks() {
-
-      private Cache cache;
-      private IndexRepositoryImpl repo;
-      private IndexWriter writer;
-
-      @Override
-      public void addObject(String key, String text) throws Exception {
-        repo.create(key, new TestObject(text));
-      }
-
-      @Override
-      public void commit()  throws Exception {
-        repo.commit();
-      }
-
-      @Override
-      public void init() throws Exception {
-        cache = new CacheFactory().set("mcast-port", "0")
-            .set("log-level", "error")
-            .create();
-        Region<String, File> fileRegion = cache.<String, File>createRegionFactory(RegionShortcut.REPLICATE).create("files");
-        Region<ChunkKey, byte[]> chunkRegion = cache.<ChunkKey, byte[]>createRegionFactory(RegionShortcut.REPLICATE).create("chunks");
-
-        RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion);
-        
-        
-        IndexWriterConfig config = new IndexWriterConfig(analyzer);
-        writer = new IndexWriter(dir, config);
-        String[] indexedFields= new String[] {"text"};
-        HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(indexedFields);
-        repo = new IndexRepositoryImpl(fileRegion, writer, mapper);
-      }
-
-      @Override
-      public void cleanup() throws IOException {
-        writer.close();
-        cache.close();
-      }
-
-      @Override
-      public void waitForAsync() throws Exception {
-        //do nothing
-      }
-
-      @Override
-      public int query(Query query) throws IOException {
-        TopEntriesCollector collector = new TopEntriesCollector();
-        repo.query(query, 100, collector);
-        return collector.size();
-      }
-    });
-  }
-  
-  /**
-   * Test our full lucene index implementation
-   * @throws Exception
-   */
-  @Test
-  public void testLuceneIndex() throws Exception {
-    
-
-    doTest("LuceneIndex", new TestCallbacks() {
-
-      private Cache cache;
-      private Region<String, TestObject> region;
-      private LuceneService service;
-
-      @Override
-      public void addObject(String key, String text) throws Exception {
-        region.create(key, new TestObject(text));
-      }
-
-      @Override
-      public void commit()  throws Exception {
-        //NA
-      }
-
-      @Override
-      public void init() throws Exception {
-        cache = new CacheFactory().set("mcast-port", "0")
-            .set("log-level", "warning")
-            .create();
-        service = LuceneServiceProvider.get(cache);
-        service.createIndex("index", "/region", "text");
-        region = cache.<String, TestObject>createRegionFactory(RegionShortcut.PARTITION)
-            .setPartitionAttributes(new PartitionAttributesFactory<>().setTotalNumBuckets(1).create())
-            .create("region");
-      }
-
-      @Override
-      public void cleanup() throws IOException {
-        cache.close();
-      }
-      
-      @Override
-      public void waitForAsync() throws Exception {
-        AsyncEventQueue aeq = cache.getAsyncEventQueue(LuceneServiceImpl.getUniqueIndexName("index", "/region"));
-        
-        //We will be at most 10 ms off
-        while(aeq.size() > 0) {
-          Thread.sleep(10);
-        }
-      }
-
-      @Override
-      public int query(final Query query) throws Exception {
-        LuceneQuery<Object, Object> luceneQuery = service.createLuceneQueryFactory().create("index", "/region", new LuceneQueryProvider() {
-          
-          @Override
-          public Query getQuery(LuceneIndex index) throws QueryException {
-            return query;
-          }
-        });
-        
-        LuceneQueryResults<Object, Object> results = luceneQuery.search();
-        return results.size();
-      }
-    });
-  }
-  
-  @Test
-  public  void testLuceneWithRegionDirectory() throws Exception {
-    doTest("RegionDirectory", new TestCallbacks() {
-
-      private IndexWriter writer;
-      private SearcherManager searcherManager;
-
-      @Override
-      public void init() throws Exception {
-        RegionDirectory dir = new RegionDirectory(new ConcurrentHashMap<String, File>(), new ConcurrentHashMap<ChunkKey, byte[]>());
-        IndexWriterConfig config = new IndexWriterConfig(analyzer);
-        writer = new IndexWriter(dir, config);
-        searcherManager = new SearcherManager(writer, true, null);
-      }
-
-      @Override
-      public void addObject(String key, String text) throws Exception {
-        Document doc = new Document();
-        doc.add(new TextField("key", key, Store.YES));
-        doc.add(new TextField("text", text, Store.NO));
-        writer.addDocument(doc);
-      }
-
-      @Override
-      public void commit() throws Exception {
-        writer.commit();
-        searcherManager.maybeRefresh();
-      }
-
-      @Override
-      public void cleanup() throws Exception {
-        writer.close();
-      }
-      
-      @Override
-      public void waitForAsync() throws Exception {
-        //do nothing
-      }
-
-      @Override
-      public int query(Query query) throws Exception {
-        IndexSearcher searcher = searcherManager.acquire();
-        try {
-          return searcher.count(query);
-        } finally {
-          searcherManager.release(searcher);
-        }
-      }
-      
-    });
-    
-  }
-  
-  @Test
-  public  void testLucene() throws Exception {
-    doTest("Lucene", new TestCallbacks() {
-
-      private IndexWriter writer;
-      private SearcherManager searcherManager;
-
-      @Override
-      public void init() throws Exception {
-        RAMDirectory dir = new RAMDirectory();
-        IndexWriterConfig config = new IndexWriterConfig(analyzer);
-        writer = new IndexWriter(dir, config);
-        searcherManager = new SearcherManager(writer, true, null);
-      }
-
-      @Override
-      public void addObject(String key, String text) throws Exception {
-        Document doc = new Document();
-        doc.add(new TextField("key", key, Store.YES));
-        doc.add(new TextField("text", text, Store.NO));
-        writer.addDocument(doc);
-      }
-
-      @Override
-      public void commit() throws Exception {
-        writer.commit();
-        searcherManager.maybeRefresh();
-      }
-
-      @Override
-      public void cleanup() throws Exception {
-        writer.close();
-      }
-      
-      @Override
-      public void waitForAsync() throws Exception {
-        //do nothing
-      }
-
-      @Override
-      public int query(Query query) throws Exception {
-        IndexSearcher searcher = searcherManager.acquire();
-        try {
-          return searcher.count(query);
-        } finally {
-          searcherManager.release(searcher);
-        }
-      }
-      
-    });
-    
-  }
-  
-  private void doTest(String testName, TestCallbacks callbacks) throws Exception {
-
-    //Create some random words. We need to be careful
-    //to make sure we get NUM_WORDS distinct words here
-    Set<String> wordSet = new HashSet<String>();
-    Random rand = new Random();
-    while(wordSet.size() < NUM_WORDS) {
-      int length = rand.nextInt(12) + 3;
-      char[] text = new char[length];
-      for(int i = 0; i < length; i++) {
-        text[i] = (char) (rand.nextInt(26) + 97);
-      }
-      wordSet.add(new String(text));
-    }
-    List<String> words = new ArrayList<String>(wordSet.size());
-    words.addAll(wordSet);
-    
-    
-    
-    //warm up
-    writeRandomWords(callbacks, words, rand, NUM_ENTRIES / 10, NUM_QUERIES / 10, COMMIT_INTERVAL[0]);
-    
-    //Do the actual test
-    
-    for(int i = 0; i < COMMIT_INTERVAL.length; i++) {
-      Results results = writeRandomWords(callbacks, words, rand, NUM_ENTRIES, NUM_QUERIES / 10, COMMIT_INTERVAL[i]);
-    
-      System.out.println(testName + " writes(entries=" + NUM_ENTRIES + ", commit=" + COMMIT_INTERVAL[i] + "): " + TimeUnit.NANOSECONDS.toMillis(results.writeTime));
-      System.out.println(testName + " queries(entries=" + NUM_ENTRIES + ", commit=" + COMMIT_INTERVAL[i] + "): " + TimeUnit.NANOSECONDS.toMillis(results.queryTime));
-    }
-  }
-
-  private Results writeRandomWords(TestCallbacks callbacks, List<String> words,
-      Random rand, int numEntries, int numQueries, int commitInterval) throws Exception {
-    Results results  = new Results();
-    callbacks.init();
-    int[] counts = new int[words.size()];
-    long start = System.nanoTime();
-    try {
-      for(int i =0; i < numEntries; i++) {
-        int word1 = rand.nextInt(words.size());
-        int word2 = rand.nextInt(words.size());
-        counts[word1]++;
-        counts[word2]++;
-        String value = words.get(word1) + " " + words.get(word2);
-        callbacks.addObject("key" + i, value);
-
-        if(i % commitInterval == 0 && i != 0) {
-          callbacks.commit();
-        }
-      }
-      callbacks.commit();
-      callbacks.waitForAsync();
-      long end = System.nanoTime();
-      results.writeTime = end - start;
-      
-      
-      start = System.nanoTime();
-      for(int i=0; i < numQueries; i++) {
-        int wordIndex = rand.nextInt(words.size());
-        String word = words.get(wordIndex);
-        Query query = new TermQuery(new Term("text", word));
-        int size  = callbacks.query(query);
-//        int size  = callbacks.query(parser.parse(word));
-        //All of my tests sometimes seem to be missing a couple of words, including the stock lucene
-//        assertEquals("Error on query " + i + " word=" + word, counts[wordIndex], size);
-      }
-      end = System.nanoTime();
-      results.queryTime = end - start;
-      
-      return results;
-    } finally {
-      callbacks.cleanup();
-    }
-  }
-
-  private static class TestObject implements DataSerializable {
-    private String text;
-
-    public TestObject() {
-      
-    }
-    
-    public TestObject(String text) {
-      super();
-      this.text = text;
-    }
-
-    @Override
-    public void toData(DataOutput out) throws IOException {
-      DataSerializer.writeString(text, out);
-    }
-
-    @Override
-    public void fromData(DataInput in)
-        throws IOException, ClassNotFoundException {
-      text = DataSerializer.readString(in);
-    }
-
-    @Override
-    public String toString() {
-      return text;
-    }
-    
-    
-  }
-  
-  private interface TestCallbacks {
-    public void init() throws Exception;
-    public int query(Query query) throws Exception;
-    public void addObject(String key, String text)  throws Exception;
-    public void commit() throws Exception;
-    public void waitForAsync() throws Exception;
-    public void cleanup() throws Exception;
-  }
-  
-  private static class Results {
-    long writeTime;
-    long queryTime;
-  }
-}


[34/50] [abbrv] incubator-geode git commit: GEODE-739: CI failure in LocatorJUnitTest.testNoThreadLeftBehind

Posted by ab...@apache.org.
GEODE-739: CI failure in LocatorJUnitTest.testNoThreadLeftBehind

The locator needs to shut down its TCPServer thread if an exception is
thrown during startup.


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

Branch: refs/heads/master
Commit: 08262a8544d349482abbc758b44921a3baf92f6b
Parents: 3e5ade7
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Thu Jan 7 09:36:25 2016 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Thu Jan 7 09:36:52 2016 -0800

----------------------------------------------------------------------
 .../gemfire/distributed/internal/InternalLocator.java         | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/08262a85/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
index 9d28d1c..50a0fa9 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
@@ -418,7 +418,12 @@ public class InternalLocator extends Locator implements ConnectListener {
 //  }
     InternalDistributedSystem sys = InternalDistributedSystem.getConnectedInstance();
     if (sys != null) {
-      slocator.startServerLocation(sys);
+      try {
+        slocator.startServerLocation(sys);
+      } catch (RuntimeException e) {
+        slocator.stop();
+        throw e;
+      }
     }
     
     slocator.endStartLocator(null);


[06/50] [abbrv] incubator-geode git commit: GEODE-718: Sanitize passwords in gfsh history file

Posted by ab...@apache.org.
GEODE-718: Sanitize passwords in gfsh history file


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

Branch: refs/heads/master
Commit: 9bca880bfce2af96c48dc6a36c0c6573431f6345
Parents: ca6148a
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed Dec 30 10:47:33 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Jan 4 09:44:27 2016 -0800

----------------------------------------------------------------------
 .../internal/cli/shell/jline/GfshHistory.java   | 11 ++-
 .../cli/shell/GfshHistoryJUnitTest.java         | 88 ++++++++++++++++++++
 2 files changed, 98 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9bca880b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/GfshHistory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/GfshHistory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/GfshHistory.java
index dc3fbe1..5d0d8dc 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/GfshHistory.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/GfshHistory.java
@@ -22,6 +22,8 @@ import jline.console.history.MemoryHistory;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 /**
  * Overrides jline.History to add History without newline characters.
@@ -31,6 +33,9 @@ import java.io.IOException;
  */
 public class GfshHistory extends MemoryHistory {
 
+  // Pattern which is intended to pick up any params containing the word 'password'.
+  private static final Pattern passwordRe = Pattern.compile("(--[^=\\s]*password[^=\\s]*\\s*=\\s*)([^\\s]*)");
+
   // let the history from history file get added initially
   private boolean autoFlush = true;
 
@@ -49,6 +54,10 @@ public class GfshHistory extends MemoryHistory {
   }
   
   public static String toHistoryLoggable(String buffer) {
-    return PreprocessorUtils.trim(buffer, false).getString();
+    String trimmed = PreprocessorUtils.trim(buffer, false).getString();
+
+    Matcher matcher = passwordRe.matcher(trimmed);
+    String sanitized = matcher.replaceAll("$1*****");
+    return sanitized;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9bca880b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
new file mode 100644
index 0000000..77e8785
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.shell;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.nio.file.Files;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * @author Jens Deppe
+ */
+@Category(UnitTest.class)
+public class GfshHistoryJUnitTest {
+
+  private File gfshHistoryFile;
+
+  private GfshConfig gfshConfig;
+
+  @Rule
+  public TemporaryFolder tempDirectory = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    gfshHistoryFile = tempDirectory.newFile("historyFile");
+    gfshConfig = new GfshConfig(gfshHistoryFile.getAbsolutePath(),
+        "",                                         // defaultPrompt
+        0,                                          // historySize
+        tempDirectory.getRoot().getAbsolutePath(),  // logDir
+        null,                                       // logLevel
+        null,                                       // logLimit
+        null,                                       // logCount
+        null                                        // initFileName
+    );
+  }
+
+  @After
+  public void teardown() throws Exception {
+    // Null out static instance so Gfsh can be reinitialised
+    Field gfshInstance = Gfsh.class.getDeclaredField("instance");
+    gfshInstance.setAccessible(true);
+    gfshInstance.set(null, null);
+  }
+
+  @Test
+  public void testHistoryFileIsCreated() throws Exception {
+    Gfsh gfsh = Gfsh.getInstance(false, new String[] {}, gfshConfig);
+    gfsh.executeScriptLine("connect --fake-param=foo");
+
+    List<String> lines = Files.readAllLines(gfshHistoryFile.toPath());
+    assertEquals(2, lines.size());
+    assertEquals(lines.get(1), "// [failed] connect --fake-param=foo");
+  }
+
+  @Test
+  public void testHistoryFileDoesNotContainPasswords() throws Exception {
+    Gfsh gfsh = Gfsh.getInstance(false, new String[] {}, gfshConfig);
+    gfsh.executeScriptLine("connect --password=foo --password = foo --password= goo --password =goo --password-param=blah --other-password-param=    gah");
+
+    List<String> lines = Files.readAllLines(gfshHistoryFile.toPath());
+    assertEquals("// [failed] connect --password=***** --password = ***** --password= ***** --password =***** --password-param=***** --other-password-param= *****",
+        lines.get(1));
+  }
+}


[10/50] [abbrv] incubator-geode git commit: GEODE-715: Fix import that was broken in merge to develop

Posted by ab...@apache.org.
GEODE-715: Fix import that was broken in merge to develop


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

Branch: refs/heads/master
Commit: 18a17d787bcf69179d1bdeb015f33141d5d8486b
Parents: c57a88c
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Jan 4 14:14:12 2016 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Mon Jan 4 14:14:12 2016 -0800

----------------------------------------------------------------------
 gemfire-core/src/test/java/dunit/DistributedTestCase.java | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/18a17d78/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 6fa560f..e89f694 100755
--- a/gemfire-core/src/test/java/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
@@ -88,8 +88,7 @@ import com.gemstone.gemfire.internal.logging.ManagerLogWriter;
 import com.gemstone.gemfire.internal.logging.log4j.LogWriterLogger;
 import com.gemstone.gemfire.management.internal.cli.LogWrapper;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-
-import dunit.standalone.DUnitLauncher;
+import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 
 /**
  * This class is the superclass of all distributed unit tests.


[22/50] [abbrv] incubator-geode git commit: GEODE-574: for --ciphers and --protocols options, if the value is "any", it should behave the same as if they were not specified (default value)

Posted by ab...@apache.org.
GEODE-574: for --ciphers and --protocols options, if the value is "any", it should behave the same as if they were not specified (default value)

Closes #65


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

Branch: refs/heads/master
Commit: f36310d78df388a926895169de7ac16ebdb07493
Parents: 8b6d8f2
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue Dec 22 14:17:46 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Wed Jan 6 09:18:46 2016 -0800

----------------------------------------------------------------------
 .../internal/cli/shell/JmxOperationInvoker.java           | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f36310d7/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
index e456bda..c2a1b2f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
@@ -114,12 +114,14 @@ public class JmxOperationInvoker implements OperationInvoker {
       Set<Entry<String, String>> entrySet = sslConfigProps.entrySet();
       for (Iterator<Entry<String, String>> it = entrySet.iterator(); it.hasNext();) {
         Entry<String, String> entry = it.next();
-        String key = entry.getKey();        
+        String key = entry.getKey();
+        String value = entry.getValue();
         if (key.startsWith("javax.") || key.startsWith("cluster-ssl") || key.startsWith("jmx-manager-ssl") ) {
           key =  checkforSystemPropertyPrefix(entry.getKey());
-          //TODO : If protocol is any lookup and set one of the following : Copied from SocketCreator 
-          // String[] knownAlgorithms = {"SSL", "SSLv2", "SSLv3", "TLS", "TLSv1", "TLSv1.1"};
-          System.setProperty(key, entry.getValue());
+          if((key.equals(Gfsh.SSL_ENABLED_CIPHERS) || key.equals(Gfsh.SSL_ENABLED_PROTOCOLS)) && "any".equals(value)){
+            continue;
+          }
+          System.setProperty(key, value);
           propsToClear.add(key);
         }
       }


[35/50] [abbrv] incubator-geode git commit: GEODE-401: upgrade log4j from 2.1 to 2.5. Remove our own configuration watcher since log4j now has an almost the same implementation.

Posted by ab...@apache.org.
GEODE-401: upgrade log4j from 2.1 to 2.5. Remove our own configuration watcher since log4j now has an almost the same implementation.


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

Branch: refs/heads/master
Commit: a097fcf32cb20f2258637b1e1f6829c632a89e46
Parents: 08262a8
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Wed Jan 6 09:02:44 2016 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Thu Jan 7 09:46:21 2016 -0800

----------------------------------------------------------------------
 .../gemfire/internal/logging/LogService.java    |  48 ---
 .../internal/logging/log4j/LogWriterLogger.java |   7 -
 .../config/GemFireFileConfigurationMonitor.java | 145 --------
 .../config/xml/GemFireXmlConfiguration.java     | 344 -------------------
 .../xml/GemFireXmlConfigurationFactory.java     |  59 ----
 .../CompressionCacheConfigDUnitTest.java        |   1 +
 .../logging/log4j/Log4J2PerformanceTest.java    |  11 -
 .../log4j/LogWriterLoggerPerformanceTest.java   |  11 -
 .../java/dunit/standalone/DUnitLauncher.java    |   2 +-
 gradle/dependency-versions.properties           |   2 +-
 10 files changed, 3 insertions(+), 627 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a097fcf3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
index 8159966..7eeb26c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
@@ -18,71 +18,23 @@ package com.gemstone.gemfire.internal.logging;
 
 import java.beans.PropertyChangeEvent;
 import java.beans.PropertyChangeListener;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.transform.Source;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import javax.xml.validation.Validator;
 
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.core.Appender;
-import org.apache.logging.log4j.core.Filter.Result;
 import org.apache.logging.log4j.core.LoggerContext;
-import org.apache.logging.log4j.core.config.AppenderRef;
 import org.apache.logging.log4j.core.config.Configuration;
-import org.apache.logging.log4j.core.config.ConfigurationFactory;
-import org.apache.logging.log4j.core.config.ConfigurationSource;
-import org.apache.logging.log4j.core.config.FileConfigurationMonitor;
 import org.apache.logging.log4j.core.config.LoggerConfig;
-import org.apache.logging.log4j.core.config.Node;
-import org.apache.logging.log4j.core.config.status.StatusConfiguration;
-import org.apache.logging.log4j.core.config.xml.XmlConfiguration;
-import org.apache.logging.log4j.core.filter.AbstractFilterable;
-import org.apache.logging.log4j.core.lookup.Interpolator;
 import org.apache.logging.log4j.core.lookup.StrLookup;
 import org.apache.logging.log4j.core.lookup.StrSubstitutor;
-import org.apache.logging.log4j.core.util.Closer;
-import org.apache.logging.log4j.core.util.Loader;
-import org.apache.logging.log4j.core.util.Patterns;
 import org.apache.logging.log4j.status.StatusLogger;
-import org.apache.logging.log4j.util.PropertiesUtil;
-import org.w3c.dom.Attr;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NamedNodeMap;
-import org.xml.sax.InputSource;
-import org.xml.sax.SAXException;
-
-import com.gemstone.gemfire.InternalGemFireError;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.logging.log4j.AppenderContext;
 import com.gemstone.gemfire.internal.logging.log4j.ConfigLocator;
 import com.gemstone.gemfire.internal.logging.log4j.Configurator;
 import com.gemstone.gemfire.internal.logging.log4j.FastLogger;
-import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.logging.log4j.LogWriterLogger;
-import com.gemstone.gemfire.internal.util.IOUtils;
-import com.gemstone.org.apache.logging.log4j.core.config.xml.GemFireXmlConfiguration;
-import com.gemstone.org.apache.logging.log4j.core.config.xml.GemFireXmlConfigurationFactory;
 import com.gemstone.org.apache.logging.log4j.message.GemFireParameterizedMessageFactory;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a097fcf3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLogger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLogger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLogger.java
index c85b3f7..1fb907d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLogger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLogger.java
@@ -22,10 +22,6 @@ import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.Marker;
-import org.apache.logging.log4j.core.Appender;
-import org.apache.logging.log4j.core.LoggerContext;
-import org.apache.logging.log4j.core.config.Configuration;
-import org.apache.logging.log4j.core.config.LoggerConfig;
 import org.apache.logging.log4j.message.Message;
 import org.apache.logging.log4j.spi.AbstractLogger;
 import org.apache.logging.log4j.spi.ExtendedLoggerWrapper;
@@ -36,9 +32,6 @@ import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.logging.GemFireHandler;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.logging.LogWriterFactory;
-import com.gemstone.gemfire.internal.logging.LogWriterImpl;
-import com.gemstone.org.apache.logging.log4j.core.config.xml.GemFireXmlConfiguration;
 import com.gemstone.org.apache.logging.log4j.message.GemFireParameterizedMessageFactory;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a097fcf3/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/GemFireFileConfigurationMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/GemFireFileConfigurationMonitor.java b/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/GemFireFileConfigurationMonitor.java
deleted file mode 100644
index e6b2bb4..0000000
--- a/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/GemFireFileConfigurationMonitor.java
+++ /dev/null
@@ -1,145 +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.org.apache.logging.log4j.core.config;
-
-import java.io.File;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.logging.log4j.core.config.ConfigurationListener;
-import org.apache.logging.log4j.core.config.ConfigurationMonitor;
-import org.apache.logging.log4j.core.config.Reconfigurable;
-
-/**
- * Configuration monitor that periodically checks the timestamp of the configuration file and calls the
- * ConfigurationListeners when an update occurs.
- */
-public class GemFireFileConfigurationMonitor implements ConfigurationMonitor {
-
-    static {
-        //System.out.println("Using GemFireFileConfigurationMonitor");
-    }
-  
-    private static boolean DISABLED = true;
-    private static boolean OPTIMIZED = true;
-  
-    private static final int MASK = 0x0f;
-
-    private static final int MIN_INTERVAL = 5;
-
-    private static final int MILLIS_PER_SECOND = 1000;
-
-    private final File file;
-
-    private long lastModified;
-
-    private final List<ConfigurationListener> listeners;
-
-    private final int interval;
-
-    private long nextCheck;
-
-    private final AtomicInteger counter = new AtomicInteger(0);
-
-    private static final Lock LOCK = new ReentrantLock();
-
-    private final Reconfigurable reconfigurable;
-
-    /**
-     * Constructor.
-     * @param reconfigurable The Configuration that can be reconfigured.
-     * @param file The File to monitor.
-     * @param listeners The List of ConfigurationListeners to notify upon a change.
-     * @param interval The monitor interval in seconds. The minimum interval is 5 seconds.
-     */
-    public GemFireFileConfigurationMonitor(final Reconfigurable reconfigurable, final File file,
-                                    final List<ConfigurationListener> listeners,
-                                    final int interval) {
-        this.reconfigurable = reconfigurable;
-        this.file = file;
-        this.lastModified = file.lastModified();
-        this.listeners = listeners;
-        this.interval = (interval < MIN_INTERVAL ? MIN_INTERVAL : interval) * MILLIS_PER_SECOND;
-        this.nextCheck = System.currentTimeMillis() + interval;
-    }
-
-    /**
-     * Called to determine if the configuration has changed.
-     */
-    @Override
-    public void checkConfiguration() {
-        if (DISABLED) {
-          return;
-        } else if (OPTIMIZED) {
-            if (((counter.incrementAndGet() & MASK) == 0)) {
-                final long current = System.currentTimeMillis();
-                if (current >= nextCheck) {
-                    LOCK.lock();
-                    try {
-                        nextCheck = current + interval;
-                        if (file.lastModified() > lastModified) {
-                            lastModified = file.lastModified();
-                            for (final ConfigurationListener listener : listeners) {
-                                final Thread thread = new Thread(new ReconfigurationWorker(listener, reconfigurable));
-                                thread.setDaemon(true);
-                                thread.start();
-                            }
-                        }
-                    } finally {
-                        LOCK.unlock();
-                    }
-                }
-            }
-        } else {
-            final long current = System.currentTimeMillis();
-            if (((counter.incrementAndGet() & MASK) == 0) && (current >= nextCheck)) {
-                LOCK.lock();
-                try {
-                    nextCheck = current + interval;
-                    if (file.lastModified() > lastModified) {
-                        lastModified = file.lastModified();
-                        for (final ConfigurationListener listener : listeners) {
-                            final Thread thread = new Thread(new ReconfigurationWorker(listener, reconfigurable));
-                            thread.setDaemon(true);
-                            thread.start();
-                        }
-                    }
-                } finally {
-                    LOCK.unlock();
-                }
-            }
-        }
-    }
-
-    private class ReconfigurationWorker implements Runnable {
-
-        private final ConfigurationListener listener;
-        private final Reconfigurable reconfigurable;
-
-        public ReconfigurationWorker(final ConfigurationListener listener, final Reconfigurable reconfigurable) {
-            this.listener = listener;
-            this.reconfigurable = reconfigurable;
-        }
-
-        @Override
-        public void run() {
-            listener.onChange(reconfigurable);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a097fcf3/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/xml/GemFireXmlConfiguration.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/xml/GemFireXmlConfiguration.java b/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/xml/GemFireXmlConfiguration.java
deleted file mode 100644
index e31db04..0000000
--- a/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/xml/GemFireXmlConfiguration.java
+++ /dev/null
@@ -1,344 +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.org.apache.logging.log4j.core.config.xml;
-
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.transform.Source;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import javax.xml.validation.Validator;
-
-import org.apache.logging.log4j.core.config.AbstractConfiguration;
-import org.apache.logging.log4j.core.config.Configuration;
-import org.apache.logging.log4j.core.config.ConfigurationSource;
-import org.apache.logging.log4j.core.config.Node;
-import org.apache.logging.log4j.core.config.Reconfigurable;
-import org.apache.logging.log4j.core.config.plugins.util.PluginManager;
-import org.apache.logging.log4j.core.config.plugins.util.PluginType;
-import org.apache.logging.log4j.core.config.plugins.util.ResolverUtil;
-import org.apache.logging.log4j.core.config.status.StatusConfiguration;
-import org.apache.logging.log4j.core.util.Closer;
-import org.apache.logging.log4j.core.util.Loader;
-import org.apache.logging.log4j.core.util.Patterns;
-import org.w3c.dom.Attr;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NamedNodeMap;
-import org.w3c.dom.NodeList;
-import org.w3c.dom.Text;
-import org.xml.sax.InputSource;
-import org.xml.sax.SAXException;
-
-/**
- * Creates a Node hierarchy from an XML file.
- */
-public class GemFireXmlConfiguration extends AbstractConfiguration implements Reconfigurable {
-
-    static {
-        //System.out.println("Using GemFireXmlConfiguration");
-    }
-  
-    private static final String XINCLUDE_FIXUP_LANGUAGE = "http://apache.org/xml/features/xinclude/fixup-language";
-    private static final String XINCLUDE_FIXUP_BASE_URIS = "http://apache.org/xml/features/xinclude/fixup-base-uris";
-    private static final String[] VERBOSE_CLASSES = new String[] { ResolverUtil.class.getName() };
-    private static final String LOG4J_XSD = "Log4j-config.xsd";
-
-    private final List<Status> status = new ArrayList<Status>();
-    private Element rootElement;
-    private boolean strict;
-    private String schema;
-
-    /**
-     * Creates a new DocumentBuilder suitable for parsing a configuration file.
-     *
-     * @return a new DocumentBuilder
-     * @throws ParserConfigurationException
-     */
-    static DocumentBuilder newDocumentBuilder() throws ParserConfigurationException {
-        final DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
-        factory.setNamespaceAware(true);
-        enableXInclude(factory);
-        return factory.newDocumentBuilder();
-    }
-
-    /**
-     * Enables XInclude for the given DocumentBuilderFactory
-     *
-     * @param factory a DocumentBuilderFactory
-     */
-    private static void enableXInclude(final DocumentBuilderFactory factory) {
-        try {
-            // Alternative: We set if a system property on the command line is set, for example:
-            // -DLog4j.XInclude=true
-            factory.setXIncludeAware(true);
-        } catch (final UnsupportedOperationException e) {
-            LOGGER.warn("The DocumentBuilderFactory does not support XInclude: {}", factory, e);
-        } catch (@SuppressWarnings("ErrorNotRethrown") final AbstractMethodError err) {
-            LOGGER.warn("The DocumentBuilderFactory is out of date and does not support XInclude: {}", factory, err);
-        }
-        try {
-            // Alternative: We could specify all features and values with system properties like:
-            // -DLog4j.DocumentBuilderFactory.Feature="http://apache.org/xml/features/xinclude/fixup-base-uris true"
-            factory.setFeature(XINCLUDE_FIXUP_BASE_URIS, true);
-        } catch (final ParserConfigurationException e) {
-            LOGGER.warn("The DocumentBuilderFactory [{}] does not support the feature [{}].", factory,
-                    XINCLUDE_FIXUP_BASE_URIS, e);
-        } catch (@SuppressWarnings("ErrorNotRethrown") final AbstractMethodError err) {
-            LOGGER.warn("The DocumentBuilderFactory is out of date and does not support setFeature: {}", factory, err);
-        }
-        try {
-            factory.setFeature(XINCLUDE_FIXUP_LANGUAGE, true);
-        } catch (final ParserConfigurationException e) {
-            LOGGER.warn("The DocumentBuilderFactory [{}] does not support the feature [{}].", factory,
-                    XINCLUDE_FIXUP_LANGUAGE, e);
-        } catch (@SuppressWarnings("ErrorNotRethrown") final AbstractMethodError err) {
-            LOGGER.warn("The DocumentBuilderFactory is out of date and does not support setFeature: {}", factory, err);
-        }
-    }
-
-    public GemFireXmlConfiguration(final ConfigurationSource configSource) {
-        super(configSource);
-        final File configFile = configSource.getFile();
-        byte[] buffer = null;
-
-        try {
-            final InputStream configStream = configSource.getInputStream();
-            try {
-                buffer = toByteArray(configStream);
-            } finally {
-                Closer.closeSilently(configStream);
-            }
-            final InputSource source = new InputSource(new ByteArrayInputStream(buffer));
-            source.setSystemId(configSource.getLocation());
-            final Document document = newDocumentBuilder().parse(source);
-            rootElement = document.getDocumentElement();
-            final Map<String, String> attrs = processAttributes(rootNode, rootElement);
-            final StatusConfiguration statusConfig = new StatusConfiguration().withVerboseClasses(VERBOSE_CLASSES)
-                    .withStatus(getDefaultStatus());
-            for (final Map.Entry<String, String> entry : attrs.entrySet()) {
-                final String key = entry.getKey();
-                final String value = getStrSubstitutor().replace(entry.getValue());
-                if ("status".equalsIgnoreCase(key)) {
-                    statusConfig.withStatus(value);
-                } else if ("dest".equalsIgnoreCase(key)) {
-                    statusConfig.withDestination(value);
-                } else if ("shutdownHook".equalsIgnoreCase(key)) {
-                    isShutdownHookEnabled = !"disable".equalsIgnoreCase(value);
-                } else if ("verbose".equalsIgnoreCase(key)) {
-                    statusConfig.withVerbosity(value);
-                } else if ("packages".equalsIgnoreCase(key)) {
-                    PluginManager.addPackages(Arrays.asList(value.split(Patterns.COMMA_SEPARATOR)));
-                } else if ("name".equalsIgnoreCase(key)) {
-                    setName(value);
-                } else if ("strict".equalsIgnoreCase(key)) {
-                    strict = Boolean.parseBoolean(value);
-                } else if ("schema".equalsIgnoreCase(key)) {
-                    schema = value;
-                } else if ("monitorInterval".equalsIgnoreCase(key)) {
-                    final int interval = Integer.parseInt(value);
-                    if (interval > 0 && configFile != null) {
-                        // GEMFIRE: this is where we can inject our own asynchronous background thread for monitor config file changes
-                        monitor = new com.gemstone.org.apache.logging.log4j.core.config.GemFireFileConfigurationMonitor(this, configFile, listeners, interval);
-                    }
-                } else if ("advertiser".equalsIgnoreCase(key)) {
-                    createAdvertiser(value, configSource, buffer, "text/xml");
-                }
-            }
-            statusConfig.initialize();
-        } catch (final SAXException domEx) {
-            LOGGER.error("Error parsing {}", configSource.getLocation(), domEx);
-        } catch (final IOException ioe) {
-            LOGGER.error("Error parsing {}", configSource.getLocation(), ioe);
-        } catch (final ParserConfigurationException pex) {
-            LOGGER.error("Error parsing {}", configSource.getLocation(), pex);
-        }
-        if (strict && schema != null && buffer != null) {
-            InputStream is = null;
-            try {
-                is = Loader.getResourceAsStream(schema, GemFireXmlConfiguration.class.getClassLoader());
-            } catch (final Exception ex) {
-                LOGGER.error("Unable to access schema {}", this.schema, ex);
-            }
-            if (is != null) {
-                final Source src = new StreamSource(is, LOG4J_XSD);
-                final SchemaFactory factory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-                Schema schema = null;
-                try {
-                    schema = factory.newSchema(src);
-                } catch (final SAXException ex) {
-                    LOGGER.error("Error parsing Log4j schema", ex);
-                }
-                if (schema != null) {
-                    final Validator validator = schema.newValidator();
-                    try {
-                        validator.validate(new StreamSource(new ByteArrayInputStream(buffer)));
-                    } catch (final IOException ioe) {
-                        LOGGER.error("Error reading configuration for validation", ioe);
-                    } catch (final SAXException ex) {
-                        LOGGER.error("Error validating configuration", ex);
-                    }
-                }
-            }
-        }
-
-        if (getName() == null) {
-            setName(configSource.getLocation());
-        }
-    }
-
-    @Override
-    public void setup() {
-        if (rootElement == null) {
-            LOGGER.error("No logging configuration");
-            return;
-        }
-        constructHierarchy(rootNode, rootElement);
-        if (status.size() > 0) {
-            for (final Status s : status) {
-                LOGGER.error("Error processing element {}: {}", s.name, s.errorType);
-            }
-            return;
-        }
-        rootElement = null;
-    }
-
-    @Override
-    public Configuration reconfigure() {
-        try {
-            final ConfigurationSource source = getConfigurationSource().resetInputStream();
-            if (source == null) {
-                return null;
-            }
-            final GemFireXmlConfiguration config = new GemFireXmlConfiguration(source);
-            return (config.rootElement == null) ? null : config;
-        } catch (final IOException ex) {
-            LOGGER.error("Cannot locate file {}", getConfigurationSource(), ex);
-        }
-        return null;
-    }
-
-    private void constructHierarchy(final Node node, final Element element) {
-        processAttributes(node, element);
-        final StringBuilder buffer = new StringBuilder();
-        final NodeList list = element.getChildNodes();
-        final List<Node> children = node.getChildren();
-        for (int i = 0; i < list.getLength(); i++) {
-            final org.w3c.dom.Node w3cNode = list.item(i);
-            if (w3cNode instanceof Element) {
-                final Element child = (Element) w3cNode;
-                final String name = getType(child);
-                final PluginType<?> type = pluginManager.getPluginType(name);
-                final Node childNode = new Node(node, name, type);
-                constructHierarchy(childNode, child);
-                if (type == null) {
-                    final String value = childNode.getValue();
-                    if (!childNode.hasChildren() && value != null) {
-                        node.getAttributes().put(name, value);
-                    } else {
-                        status.add(new Status(name, element, ErrorType.CLASS_NOT_FOUND));
-                    }
-                } else {
-                    children.add(childNode);
-                }
-            } else if (w3cNode instanceof Text) {
-                final Text data = (Text) w3cNode;
-                buffer.append(data.getData());
-            }
-        }
-
-        final String text = buffer.toString().trim();
-        if (text.length() > 0 || (!node.hasChildren() && !node.isRoot())) {
-            node.setValue(text);
-        }
-    }
-
-    private String getType(final Element element) {
-        if (strict) {
-            final NamedNodeMap attrs = element.getAttributes();
-            for (int i = 0; i < attrs.getLength(); ++i) {
-                final org.w3c.dom.Node w3cNode = attrs.item(i);
-                if (w3cNode instanceof Attr) {
-                    final Attr attr = (Attr) w3cNode;
-                    if (attr.getName().equalsIgnoreCase("type")) {
-                        final String type = attr.getValue();
-                        attrs.removeNamedItem(attr.getName());
-                        return type;
-                    }
-                }
-            }
-        }
-        return element.getTagName();
-    }
-
-    private Map<String, String> processAttributes(final Node node, final Element element) {
-        final NamedNodeMap attrs = element.getAttributes();
-        final Map<String, String> attributes = node.getAttributes();
-
-        for (int i = 0; i < attrs.getLength(); ++i) {
-            final org.w3c.dom.Node w3cNode = attrs.item(i);
-            if (w3cNode instanceof Attr) {
-                final Attr attr = (Attr) w3cNode;
-                if (attr.getName().equals("xml:base")) {
-                    continue;
-                }
-                attributes.put(attr.getName(), attr.getValue());
-            }
-        }
-        return attributes;
-    }
-
-    @Override
-    public String toString() {
-        return getClass().getSimpleName() + "[location=" + getConfigurationSource() + "]";
-    }
-
-    /**
-     * The error that occurred.
-     */
-    private enum ErrorType {
-        CLASS_NOT_FOUND
-    }
-
-    /**
-     * Status for recording errors.
-     */
-    private static class Status {
-        private final Element element;
-        private final String name;
-        private final ErrorType errorType;
-
-        public Status(final String name, final Element element, final ErrorType errorType) {
-            this.name = name;
-            this.element = element;
-            this.errorType = errorType;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a097fcf3/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/xml/GemFireXmlConfigurationFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/xml/GemFireXmlConfigurationFactory.java b/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/xml/GemFireXmlConfigurationFactory.java
deleted file mode 100644
index 4e9d7af..0000000
--- a/gemfire-core/src/main/java/com/gemstone/org/apache/logging/log4j/core/config/xml/GemFireXmlConfigurationFactory.java
+++ /dev/null
@@ -1,59 +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.org.apache.logging.log4j.core.config.xml;
-
-import org.apache.logging.log4j.core.config.Configuration;
-import org.apache.logging.log4j.core.config.ConfigurationFactory;
-import org.apache.logging.log4j.core.config.ConfigurationSource;
-import org.apache.logging.log4j.core.config.Order;
-import org.apache.logging.log4j.core.config.plugins.Plugin;
-
-/**
- * Factory to construct an XmlConfiguration.
- */
-@Plugin(name = "GemFireXmlConfigurationFactory", category = "ConfigurationFactory")
-@Order(1)
-public class GemFireXmlConfigurationFactory extends ConfigurationFactory {
-
-    static {
-        //System.out.println("KIRK: loading GemFireXmlConfigurationFactory");
-    }
-  
-    /**
-     * Valid file extensions for XML files.
-     */
-    public static final String[] SUFFIXES = new String[] {".xml", "*"};
-
-    /**
-     * Returns the Configuration.
-     * @param source The InputSource.
-     * @return The Configuration.
-     */
-    @Override
-    public Configuration getConfiguration(final ConfigurationSource source) {
-        return new com.gemstone.org.apache.logging.log4j.core.config.xml.GemFireXmlConfiguration(source);
-    }
-
-    /**
-     * Returns the file suffixes for XML files.
-     * @return An array of File extensions.
-     */
-    @Override
-    public String[] getSupportedTypes() {
-        return SUFFIXES;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a097fcf3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
index 9eb51c5..b7167bc 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
@@ -93,6 +93,7 @@ public class CompressionCacheConfigDUnitTest extends CacheTestCase {
    * @throws Exception
    */
   public void testCreateCacheWithBadCompressor() throws Exception {
+    addExpectedException("Unable to load class BAD_COMPRESSOR");
     File cacheXml = createCacheXml(BAD_COMPRESSOR);
     ExpectedException expectedException = DistributedTestCase.addExpectedException("While reading Cache XML file");
     try {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a097fcf3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
index a002389..6d75f80 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
@@ -28,12 +28,10 @@ import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
-import org.apache.logging.log4j.util.PropertiesUtil;
 
 import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.logging.LoggingPerformanceTestCase;
 import com.gemstone.gemfire.internal.util.IOUtils;
-import com.gemstone.org.apache.logging.log4j.core.config.xml.GemFireXmlConfigurationFactory;
 
 public class Log4J2PerformanceTest extends LoggingPerformanceTestCase {
 
@@ -44,15 +42,6 @@ public class Log4J2PerformanceTest extends LoggingPerformanceTestCase {
   protected static final String SYS_LOG_FILE_SIZE_LIMIT = "gemfire-log-file-size-limit";
   protected static final String SYS_LOG_FILE_COUNT_LIMIT = "gemfire-log-file-count-limit";
   
-  static {
-    // set log4j.configurationFactory to be our optimized version
-    final String factory = GemFireXmlConfigurationFactory.class.getName();
-    System.setProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY, factory);
-    System.out.println("Set "+ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY+" to "+factory);
-    final String factoryClass = PropertiesUtil.getProperties().getStringProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY);
-    System.out.println("KIRK: factoryClass is " + factoryClass);
-  }
-  
   private File config = null;
   
   public Log4J2PerformanceTest(String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a097fcf3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
index 61b5131..506e487 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
@@ -27,12 +27,10 @@ import org.apache.commons.io.FileUtils;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
-import org.apache.logging.log4j.util.PropertiesUtil;
 
 import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.logging.LoggingPerformanceTestCase;
 import com.gemstone.gemfire.internal.util.IOUtils;
-import com.gemstone.org.apache.logging.log4j.core.config.xml.GemFireXmlConfigurationFactory;
 
 public class LogWriterLoggerPerformanceTest extends LoggingPerformanceTestCase {
 
@@ -43,15 +41,6 @@ public class LogWriterLoggerPerformanceTest extends LoggingPerformanceTestCase {
   protected static final String SYS_LOG_FILE_SIZE_LIMIT = "gemfire-log-file-size-limit";
   protected static final String SYS_LOG_FILE_COUNT_LIMIT = "gemfire-log-file-count-limit";
   
-  static {
-    // set log4j.configurationFactory to be our optimized version
-    final String factory = GemFireXmlConfigurationFactory.class.getName();
-    System.setProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY, factory);
-    System.out.println("Set "+ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY+" to "+factory);
-    final String factoryClass = PropertiesUtil.getProperties().getStringProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY);
-    System.out.println("KIRK: factoryClass is " + factoryClass);
-  }
-  
   private File config = null;
   
   public LogWriterLoggerPerformanceTest(String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a097fcf3/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 72c33d6..6433389 100644
--- a/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
+++ b/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java
@@ -242,7 +242,7 @@ public class DUnitLauncher {
         LogManager.getLogger(LogService.BASE_LOGGER_NAME)).getContext();
 
     final PatternLayout layout = PatternLayout.createLayout(
-        "[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} <%thread> tid=%tid] %message%n%throwable%n", null, null,
+        "[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} <%thread> tid=%tid] %message%n%throwable%n", null, null,null,
         Charset.defaultCharset(), true, false, "", "");
     
     final FileAppender fileAppender = FileAppender.createAppender(suspectFilename, "true", "false",

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a097fcf3/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 30a7fd8..0bbbccd 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -57,7 +57,7 @@ jna.version = 4.0.0
 json4s.version = 3.2.4
 junit.version = 4.12
 JUnitParams.version = 1.0.4
-log4j.version = 2.1
+log4j.version = 2.5
 lucene.version = 5.3.0
 mockito-core.version = 1.10.19
 multithreadedtc.version = 1.01



[46/50] [abbrv] incubator-geode git commit: GEODE-769 Update 3 issues in README.md

Posted by ab...@apache.org.
GEODE-769  Update 3 issues in README.md


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

Branch: refs/heads/master
Commit: 90efd883853d8fb7a8f77259a6fe35e87a087db5
Parents: e5a7b9a
Author: Karen Miller <km...@pivotal.io>
Authored: Thu Jan 7 15:53:50 2016 -0800
Committer: Anthony Baker <ab...@apache.org>
Committed: Wed Jan 27 10:56:19 2016 -0800

----------------------------------------------------------------------
 README.md | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/90efd883/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 6e08e79..e90b769 100755
--- a/README.md
+++ b/README.md
@@ -12,9 +12,10 @@ _Caches_ are an abstraction that describe a node in a Geode distributed system.
 
 Within each cache, you define data _regions_. Data regions are analogous to tables in a relational database and manage data in a distributed fashion as name/value pairs. A _replicated_ region stores identical copies of the data on each cache member of a distributed system. A _partitioned_ region spreads the data among cache members. After the system is configured, client applications can access the distributed data in regions without knowledge of the underlying system architecture. You can define listeners to receive notifications when data has changed, and you can define expiration criteria to delete obsolete data in a region.
 
-_Locators_ provide both discovery and load balancing services. You configure clients with a list of locator services and the locators maintain a dynamic list of member servers. By default, Geode clients and servers use port 40404 and multicast to discover each other.
+_Locators_ provide clients with both discovery and server load balancing services. Clients are configured with locator information, and the locators maintain a dynamic list of member servers. The locators provide clients with connection information to a server. 
 
 Geode includes the following features:
+
 * Combines redundancy, replication, and a "shared nothing" persistence architecture to deliver fail-safe reliability and performance.
 * Horizontally scalable to thousands of cache members, with multiple cache topologies to meet different enterprise needs. The cache can be distributed across multiple computers.
 * Asynchronous and synchronous cache update propagation.
@@ -31,7 +32,8 @@ Geode includes the following features:
 
 # Geode in 5 minutes
 
-Extract and build from source (note: currently Geode supports jdk1.7.75):
+With JDK 1.8 or a more recent version installed, obtain the source archive. 
+Extract and build from source:
 
     $ cd geode
     $ ./gradlew build installDist


[03/50] [abbrv] incubator-geode git commit: GEODE-714: Modify all tests to use JUnit Categories

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestTestCase.java
new file mode 100755
index 0000000..481863c
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestTestCase.java
@@ -0,0 +1,1018 @@
+/*
+ * 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.tier.sockets;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.InterestResultPolicy;
+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.PoolManager;
+import com.gemstone.gemfire.cache.client.internal.Connection;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.cache.client.internal.RegisterInterestTracker;
+import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.tier.InterestType;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.VM;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests Interest Registration Functionality
+ */
+@SuppressWarnings({"deprecation", "rawtypes", "serial", "unchecked"})
+public class HAInterestTestCase extends DistributedTestCase {
+  
+  protected static final int TIMEOUT_MILLIS = 60 * 1000;
+  protected static final int INTERVAL_MILLIS = 10;
+  
+  protected static final String REGION_NAME = "HAInterestBaseTest_region";
+  
+  protected static final String k1 = "k1";
+  protected static final String k2 = "k2";
+  protected static final String client_k1 = "client-k1";
+  protected static final String client_k2 = "client-k2";
+  protected static final String server_k1 = "server-k1";
+  protected static final String server_k2 = "server-k2";
+  protected static final String server_k1_updated = "server_k1_updated";
+
+  protected static Cache cache = null;
+  protected static PoolImpl pool = null;
+  protected static Connection conn = null;
+
+  protected static int PORT1;
+  protected static int PORT2;
+  protected static int PORT3;
+
+  protected static boolean isBeforeRegistrationCallbackCalled = false;
+  protected static boolean isBeforeInterestRecoveryCallbackCalled = false;
+  protected static boolean isAfterRegistrationCallbackCalled = false;
+
+  protected static Host host = null;
+  protected static VM server1 = null;
+  protected static VM server2 = null;
+  protected static VM server3 = null;
+  
+  protected volatile static boolean exceptionOccured = false;
+
+  public HAInterestTestCase(String name) {
+    super(name);
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    host = Host.getHost(0);
+    server1 = host.getVM(0);
+    server2 = host.getVM(1);
+    server3 = host.getVM(2);
+    CacheServerTestUtil.disableShufflingOfEndpoints();
+    // start servers first
+    PORT1 = ((Integer) server1.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
+    PORT2 = ((Integer) server2.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
+    PORT3 = ((Integer) server3.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
+    exceptionOccured = false;
+    addExpectedException("java.net.ConnectException: Connection refused: connect");
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    // close the clients first
+    closeCache();
+
+    // then close the servers
+    server1.invoke(HAInterestTestCase.class, "closeCache");
+    server2.invoke(HAInterestTestCase.class, "closeCache");
+    server3.invoke(HAInterestTestCase.class, "closeCache");
+    CacheServerTestUtil.resetDisableShufflingOfEndpointsFlag();
+  }
+
+  public static void closeCache() {
+    PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
+    PoolImpl.BEFORE_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = false;
+    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
+    PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
+    HAInterestTestCase.isAfterRegistrationCallbackCalled = false;
+    HAInterestTestCase.isBeforeInterestRecoveryCallbackCalled = false;
+    HAInterestTestCase.isBeforeRegistrationCallbackCalled = false;
+    if (cache != null && !cache.isClosed()) {
+      cache.close();
+      cache.getDistributedSystem().disconnect();
+    }
+    cache = null;
+    pool = null;
+    conn = null;
+  }
+  
+  /**
+   * Return the current primary waiting for a primary to exist.
+   * 
+   * @since 5.7
+   */
+  public static VM getPrimaryVM() {
+    return getPrimaryVM(null);
+  }
+
+  /**
+   * Return the current primary waiting for a primary to exist and for it not to
+   * be the oldPrimary (if oldPrimary is NOT null).
+   * 
+   * @since 5.7
+   */
+  public static VM getPrimaryVM(final VM oldPrimary) {
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        int primaryPort = pool.getPrimaryPort();
+        if (primaryPort == -1) {
+          return false;
+        }
+        // we have a primary
+        VM currentPrimary = getServerVM(primaryPort);
+        if (currentPrimary != oldPrimary) {
+          return true;
+        }
+        return false;
+      }
+      @Override
+      public String description() {
+        return "waiting for primary";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    int primaryPort = pool.getPrimaryPort();
+    assertTrue(primaryPort != -1);
+    VM currentPrimary = getServerVM(primaryPort);
+    assertTrue(currentPrimary != oldPrimary);
+    return currentPrimary;
+  }
+
+  public static VM getBackupVM() {
+    return getBackupVM(null);
+  }
+
+  public static VM getBackupVM(VM stoppedBackup) {
+    VM currentPrimary = getPrimaryVM(null);
+    if (currentPrimary != server2 && server2 != stoppedBackup) {
+      return server2;
+    } else if (currentPrimary != server3 && server3 != stoppedBackup) {
+      return server3;
+    } else if (currentPrimary != server1 && server1 != stoppedBackup) {
+      return server1;
+    } else {
+      fail("expected currentPrimary " + currentPrimary + " to be " + server1 + ", or " + server2 + ", or " + server3);
+      return null;
+    }
+  }
+
+  /**
+   * Given a server vm (server1, server2, or server3) return its port.
+   * 
+   * @since 5.7
+   */
+  public static int getServerPort(VM vm) {
+    if (vm == server1) {
+      return PORT1;
+    } else if (vm == server2) {
+      return PORT2;
+    } else if (vm == server3) {
+      return PORT3;
+    } else {
+      fail("expected vm " + vm + " to be " + server1 + ", or " + server2 + ", or " + server3);
+      return -1;
+    }
+  }
+
+  /**
+   * Given a server port (PORT1, PORT2, or PORT3) return its vm.
+   * 
+   * @since 5.7
+   */
+  public static VM getServerVM(int port) {
+    if (port == PORT1) {
+      return server1;
+    } else if (port == PORT2) {
+      return server2;
+    } else if (port == PORT3) {
+      return server3;
+    } else {
+      fail("expected port " + port + " to be " + PORT1 + ", or " + PORT2 + ", or " + PORT3);
+      return null;
+    }
+  }
+
+  public static void verifyRefreshedEntriesFromServer() {
+    final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r1);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        Region.Entry re = r1.getEntry(k1);
+        if (re == null)
+          return false;
+        Object val = re.getValue();
+        return client_k1.equals(val);
+      }
+      @Override
+      public String description() {
+        return "waiting for client_k1 refresh from server";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        Region.Entry re = r1.getEntry(k2);
+        if (re == null)
+          return false;
+        Object val = re.getValue();
+        return client_k2.equals(val);
+      }
+      @Override
+      public String description() {
+        return "waiting for client_k2 refresh from server";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+  }
+
+  public static void verifyDeadAndLiveServers(final int expectedDeadServers, final int expectedLiveServers) {
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == expectedLiveServers;
+      }
+      @Override
+      public String description() {
+        return "waiting for pool.getConnectedServerCount() == expectedLiveServer";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+  }
+
+  public static void putK1andK2() {
+    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r1);
+    r1.put(k1, server_k1);
+    r1.put(k2, server_k2);
+  }
+
+  public static void setClientServerObserverForBeforeInterestRecoveryFailure() {
+    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
+      public void beforeInterestRecovery() {
+        synchronized (HAInterestTestCase.class) {
+          Thread t = new Thread() {
+            public void run() {
+              getBackupVM().invoke(HAInterestTestCase.class, "startServer");
+              getPrimaryVM().invoke(HAInterestTestCase.class, "stopServer");
+            }
+          };
+          t.start();
+          try {
+            DistributedTestCase.join(t, 30 * 1000, getLogWriter());
+          } catch (Exception ignore) {
+            exceptionOccured = true;
+          }
+          HAInterestTestCase.isBeforeInterestRecoveryCallbackCalled = true;
+          HAInterestTestCase.class.notify();
+          PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
+        }
+      }
+    });
+  }
+
+  public static void setClientServerObserverForBeforeInterestRecovery() {
+    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
+      public void beforeInterestRecovery() {
+        synchronized (HAInterestTestCase.class) {
+          Thread t = new Thread() {
+            public void run() {
+              Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+              assertNotNull(r1);
+              r1.put(k1, server_k1_updated);
+            }
+          };
+          t.start();
+
+          HAInterestTestCase.isBeforeInterestRecoveryCallbackCalled = true;
+          HAInterestTestCase.class.notify();
+          PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
+        }
+      }
+    });
+  }
+
+  public static void waitForBeforeInterestRecoveryCallBack() throws InterruptedException {
+    assertNotNull(cache);
+    synchronized (HAInterestTestCase.class) {
+      while (!isBeforeInterestRecoveryCallbackCalled) {
+        HAInterestTestCase.class.wait();
+      }
+    }
+  }
+
+  public static void setClientServerObserverForBeforeRegistration(final VM vm) {
+    PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = true;
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
+      public void beforeInterestRegistration() {
+        synchronized (HAInterestTestCase.class) {
+          vm.invoke(HAInterestTestCase.class, "startServer");
+          HAInterestTestCase.isBeforeRegistrationCallbackCalled = true;
+          HAInterestTestCase.class.notify();
+          PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
+        }
+      }
+    });
+  }
+
+  public static void waitForBeforeRegistrationCallback() throws InterruptedException {
+    assertNotNull(cache);
+    synchronized (HAInterestTestCase.class) {
+      while (!isBeforeRegistrationCallbackCalled) {
+        HAInterestTestCase.class.wait();
+      }
+    }
+  }
+
+  public static void setClientServerObserverForAfterRegistration(final VM vm) {
+    PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = true;
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
+      public void afterInterestRegistration() {
+        synchronized (HAInterestTestCase.class) {
+          vm.invoke(HAInterestTestCase.class, "startServer");
+          HAInterestTestCase.isAfterRegistrationCallbackCalled = true;
+          HAInterestTestCase.class.notify();
+          PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
+        }
+      }
+    });
+  }
+
+  public static void waitForAfterRegistrationCallback() throws InterruptedException {
+    assertNotNull(cache);
+    if (!isAfterRegistrationCallbackCalled) {
+      synchronized (HAInterestTestCase.class) {
+        while (!isAfterRegistrationCallbackCalled) {
+          HAInterestTestCase.class.wait();
+        }
+      }
+    }
+  }
+
+  public static void unSetClientServerObserverForRegistrationCallback() {
+    synchronized (HAInterestTestCase.class) {
+      PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
+      PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
+      HAInterestTestCase.isBeforeRegistrationCallbackCalled = false;
+      HAInterestTestCase.isAfterRegistrationCallbackCalled = false;
+    }
+  }
+
+  public static void verifyDispatcherIsAlive() {
+    assertEquals("More than one BridgeServer", 1, cache.getCacheServers().size());
+    
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return cache.getCacheServers().size() == 1;
+      }
+      @Override
+      public String description() {
+        return "waiting for cache.getCacheServers().size() == 1";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    assertNotNull(bs.getAcceptor());
+    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
+    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
+
+    wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return ccn.getClientProxies().size() > 0;
+      }
+      @Override
+      public String description() {
+        return "waiting for ccn.getClientProxies().size() > 0";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    wc = new WaitCriterion() {
+      Iterator iter_prox;
+      CacheClientProxy proxy;
+
+      @Override
+      public boolean done() {
+        iter_prox = ccn.getClientProxies().iterator();
+        if (iter_prox.hasNext()) {
+          proxy = (CacheClientProxy) iter_prox.next();
+          return proxy._messageDispatcher.isAlive();
+        } else {
+          return false;
+        }
+      }
+
+      @Override
+      public String description() {
+        return "waiting for CacheClientProxy _messageDispatcher to be alive";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+  }
+
+  public static void verifyDispatcherIsNotAlive() {
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return cache.getCacheServers().size() == 1;
+      }
+      @Override
+      public String description() {
+        return "cache.getCacheServers().size() == 1";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    assertNotNull(bs.getAcceptor());
+    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
+    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
+    
+    wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return ccn.getClientProxies().size() > 0;
+      }
+      @Override
+      public String description() {
+        return "waiting for ccn.getClientProxies().size() > 0";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    Iterator iter_prox = ccn.getClientProxies().iterator();
+    if (iter_prox.hasNext()) {
+      CacheClientProxy proxy = (CacheClientProxy) iter_prox.next();
+      assertFalse("Dispatcher on secondary should not be alive", proxy._messageDispatcher.isAlive());
+    }
+  }
+
+  public static void createEntriesK1andK2OnServer() {
+    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r1);
+    if (!r1.containsKey(k1)) {
+      r1.create(k1, server_k1);
+    }
+    if (!r1.containsKey(k2)) {
+      r1.create(k2, server_k2);
+    }
+    assertEquals(r1.getEntry(k1).getValue(), server_k1);
+    assertEquals(r1.getEntry(k2).getValue(), server_k2);
+  }
+
+  public static void createEntriesK1andK2() {
+    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r1);
+    if (!r1.containsKey(k1)) {
+      r1.create(k1, client_k1);
+    }
+    if (!r1.containsKey(k2)) {
+      r1.create(k2, client_k2);
+    }
+    assertEquals(r1.getEntry(k1).getValue(), client_k1);
+    assertEquals(r1.getEntry(k2).getValue(), client_k2);
+  }
+
+  public static void createServerEntriesK1andK2() {
+    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r1);
+    if (!r1.containsKey(k1)) {
+      r1.create(k1, server_k1);
+    }
+    if (!r1.containsKey(k2)) {
+      r1.create(k2, server_k2);
+    }
+    assertEquals(r1.getEntry(k1).getValue(), server_k1);
+    assertEquals(r1.getEntry(k2).getValue(), server_k2);
+  }
+
+  public static void registerK1AndK2() {
+    Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+    r.registerInterest(list, InterestResultPolicy.KEYS_VALUES);
+  }
+
+  public static void reRegisterK1AndK2() {
+    Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+    r.registerInterest(list);
+  }
+
+  public static void startServer() throws IOException {
+    Cache c = CacheFactory.getAnyInstance();
+    assertEquals("More than one BridgeServer", 1, c.getCacheServers().size());
+    CacheServerImpl bs = (CacheServerImpl) c.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    bs.start();
+  }
+
+  public static void stopServer() {
+    assertEquals("More than one BridgeServer", 1, cache.getCacheServers().size());
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    bs.stop();
+  }
+
+  public static void stopPrimaryAndRegisterK1AndK2AndVerifyResponse() {
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == 3;
+      }
+      @Override
+      public String description() {
+        return "connected server count never became 3";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    // close primaryEP
+    getPrimaryVM().invoke(HAInterestTestCase.class, "stopServer");
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
+    assertNotNull(serverKeys);
+    List resultKeys = (List) serverKeys.get(0);
+    assertEquals(2, resultKeys.size());
+    assertTrue(resultKeys.contains(k1));
+    assertTrue(resultKeys.contains(k2));
+  }
+
+  public static void stopPrimaryAndUnregisterRegisterK1() {
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == 3;
+      }
+      @Override
+      public String description() {
+        return "connected server count never became 3";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    // close primaryEP
+    getPrimaryVM().invoke(HAInterestTestCase.class, "stopServer");
+    List list = new ArrayList();
+    list.add(k1);
+    srp.unregisterInterest(list, InterestType.KEY, false, false);
+  }
+
+  public static void stopBothPrimaryAndSecondaryAndRegisterK1AndK2AndVerifyResponse() {
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == 3;
+      }
+      @Override
+      public String description() {
+        return "connected server count never became 3";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    // close primaryEP
+    VM backup = getBackupVM();
+    getPrimaryVM().invoke(HAInterestTestCase.class, "stopServer");
+    // close secondary
+    backup.invoke(HAInterestTestCase.class, "stopServer");
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
+
+    assertNotNull(serverKeys);
+    List resultKeys = (List) serverKeys.get(0);
+    assertEquals(2, resultKeys.size());
+    assertTrue(resultKeys.contains(k1));
+    assertTrue(resultKeys.contains(k2));
+  }
+
+  /**
+   * returns the secondary that was stopped
+   */
+  public static VM stopSecondaryAndRegisterK1AndK2AndVerifyResponse() {
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == 3;
+      }
+      @Override
+      public String description() {
+        return "Never got three connected servers";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    // close secondary EP
+    VM result = getBackupVM();
+    result.invoke(HAInterestTestCase.class, "stopServer");
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
+
+    assertNotNull(serverKeys);
+    List resultKeys = (List) serverKeys.get(0);
+    assertEquals(2, resultKeys.size());
+    assertTrue(resultKeys.contains(k1));
+    assertTrue(resultKeys.contains(k2));
+    return result;
+  }
+
+  /**
+   * returns the secondary that was stopped
+   */
+  public static VM stopSecondaryAndUNregisterK1() {
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return pool.getConnectedServerCount() == 3;
+      }
+      @Override
+      public String description() {
+        return "connected server count never became 3";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    // close secondary EP
+    VM result = getBackupVM();
+    result.invoke(HAInterestTestCase.class, "stopServer");
+    List list = new ArrayList();
+    list.add(k1);
+    srp.unregisterInterest(list, InterestType.KEY, false, false);
+    return result;
+  }
+
+  public static void registerK1AndK2OnPrimaryAndSecondaryAndVerifyResponse() {
+    ServerLocation primary = pool.getPrimary();
+    ServerLocation secondary = (ServerLocation) pool.getRedundants().get(0);
+    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
+    assertNotNull(r);
+    ServerRegionProxy srp = new ServerRegionProxy(r);
+    List list = new ArrayList();
+    list.add(k1);
+    list.add(k2);
+
+    // Primary server
+    List serverKeys1 = srp.registerInterestOn(primary, list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
+    assertNotNull(serverKeys1);
+    // expect serverKeys in response from primary
+    List resultKeys = (List) serverKeys1.get(0);
+    assertEquals(2, resultKeys.size());
+    assertTrue(resultKeys.contains(k1));
+    assertTrue(resultKeys.contains(k2));
+
+    // Secondary server
+    List serverKeys2 = srp.registerInterestOn(secondary, list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
+    // if the list is null then it is empty
+    if (serverKeys2 != null) {
+      // no serverKeys in response from secondary
+      assertTrue(serverKeys2.isEmpty());
+    }
+  }
+
+  public static void verifyInterestRegistration() {
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return cache.getCacheServers().size() == 1;
+      }
+      @Override
+      public String description() {
+        return "waiting for cache.getCacheServers().size() == 1";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    assertNotNull(bs.getAcceptor());
+    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
+    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
+    
+    wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return ccn.getClientProxies().size() > 0;
+      }
+      @Override
+      public String description() {
+        return "waiting for ccn.getClientProxies().size() > 0";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    Iterator iter_prox = ccn.getClientProxies().iterator();
+
+    if (iter_prox.hasNext()) {
+      final CacheClientProxy ccp = (CacheClientProxy) iter_prox.next();
+      
+      wc = new WaitCriterion() {
+        @Override
+        public boolean done() {
+          Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
+              .getProfile(Region.SEPARATOR + REGION_NAME)
+              .getKeysOfInterestFor(ccp.getProxyID());
+          return keysMap != null && keysMap.size() == 2;
+        }
+        @Override
+        public String description() {
+          return "waiting for keys of interest to include 2 keys";
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+      Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex].getProfile(Region.SEPARATOR + REGION_NAME)
+          .getKeysOfInterestFor(ccp.getProxyID());
+      assertNotNull(keysMap);
+      assertEquals(2, keysMap.size());
+      assertTrue(keysMap.contains(k1));
+      assertTrue(keysMap.contains(k2));
+    }
+  }
+
+  public static void verifyInterestUNRegistration() {
+    WaitCriterion wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return cache.getCacheServers().size() == 1;
+      }
+      @Override
+      public String description() {
+        return "waiting for cache.getCacheServers().size() == 1";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
+    assertNotNull(bs);
+    assertNotNull(bs.getAcceptor());
+    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
+    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
+    
+    wc = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        return ccn.getClientProxies().size() > 0;
+      }
+      @Override
+      public String description() {
+        return "waiting for ccn.getClientProxies().size() > 0";
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+    Iterator iter_prox = ccn.getClientProxies().iterator();
+    if (iter_prox.hasNext()) {
+      final CacheClientProxy ccp = (CacheClientProxy) iter_prox.next();
+      
+      wc = new WaitCriterion() {
+        @Override
+        public boolean done() {
+          Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
+              .getProfile(Region.SEPARATOR + REGION_NAME)
+              .getKeysOfInterestFor(ccp.getProxyID());
+          return keysMap != null;
+        }
+        @Override
+        public String description() {
+          return "waiting for keys of interest to not be null";
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
+
+      Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
+          .getProfile(Region.SEPARATOR + REGION_NAME)
+          .getKeysOfInterestFor(ccp.getProxyID());
+      assertNotNull(keysMap);
+      assertEquals(1, keysMap.size());
+      assertFalse(keysMap.contains(k1));
+      assertTrue(keysMap.contains(k2));
+    }
+  }
+
+  private void createCache(Properties props) throws Exception {
+    DistributedSystem ds = getSystem(props);
+    assertNotNull(ds);
+    ds.disconnect();
+    ds = getSystem(props);
+    cache = CacheFactory.create(ds);
+    assertNotNull(cache);
+  }
+
+  public static void createClientPoolCache(String testName, String host) throws Exception {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    new HAInterestTestCase("temp").createCache(props);
+    CacheServerTestUtil.disableShufflingOfEndpoints();
+    PoolImpl p;
+    try {
+      p = (PoolImpl) PoolManager.createFactory()
+          .addServer(host, PORT1)
+          .addServer(host, PORT2)
+          .addServer(host, PORT3)
+          .setSubscriptionEnabled(true)
+          .setSubscriptionRedundancy(-1)
+          .setReadTimeout(1000)
+          .setPingInterval(1000)
+          // retryInterval should be more so that only registerInterste thread
+          // will initiate failover
+          // .setRetryInterval(20000)
+          .create("HAInterestBaseTestPool");
+    } finally {
+      CacheServerTestUtil.enableShufflingOfEndpoints();
+    }
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    factory.setConcurrencyChecksEnabled(true);
+    factory.setPoolName(p.getName());
+
+    cache.createRegion(REGION_NAME, factory.create());
+    pool = p;
+    conn = pool.acquireConnection();
+    assertNotNull(conn);
+  }
+
+  public static void createClientPoolCacheWithSmallRetryInterval(String testName, String host) throws Exception {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    new HAInterestTestCase("temp").createCache(props);
+    CacheServerTestUtil.disableShufflingOfEndpoints();
+    PoolImpl p;
+    try {
+      p = (PoolImpl) PoolManager.createFactory()
+          .addServer(host, PORT1)
+          .addServer(host, PORT2)
+          .setSubscriptionEnabled(true)
+          .setSubscriptionRedundancy(-1)
+          .setReadTimeout(1000)
+          .setSocketBufferSize(32768)
+          .setMinConnections(6)
+          .setPingInterval(200)
+          // .setRetryInterval(200)
+          // retryAttempts 3
+          .create("HAInterestBaseTestPool");
+    } finally {
+      CacheServerTestUtil.enableShufflingOfEndpoints();
+    }
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    factory.setConcurrencyChecksEnabled(true);
+    factory.setPoolName(p.getName());
+
+    cache.createRegion(REGION_NAME, factory.create());
+
+    pool = p;
+    conn = pool.acquireConnection();
+    assertNotNull(conn);
+  }
+
+  public static void createClientPoolCacheConnectionToSingleServer(String testName, String hostName) throws Exception {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    new HAInterestTestCase("temp").createCache(props);
+    PoolImpl p = (PoolImpl) PoolManager.createFactory()
+        .addServer(hostName, PORT1)
+        .setSubscriptionEnabled(true)
+        .setSubscriptionRedundancy(-1)
+        .setReadTimeout(1000)
+        // .setRetryInterval(20)
+        .create("HAInterestBaseTestPool");
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    factory.setConcurrencyChecksEnabled(true);
+    factory.setPoolName(p.getName());
+
+    cache.createRegion(REGION_NAME, factory.create());
+
+    pool = p;
+    conn = pool.acquireConnection();
+    assertNotNull(conn);
+  }
+
+  public static Integer createServerCache() throws Exception {
+    new HAInterestTestCase("temp").createCache(new Properties());
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setEnableBridgeConflation(true);
+    factory.setMirrorType(MirrorType.KEYS_VALUES);
+    factory.setConcurrencyChecksEnabled(true);
+    cache.createRegion(REGION_NAME, factory.create());
+
+    CacheServer server = cache.addCacheServer();
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    server.setPort(port);
+    server.setMaximumTimeBetweenPings(180000);
+    // ensures updates to be sent instead of invalidations
+    server.setNotifyBySubscription(true);
+    server.start();
+    return new Integer(server.getPort());
+  }
+
+  public static Integer createServerCacheWithLocalRegion() throws Exception {
+    new HAInterestTestCase("temp").createCache(new Properties());
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    factory.setConcurrencyChecksEnabled(true);
+    RegionAttributes attrs = factory.create();
+    cache.createRegion(REGION_NAME, attrs);
+
+    CacheServer server = cache.addCacheServer();
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    server.setPort(port);
+    // ensures updates to be sent instead of invalidations
+    server.setNotifyBySubscription(true);
+    server.setMaximumTimeBetweenPings(180000);
+    server.start();
+    return new Integer(server.getPort());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
index b12f55b..b6bfe22 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
@@ -22,12 +22,18 @@ import static org.mockito.Mockito.when;
 import java.io.IOException;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
+/**
+ * Exposes GEODE-537: NPE in JTA AFTER_COMPLETION command processing
+ */
+@Category(UnitTest.class)
 public class CommitCommandTest {
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogWriterPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogWriterPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogWriterPerformanceTest.java
index 77d7995..1f72a6b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogWriterPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogWriterPerformanceTest.java
@@ -21,16 +21,22 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.Properties;
 
+import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.internal.util.IOUtils;
+import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 
 /**
  * Tests performance of logging when level is OFF.
  * 
  * @author Kirk Lund
  */
+@Category(PerformanceTest.class)
+@Ignore("Tests have no assertions")
 public class LogWriterPerformanceTest extends LoggingPerformanceTestCase {
 
   public LogWriterPerformanceTest(String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2DisabledPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2DisabledPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2DisabledPerformanceTest.java
index f98868b..caedadc 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2DisabledPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2DisabledPerformanceTest.java
@@ -20,7 +20,13 @@ import java.io.IOException;
 
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
+import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
+
+@Category(PerformanceTest.class)
+@Ignore("Tests have no assertions")
 public class Log4J2DisabledPerformanceTest extends Log4J2PerformanceTest {
 
   public Log4J2DisabledPerformanceTest(String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
index a002389..ddf106d 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4J2PerformanceTest.java
@@ -29,12 +29,17 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
 import org.apache.logging.log4j.util.PropertiesUtil;
+import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.logging.LoggingPerformanceTestCase;
 import com.gemstone.gemfire.internal.util.IOUtils;
+import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 import com.gemstone.org.apache.logging.log4j.core.config.xml.GemFireXmlConfigurationFactory;
 
+@Category(PerformanceTest.class)
+@Ignore("Tests have no assertions")
 public class Log4J2PerformanceTest extends LoggingPerformanceTestCase {
 
   protected static final int DEFAULT_LOG_FILE_SIZE_LIMIT = Integer.MAX_VALUE;
@@ -44,13 +49,11 @@ public class Log4J2PerformanceTest extends LoggingPerformanceTestCase {
   protected static final String SYS_LOG_FILE_SIZE_LIMIT = "gemfire-log-file-size-limit";
   protected static final String SYS_LOG_FILE_COUNT_LIMIT = "gemfire-log-file-count-limit";
   
-  static {
+  private static void init() { // was a static initializer
     // set log4j.configurationFactory to be our optimized version
     final String factory = GemFireXmlConfigurationFactory.class.getName();
     System.setProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY, factory);
-    System.out.println("Set "+ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY+" to "+factory);
     final String factoryClass = PropertiesUtil.getProperties().getStringProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY);
-    System.out.println("KIRK: factoryClass is " + factoryClass);
   }
   
   private File config = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerDisabledPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerDisabledPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerDisabledPerformanceTest.java
index f964208..4be34c7 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerDisabledPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerDisabledPerformanceTest.java
@@ -20,7 +20,13 @@ import java.io.IOException;
 
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
+import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
+
+@Category(PerformanceTest.class)
+@Ignore("Tests have no assertions")
 public class LogWriterLoggerDisabledPerformanceTest extends LogWriterLoggerPerformanceTest {
 
   public LogWriterLoggerDisabledPerformanceTest(String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
index 61b5131..926bc75 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterLoggerPerformanceTest.java
@@ -28,12 +28,17 @@ import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
 import org.apache.logging.log4j.util.PropertiesUtil;
+import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.logging.LoggingPerformanceTestCase;
 import com.gemstone.gemfire.internal.util.IOUtils;
+import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 import com.gemstone.org.apache.logging.log4j.core.config.xml.GemFireXmlConfigurationFactory;
 
+@Category(PerformanceTest.class)
+@Ignore("Tests have no assertions")
 public class LogWriterLoggerPerformanceTest extends LoggingPerformanceTestCase {
 
   protected static final int DEFAULT_LOG_FILE_SIZE_LIMIT = Integer.MAX_VALUE;
@@ -43,13 +48,11 @@ public class LogWriterLoggerPerformanceTest extends LoggingPerformanceTestCase {
   protected static final String SYS_LOG_FILE_SIZE_LIMIT = "gemfire-log-file-size-limit";
   protected static final String SYS_LOG_FILE_COUNT_LIMIT = "gemfire-log-file-count-limit";
   
-  static {
+  private static void init() { // was a static initializer
     // set log4j.configurationFactory to be our optimized version
     final String factory = GemFireXmlConfigurationFactory.class.getName();
     System.setProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY, factory);
-    System.out.println("Set "+ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY+" to "+factory);
     final String factoryClass = PropertiesUtil.getProperties().getStringProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY);
-    System.out.println("KIRK: factoryClass is " + factoryClass);
   }
   
   private File config = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/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 a3d4785..6fa560f 100755
--- a/gemfire-core/src/test/java/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
@@ -34,6 +34,7 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
+import org.junit.experimental.categories.Category;
 import org.springframework.data.gemfire.support.GemfireCache;
 
 import junit.framework.TestCase;
@@ -86,6 +87,7 @@ import com.gemstone.gemfire.internal.logging.LogWriterImpl;
 import com.gemstone.gemfire.internal.logging.ManagerLogWriter;
 import com.gemstone.gemfire.internal.logging.log4j.LogWriterLogger;
 import com.gemstone.gemfire.management.internal.cli.LogWrapper;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 import dunit.standalone.DUnitLauncher;
 
@@ -101,6 +103,7 @@ import dunit.standalone.DUnitLauncher;
  *
  * @author David Whitlock
  */
+@Category(DistributedTest.class)
 @SuppressWarnings("serial")
 public abstract class DistributedTestCase extends TestCase implements java.io.Serializable {
   private static final Logger logger = LogService.getLogger();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitPerformanceTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitPerformanceTest.java
deleted file mode 100644
index ab2db78..0000000
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitPerformanceTest.java
+++ /dev/null
@@ -1,437 +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.cache.lucene.internal.repository;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.SearcherManager;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.store.RAMDirectory;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.DataSerializable;
-import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.PartitionAttributesFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
-import com.gemstone.gemfire.cache.lucene.LuceneIndex;
-import com.gemstone.gemfire.cache.lucene.LuceneQuery;
-import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
-import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
-import com.gemstone.gemfire.cache.lucene.LuceneService;
-import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
-import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
-import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
-import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
-import com.gemstone.gemfire.cache.query.QueryException;
-import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
-
-/**
- * Microbenchmark of the IndexRepository to compare an
- * IndexRepository built on top of cache with a 
- * stock lucene IndexWriter with a RAMDirectory.
- */
-@Category(PerformanceTest.class)
-public class IndexRepositoryImplJUnitPerformanceTest {
-  
-  private static final int NUM_WORDS = 1000;
-  private static int[] COMMIT_INTERVAL = new int[] {100, 1000, 5000};
-  private static int NUM_ENTRIES = 500_000;
-  private static int NUM_QUERIES = 500_000;
-
-  private StandardAnalyzer analyzer = new StandardAnalyzer();
-  
-  @Test
-  public  void testIndexRepository() throws Exception {
-    
-
-    doTest("IndexRepository", new TestCallbacks() {
-
-      private Cache cache;
-      private IndexRepositoryImpl repo;
-      private IndexWriter writer;
-
-      @Override
-      public void addObject(String key, String text) throws Exception {
-        repo.create(key, new TestObject(text));
-      }
-
-      @Override
-      public void commit()  throws Exception {
-        repo.commit();
-      }
-
-      @Override
-      public void init() throws Exception {
-        cache = new CacheFactory().set("mcast-port", "0")
-            .set("log-level", "error")
-            .create();
-        Region<String, File> fileRegion = cache.<String, File>createRegionFactory(RegionShortcut.REPLICATE).create("files");
-        Region<ChunkKey, byte[]> chunkRegion = cache.<ChunkKey, byte[]>createRegionFactory(RegionShortcut.REPLICATE).create("chunks");
-
-        RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion);
-        
-        
-        IndexWriterConfig config = new IndexWriterConfig(analyzer);
-        writer = new IndexWriter(dir, config);
-        String[] indexedFields= new String[] {"text"};
-        HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(indexedFields);
-        repo = new IndexRepositoryImpl(fileRegion, writer, mapper);
-      }
-
-      @Override
-      public void cleanup() throws IOException {
-        writer.close();
-        cache.close();
-      }
-
-      @Override
-      public void waitForAsync() throws Exception {
-        //do nothing
-      }
-
-      @Override
-      public int query(Query query) throws IOException {
-        TopEntriesCollector collector = new TopEntriesCollector();
-        repo.query(query, 100, collector);
-        return collector.size();
-      }
-    });
-  }
-  
-  /**
-   * Test our full lucene index implementation
-   * @throws Exception
-   */
-  @Test
-  public void testLuceneIndex() throws Exception {
-    
-
-    doTest("LuceneIndex", new TestCallbacks() {
-
-      private Cache cache;
-      private Region<String, TestObject> region;
-      private LuceneService service;
-
-      @Override
-      public void addObject(String key, String text) throws Exception {
-        region.create(key, new TestObject(text));
-      }
-
-      @Override
-      public void commit()  throws Exception {
-        //NA
-      }
-
-      @Override
-      public void init() throws Exception {
-        cache = new CacheFactory().set("mcast-port", "0")
-            .set("log-level", "warning")
-            .create();
-        service = LuceneServiceProvider.get(cache);
-        service.createIndex("index", "/region", "text");
-        region = cache.<String, TestObject>createRegionFactory(RegionShortcut.PARTITION)
-            .setPartitionAttributes(new PartitionAttributesFactory<>().setTotalNumBuckets(1).create())
-            .create("region");
-      }
-
-      @Override
-      public void cleanup() throws IOException {
-        cache.close();
-      }
-      
-      @Override
-      public void waitForAsync() throws Exception {
-        AsyncEventQueue aeq = cache.getAsyncEventQueue(LuceneServiceImpl.getUniqueIndexName("index", "/region"));
-        
-        //We will be at most 10 ms off
-        while(aeq.size() > 0) {
-          Thread.sleep(10);
-        }
-      }
-
-      @Override
-      public int query(final Query query) throws Exception {
-        LuceneQuery<Object, Object> luceneQuery = service.createLuceneQueryFactory().create("index", "/region", new LuceneQueryProvider() {
-          
-          @Override
-          public Query getQuery(LuceneIndex index) throws QueryException {
-            return query;
-          }
-        });
-        
-        LuceneQueryResults<Object, Object> results = luceneQuery.search();
-        return results.size();
-      }
-    });
-  }
-  
-  @Test
-  public  void testLuceneWithRegionDirectory() throws Exception {
-    doTest("RegionDirectory", new TestCallbacks() {
-
-      private IndexWriter writer;
-      private SearcherManager searcherManager;
-
-      @Override
-      public void init() throws Exception {
-        RegionDirectory dir = new RegionDirectory(new ConcurrentHashMap<String, File>(), new ConcurrentHashMap<ChunkKey, byte[]>());
-        IndexWriterConfig config = new IndexWriterConfig(analyzer);
-        writer = new IndexWriter(dir, config);
-        searcherManager = new SearcherManager(writer, true, null);
-      }
-
-      @Override
-      public void addObject(String key, String text) throws Exception {
-        Document doc = new Document();
-        doc.add(new TextField("key", key, Store.YES));
-        doc.add(new TextField("text", text, Store.NO));
-        writer.addDocument(doc);
-      }
-
-      @Override
-      public void commit() throws Exception {
-        writer.commit();
-        searcherManager.maybeRefresh();
-      }
-
-      @Override
-      public void cleanup() throws Exception {
-        writer.close();
-      }
-      
-      @Override
-      public void waitForAsync() throws Exception {
-        //do nothing
-      }
-
-      @Override
-      public int query(Query query) throws Exception {
-        IndexSearcher searcher = searcherManager.acquire();
-        try {
-          return searcher.count(query);
-        } finally {
-          searcherManager.release(searcher);
-        }
-      }
-      
-    });
-    
-  }
-  
-  @Test
-  public  void testLucene() throws Exception {
-    doTest("Lucene", new TestCallbacks() {
-
-      private IndexWriter writer;
-      private SearcherManager searcherManager;
-
-      @Override
-      public void init() throws Exception {
-        RAMDirectory dir = new RAMDirectory();
-        IndexWriterConfig config = new IndexWriterConfig(analyzer);
-        writer = new IndexWriter(dir, config);
-        searcherManager = new SearcherManager(writer, true, null);
-      }
-
-      @Override
-      public void addObject(String key, String text) throws Exception {
-        Document doc = new Document();
-        doc.add(new TextField("key", key, Store.YES));
-        doc.add(new TextField("text", text, Store.NO));
-        writer.addDocument(doc);
-      }
-
-      @Override
-      public void commit() throws Exception {
-        writer.commit();
-        searcherManager.maybeRefresh();
-      }
-
-      @Override
-      public void cleanup() throws Exception {
-        writer.close();
-      }
-      
-      @Override
-      public void waitForAsync() throws Exception {
-        //do nothing
-      }
-
-      @Override
-      public int query(Query query) throws Exception {
-        IndexSearcher searcher = searcherManager.acquire();
-        try {
-          return searcher.count(query);
-        } finally {
-          searcherManager.release(searcher);
-        }
-      }
-      
-    });
-    
-  }
-  
-  private void doTest(String testName, TestCallbacks callbacks) throws Exception {
-
-    //Create some random words. We need to be careful
-    //to make sure we get NUM_WORDS distinct words here
-    Set<String> wordSet = new HashSet<String>();
-    Random rand = new Random();
-    while(wordSet.size() < NUM_WORDS) {
-      int length = rand.nextInt(12) + 3;
-      char[] text = new char[length];
-      for(int i = 0; i < length; i++) {
-        text[i] = (char) (rand.nextInt(26) + 97);
-      }
-      wordSet.add(new String(text));
-    }
-    List<String> words = new ArrayList<String>(wordSet.size());
-    words.addAll(wordSet);
-    
-    
-    
-    //warm up
-    writeRandomWords(callbacks, words, rand, NUM_ENTRIES / 10, NUM_QUERIES / 10, COMMIT_INTERVAL[0]);
-    
-    //Do the actual test
-    
-    for(int i = 0; i < COMMIT_INTERVAL.length; i++) {
-      Results results = writeRandomWords(callbacks, words, rand, NUM_ENTRIES, NUM_QUERIES / 10, COMMIT_INTERVAL[i]);
-    
-      System.out.println(testName + " writes(entries=" + NUM_ENTRIES + ", commit=" + COMMIT_INTERVAL[i] + "): " + TimeUnit.NANOSECONDS.toMillis(results.writeTime));
-      System.out.println(testName + " queries(entries=" + NUM_ENTRIES + ", commit=" + COMMIT_INTERVAL[i] + "): " + TimeUnit.NANOSECONDS.toMillis(results.queryTime));
-    }
-  }
-
-  private Results writeRandomWords(TestCallbacks callbacks, List<String> words,
-      Random rand, int numEntries, int numQueries, int commitInterval) throws Exception {
-    Results results  = new Results();
-    callbacks.init();
-    int[] counts = new int[words.size()];
-    long start = System.nanoTime();
-    try {
-      for(int i =0; i < numEntries; i++) {
-        int word1 = rand.nextInt(words.size());
-        int word2 = rand.nextInt(words.size());
-        counts[word1]++;
-        counts[word2]++;
-        String value = words.get(word1) + " " + words.get(word2);
-        callbacks.addObject("key" + i, value);
-
-        if(i % commitInterval == 0 && i != 0) {
-          callbacks.commit();
-        }
-      }
-      callbacks.commit();
-      callbacks.waitForAsync();
-      long end = System.nanoTime();
-      results.writeTime = end - start;
-      
-      
-      start = System.nanoTime();
-      for(int i=0; i < numQueries; i++) {
-        int wordIndex = rand.nextInt(words.size());
-        String word = words.get(wordIndex);
-        Query query = new TermQuery(new Term("text", word));
-        int size  = callbacks.query(query);
-//        int size  = callbacks.query(parser.parse(word));
-        //All of my tests sometimes seem to be missing a couple of words, including the stock lucene
-//        assertEquals("Error on query " + i + " word=" + word, counts[wordIndex], size);
-      }
-      end = System.nanoTime();
-      results.queryTime = end - start;
-      
-      return results;
-    } finally {
-      callbacks.cleanup();
-    }
-  }
-
-  private static class TestObject implements DataSerializable {
-    private String text;
-
-    public TestObject() {
-      
-    }
-    
-    public TestObject(String text) {
-      super();
-      this.text = text;
-    }
-
-    @Override
-    public void toData(DataOutput out) throws IOException {
-      DataSerializer.writeString(text, out);
-    }
-
-    @Override
-    public void fromData(DataInput in)
-        throws IOException, ClassNotFoundException {
-      text = DataSerializer.readString(in);
-    }
-
-    @Override
-    public String toString() {
-      return text;
-    }
-    
-    
-  }
-  
-  private interface TestCallbacks {
-    public void init() throws Exception;
-    public int query(Query query) throws Exception;
-    public void addObject(String key, String text)  throws Exception;
-    public void commit() throws Exception;
-    public void waitForAsync() throws Exception;
-    public void cleanup() throws Exception;
-  }
-  
-  private static class Results {
-    long writeTime;
-    long queryTime;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
new file mode 100644
index 0000000..74f3742
--- /dev/null
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
@@ -0,0 +1,439 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.gemstone.gemfire.cache.lucene.internal.repository;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.RAMDirectory;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.DataSerializable;
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneQuery;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
+import com.gemstone.gemfire.cache.query.QueryException;
+import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
+
+/**
+ * Microbenchmark of the IndexRepository to compare an
+ * IndexRepository built on top of cache with a 
+ * stock lucene IndexWriter with a RAMDirectory.
+ */
+@Category(PerformanceTest.class)
+@Ignore("Tests have no assertions")
+public class IndexRepositoryImplPerformanceTest {
+  
+  private static final int NUM_WORDS = 1000;
+  private static int[] COMMIT_INTERVAL = new int[] {100, 1000, 5000};
+  private static int NUM_ENTRIES = 500_000;
+  private static int NUM_QUERIES = 500_000;
+
+  private StandardAnalyzer analyzer = new StandardAnalyzer();
+  
+  @Test
+  public  void testIndexRepository() throws Exception {
+    
+
+    doTest("IndexRepository", new TestCallbacks() {
+
+      private Cache cache;
+      private IndexRepositoryImpl repo;
+      private IndexWriter writer;
+
+      @Override
+      public void addObject(String key, String text) throws Exception {
+        repo.create(key, new TestObject(text));
+      }
+
+      @Override
+      public void commit()  throws Exception {
+        repo.commit();
+      }
+
+      @Override
+      public void init() throws Exception {
+        cache = new CacheFactory().set("mcast-port", "0")
+            .set("log-level", "error")
+            .create();
+        Region<String, File> fileRegion = cache.<String, File>createRegionFactory(RegionShortcut.REPLICATE).create("files");
+        Region<ChunkKey, byte[]> chunkRegion = cache.<ChunkKey, byte[]>createRegionFactory(RegionShortcut.REPLICATE).create("chunks");
+
+        RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion);
+        
+        
+        IndexWriterConfig config = new IndexWriterConfig(analyzer);
+        writer = new IndexWriter(dir, config);
+        String[] indexedFields= new String[] {"text"};
+        HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(indexedFields);
+        repo = new IndexRepositoryImpl(fileRegion, writer, mapper);
+      }
+
+      @Override
+      public void cleanup() throws IOException {
+        writer.close();
+        cache.close();
+      }
+
+      @Override
+      public void waitForAsync() throws Exception {
+        //do nothing
+      }
+
+      @Override
+      public int query(Query query) throws IOException {
+        TopEntriesCollector collector = new TopEntriesCollector();
+        repo.query(query, 100, collector);
+        return collector.size();
+      }
+    });
+  }
+  
+  /**
+   * Test our full lucene index implementation
+   * @throws Exception
+   */
+  @Test
+  public void testLuceneIndex() throws Exception {
+    
+
+    doTest("LuceneIndex", new TestCallbacks() {
+
+      private Cache cache;
+      private Region<String, TestObject> region;
+      private LuceneService service;
+
+      @Override
+      public void addObject(String key, String text) throws Exception {
+        region.create(key, new TestObject(text));
+      }
+
+      @Override
+      public void commit()  throws Exception {
+        //NA
+      }
+
+      @Override
+      public void init() throws Exception {
+        cache = new CacheFactory().set("mcast-port", "0")
+            .set("log-level", "warning")
+            .create();
+        service = LuceneServiceProvider.get(cache);
+        service.createIndex("index", "/region", "text");
+        region = cache.<String, TestObject>createRegionFactory(RegionShortcut.PARTITION)
+            .setPartitionAttributes(new PartitionAttributesFactory<>().setTotalNumBuckets(1).create())
+            .create("region");
+      }
+
+      @Override
+      public void cleanup() throws IOException {
+        cache.close();
+      }
+      
+      @Override
+      public void waitForAsync() throws Exception {
+        AsyncEventQueue aeq = cache.getAsyncEventQueue(LuceneServiceImpl.getUniqueIndexName("index", "/region"));
+        
+        //We will be at most 10 ms off
+        while(aeq.size() > 0) {
+          Thread.sleep(10);
+        }
+      }
+
+      @Override
+      public int query(final Query query) throws Exception {
+        LuceneQuery<Object, Object> luceneQuery = service.createLuceneQueryFactory().create("index", "/region", new LuceneQueryProvider() {
+          
+          @Override
+          public Query getQuery(LuceneIndex index) throws QueryException {
+            return query;
+          }
+        });
+        
+        LuceneQueryResults<Object, Object> results = luceneQuery.search();
+        return results.size();
+      }
+    });
+  }
+  
+  @Test
+  public  void testLuceneWithRegionDirectory() throws Exception {
+    doTest("RegionDirectory", new TestCallbacks() {
+
+      private IndexWriter writer;
+      private SearcherManager searcherManager;
+
+      @Override
+      public void init() throws Exception {
+        RegionDirectory dir = new RegionDirectory(new ConcurrentHashMap<String, File>(), new ConcurrentHashMap<ChunkKey, byte[]>());
+        IndexWriterConfig config = new IndexWriterConfig(analyzer);
+        writer = new IndexWriter(dir, config);
+        searcherManager = new SearcherManager(writer, true, null);
+      }
+
+      @Override
+      public void addObject(String key, String text) throws Exception {
+        Document doc = new Document();
+        doc.add(new TextField("key", key, Store.YES));
+        doc.add(new TextField("text", text, Store.NO));
+        writer.addDocument(doc);
+      }
+
+      @Override
+      public void commit() throws Exception {
+        writer.commit();
+        searcherManager.maybeRefresh();
+      }
+
+      @Override
+      public void cleanup() throws Exception {
+        writer.close();
+      }
+      
+      @Override
+      public void waitForAsync() throws Exception {
+        //do nothing
+      }
+
+      @Override
+      public int query(Query query) throws Exception {
+        IndexSearcher searcher = searcherManager.acquire();
+        try {
+          return searcher.count(query);
+        } finally {
+          searcherManager.release(searcher);
+        }
+      }
+      
+    });
+    
+  }
+  
+  @Test
+  public  void testLucene() throws Exception {
+    doTest("Lucene", new TestCallbacks() {
+
+      private IndexWriter writer;
+      private SearcherManager searcherManager;
+
+      @Override
+      public void init() throws Exception {
+        RAMDirectory dir = new RAMDirectory();
+        IndexWriterConfig config = new IndexWriterConfig(analyzer);
+        writer = new IndexWriter(dir, config);
+        searcherManager = new SearcherManager(writer, true, null);
+      }
+
+      @Override
+      public void addObject(String key, String text) throws Exception {
+        Document doc = new Document();
+        doc.add(new TextField("key", key, Store.YES));
+        doc.add(new TextField("text", text, Store.NO));
+        writer.addDocument(doc);
+      }
+
+      @Override
+      public void commit() throws Exception {
+        writer.commit();
+        searcherManager.maybeRefresh();
+      }
+
+      @Override
+      public void cleanup() throws Exception {
+        writer.close();
+      }
+      
+      @Override
+      public void waitForAsync() throws Exception {
+        //do nothing
+      }
+
+      @Override
+      public int query(Query query) throws Exception {
+        IndexSearcher searcher = searcherManager.acquire();
+        try {
+          return searcher.count(query);
+        } finally {
+          searcherManager.release(searcher);
+        }
+      }
+      
+    });
+    
+  }
+  
+  private void doTest(String testName, TestCallbacks callbacks) throws Exception {
+
+    //Create some random words. We need to be careful
+    //to make sure we get NUM_WORDS distinct words here
+    Set<String> wordSet = new HashSet<String>();
+    Random rand = new Random();
+    while(wordSet.size() < NUM_WORDS) {
+      int length = rand.nextInt(12) + 3;
+      char[] text = new char[length];
+      for(int i = 0; i < length; i++) {
+        text[i] = (char) (rand.nextInt(26) + 97);
+      }
+      wordSet.add(new String(text));
+    }
+    List<String> words = new ArrayList<String>(wordSet.size());
+    words.addAll(wordSet);
+    
+    
+    
+    //warm up
+    writeRandomWords(callbacks, words, rand, NUM_ENTRIES / 10, NUM_QUERIES / 10, COMMIT_INTERVAL[0]);
+    
+    //Do the actual test
+    
+    for(int i = 0; i < COMMIT_INTERVAL.length; i++) {
+      Results results = writeRandomWords(callbacks, words, rand, NUM_ENTRIES, NUM_QUERIES / 10, COMMIT_INTERVAL[i]);
+    
+      System.out.println(testName + " writes(entries=" + NUM_ENTRIES + ", commit=" + COMMIT_INTERVAL[i] + "): " + TimeUnit.NANOSECONDS.toMillis(results.writeTime));
+      System.out.println(testName + " queries(entries=" + NUM_ENTRIES + ", commit=" + COMMIT_INTERVAL[i] + "): " + TimeUnit.NANOSECONDS.toMillis(results.queryTime));
+    }
+  }
+
+  private Results writeRandomWords(TestCallbacks callbacks, List<String> words,
+      Random rand, int numEntries, int numQueries, int commitInterval) throws Exception {
+    Results results  = new Results();
+    callbacks.init();
+    int[] counts = new int[words.size()];
+    long start = System.nanoTime();
+    try {
+      for(int i =0; i < numEntries; i++) {
+        int word1 = rand.nextInt(words.size());
+        int word2 = rand.nextInt(words.size());
+        counts[word1]++;
+        counts[word2]++;
+        String value = words.get(word1) + " " + words.get(word2);
+        callbacks.addObject("key" + i, value);
+
+        if(i % commitInterval == 0 && i != 0) {
+          callbacks.commit();
+        }
+      }
+      callbacks.commit();
+      callbacks.waitForAsync();
+      long end = System.nanoTime();
+      results.writeTime = end - start;
+      
+      
+      start = System.nanoTime();
+      for(int i=0; i < numQueries; i++) {
+        int wordIndex = rand.nextInt(words.size());
+        String word = words.get(wordIndex);
+        Query query = new TermQuery(new Term("text", word));
+        int size  = callbacks.query(query);
+//        int size  = callbacks.query(parser.parse(word));
+        //All of my tests sometimes seem to be missing a couple of words, including the stock lucene
+//        assertEquals("Error on query " + i + " word=" + word, counts[wordIndex], size);
+      }
+      end = System.nanoTime();
+      results.queryTime = end - start;
+      
+      return results;
+    } finally {
+      callbacks.cleanup();
+    }
+  }
+
+  private static class TestObject implements DataSerializable {
+    private String text;
+
+    public TestObject() {
+      
+    }
+    
+    public TestObject(String text) {
+      super();
+      this.text = text;
+    }
+
+    @Override
+    public void toData(DataOutput out) throws IOException {
+      DataSerializer.writeString(text, out);
+    }
+
+    @Override
+    public void fromData(DataInput in)
+        throws IOException, ClassNotFoundException {
+      text = DataSerializer.readString(in);
+    }
+
+    @Override
+    public String toString() {
+      return text;
+    }
+    
+    
+  }
+  
+  private interface TestCallbacks {
+    public void init() throws Exception;
+    public int query(Query query) throws Exception;
+    public void addObject(String key, String text)  throws Exception;
+    public void commit() throws Exception;
+    public void waitForAsync() throws Exception;
+    public void cleanup() throws Exception;
+  }
+  
+  private static class Results {
+    long writeTime;
+    long queryTime;
+  }
+}


[13/50] [abbrv] incubator-geode git commit: Revert "GEODE-715: Fix import that was broken in merge to develop"

Posted by ab...@apache.org.
Revert "GEODE-715: Fix import that was broken in merge to develop"

This reverts commit 18a17d787bcf69179d1bdeb015f33141d5d8486b.


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

Branch: refs/heads/master
Commit: a8262812a0c85163b21eeaf0e34cff6af0b15d70
Parents: 4bc1b28
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Jan 4 16:27:18 2016 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Mon Jan 4 16:27:18 2016 -0800

----------------------------------------------------------------------
 gemfire-core/src/test/java/dunit/DistributedTestCase.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a8262812/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 e89f694..6fa560f 100755
--- a/gemfire-core/src/test/java/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
@@ -88,7 +88,8 @@ import com.gemstone.gemfire.internal.logging.ManagerLogWriter;
 import com.gemstone.gemfire.internal.logging.log4j.LogWriterLogger;
 import com.gemstone.gemfire.management.internal.cli.LogWrapper;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
+
+import dunit.standalone.DUnitLauncher;
 
 /**
  * This class is the superclass of all distributed unit tests.


[12/50] [abbrv] incubator-geode git commit: merge of hot-fix from closed-source 8.2x maintenance branch

Posted by ab...@apache.org.
merge of hot-fix from closed-source 8.2x maintenance branch

The tombstone GC service ran into an NPE removing a tombstone and then
the thread went hot because exception-handling was not advancing to the
next GC candidate.  This revision attempts to avoid the NPE (which was
not reported with line numbers) and alters the exception-handling to
advance to the next tombstone if there is an unexpected problem removing
the current tombstone.


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

Branch: refs/heads/master
Commit: 4bc1b28a13b952f6f043ee9f7f28a8d3a9220f1f
Parents: d19bb41
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Jan 4 15:26:05 2016 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Jan 4 15:28:43 2016 -0800

----------------------------------------------------------------------
 .../gemfire/internal/cache/AbstractRegionMap.java         |  6 ++++--
 .../gemstone/gemfire/internal/cache/TombstoneService.java | 10 ++++++++++
 2 files changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bc1b28a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
index 6d49d74..f4fb044 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
@@ -4269,7 +4269,6 @@ RETRY_LOOP:
   public final boolean removeTombstone(RegionEntry re, VersionHolder version, boolean isEviction, boolean isScheduledTombstone)  {
     boolean result = false;
     int destroyedVersion = version.getEntryVersion();
-    DiskRegion dr = this._getOwner().getDiskRegion();
 
     synchronized(this._getOwner().getSizeGuard()) { // do this sync first; see bug 51985
         synchronized (re) {
@@ -4303,7 +4302,10 @@ RETRY_LOOP:
                 if (isScheduledTombstone) {
                   _getOwner().incTombstoneCount(-1);
                 }
-                _getOwner().getVersionVector().recordGCVersion(version.getMemberID(), version.getRegionVersion());
+                RegionVersionVector vector = _getOwner().getVersionVector();
+                if (vector != null) {
+                  vector.recordGCVersion(version.getMemberID(), version.getRegionVersion());
+                }
               }
             } catch (RegionClearedException e) {
               // if the region has been cleared we don't need to remove the tombstone

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bc1b28a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TombstoneService.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TombstoneService.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TombstoneService.java
index 9fc5032..0814b34 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TombstoneService.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TombstoneService.java
@@ -204,6 +204,10 @@ public class TombstoneService  implements ResourceListener<MemoryEvent> {
    * @param destroyedVersion the version that was destroyed
    */
   public void scheduleTombstone(LocalRegion r, RegionEntry entry, VersionTag destroyedVersion) {
+    if (entry.getVersionStamp() == null) {
+      logger.warn("Detected an attempt to schedule a tombstone for an entry that is not versioned in region " + r.getFullPath(), new Exception("stack trace"));
+      return;
+    }
     boolean useReplicated = useReplicatedQueue(r);
     Tombstone ts = new Tombstone(entry, r, destroyedVersion);
     if (useReplicated) {
@@ -871,6 +875,12 @@ public class TombstoneService  implements ResourceListener<MemoryEvent> {
               return;
             } catch (Exception e) {
               logger.warn(LocalizedMessage.create(LocalizedStrings.GemFireCacheImpl_TOMBSTONE_ERROR), e);
+              currentTombstoneLock.lock();
+              try {
+                currentTombstone = null;
+              } finally {
+                currentTombstoneLock.unlock();
+              }
             }
           }
           if (sleepTime > 0) {


[08/50] [abbrv] incubator-geode git commit: GEODE-715: Move dunit.standalone under com.gemstone.gemfire.test

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/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
deleted file mode 100644
index 7fc762f..0000000
--- a/gemfire-core/src/test/java/dunit/standalone/ProcessManager.java
+++ /dev/null
@@ -1,261 +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 dunit.standalone;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.PrintStream;
-import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-import java.rmi.AccessException;
-import java.rmi.NotBoundException;
-import java.rmi.RemoteException;
-import java.rmi.registry.Registry;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.commons.io.FileUtils;
-
-import com.gemstone.gemfire.internal.FileUtil;
-import com.gemstone.gemfire.internal.logging.LogService;
-
-import dunit.RemoteDUnitVMIF;
-
-/**
- * @author dsmith
- *
- */
-public class ProcessManager {
-  private int namingPort;
-  private Map<Integer, ProcessHolder> processes = new HashMap<Integer, ProcessHolder>();
-  private File log4jConfig;
-  private int pendingVMs;
-  private Registry registry;
-  private int debugPort = Integer.getInteger("dunit.debug.basePort", 0);
-  private int suspendVM = Integer.getInteger("dunit.debug.suspendVM", -100);
-
-  public ProcessManager(int namingPort, Registry registry) {
-    this.namingPort = namingPort;
-    this.registry = registry;
-  }
-  
-  public void launchVMs() throws IOException, NotBoundException {
-    log4jConfig = LogService.findLog4jConfigInCurrentDir();
-  }
-
-  public synchronized void launchVM(int vmNum) throws IOException {
-    if(processes.containsKey(vmNum)) {
-      throw new IllegalStateException("VM " + vmNum + " is already running.");
-    }
-    
-    String[] cmd = buildJavaCommand(vmNum, namingPort);
-    System.out.println("Executing " + Arrays.asList(cmd));
-    File workingDir = getVMDir(vmNum);
-    try {
-      FileUtil.delete(workingDir);
-    } catch(IOException e) {
-      //This delete is occasionally failing on some platforms, maybe due to a lingering
-      //process. Allow the process to be launched anyway.
-      System.err.println("Unable to delete " + workingDir + ". Currently contains " 
-                          + Arrays.asList(workingDir.list()));
-    }
-    workingDir.mkdirs();
-    if (log4jConfig != null) {
-      FileUtils.copyFileToDirectory(log4jConfig, workingDir);
-    }
-    
-    //TODO - delete directory contents, preferably with commons io FileUtils
-    Process process = Runtime.getRuntime().exec(cmd, null, workingDir);
-    pendingVMs++;
-    ProcessHolder holder = new ProcessHolder(process);
-    processes.put(vmNum, holder);
-    linkStreams(vmNum, holder, process.getErrorStream(), System.err);
-    linkStreams(vmNum, holder, process.getInputStream(), System.out);
-  }
-
-  public static File getVMDir(int vmNum) {
-    return new File(DUnitLauncher.DUNIT_DIR, "vm" + vmNum);
-  }
-  
-  public synchronized void killVMs() {
-    for(ProcessHolder process : processes.values()) {
-      if(process != null) {
-        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);
-    }
-    try {
-      ProcessHolder holder = processes.remove(vmNum);
-      holder.kill();
-      holder.getProcess().waitFor();
-      launchVM(vmNum);
-    } catch (InterruptedException | IOException e) {
-      throw new RuntimeException("Unable to restart VM " + vmNum, e);
-    }
-  }
-   
-  private void linkStreams(final int vmNum, final ProcessHolder holder, final InputStream in, final PrintStream out) {
-    Thread ioTransport = new Thread() {
-      public void run() {
-        BufferedReader reader = new BufferedReader(new InputStreamReader(in));
-        String vmName = (vmNum==-2)? "[locator]" : "[vm_"+vmNum+"]";
-        try {
-          String line = reader.readLine();
-          while(line != null) {
-            if (line.length() == 0) {
-              out.println();
-            } else {
-              out.print(vmName);
-              out.println(line);
-            }
-            line = reader.readLine();
-          }
-        } catch(Exception e) {
-          if(!holder.isKilled()) {
-            out.println("Error transporting IO from child process");
-            e.printStackTrace(out);
-          }
-        }
-      }
-    };
-
-    ioTransport.setDaemon(true);
-    ioTransport.start();
-  }
-
-  private String[] buildJavaCommand(int vmNum, int namingPort) {
-    String cmd = System.getProperty( "java.home" ) + File.separator + "bin" + File.separator + "java";
-    String classPath = System.getProperty("java.class.path");
-    //String tmpDir = System.getProperty("java.io.tmpdir");
-    String agent = getAgentString();
-
-    String jdkDebug = "";
-    if (debugPort > 0) {
-      jdkDebug += ",address=" + debugPort;
-      debugPort++;
-    }
-
-    String jdkSuspend = vmNum == suspendVM ? "y" : "n";
-
-    return new String[] {
-      cmd, "-classpath", classPath,
-      "-D" + DUnitLauncher.RMI_PORT_PARAM + "=" + namingPort,
-      "-D" + DUnitLauncher.VM_NUM_PARAM + "=" + vmNum,
-      "-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=" + jdkSuspend + jdkDebug,
-      "-XX:+HeapDumpOnOutOfMemoryError",
-      "-Xmx512m",
-      "-Dgemfire.DEFAULT_MAX_OPLOG_SIZE=10",
-      "-Dgemfire.disallowMcastDefaults=true",
-      "-ea",
-      agent,
-      "dunit.standalone.ChildVM"
-    };
-  }
-  
-  /**
-   * Get the java agent passed to this process and pass it to the child VMs.
-   * This was added to support jacoco code coverage reports
-   */
-  private String getAgentString() {
-    RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
-    if (runtimeBean != null) {
-      for(String arg: runtimeBean.getInputArguments()) {
-        if(arg.contains("-javaagent:")) {
-          //HACK for gradle bug  GRADLE-2859. Jacoco is passing a relative path
-          //That won't work when we pass this to dunit VMs in a different 
-          //directory
-          arg = arg.replace("-javaagent:..", "-javaagent:" + System.getProperty("user.dir") + File.separator + "..");
-          arg = arg.replace("destfile=..", "destfile=" + System.getProperty("user.dir") + File.separator + "..");
-          return arg;
-        }
-      }
-    }
-    
-    return "-DdummyArg=true";
-  }
-
-  synchronized void signalVMReady() {
-    pendingVMs--;
-    this.notifyAll();
-  }
-  
-  public synchronized boolean waitForVMs(long timeout) throws InterruptedException {
-    long end = System.currentTimeMillis() + timeout;
-    while(pendingVMs > 0) {
-      long remaining = end - System.currentTimeMillis();
-      if(remaining <= 0) {
-        return false;
-      }
-      this.wait(remaining);
-    }
-    
-    return true;
-  }
-  
-  private static class ProcessHolder {
-    private final Process process;
-    private volatile boolean killed = false;
-    
-    public ProcessHolder(Process process) {
-      this.process = process;
-    }
-
-    public void kill() {
-      this.killed = true;
-      process.destroy();
-      
-    }
-
-    public Process getProcess() {
-      return process;
-    }
-
-    public boolean isKilled() {
-      return killed;
-    }
-    
-    public boolean isAlive() {
-      return !killed && process.isAlive();
-    }
-  }
-
-  public RemoteDUnitVMIF getStub(int i) throws AccessException, RemoteException, NotBoundException, InterruptedException {
-    waitForVMs(DUnitLauncher.STARTUP_TIMEOUT);
-    return (RemoteDUnitVMIF) registry.lookup("vm" + i);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/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
deleted file mode 100644
index 742dc55..0000000
--- a/gemfire-core/src/test/java/dunit/standalone/RemoteDUnitVM.java
+++ /dev/null
@@ -1,144 +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 dunit.standalone;
-
-import hydra.MethExecutor;
-import hydra.MethExecutorResult;
-
-import java.rmi.RemoteException;
-import java.rmi.server.UnicastRemoteObject;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.internal.logging.LogService;
-
-import dunit.RemoteDUnitVMIF;
-
-/**
- * @author dsmith
- *
- */
-public class RemoteDUnitVM extends UnicastRemoteObject implements RemoteDUnitVMIF {
-  
-  private static final Logger logger = LogService.getLogger();
-  
-  public RemoteDUnitVM() throws RemoteException {
-    super();
-  }
-
-  /** 
-   * Called remotely by the master controller to cause the client to execute 
-   * the instance method on the object.  Does this synchronously (does not spawn
-   * a thread).  This method is used by the unit test framework, dunit.
-   *
-   * @param obj the object to execute the method on
-   * @param methodName the name of the method to execute
-   * @return the result of method execution
-   */ 
-   public MethExecutorResult executeMethodOnObject( Object obj, String methodName ) {
-     String name = obj.getClass().getName() + "." + methodName + 
-       " on object: " + obj;
-     logger.info("Received method: " + name);
-     long start = System.currentTimeMillis();
-     MethExecutorResult result = MethExecutor.executeObject( obj, methodName );
-     long delta = System.currentTimeMillis() - start;
-     logger.info( "Got result: " + result.toString().trim()  + " from " +
-               name + " (took " + delta + " ms)");
-     return result;
-   }
-
-   /**
-    * Executes a given instance method on a given object with the given
-    * arguments. 
-    */
-   public MethExecutorResult executeMethodOnObject(Object obj,
-                                                   String methodName,
-                                                   Object[] args) {
-     String name = obj.getClass().getName() + "." + methodName + 
-              (args != null ? " with " + args.length + " args": "") +
-       " on object: " + obj;
-     logger.info("Received method: " + name);
-     long start = System.currentTimeMillis();
-     MethExecutorResult result = 
-       MethExecutor.executeObject(obj, methodName, args);
-     long delta = System.currentTimeMillis() - start;
-     logger.info( "Got result: " + result.toString() + " from " + name + 
-               " (took " + delta + " ms)");
-     return result;
-   }
-
-  /** 
-   * Called remotely by the master controller to cause the client to execute 
-   * the method on the class.  Does this synchronously (does not spawn a thread).
-   * This method is used by the unit test framework, dunit.
-   *
-   * @param className the name of the class execute
-   * @param methodName the name of the method to execute
-   * @return the result of method execution
-   */ 
-   public MethExecutorResult executeMethodOnClass( String className, String methodName ) {
-     String name = className + "." + methodName;
-     logger.info("Received method: " +  name);
-     long start = System.currentTimeMillis();
-     MethExecutorResult result = MethExecutor.execute( className, methodName );
-     long delta = System.currentTimeMillis() - start;
-     logger.info( "Got result: " + result.toString() + " from " + name + 
-               " (took " + delta + " ms)");
-     
-     return result;
-   }
-
-   /**
-    * Executes a given static method in a given class with the given
-    * arguments. 
-    */
-   public MethExecutorResult executeMethodOnClass(String className,
-                                                  String methodName,
-                                                  Object[] args) {
-     String name = className + "." + methodName + 
-       (args != null ? " with " + args.length + " args": "");
-     logger.info("Received method: " + name);
-     long start = System.currentTimeMillis();
-     MethExecutorResult result = 
-       MethExecutor.execute(className, methodName, args);
-     long delta = System.currentTimeMillis() - start;
-     logger.info( "Got result: " + result.toString() + " from " + name +
-               " (took " + delta + " ms)");
-     return result;
-   }
-
-  public void executeTask(int tsid, int type, int index) throws RemoteException {
-    throw new UnsupportedOperationException();
-    
-  }
-  
-  public void runShutdownHook() throws RemoteException {
-    
-  }
-
-  public void notifyDynamicActionComplete(int actionId) throws RemoteException {
-    throw new UnsupportedOperationException();
-    
-  }
-
-  public void shutDownVM() throws RemoteException {
-    ChildVM.stopVM();
-  }
-
-  public void disconnectVM() throws RemoteException {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/standalone/StandAloneDUnitEnv.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/standalone/StandAloneDUnitEnv.java b/gemfire-core/src/test/java/dunit/standalone/StandAloneDUnitEnv.java
deleted file mode 100644
index 085035d..0000000
--- a/gemfire-core/src/test/java/dunit/standalone/StandAloneDUnitEnv.java
+++ /dev/null
@@ -1,75 +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 dunit.standalone;
-
-import java.io.File;
-import java.rmi.RemoteException;
-import java.util.Properties;
-
-import dunit.BounceResult;
-import dunit.DUnitEnv;
-import dunit.standalone.DUnitLauncher.MasterRemote;
-
-public class StandAloneDUnitEnv extends DUnitEnv {
-
-  private MasterRemote master;
-
-  public StandAloneDUnitEnv(MasterRemote master) {
-    this.master = master;
-  }
-
-  @Override
-  public String getLocatorString() {
-    return DUnitLauncher.getLocatorString();
-  }
-
-  @Override
-  public String getLocatorAddress() {
-    return "localhost";
-  }
-  
-  @Override
-  public int getLocatorPort() {
-    return DUnitLauncher.locatorPort;
-  }
-
-  @Override
-  public Properties getDistributedSystemProperties() {
-    return DUnitLauncher.getDistributedSystemProperties();
-  }
-
-  @Override
-  public int getPid() {
-    return Integer.getInteger(DUnitLauncher.VM_NUM_PARAM, -1).intValue();
-  }
-
-  @Override
-  public int getVMID() {
-    return getPid();
-  }
-
-  @Override
-  public BounceResult bounce(int pid) throws RemoteException {
-    return master.bounce(pid);
-  }
-
-  @Override
-  public File getWorkingDirectory(int pid) {
-    return ProcessManager.getVMDir(pid);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/tests/BasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/tests/BasicDUnitTest.java b/gemfire-core/src/test/java/dunit/tests/BasicDUnitTest.java
deleted file mode 100644
index c284b3a..0000000
--- a/gemfire-core/src/test/java/dunit/tests/BasicDUnitTest.java
+++ /dev/null
@@ -1,132 +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 dunit.tests;
-
-import java.util.Properties;
-
-import dunit.AsyncInvocation;
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.RMIException;
-import dunit.VM;
-
-/**
- * This class tests the basic functionality of the distributed unit
- * test framework.
- */
-public class BasicDUnitTest extends DistributedTestCase {
-
-  public BasicDUnitTest(String name) {
-    super(name);
-  }
-
-  ////////  Test Methods
-
-  /**
-   * Tests how the Hydra framework handles an error
-   */
-  public void _testDontCatchRemoteException() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    vm.invoke(this.getClass(), "remoteThrowException");
-  }
-
-  public void testRemoteInvocationWithException() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(this.getClass(), "remoteThrowException");
-      fail("Should have thrown a BasicTestException");
-
-    } catch (RMIException ex) {
-      assertTrue(ex.getCause() instanceof BasicTestException);
-    }
-  } 
-
-  static class BasicTestException extends RuntimeException {
-    BasicTestException() {
-      this("Test exception.  Please ignore.");
-    }
-
-    BasicTestException(String s) {
-      super(s);
-    }
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   *
-   */
-  protected static void remoteThrowException() {
-    String s = "Test exception.  Please ignore.";
-    throw new BasicTestException(s);
-  }
-
-  public void _testRemoteInvocationBoolean() {
-
-  }
-
-  public void testRemoteInvokeAsync() throws InterruptedException {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    String name = this.getUniqueName();
-    String value = "Hello";
-
-    AsyncInvocation ai =
-      vm.invokeAsync(this.getClass(), "remoteBind", 
-                     new Object[] { name, value });
-    ai.join();
-    // TODO shouldn't we call fail() here?
-    if (ai.exceptionOccurred()) {
-      fail("remoteBind failed", ai.getException());
-    }
-
-    ai = vm.invokeAsync(this.getClass(), "remoteValidateBind",
-                        new Object[] {name, value });
-    ai.join();
-    if (ai.exceptionOccurred()) {
-      fail("remoteValidateBind failed", ai.getException());
-    }
-  }
-
-  private static Properties bindings = new Properties();
-  private static void remoteBind(String name, String s) {
-    new BasicDUnitTest("bogus").getSystem(); // forces connection
-    bindings.setProperty(name, s);
-  }
-
-  private static void remoteValidateBind(String name, String expected)
-  {
-    assertEquals(expected, bindings.getProperty(name));
-  }
-
-  public void testRemoteInvokeAsyncWithException() 
-    throws InterruptedException {
-
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-//    String name = this.getUniqueName();
-//    String value = "Hello";
-
-    AsyncInvocation ai =
-      vm.invokeAsync(this.getClass(), "remoteThrowException");
-    ai.join();
-    assertTrue(ai.exceptionOccurred());
-    Throwable ex = ai.getException();
-    assertTrue(ex instanceof BasicTestException);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/tests/TestFailure.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/tests/TestFailure.java b/gemfire-core/src/test/java/dunit/tests/TestFailure.java
deleted file mode 100644
index 17a39fa..0000000
--- a/gemfire-core/src/test/java/dunit/tests/TestFailure.java
+++ /dev/null
@@ -1,50 +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 dunit.tests;
-
-import dunit.*;
-
-/**
- * The tests in this class always fail.  It is used when developing
- * DUnit to give us an idea of how test failure are logged, etc.
- *
- * @author David Whitlock
- *
- * @since 3.0
- */
-public class TestFailure extends DistributedTestCase {
-
-  public TestFailure(String name) {
-    super(name);
-  }
-
-  ////////  Test Methods
-
-  public void testFailure() {
-    assertTrue("Test Failure", false);
-  }
-
-  public void testError() {
-    String s = "Test Error";
-    throw new Error(s);
-  }
-
-  public void testHang() throws InterruptedException {
-    Thread.sleep(100000 * 1000);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/tests/VMDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/tests/VMDUnitTest.java b/gemfire-core/src/test/java/dunit/tests/VMDUnitTest.java
deleted file mode 100644
index 27736dc..0000000
--- a/gemfire-core/src/test/java/dunit/tests/VMDUnitTest.java
+++ /dev/null
@@ -1,237 +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 dunit.tests;
-
-import dunit.*;
-
-import java.io.Serializable;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * This class tests the functionality of the {@link VM} class.
- */
-public class VMDUnitTest extends DistributedTestCase {
-
-  private static final boolean BOOLEAN_VALUE = true;
-  private static final byte BYTE_VALUE = (byte) 40;
-  private static final long LONG_VALUE = 42L;
-  private static final String STRING_VALUE = "BLAH BLAH BLAH";
-
-  public VMDUnitTest(String name) {
-    super(name);
-  }
-
-  ////////  Test Methods
-
-  public void notestInvokeNonExistentMethod() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(VMDUnitTest.class, "nonExistentMethod");
-      fail("Should have thrown an RMIException");
-
-    } catch (RMIException ex) {
-      String s = "Excepted a NoSuchMethodException, got a " +
-        ex.getCause();;
-      assertTrue(s, ex.getCause() instanceof NoSuchMethodException);
-    }
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   * @return
-   */
-  protected static byte remoteByteMethod() {
-    return BYTE_VALUE;
-  }
-
-  public void notestInvokeStaticBoolean() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(BOOLEAN_VALUE,
-                 vm.invokeBoolean(VMDUnitTest.class, "remoteBooleanMethod")); 
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   * @return
-   */
-  protected static boolean remoteBooleanMethod() {
-    return BOOLEAN_VALUE;
-  }
-
-  public void notestInvokeStaticBooleanNotBoolean() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invokeBoolean(VMDUnitTest.class, "remoteByteMethod");
-      fail("Should have thrown an IllegalArgumentException");
-
-    } catch (IllegalArgumentException ex) {
-      
-    }
-  }
-
-  public void notestInvokeStaticLong() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(LONG_VALUE,
-                 vm.invokeLong(VMDUnitTest.class, "remoteLongMethod")); 
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   * @return
-   */
-  protected static long remoteLongMethod() {
-    return LONG_VALUE;
-  }
-
-  public void notestInvokeStaticLongNotLong() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invokeLong(VMDUnitTest.class, "remoteByteMethod");
-      fail("Should have thrown an IllegalArgumentException");
-
-    } catch (IllegalArgumentException ex) {
-      
-    }
-  }
-
-  protected static class ClassWithLong implements Serializable {
-    public long getLong() {
-      return LONG_VALUE;
-    }
-  }
-
-  protected static class ClassWithByte implements Serializable {
-    public byte getByte() {
-      return BYTE_VALUE;
-    }
-  }
-
-  public void notestInvokeInstanceLong() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(LONG_VALUE,
-                 vm.invokeLong(new ClassWithLong(), "getLong"));
-  }
-
-  public void notestInvokeInstanceLongNotLong() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invokeLong(new ClassWithByte(), "getByte");
-      fail("Should have thrown an IllegalArgumentException");
-
-    } catch (IllegalArgumentException ex) {
-
-    }
-  }
-
-  protected static class InvokeRunnable
-    implements Serializable, Runnable {
-
-    public void run() {
-      throw new BasicDUnitTest.BasicTestException();
-    }
-  }
-
-  protected static class ClassWithString implements Serializable {
-    public String getString() {
-      return STRING_VALUE;
-    }
-  }
-
-  public void notestInvokeInstance() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(STRING_VALUE,
-                 vm.invoke(new ClassWithString(), "getString"));
-  }
-
-  public void notestInvokeRunnable() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(new InvokeRunnable());
-      fail("Should have thrown a BasicTestException");
-
-    } catch (RMIException ex) {
-      assertTrue(ex.getCause() instanceof BasicDUnitTest.BasicTestException);
-    }
-  }
-  
-  private static final AtomicInteger COUNTER = new AtomicInteger();
-  public static Integer getAndIncStaticCount() {
-    return new Integer(COUNTER.getAndIncrement());
-  }
-  public static Integer incrementStaticCount(Integer inc) {
-    return new Integer(COUNTER.addAndGet(inc.intValue()));
-  }
-  public static void incStaticCount() {
-    COUNTER.incrementAndGet();
-  }
-  public static class VMTestObject implements Serializable {
-    private static final long serialVersionUID = 1L;
-    private final AtomicInteger val;
-    public VMTestObject(int init) {
-      this.val = new AtomicInteger(init);
-    }
-    public Integer get() {
-      return new Integer(this.val.get());
-    }
-    public Integer incrementAndGet() {
-      return new Integer(this.val.incrementAndGet());
-    }
-    public void set(Integer newVal) {
-      this.val.set(newVal.intValue());
-    }
-  }
-  public void testReturnValue() throws Exception {
-    final Host host = Host.getHost(0);
-    final VM vm = host.getVM(0);
-    // Assert class static invocation works
-    AsyncInvocation a1 = vm.invokeAsync(getClass(), "getAndIncStaticCount");
-    a1.join();
-    assertEquals(new Integer(0), a1.getReturnValue());
-    // Assert class static invocation with args works
-    a1 = vm.invokeAsync(getClass(), "incrementStaticCount", new Object[] {new Integer(2)});
-    a1.join();
-    assertEquals(new Integer(3), a1.getReturnValue());
-    // Assert that previous values are not returned when invoking method w/ no return val
-    a1 = vm.invokeAsync(getClass(), "incStaticCount");
-    a1.join();
-    assertNull(a1.getReturnValue());
-    // Assert that previous null returns are over-written 
-    a1 = vm.invokeAsync(getClass(), "getAndIncStaticCount");
-    a1.join();
-    assertEquals(new Integer(4), a1.getReturnValue());
-
-    // Assert object method invocation works with zero arg method
-    final VMTestObject o = new VMTestObject(0);
-    a1 = vm.invokeAsync(o, "incrementAndGet", new Object[] {});
-    a1.join();
-    assertEquals(new Integer(1), a1.getReturnValue());
-    // Assert object method invocation works with no return
-    a1 = vm.invokeAsync(o, "set", new Object[] {new Integer(3)});
-    a1.join();
-    assertNull(a1.getReturnValue());
-  }
-}


[04/50] [abbrv] incubator-geode git commit: GEODE-714: Modify all tests to use JUnit Categories

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java
deleted file mode 100755
index 90679d9..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java
+++ /dev/null
@@ -1,1015 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.cache.tier.sockets;
-
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.InterestResultPolicy;
-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.PoolManager;
-import com.gemstone.gemfire.cache.client.internal.Connection;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.client.internal.RegisterInterestTracker;
-import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
-import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
-import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
-import com.gemstone.gemfire.internal.cache.CacheServerImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.tier.InterestType;
-
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.VM;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-
-/**
- * Tests Interest Registration Functionality
- */
-@SuppressWarnings({"deprecation", "rawtypes", "serial", "unchecked"})
-public class HAInterestBaseTest extends DistributedTestCase {
-  
-  protected static final int TIMEOUT_MILLIS = 60 * 1000;
-  protected static final int INTERVAL_MILLIS = 10;
-  
-  protected static final String REGION_NAME = "HAInterestBaseTest_region";
-  
-  protected static final String k1 = "k1";
-  protected static final String k2 = "k2";
-  protected static final String client_k1 = "client-k1";
-  protected static final String client_k2 = "client-k2";
-  protected static final String server_k1 = "server-k1";
-  protected static final String server_k2 = "server-k2";
-  protected static final String server_k1_updated = "server_k1_updated";
-
-  protected static Cache cache = null;
-  protected static PoolImpl pool = null;
-  protected static Connection conn = null;
-
-  protected static int PORT1;
-  protected static int PORT2;
-  protected static int PORT3;
-
-  protected static boolean isBeforeRegistrationCallbackCalled = false;
-  protected static boolean isBeforeInterestRecoveryCallbackCalled = false;
-  protected static boolean isAfterRegistrationCallbackCalled = false;
-
-  protected static Host host = null;
-  protected static VM server1 = null;
-  protected static VM server2 = null;
-  protected static VM server3 = null;
-  
-  protected volatile static boolean exceptionOccured = false;
-
-  public HAInterestBaseTest(String name) {
-    super(name);
-  }
-
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-    host = Host.getHost(0);
-    server1 = host.getVM(0);
-    server2 = host.getVM(1);
-    server3 = host.getVM(2);
-    CacheServerTestUtil.disableShufflingOfEndpoints();
-    // start servers first
-    PORT1 = ((Integer) server1.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
-    PORT2 = ((Integer) server2.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
-    PORT3 = ((Integer) server3.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
-    exceptionOccured = false;
-    addExpectedException("java.net.ConnectException: Connection refused: connect");
-  }
-
-  @Override
-  public void tearDown2() throws Exception {
-    // close the clients first
-    closeCache();
-
-    // then close the servers
-    server1.invoke(HAInterestBaseTest.class, "closeCache");
-    server2.invoke(HAInterestBaseTest.class, "closeCache");
-    server3.invoke(HAInterestBaseTest.class, "closeCache");
-    CacheServerTestUtil.resetDisableShufflingOfEndpointsFlag();
-  }
-
-  public static void closeCache() {
-    PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
-    PoolImpl.BEFORE_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = false;
-    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
-    PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
-    HAInterestBaseTest.isAfterRegistrationCallbackCalled = false;
-    HAInterestBaseTest.isBeforeInterestRecoveryCallbackCalled = false;
-    HAInterestBaseTest.isBeforeRegistrationCallbackCalled = false;
-    if (cache != null && !cache.isClosed()) {
-      cache.close();
-      cache.getDistributedSystem().disconnect();
-    }
-    cache = null;
-    pool = null;
-    conn = null;
-  }
-  
-  /**
-   * Return the current primary waiting for a primary to exist.
-   * 
-   * @since 5.7
-   */
-  public static VM getPrimaryVM() {
-    return getPrimaryVM(null);
-  }
-
-  /**
-   * Return the current primary waiting for a primary to exist and for it not to
-   * be the oldPrimary (if oldPrimary is NOT null).
-   * 
-   * @since 5.7
-   */
-  public static VM getPrimaryVM(final VM oldPrimary) {
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        int primaryPort = pool.getPrimaryPort();
-        if (primaryPort == -1) {
-          return false;
-        }
-        // we have a primary
-        VM currentPrimary = getServerVM(primaryPort);
-        if (currentPrimary != oldPrimary) {
-          return true;
-        }
-        return false;
-      }
-      @Override
-      public String description() {
-        return "waiting for primary";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    int primaryPort = pool.getPrimaryPort();
-    assertTrue(primaryPort != -1);
-    VM currentPrimary = getServerVM(primaryPort);
-    assertTrue(currentPrimary != oldPrimary);
-    return currentPrimary;
-  }
-
-  public static VM getBackupVM() {
-    return getBackupVM(null);
-  }
-
-  public static VM getBackupVM(VM stoppedBackup) {
-    VM currentPrimary = getPrimaryVM(null);
-    if (currentPrimary != server2 && server2 != stoppedBackup) {
-      return server2;
-    } else if (currentPrimary != server3 && server3 != stoppedBackup) {
-      return server3;
-    } else if (currentPrimary != server1 && server1 != stoppedBackup) {
-      return server1;
-    } else {
-      fail("expected currentPrimary " + currentPrimary + " to be " + server1 + ", or " + server2 + ", or " + server3);
-      return null;
-    }
-  }
-
-  /**
-   * Given a server vm (server1, server2, or server3) return its port.
-   * 
-   * @since 5.7
-   */
-  public static int getServerPort(VM vm) {
-    if (vm == server1) {
-      return PORT1;
-    } else if (vm == server2) {
-      return PORT2;
-    } else if (vm == server3) {
-      return PORT3;
-    } else {
-      fail("expected vm " + vm + " to be " + server1 + ", or " + server2 + ", or " + server3);
-      return -1;
-    }
-  }
-
-  /**
-   * Given a server port (PORT1, PORT2, or PORT3) return its vm.
-   * 
-   * @since 5.7
-   */
-  public static VM getServerVM(int port) {
-    if (port == PORT1) {
-      return server1;
-    } else if (port == PORT2) {
-      return server2;
-    } else if (port == PORT3) {
-      return server3;
-    } else {
-      fail("expected port " + port + " to be " + PORT1 + ", or " + PORT2 + ", or " + PORT3);
-      return null;
-    }
-  }
-
-  public static void verifyRefreshedEntriesFromServer() {
-    final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r1);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        Region.Entry re = r1.getEntry(k1);
-        if (re == null)
-          return false;
-        Object val = re.getValue();
-        return client_k1.equals(val);
-      }
-      @Override
-      public String description() {
-        return "waiting for client_k1 refresh from server";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        Region.Entry re = r1.getEntry(k2);
-        if (re == null)
-          return false;
-        Object val = re.getValue();
-        return client_k2.equals(val);
-      }
-      @Override
-      public String description() {
-        return "waiting for client_k2 refresh from server";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-  }
-
-  public static void verifyDeadAndLiveServers(final int expectedDeadServers, final int expectedLiveServers) {
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == expectedLiveServers;
-      }
-      @Override
-      public String description() {
-        return "waiting for pool.getConnectedServerCount() == expectedLiveServer";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-  }
-
-  public static void putK1andK2() {
-    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r1);
-    r1.put(k1, server_k1);
-    r1.put(k2, server_k2);
-  }
-
-  public static void setClientServerObserverForBeforeInterestRecoveryFailure() {
-    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
-    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
-      public void beforeInterestRecovery() {
-        synchronized (HAInterestBaseTest.class) {
-          Thread t = new Thread() {
-            public void run() {
-              getBackupVM().invoke(HAInterestBaseTest.class, "startServer");
-              getPrimaryVM().invoke(HAInterestBaseTest.class, "stopServer");
-            }
-          };
-          t.start();
-          try {
-            DistributedTestCase.join(t, 30 * 1000, getLogWriter());
-          } catch (Exception ignore) {
-            exceptionOccured = true;
-          }
-          HAInterestBaseTest.isBeforeInterestRecoveryCallbackCalled = true;
-          HAInterestBaseTest.class.notify();
-          PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
-        }
-      }
-    });
-  }
-
-  public static void setClientServerObserverForBeforeInterestRecovery() {
-    PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
-    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
-      public void beforeInterestRecovery() {
-        synchronized (HAInterestBaseTest.class) {
-          Thread t = new Thread() {
-            public void run() {
-              Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-              assertNotNull(r1);
-              r1.put(k1, server_k1_updated);
-            }
-          };
-          t.start();
-
-          HAInterestBaseTest.isBeforeInterestRecoveryCallbackCalled = true;
-          HAInterestBaseTest.class.notify();
-          PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
-        }
-      }
-    });
-  }
-
-  public static void waitForBeforeInterestRecoveryCallBack() throws InterruptedException {
-    assertNotNull(cache);
-    synchronized (HAInterestBaseTest.class) {
-      while (!isBeforeInterestRecoveryCallbackCalled) {
-        HAInterestBaseTest.class.wait();
-      }
-    }
-  }
-
-  public static void setClientServerObserverForBeforeRegistration(final VM vm) {
-    PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = true;
-    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
-      public void beforeInterestRegistration() {
-        synchronized (HAInterestBaseTest.class) {
-          vm.invoke(HAInterestBaseTest.class, "startServer");
-          HAInterestBaseTest.isBeforeRegistrationCallbackCalled = true;
-          HAInterestBaseTest.class.notify();
-          PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
-        }
-      }
-    });
-  }
-
-  public static void waitForBeforeRegistrationCallback() throws InterruptedException {
-    assertNotNull(cache);
-    synchronized (HAInterestBaseTest.class) {
-      while (!isBeforeRegistrationCallbackCalled) {
-        HAInterestBaseTest.class.wait();
-      }
-    }
-  }
-
-  public static void setClientServerObserverForAfterRegistration(final VM vm) {
-    PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = true;
-    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
-      public void afterInterestRegistration() {
-        synchronized (HAInterestBaseTest.class) {
-          vm.invoke(HAInterestBaseTest.class, "startServer");
-          HAInterestBaseTest.isAfterRegistrationCallbackCalled = true;
-          HAInterestBaseTest.class.notify();
-          PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
-        }
-      }
-    });
-  }
-
-  public static void waitForAfterRegistrationCallback() throws InterruptedException {
-    assertNotNull(cache);
-    if (!isAfterRegistrationCallbackCalled) {
-      synchronized (HAInterestBaseTest.class) {
-        while (!isAfterRegistrationCallbackCalled) {
-          HAInterestBaseTest.class.wait();
-        }
-      }
-    }
-  }
-
-  public static void unSetClientServerObserverForRegistrationCallback() {
-    synchronized (HAInterestBaseTest.class) {
-      PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
-      PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
-      HAInterestBaseTest.isBeforeRegistrationCallbackCalled = false;
-      HAInterestBaseTest.isAfterRegistrationCallbackCalled = false;
-    }
-  }
-
-  public static void verifyDispatcherIsAlive() {
-    assertEquals("More than one BridgeServer", 1, cache.getCacheServers().size());
-    
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return cache.getCacheServers().size() == 1;
-      }
-      @Override
-      public String description() {
-        return "waiting for cache.getCacheServers().size() == 1";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    assertNotNull(bs.getAcceptor());
-    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
-    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
-
-    wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return ccn.getClientProxies().size() > 0;
-      }
-      @Override
-      public String description() {
-        return "waiting for ccn.getClientProxies().size() > 0";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    wc = new WaitCriterion() {
-      Iterator iter_prox;
-      CacheClientProxy proxy;
-
-      @Override
-      public boolean done() {
-        iter_prox = ccn.getClientProxies().iterator();
-        if (iter_prox.hasNext()) {
-          proxy = (CacheClientProxy) iter_prox.next();
-          return proxy._messageDispatcher.isAlive();
-        } else {
-          return false;
-        }
-      }
-
-      @Override
-      public String description() {
-        return "waiting for CacheClientProxy _messageDispatcher to be alive";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-  }
-
-  public static void verifyDispatcherIsNotAlive() {
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return cache.getCacheServers().size() == 1;
-      }
-      @Override
-      public String description() {
-        return "cache.getCacheServers().size() == 1";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    assertNotNull(bs.getAcceptor());
-    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
-    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
-    
-    wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return ccn.getClientProxies().size() > 0;
-      }
-      @Override
-      public String description() {
-        return "waiting for ccn.getClientProxies().size() > 0";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    Iterator iter_prox = ccn.getClientProxies().iterator();
-    if (iter_prox.hasNext()) {
-      CacheClientProxy proxy = (CacheClientProxy) iter_prox.next();
-      assertFalse("Dispatcher on secondary should not be alive", proxy._messageDispatcher.isAlive());
-    }
-  }
-
-  public static void createEntriesK1andK2OnServer() {
-    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r1);
-    if (!r1.containsKey(k1)) {
-      r1.create(k1, server_k1);
-    }
-    if (!r1.containsKey(k2)) {
-      r1.create(k2, server_k2);
-    }
-    assertEquals(r1.getEntry(k1).getValue(), server_k1);
-    assertEquals(r1.getEntry(k2).getValue(), server_k2);
-  }
-
-  public static void createEntriesK1andK2() {
-    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r1);
-    if (!r1.containsKey(k1)) {
-      r1.create(k1, client_k1);
-    }
-    if (!r1.containsKey(k2)) {
-      r1.create(k2, client_k2);
-    }
-    assertEquals(r1.getEntry(k1).getValue(), client_k1);
-    assertEquals(r1.getEntry(k2).getValue(), client_k2);
-  }
-
-  public static void createServerEntriesK1andK2() {
-    Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r1);
-    if (!r1.containsKey(k1)) {
-      r1.create(k1, server_k1);
-    }
-    if (!r1.containsKey(k2)) {
-      r1.create(k2, server_k2);
-    }
-    assertEquals(r1.getEntry(k1).getValue(), server_k1);
-    assertEquals(r1.getEntry(k2).getValue(), server_k2);
-  }
-
-  public static void registerK1AndK2() {
-    Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-    r.registerInterest(list, InterestResultPolicy.KEYS_VALUES);
-  }
-
-  public static void reRegisterK1AndK2() {
-    Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-    r.registerInterest(list);
-  }
-
-  public static void startServer() throws IOException {
-    Cache c = CacheFactory.getAnyInstance();
-    assertEquals("More than one BridgeServer", 1, c.getCacheServers().size());
-    CacheServerImpl bs = (CacheServerImpl) c.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    bs.start();
-  }
-
-  public static void stopServer() {
-    assertEquals("More than one BridgeServer", 1, cache.getCacheServers().size());
-    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    bs.stop();
-  }
-
-  public static void stopPrimaryAndRegisterK1AndK2AndVerifyResponse() {
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == 3;
-      }
-      @Override
-      public String description() {
-        return "connected server count never became 3";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    // close primaryEP
-    getPrimaryVM().invoke(HAInterestBaseTest.class, "stopServer");
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
-    assertNotNull(serverKeys);
-    List resultKeys = (List) serverKeys.get(0);
-    assertEquals(2, resultKeys.size());
-    assertTrue(resultKeys.contains(k1));
-    assertTrue(resultKeys.contains(k2));
-  }
-
-  public static void stopPrimaryAndUnregisterRegisterK1() {
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == 3;
-      }
-      @Override
-      public String description() {
-        return "connected server count never became 3";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    // close primaryEP
-    getPrimaryVM().invoke(HAInterestBaseTest.class, "stopServer");
-    List list = new ArrayList();
-    list.add(k1);
-    srp.unregisterInterest(list, InterestType.KEY, false, false);
-  }
-
-  public static void stopBothPrimaryAndSecondaryAndRegisterK1AndK2AndVerifyResponse() {
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == 3;
-      }
-      @Override
-      public String description() {
-        return "connected server count never became 3";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    // close primaryEP
-    VM backup = getBackupVM();
-    getPrimaryVM().invoke(HAInterestBaseTest.class, "stopServer");
-    // close secondary
-    backup.invoke(HAInterestBaseTest.class, "stopServer");
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
-
-    assertNotNull(serverKeys);
-    List resultKeys = (List) serverKeys.get(0);
-    assertEquals(2, resultKeys.size());
-    assertTrue(resultKeys.contains(k1));
-    assertTrue(resultKeys.contains(k2));
-  }
-
-  /**
-   * returns the secondary that was stopped
-   */
-  public static VM stopSecondaryAndRegisterK1AndK2AndVerifyResponse() {
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == 3;
-      }
-      @Override
-      public String description() {
-        return "Never got three connected servers";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    // close secondary EP
-    VM result = getBackupVM();
-    result.invoke(HAInterestBaseTest.class, "stopServer");
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-    List serverKeys = srp.registerInterest(list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
-
-    assertNotNull(serverKeys);
-    List resultKeys = (List) serverKeys.get(0);
-    assertEquals(2, resultKeys.size());
-    assertTrue(resultKeys.contains(k1));
-    assertTrue(resultKeys.contains(k2));
-    return result;
-  }
-
-  /**
-   * returns the secondary that was stopped
-   */
-  public static VM stopSecondaryAndUNregisterK1() {
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return pool.getConnectedServerCount() == 3;
-      }
-      @Override
-      public String description() {
-        return "connected server count never became 3";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    // close secondary EP
-    VM result = getBackupVM();
-    result.invoke(HAInterestBaseTest.class, "stopServer");
-    List list = new ArrayList();
-    list.add(k1);
-    srp.unregisterInterest(list, InterestType.KEY, false, false);
-    return result;
-  }
-
-  public static void registerK1AndK2OnPrimaryAndSecondaryAndVerifyResponse() {
-    ServerLocation primary = pool.getPrimary();
-    ServerLocation secondary = (ServerLocation) pool.getRedundants().get(0);
-    LocalRegion r = (LocalRegion) cache.getRegion(Region.SEPARATOR + REGION_NAME);
-    assertNotNull(r);
-    ServerRegionProxy srp = new ServerRegionProxy(r);
-    List list = new ArrayList();
-    list.add(k1);
-    list.add(k2);
-
-    // Primary server
-    List serverKeys1 = srp.registerInterestOn(primary, list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
-    assertNotNull(serverKeys1);
-    // expect serverKeys in response from primary
-    List resultKeys = (List) serverKeys1.get(0);
-    assertEquals(2, resultKeys.size());
-    assertTrue(resultKeys.contains(k1));
-    assertTrue(resultKeys.contains(k2));
-
-    // Secondary server
-    List serverKeys2 = srp.registerInterestOn(secondary, list, InterestType.KEY, InterestResultPolicy.KEYS, false, r.getAttributes().getDataPolicy().ordinal);
-    // if the list is null then it is empty
-    if (serverKeys2 != null) {
-      // no serverKeys in response from secondary
-      assertTrue(serverKeys2.isEmpty());
-    }
-  }
-
-  public static void verifyInterestRegistration() {
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return cache.getCacheServers().size() == 1;
-      }
-      @Override
-      public String description() {
-        return "waiting for cache.getCacheServers().size() == 1";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    assertNotNull(bs.getAcceptor());
-    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
-    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
-    
-    wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return ccn.getClientProxies().size() > 0;
-      }
-      @Override
-      public String description() {
-        return "waiting for ccn.getClientProxies().size() > 0";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    Iterator iter_prox = ccn.getClientProxies().iterator();
-
-    if (iter_prox.hasNext()) {
-      final CacheClientProxy ccp = (CacheClientProxy) iter_prox.next();
-      
-      wc = new WaitCriterion() {
-        @Override
-        public boolean done() {
-          Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
-              .getProfile(Region.SEPARATOR + REGION_NAME)
-              .getKeysOfInterestFor(ccp.getProxyID());
-          return keysMap != null && keysMap.size() == 2;
-        }
-        @Override
-        public String description() {
-          return "waiting for keys of interest to include 2 keys";
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-      Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex].getProfile(Region.SEPARATOR + REGION_NAME)
-          .getKeysOfInterestFor(ccp.getProxyID());
-      assertNotNull(keysMap);
-      assertEquals(2, keysMap.size());
-      assertTrue(keysMap.contains(k1));
-      assertTrue(keysMap.contains(k2));
-    }
-  }
-
-  public static void verifyInterestUNRegistration() {
-    WaitCriterion wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return cache.getCacheServers().size() == 1;
-      }
-      @Override
-      public String description() {
-        return "waiting for cache.getCacheServers().size() == 1";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
-    assertNotNull(bs);
-    assertNotNull(bs.getAcceptor());
-    assertNotNull(bs.getAcceptor().getCacheClientNotifier());
-    final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
-    
-    wc = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        return ccn.getClientProxies().size() > 0;
-      }
-      @Override
-      public String description() {
-        return "waiting for ccn.getClientProxies().size() > 0";
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-    Iterator iter_prox = ccn.getClientProxies().iterator();
-    if (iter_prox.hasNext()) {
-      final CacheClientProxy ccp = (CacheClientProxy) iter_prox.next();
-      
-      wc = new WaitCriterion() {
-        @Override
-        public boolean done() {
-          Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
-              .getProfile(Region.SEPARATOR + REGION_NAME)
-              .getKeysOfInterestFor(ccp.getProxyID());
-          return keysMap != null;
-        }
-        @Override
-        public String description() {
-          return "waiting for keys of interest to not be null";
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
-
-      Set keysMap = (Set) ccp.cils[RegisterInterestTracker.interestListIndex]
-          .getProfile(Region.SEPARATOR + REGION_NAME)
-          .getKeysOfInterestFor(ccp.getProxyID());
-      assertNotNull(keysMap);
-      assertEquals(1, keysMap.size());
-      assertFalse(keysMap.contains(k1));
-      assertTrue(keysMap.contains(k2));
-    }
-  }
-
-  private void createCache(Properties props) throws Exception {
-    DistributedSystem ds = getSystem(props);
-    assertNotNull(ds);
-    ds.disconnect();
-    ds = getSystem(props);
-    cache = CacheFactory.create(ds);
-    assertNotNull(cache);
-  }
-
-  public static void createClientPoolCache(String testName, String host) throws Exception {
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new HAInterestBaseTest("temp").createCache(props);
-    CacheServerTestUtil.disableShufflingOfEndpoints();
-    PoolImpl p;
-    try {
-      p = (PoolImpl) PoolManager.createFactory()
-          .addServer(host, PORT1)
-          .addServer(host, PORT2)
-          .addServer(host, PORT3)
-          .setSubscriptionEnabled(true)
-          .setSubscriptionRedundancy(-1)
-          .setReadTimeout(1000)
-          .setPingInterval(1000)
-          // retryInterval should be more so that only registerInterste thread
-          // will initiate failover
-          // .setRetryInterval(20000)
-          .create("HAInterestBaseTestPool");
-    } finally {
-      CacheServerTestUtil.enableShufflingOfEndpoints();
-    }
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    factory.setConcurrencyChecksEnabled(true);
-    factory.setPoolName(p.getName());
-
-    cache.createRegion(REGION_NAME, factory.create());
-    pool = p;
-    conn = pool.acquireConnection();
-    assertNotNull(conn);
-  }
-
-  public static void createClientPoolCacheWithSmallRetryInterval(String testName, String host) throws Exception {
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new HAInterestBaseTest("temp").createCache(props);
-    CacheServerTestUtil.disableShufflingOfEndpoints();
-    PoolImpl p;
-    try {
-      p = (PoolImpl) PoolManager.createFactory()
-          .addServer(host, PORT1)
-          .addServer(host, PORT2)
-          .setSubscriptionEnabled(true)
-          .setSubscriptionRedundancy(-1)
-          .setReadTimeout(1000)
-          .setSocketBufferSize(32768)
-          .setMinConnections(6)
-          .setPingInterval(200)
-          // .setRetryInterval(200)
-          // retryAttempts 3
-          .create("HAInterestBaseTestPool");
-    } finally {
-      CacheServerTestUtil.enableShufflingOfEndpoints();
-    }
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    factory.setConcurrencyChecksEnabled(true);
-    factory.setPoolName(p.getName());
-
-    cache.createRegion(REGION_NAME, factory.create());
-
-    pool = p;
-    conn = pool.acquireConnection();
-    assertNotNull(conn);
-  }
-
-  public static void createClientPoolCacheConnectionToSingleServer(String testName, String hostName) throws Exception {
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new HAInterestBaseTest("temp").createCache(props);
-    PoolImpl p = (PoolImpl) PoolManager.createFactory()
-        .addServer(hostName, PORT1)
-        .setSubscriptionEnabled(true)
-        .setSubscriptionRedundancy(-1)
-        .setReadTimeout(1000)
-        // .setRetryInterval(20)
-        .create("HAInterestBaseTestPool");
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    factory.setConcurrencyChecksEnabled(true);
-    factory.setPoolName(p.getName());
-
-    cache.createRegion(REGION_NAME, factory.create());
-
-    pool = p;
-    conn = pool.acquireConnection();
-    assertNotNull(conn);
-  }
-
-  public static Integer createServerCache() throws Exception {
-    new HAInterestBaseTest("temp").createCache(new Properties());
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setEnableBridgeConflation(true);
-    factory.setMirrorType(MirrorType.KEYS_VALUES);
-    factory.setConcurrencyChecksEnabled(true);
-    cache.createRegion(REGION_NAME, factory.create());
-
-    CacheServer server = cache.addCacheServer();
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    server.setPort(port);
-    server.setMaximumTimeBetweenPings(180000);
-    // ensures updates to be sent instead of invalidations
-    server.setNotifyBySubscription(true);
-    server.start();
-    return new Integer(server.getPort());
-  }
-
-  public static Integer createServerCacheWithLocalRegion() throws Exception {
-    new HAInterestBaseTest("temp").createCache(new Properties());
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    factory.setConcurrencyChecksEnabled(true);
-    RegionAttributes attrs = factory.create();
-    cache.createRegion(REGION_NAME, attrs);
-
-    CacheServer server = cache.addCacheServer();
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    server.setPort(port);
-    // ensures updates to be sent instead of invalidations
-    server.setNotifyBySubscription(true);
-    server.setMaximumTimeBetweenPings(180000);
-    server.start();
-    return new Integer(server.getPort());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart1DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart1DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart1DUnitTest.java
index 482fca9..27779a6 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart1DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart1DUnitTest.java
@@ -19,7 +19,7 @@ package com.gemstone.gemfire.internal.cache.tier.sockets;
 import dunit.VM;
 
 @SuppressWarnings("serial")
-public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
+public class HAInterestPart1DUnitTest extends HAInterestTestCase {
 
   public HAInterestPart1DUnitTest(String name) {
     super(name);
@@ -31,14 +31,14 @@ public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
   public void testInterestRegistrationOnBothPrimaryAndSecondary() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
     // register K1 and K2
     registerK1AndK2();
-    server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
-    server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
-    server3.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server1.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server2.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server3.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
   }
 
   /**
@@ -48,9 +48,9 @@ public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
   public void testInterestRegistrationResponseOnBothPrimaryAndSecondary() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
     // register interest and verify response
     registerK1AndK2OnPrimaryAndSecondaryAndVerifyResponse();
   }
@@ -62,15 +62,15 @@ public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
   public void testRERegistrationWillNotCreateDuplicateKeysOnServerInterstMaps() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
     // register multiple times
     reRegisterK1AndK2();
 
-    server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
-    server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
-    server3.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server1.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server2.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server3.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
   }
 
   /**
@@ -81,9 +81,9 @@ public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
   public void testPrimaryFailureInRegisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
     // stop primary
     VM oldPrimary = getPrimaryVM();
     stopPrimaryAndRegisterK1AndK2AndVerifyResponse();
@@ -91,8 +91,8 @@ public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
     verifyDeadAndLiveServers(1, 2);
     // new primary
     VM newPrimary = getPrimaryVM(oldPrimary);
-    newPrimary.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
-    newPrimary.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    newPrimary.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
+    newPrimary.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
   }
 
   /**
@@ -102,17 +102,17 @@ public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
   public void testSecondaryFailureInRegisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
 
     VM primary = getPrimaryVM();
     stopSecondaryAndRegisterK1AndK2AndVerifyResponse();
 
     verifyDeadAndLiveServers(1, 2);
     // still primary
-    primary.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
-    primary.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    primary.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
+    primary.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
   }
 
   /**
@@ -124,17 +124,17 @@ public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
   public void testBothPrimaryAndSecondaryFailureInRegisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
     // stop server1 and server2
     VM oldPrimary = getPrimaryVM();
     stopBothPrimaryAndSecondaryAndRegisterK1AndK2AndVerifyResponse();
 
     verifyDeadAndLiveServers(2, 1);
     VM newPrimary = getPrimaryVM(oldPrimary);
-    newPrimary.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
-    newPrimary.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    newPrimary.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
+    newPrimary.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
   }
 
   /**
@@ -148,17 +148,17 @@ public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
   public void testProbablePrimaryFailureInRegisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
 
     VM oldPrimary = getPrimaryVM();
     stopPrimaryAndRegisterK1AndK2AndVerifyResponse();
 
     verifyDeadAndLiveServers(1, 2);
     VM newPrimary = getPrimaryVM(oldPrimary);
-    newPrimary.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
-    newPrimary.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    newPrimary.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
+    newPrimary.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
   }
 
   /**
@@ -172,40 +172,40 @@ public class HAInterestPart1DUnitTest extends HAInterestBaseTest {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
     registerK1AndK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
 
-    server1.invoke(HAInterestBaseTest.class, "stopServer");
-    server2.invoke(HAInterestBaseTest.class, "stopServer");
-    server3.invoke(HAInterestBaseTest.class, "stopServer");
+    server1.invoke(HAInterestTestCase.class, "stopServer");
+    server2.invoke(HAInterestTestCase.class, "stopServer");
+    server3.invoke(HAInterestTestCase.class, "stopServer");
     // All servers are dead at this point , no primary in the system.
     verifyDeadAndLiveServers(3, 0);
 
     // now start one of the servers
-    server2.invoke(HAInterestBaseTest.class, "startServer");
+    server2.invoke(HAInterestTestCase.class, "startServer");
     verifyDeadAndLiveServers(2, 1);
     // verify that is it primary , and dispatcher is running
-    server2.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
+    server2.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
     // verify that interest is registered on this recovered EP
-    server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server2.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
 
     // now start one more server ; this should be now secondary
-    server1.invoke(HAInterestBaseTest.class, "startServer");
+    server1.invoke(HAInterestTestCase.class, "startServer");
     verifyDeadAndLiveServers(1, 2);
 
     // verify that is it secondary , dispatcher should not be runnig
-    server1.invoke(HAInterestBaseTest.class, "verifyDispatcherIsNotAlive");
+    server1.invoke(HAInterestTestCase.class, "verifyDispatcherIsNotAlive");
     // verify that interest is registered on this recovered EP as well
-    server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server1.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
 
     // now start one more server ; this should be now secondary
-    server3.invoke(HAInterestBaseTest.class, "startServer");
+    server3.invoke(HAInterestTestCase.class, "startServer");
     verifyDeadAndLiveServers(0, 3);
 
     // verify that is it secondary , dispatcher should not be runnig
-    server3.invoke(HAInterestBaseTest.class, "verifyDispatcherIsNotAlive");
+    server3.invoke(HAInterestTestCase.class, "verifyDispatcherIsNotAlive");
     // verify that interest is registered on this recovered EP as well
-    server3.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server3.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca6148aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
index eaa1ca1..31a2811 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
@@ -24,7 +24,7 @@ import dunit.DistributedTestCase;
 import dunit.VM;
 
 @SuppressWarnings({"rawtypes", "serial"})
-public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
+public class HAInterestPart2DUnitTest extends HAInterestTestCase {
 
   public HAInterestPart2DUnitTest(String name) {
     super(name);
@@ -37,9 +37,9 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
   public void testPrimaryFailureInUNregisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
 
     registerK1AndK2();
 
@@ -49,11 +49,11 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
     verifyDeadAndLiveServers(1, 2);
 
     VM newPrimary = getPrimaryVM(oldPrimary);
-    newPrimary.invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
+    newPrimary.invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
     // primary
-    newPrimary.invoke(HAInterestBaseTest.class, "verifyInterestUNRegistration");
+    newPrimary.invoke(HAInterestTestCase.class, "verifyInterestUNRegistration");
     // secondary
-    getBackupVM().invoke(HAInterestBaseTest.class, "verifyInterestUNRegistration");
+    getBackupVM().invoke(HAInterestTestCase.class, "verifyInterestUNRegistration");
   }
 
   /**
@@ -63,18 +63,18 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
   public void testSecondaryFailureInUNRegisterInterest() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
     registerK1AndK2();
     VM stoppedBackup = stopSecondaryAndUNregisterK1();
     verifyDeadAndLiveServers(1, 2);
     // still primary
-    getPrimaryVM().invoke(HAInterestBaseTest.class, "verifyDispatcherIsAlive");
+    getPrimaryVM().invoke(HAInterestTestCase.class, "verifyDispatcherIsAlive");
     // primary
-    getPrimaryVM().invoke(HAInterestBaseTest.class, "verifyInterestUNRegistration");
+    getPrimaryVM().invoke(HAInterestTestCase.class, "verifyInterestUNRegistration");
     // secondary
-    getBackupVM(stoppedBackup).invoke(HAInterestBaseTest.class, "verifyInterestUNRegistration");
+    getBackupVM(stoppedBackup).invoke(HAInterestTestCase.class, "verifyInterestUNRegistration");
   }
 
   /**
@@ -85,11 +85,11 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
   public void testDSMDetectsServerLiveJustBeforeInterestRegistration() throws Exception {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
     VM backup = getBackupVM();
-    backup.invoke(HAInterestBaseTest.class, "stopServer");
+    backup.invoke(HAInterestTestCase.class, "stopServer");
     verifyDeadAndLiveServers(1, 2);
     setClientServerObserverForBeforeRegistration(backup);
     try {
@@ -98,9 +98,9 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
     } finally {
       unSetClientServerObserverForRegistrationCallback();
     }
-    server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
-    server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
-    server3.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server1.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server2.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server3.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
   }
 
   /**
@@ -112,12 +112,12 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
 
     createEntriesK1andK2();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
 
     VM backup = getBackupVM();
-    backup.invoke(HAInterestBaseTest.class, "stopServer");
+    backup.invoke(HAInterestTestCase.class, "stopServer");
     verifyDeadAndLiveServers(1, 2);
 
     setClientServerObserverForAfterRegistration(backup);
@@ -128,9 +128,9 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
       unSetClientServerObserverForRegistrationCallback();
     }
 
-    server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
-    server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
-    server3.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
+    server1.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server2.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
+    server3.invoke(HAInterestTestCase.class, "verifyInterestRegistration");
   }
 
   /**
@@ -143,16 +143,16 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
   public void testRefreshEntriesFromPrimaryWhenDSMDetectsServerLive() throws Exception {
     addExpectedException(ServerConnectivityException.class.getName());
     
-    PORT1 = ((Integer) server1.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    PORT1 = ((Integer) server1.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
     createClientPoolCacheConnectionToSingleServer(this.getName(), getServerHostName(server1.getHost()));
     registerK1AndK2();
     verifyRefreshedEntriesFromServer();
 
-    server1.invoke(HAInterestBaseTest.class, "stopServer");
+    server1.invoke(HAInterestTestCase.class, "stopServer");
     verifyDeadAndLiveServers(1, 0);
-    server1.invoke(HAInterestBaseTest.class, "putK1andK2");
-    server1.invoke(HAInterestBaseTest.class, "startServer");
+    server1.invoke(HAInterestTestCase.class, "putK1andK2");
+    server1.invoke(HAInterestTestCase.class, "startServer");
     verifyDeadAndLiveServers(0, 1);
     final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
     assertNotNull(r1);
@@ -211,29 +211,29 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
    * refreshes registered entries from the server, because it is secondary
    */
   public void testGIIFromSecondaryWhenDSMDetectsServerLive() throws Exception {
-    server1.invoke(HAInterestBaseTest.class, "closeCache");
-    server2.invoke(HAInterestBaseTest.class, "closeCache");
-    server3.invoke(HAInterestBaseTest.class, "closeCache");
+    server1.invoke(HAInterestTestCase.class, "closeCache");
+    server2.invoke(HAInterestTestCase.class, "closeCache");
+    server3.invoke(HAInterestTestCase.class, "closeCache");
 
-    PORT1 = ((Integer) server1.invoke(HAInterestBaseTest.class, "createServerCacheWithLocalRegion")).intValue();
-    PORT2 = ((Integer) server2.invoke(HAInterestBaseTest.class, "createServerCacheWithLocalRegion")).intValue();
-    PORT3 = ((Integer) server3.invoke(HAInterestBaseTest.class, "createServerCacheWithLocalRegion")).intValue();
+    PORT1 = ((Integer) server1.invoke(HAInterestTestCase.class, "createServerCacheWithLocalRegion")).intValue();
+    PORT2 = ((Integer) server2.invoke(HAInterestTestCase.class, "createServerCacheWithLocalRegion")).intValue();
+    PORT3 = ((Integer) server3.invoke(HAInterestTestCase.class, "createServerCacheWithLocalRegion")).intValue();
 
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    server3.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    server3.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
 
     createClientPoolCache(this.getName(), getServerHostName(server1.getHost()));
 
     VM backup1 = getBackupVM();
     VM backup2 = getBackupVM(backup1);
-    backup1.invoke(HAInterestBaseTest.class, "stopServer");
-    backup2.invoke(HAInterestBaseTest.class, "stopServer");
+    backup1.invoke(HAInterestTestCase.class, "stopServer");
+    backup2.invoke(HAInterestTestCase.class, "stopServer");
     verifyDeadAndLiveServers(2, 1);
     registerK1AndK2();
     verifyRefreshedEntriesFromServer();
-    backup1.invoke(HAInterestBaseTest.class, "putK1andK2");
-    backup1.invoke(HAInterestBaseTest.class, "startServer");
+    backup1.invoke(HAInterestTestCase.class, "putK1andK2");
+    backup1.invoke(HAInterestTestCase.class, "startServer");
     verifyDeadAndLiveServers(1, 2);
     verifyRefreshedEntriesFromServer();
   }
@@ -246,19 +246,19 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
    * @throws Exception
    */
   public void testBug35945() throws Exception {
-    PORT1 = ((Integer) server1.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    PORT1 = ((Integer) server1.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
     createClientPoolCacheConnectionToSingleServer(this.getName(), getServerHostName(server1.getHost()));
     registerK1AndK2();
     verifyRefreshedEntriesFromServer();
 
-    server1.invoke(HAInterestBaseTest.class, "stopServer");
+    server1.invoke(HAInterestTestCase.class, "stopServer");
     verifyDeadAndLiveServers(1, 0);
     // put on stopped server
-    server1.invoke(HAInterestBaseTest.class, "putK1andK2");
+    server1.invoke(HAInterestTestCase.class, "putK1andK2");
     // spawn a thread to put on server , which will acquire a lock on entry
     setClientServerObserverForBeforeInterestRecovery();
-    server1.invoke(HAInterestBaseTest.class, "startServer");
+    server1.invoke(HAInterestTestCase.class, "startServer");
     verifyDeadAndLiveServers(0, 1);
     waitForBeforeInterestRecoveryCallBack();
     // verify updated value of k1 as a refreshEntriesFromServer
@@ -314,23 +314,23 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
   public void testInterestRecoveryFailure() throws Exception {
     addExpectedException("Server unreachable");
     
-    PORT1 = ((Integer) server1.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
-    server1.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
-    PORT2 = ((Integer) server2.invoke(HAInterestBaseTest.class, "createServerCache")).intValue();
-    server2.invoke(HAInterestBaseTest.class, "createEntriesK1andK2");
+    PORT1 = ((Integer) server1.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
+    server1.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
+    PORT2 = ((Integer) server2.invoke(HAInterestTestCase.class, "createServerCache")).intValue();
+    server2.invoke(HAInterestTestCase.class, "createEntriesK1andK2");
     createClientPoolCacheWithSmallRetryInterval(this.getName(), getServerHostName(server1.getHost()));
     registerK1AndK2();
     verifyRefreshedEntriesFromServer();
     VM backup = getBackupVM();
     VM primary = getPrimaryVM();
 
-    backup.invoke(HAInterestBaseTest.class, "stopServer");
-    primary.invoke(HAInterestBaseTest.class, "stopServer");
+    backup.invoke(HAInterestTestCase.class, "stopServer");
+    primary.invoke(HAInterestTestCase.class, "stopServer");
     verifyDeadAndLiveServers(2, 0);
 
-    primary.invoke(HAInterestBaseTest.class, "putK1andK2");
+    primary.invoke(HAInterestTestCase.class, "putK1andK2");
     setClientServerObserverForBeforeInterestRecoveryFailure();
-    primary.invoke(HAInterestBaseTest.class, "startServer");
+    primary.invoke(HAInterestTestCase.class, "startServer");
     waitForBeforeInterestRecoveryCallBack();
     if (exceptionOccured) {
       fail("The DSM could not ensure that server 1 is started & serevr 2 is stopped");


[24/50] [abbrv] incubator-geode git commit: GEODE-356: Fix unexpected UNDEFINED in OQL query result set

Posted by ab...@apache.org.
GEODE-356: Fix unexpected UNDEFINED in OQL query result set


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

Branch: refs/heads/master
Commit: b6a89ad85376aafe16e47b2c3d0428cdff9c38d5
Parents: fb38186
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Wed Jan 6 11:01:06 2016 -0800
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Wed Jan 6 11:01:06 2016 -0800

----------------------------------------------------------------------
 .../query/internal/index/CompactRangeIndex.java |   8 +-
 .../query/internal/index/MemoryIndexStore.java  |  33 +-
 .../functional/IndexOnEntrySetJUnitTest.java    | 335 +++++++++++++++++++
 3 files changed, 372 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b6a89ad8/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/CompactRangeIndex.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/CompactRangeIndex.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/CompactRangeIndex.java
index 41e2ea4..79d0c54 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/CompactRangeIndex.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/CompactRangeIndex.java
@@ -16,7 +16,6 @@
  */
 
 package com.gemstone.gemfire.cache.query.internal.index;
-import  com.gemstone.gemfire.internal.cache.CachedDeserializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
@@ -28,6 +27,7 @@ import java.util.Set;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.EntryDestroyedException;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.query.AmbiguousNameException;
 import com.gemstone.gemfire.cache.query.FunctionDomainException;
@@ -908,6 +908,9 @@ public class CompactRangeIndex extends AbstractIndex {
                 result.add(new CqEntry(indexEntry.getDeserializedRegionKey(),
                     value));
               } else {
+                if (IndexManager.testHook != null) {
+                  IndexManager.testHook.hook(200);
+                }
                 applyProjection(projAttrib, context, result, value,
                     intermediateResults, isIntersection);
               }
@@ -921,6 +924,9 @@ public class CompactRangeIndex extends AbstractIndex {
       } catch (ClassCastException e) {
         
       }
+      catch(EntryDestroyedException e) {
+        //ignore it
+      }
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b6a89ad8/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStore.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStore.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStore.java
index 966d82a..41de5ea 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStore.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStore.java
@@ -503,7 +503,7 @@ public class MemoryIndexStore implements IndexStore {
     } else if (indexOnRegionKeys) {
       return entry.getKey();
     }
-    return ((LocalRegion) this.region).new NonTXEntry(entry);
+    return new CachedEntryWrapper(((LocalRegion) this.region).new NonTXEntry(entry));
   }
 
   public Object getTargetObjectInVM(RegionEntry entry) {
@@ -588,6 +588,7 @@ public class MemoryIndexStore implements IndexStore {
     protected Iterator<Map.Entry> mapIterator;
     protected Iterator valuesIterator;
     protected Object currKey;
+    protected Object currValue; //RegionEntry
     final long iteratorStartTime = GemFireCacheImpl.getInstance().cacheTimeMillis();
     protected MemoryIndexStoreEntry currentEntry = new MemoryIndexStoreEntry(iteratorStartTime);
     
@@ -640,8 +641,8 @@ public class MemoryIndexStore implements IndexStore {
         if (values instanceof Collection) {
           this.valuesIterator = ((Collection) values).iterator();
         } else {
-          currentEntry.setMemoryIndexStoreEntry(currKey, (RegionEntry) values);
           this.valuesIterator = null;
+          currValue = values;
         }
         return values != null &&
                 (values instanceof RegionEntry
@@ -659,6 +660,7 @@ public class MemoryIndexStore implements IndexStore {
      */
     public MemoryIndexStoreEntry next() {
       if (valuesIterator == null) {
+        currentEntry.setMemoryIndexStoreEntry(currKey, (RegionEntry) currValue);
         return currentEntry;
       }
 
@@ -728,7 +730,7 @@ public class MemoryIndexStore implements IndexStore {
     private RegionEntry regionEntry;
     private boolean updateInProgress;
     private Object value;
-    private long iteratorStartTime;
+    private long iteratorStartTime;    
 
     private MemoryIndexStoreEntry(long iteratorStartTime) {
     	this.iteratorStartTime = iteratorStartTime;
@@ -770,5 +772,30 @@ public class MemoryIndexStore implements IndexStore {
           ||  IndexManager.needsRecalculation(iteratorStartTime, regionEntry.getLastModified());
     }
   }
+  
+  class CachedEntryWrapper {
+
+    private Object key, value;
+
+    public CachedEntryWrapper(LocalRegion.NonTXEntry entry) {
+      this.key = entry.getKey();
+      this.value = entry.getValue();
+    }
+
+    public Object getKey() {
+      return this.key;
+    }
+
+    public Object getValue() {
+      return this.value;
+    }
+
+    public String toString() {
+      return new StringBuilder("CachedEntryWrapper@").append(
+          Integer.toHexString(System.identityHashCode(this))).append(' ')
+          .append(this.key).append(' ').append(this.value).toString();
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b6a89ad8/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IndexOnEntrySetJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IndexOnEntrySetJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IndexOnEntrySetJUnitTest.java
new file mode 100644
index 0000000..e375b90
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IndexOnEntrySetJUnitTest.java
@@ -0,0 +1,335 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.cache.query.functional;
+
+import java.text.ParseException;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.query.CacheUtils;
+import com.gemstone.gemfire.cache.query.Index;
+import com.gemstone.gemfire.cache.query.QueryService;
+import com.gemstone.gemfire.cache.query.SelectResults;
+import com.gemstone.gemfire.cache.query.Struct;
+import com.gemstone.gemfire.cache.query.internal.QueryObserverAdapter;
+import com.gemstone.gemfire.cache.query.internal.QueryObserverHolder;
+import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class IndexOnEntrySetJUnitTest {
+
+  private static String testRegionName = "regionName";
+  private static Region testRegion;
+  private static int numElem = 100;
+  private String newValue = "NEW VALUE";
+
+   @Before
+  public void setUp() throws Exception {    
+    System.setProperty("gemfire.Query.VERBOSE", "true");
+    CacheUtils.startCache();
+  }
+
+   @After
+  public void tearDown() throws Exception {
+    // Destroy current Region for other tests    
+    IndexManager.testHook = null;
+    if (testRegion != null) {
+      testRegion.destroyRegion();
+    }
+    CacheUtils.closeCache();    
+  }
+
+  private String[] getQueriesOnRegion(String regionName) {
+    return new String[] {"SELECT DISTINCT entry.value, entry.key FROM /" + regionName + ".entrySet entry WHERE entry.key.PartitionID > 0 AND " + "entry.key.Index > 1 ORDER BY entry.key.Index ASC LIMIT 2",
+    "SELECT DISTINCT entry.value, entry.key FROM /" + regionName + ".entrySet entry WHERE entry.key.Index > 1 ORDER BY entry.key.Index ASC LIMIT 2",
+    "SELECT DISTINCT * FROM /" + regionName + ".entrySet entry WHERE entry.key.PartitionID > 0 AND " + "entry.key.Index > 1 ORDER BY entry.key.Index ASC LIMIT 2",
+    "SELECT DISTINCT entry.value, entry.key FROM /" + regionName + ".entrySet entry WHERE entry.key.PartitionID > 0 AND " + "entry.key.Index > 1 LIMIT 2",
+    "SELECT DISTINCT entry.value, entry.key FROM /" + regionName + ".entrySet entry WHERE entry.key.PartitionID > 0 AND " + "entry.key.Index > 1 ORDER BY entry.key.Index ASC",};
+  }
+  
+  private String[] getQueriesOnRegionForPut(String regionName) {
+    return new String[] {"SELECT DISTINCT entry.value, entry.key FROM /" + regionName + ".entrySet entry WHERE entry.key.PartitionID = 50 AND " + "entry.key.Index > 1 ORDER BY entry.key.Index ASC LIMIT 2",
+        "SELECT DISTINCT entry.value, entry.key FROM /" + regionName + ".entrySet entry WHERE entry.value = 50 AND " + "entry.key.Index > 1 ORDER BY entry.key.Index ASC LIMIT 2"};
+  }
+
+  /**
+   * Test queries with index on replicated regions and concurrent PUT, DESTORY, INVALIDATE operations.
+   * Make sure there is no UNDEFINED in the query result.
+   */
+  @Test
+  public void testQueriesOnReplicatedRegion() throws Exception {
+    testRegion = createReplicatedRegion(testRegionName);
+    String regionPath = "/" + testRegionName + ".entrySet entry";
+    executeQueryTest(getQueriesOnRegion(testRegionName), "entry.key.Index", regionPath);
+  }
+
+  /**
+   * Test queries with index on partitioned regions and concurrent PUT, DESTORY, INVALIDATE operations.
+   * Make sure there is no UNDEFINED in the query result.
+   */
+  @Test
+  public void testQueriesOnPartitionedRegion() throws Exception {
+    testRegion = createPartitionedRegion(testRegionName);
+    String regionPath = "/" + testRegionName + ".entrySet entry";
+    executeQueryTest(getQueriesOnRegion(testRegionName), "entry.key.Index", regionPath);
+  }
+
+  private Region createReplicatedRegion(String regionName) throws ParseException {
+    Cache cache = CacheUtils.getCache();
+    AttributesFactory attributesFactory = new AttributesFactory();
+    attributesFactory.setDataPolicy(DataPolicy.REPLICATE);
+    RegionAttributes regionAttributes = attributesFactory.create();
+    return cache.createRegion(regionName, regionAttributes);
+  }
+
+  private Region createPartitionedRegion(String regionName) throws ParseException {
+    Cache cache = CacheUtils.getCache();
+    PartitionAttributesFactory prAttFactory = new PartitionAttributesFactory();
+    AttributesFactory attributesFactory = new AttributesFactory();
+    attributesFactory.setPartitionAttributes(prAttFactory.create());
+    RegionAttributes regionAttributes = attributesFactory.create();
+    return cache.createRegion(regionName, regionAttributes);
+  }
+
+  private void populateRegion(Region region) throws Exception {
+    for (int i = 1; i <= numElem; i++) {
+      putData(i, region);
+    }
+  }
+
+  private void putData(int id, Region region) throws ParseException {
+    region.put(new SomeKey(id, id), id);
+  }
+  
+  private void clearData(Region region) {
+    Iterator it = region.entrySet().iterator();
+    while (it.hasNext()) {
+      Region.Entry entry = (Region.Entry) it.next();
+      region.destroy(entry.getKey());
+    }
+  }
+
+  /**** Query Execution Helpers ****/
+
+  private void executeQueryTest(String[] queries, String indexedExpression, String regionPath) throws Exception {
+    Cache cache = CacheUtils.getCache();
+    boolean[] booleanVals = {true, false};
+    for (String query : queries) {
+      for (boolean isDestroy : booleanVals) {
+        clearData(testRegion);
+        populateRegion(testRegion);
+        Assert.assertNotNull(cache.getRegion(testRegionName));
+        Assert.assertEquals(numElem, cache.getRegion(testRegionName).size());
+        if (isDestroy) {
+          helpTestFunctionalIndexForQuery(query, indexedExpression, regionPath, new DestroyEntryTestHook(testRegion));
+        }
+        else {
+          helpTestFunctionalIndexForQuery(query, indexedExpression, regionPath, new InvalidateEntryTestHook(testRegion));
+        }
+      }
+    }
+    
+    queries = getQueriesOnRegionForPut(testRegionName);
+    for (String query : queries) {
+      clearData(testRegion);
+      populateRegion(testRegion);
+      Assert.assertNotNull(cache.getRegion(testRegionName));
+      Assert.assertEquals(numElem, cache.getRegion(testRegionName).size());
+      helpTestFunctionalIndexForQuery(query, indexedExpression, regionPath, new PutEntryTestHook(testRegion));
+    }
+  }
+
+  /**
+   *  helper method to test against a functional index
+   *  make sure there is no UNDEFINED result
+   * 
+   * @param query
+   * 
+   * @throws Exception */
+  private SelectResults helpTestFunctionalIndexForQuery(String query, String indexedExpression, String regionPath, AbstractTestHook testHook) throws Exception {
+    MyQueryObserverAdapter observer = new MyQueryObserverAdapter();
+    QueryObserverHolder.setInstance(observer);
+    IndexManager.testHook = testHook;
+    QueryService qs = CacheUtils.getQueryService();
+    Index index = qs.createIndex("testIndex", indexedExpression, regionPath);
+    SelectResults indexedResults = (SelectResults) qs.newQuery(query).execute();
+    Iterator iterator = indexedResults.iterator();
+    while (iterator.hasNext()) {
+      Object row = iterator.next();
+      if (row instanceof Struct) {
+        Object[] fields = ((Struct) row).getFieldValues();
+        for (Object field: fields) {
+          Assert.assertTrue(field != QueryService.UNDEFINED);
+          if (field instanceof String) {
+            Assert.assertTrue(((String) field).compareTo(newValue) != 0);
+          }
+        }
+      }
+      else {
+        Assert.assertTrue(row != QueryService.UNDEFINED);
+        if (row instanceof String) {
+          Assert.assertTrue(((String) row).compareTo(newValue) != 0);
+        }
+      }
+    }
+    Assert.assertTrue(indexedResults.size() > 0);
+    Assert.assertTrue(observer.indexUsed);
+    Assert.assertTrue(((AbstractTestHook) IndexManager.testHook).isTestHookCalled());
+    ((AbstractTestHook) IndexManager.testHook).reset();
+    qs.removeIndex(index);
+   
+    return indexedResults;
+  }
+
+  class MyQueryObserverAdapter extends QueryObserverAdapter {
+    public boolean indexUsed = false;
+
+    public void afterIndexLookup(Collection results) {
+      super.afterIndexLookup(results);
+      indexUsed = true;
+    }
+  }
+
+  class SomeKey {
+    public int Index = 1;
+    public int PartitionID = 1;
+
+    public SomeKey(int index, int partitionId) {
+      this.Index = index;
+      this.PartitionID = partitionId;
+    }
+
+    public boolean equals(Object other) {
+      if (other instanceof SomeKey) {
+        SomeKey otherKey = (SomeKey) other;
+        return this.Index == otherKey.Index && this.PartitionID == otherKey.PartitionID;
+      }
+      return false;
+    }
+
+    public String toString() {
+      return "somekey:" + Index + "," + PartitionID;
+
+    }
+  }
+
+  /**
+   * Test hook
+   */
+  abstract class AbstractTestHook implements IndexManager.TestHook {
+    boolean isTestHookCalled = false;
+    Object waitObj = new Object();
+    Region r;
+
+    public void reset() {
+      isTestHookCalled = false;
+    }
+    
+    public boolean isTestHookCalled() {
+      return isTestHookCalled;
+    }
+    
+    /**
+     * Subclass override with different operation
+     */
+    public abstract void doOp();
+    
+    @Override
+    public void hook(int spot) throws RuntimeException {
+      if (spot == 200) {
+        if (!isTestHookCalled) {
+          isTestHookCalled = true;
+          try {
+            new Thread(new Runnable() {
+              public void run() {
+                doOp();
+                synchronized (waitObj) {
+                  waitObj.notifyAll();
+                }
+              }
+            }).start();
+            synchronized (waitObj) {
+              waitObj.wait();
+            }
+          } catch (InterruptedException e) {
+            throw new Error(e);
+          }
+        }
+      }
+
+    }
+
+  }
+  
+  class DestroyEntryTestHook extends AbstractTestHook {
+    
+    DestroyEntryTestHook(Region r) {
+      this.r = r;
+    }
+    
+    public void doOp() {
+      Iterator it = r.entrySet().iterator();
+      while (it.hasNext()) {
+        Region.Entry entry = (Region.Entry) it.next();
+          r.destroy(entry.getKey());
+        }      
+    }
+  }
+  
+  class InvalidateEntryTestHook extends AbstractTestHook {
+    
+    InvalidateEntryTestHook(Region r) {
+      this.r = r;
+    }
+    
+    public void doOp() {
+      Iterator it = r.entrySet().iterator();
+      while (it.hasNext()) {
+        Region.Entry entry = (Region.Entry) it.next();
+          r.invalidate(entry.getKey());
+      }      
+    }
+  }
+  
+  class PutEntryTestHook extends AbstractTestHook {
+    
+    PutEntryTestHook(Region r) {
+      this.r = r;
+    }
+    
+    public void doOp() {
+      Iterator it = r.entrySet().iterator();
+      while (it.hasNext()) {
+        Region.Entry entry = (Region.Entry) it.next();
+        r.put(entry.getKey(), newValue);
+      }      
+    }
+  }
+}