You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by je...@apache.org on 2015/12/30 01:28:01 UTC

[01/33] incubator-geode git commit: GEODE-588: CI failure in LocatorDUnitTest

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-14 b15ddfd8d -> f23e6fd72


GEODE-588: CI failure in LocatorDUnitTest

TcpServer waits up to 60 seconds for its executor pool to shut down
but InternalLocator.stop() also waits for 60 seconds for TcpServer
to shut down.  I made the stop() wait time dependent on TcpServer's
wait time and added some buffer.  I also added an expected-exception
for the suspect string that causes GEODE-588.


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

Branch: refs/heads/feature/GEODE-14
Commit: af654ca6c3cc5a5aa447cb1b8eb710d095558414
Parents: 0851f18
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Dec 18 15:59:56 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri Dec 18 15:59:56 2015 -0800

----------------------------------------------------------------------
 .../gemstone/gemfire/distributed/internal/InternalLocator.java    | 2 +-
 .../gemfire/distributed/internal/tcpserver/TcpServer.java         | 2 +-
 .../java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java   | 3 ++-
 3 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/af654ca6/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 5090db8..9d28d1c 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
@@ -965,7 +965,7 @@ public class InternalLocator extends Locator implements ConnectListener {
       }
       boolean interrupted = Thread.interrupted();
       try {
-        this.server.join(60 * 1000);
+        this.server.join(TcpServer.SHUTDOWN_WAIT_TIME * 1000 + 10000);
   
       } catch (InterruptedException ex) {
         interrupted = true;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/af654ca6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
index 92793ae..f52b9ab 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
@@ -99,7 +99,7 @@ public class TcpServer {
   public static int TESTVERSION = GOSSIPVERSION;
   public static int OLDTESTVERSION = OLDGOSSIPVERSION;
 
-  private static final long SHUTDOWN_WAIT_TIME = 60 * 1000;
+  public static final long SHUTDOWN_WAIT_TIME = 60 * 1000;
   private static int MAX_POOL_SIZE = Integer.getInteger("gemfire.TcpServer.MAX_POOL_SIZE", 100).intValue();
   private static int POOL_IDLE_TIMEOUT = 60 * 1000;
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/af654ca6/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
index 80f527b..e13d0f3 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
@@ -1436,12 +1436,13 @@ public class LocatorDUnitTest extends DistributedTestCase {
       vm0.invoke(getStopLocatorRunnable());
     }
   }
-
+  
   /**
    * Tests starting multiple locators in multiple VMs.
    */
   public void testMultipleMcastLocators() throws Exception {
     disconnectAllFromDS();
+    addExpectedException("Could not stop  Distribution Locator"); // shutdown timing issue in InternalLocator
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);


[16/33] incubator-geode git commit: GEODE-299

Posted by je...@apache.org.
GEODE-299

when shutdown distributed system, Wait until cache is closed.


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

Branch: refs/heads/feature/GEODE-14
Commit: adacdd72ed96b4fdaa8d4d78b924e2648b66865f
Parents: 26ce9a4
Author: zhouxh <gz...@pivotal.io>
Authored: Mon Dec 21 11:45:17 2015 -0800
Committer: zhouxh <gz...@pivotal.io>
Committed: Mon Dec 21 11:49:58 2015 -0800

----------------------------------------------------------------------
 .../partitioned/PersistentPartitionedRegionDUnitTest.java      | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/adacdd72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
index 39ffb11..d043df8 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.internal.cache.partitioned;
 
+import static com.jayway.awaitility.Awaitility.await;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.File;
@@ -1439,7 +1441,11 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
           public void beforeSendMessage(DistributionManager dm,
               DistributionMessage msg) {
             if(msg instanceof ManageBucketReplyMessage) {
+              Cache cache = getCache();
               DistributedTestCase.disconnectFromDS();
+              
+              await().atMost(30, SECONDS).until(() -> {return (cache == null || cache.isClosed());});
+              getLogWriter().info("Cache is confirmed closed");
             }
           }
         });


[28/33] incubator-geode git commit: Fix for suspect string causing a number of CI failures

Posted by je...@apache.org.
Fix for suspect string causing a number of CI failures

During shutdown it's possible for the DirectChannel communication service
to throw a ConnectExceptions containing a ShunnedMemberException for one
or more members.  These indicate that the message couldn't be sent, but this
is expected behavior and the ShunnedMemberExceptions can be ignored.
This fixes GEODE-355, GEODE-682, GEODE-683, GEODE-684, GEODE-685, GEODE-687,
GEODE-689, GEODE-691, GEODE-702 and GEODE-703.


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

Branch: refs/heads/feature/GEODE-14
Commit: 2a21b70477123e679cbb76c8ffa07e44b6e45e6c
Parents: 1c2cad1
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Wed Dec 23 09:48:39 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Wed Dec 23 09:48:39 2015 -0800

----------------------------------------------------------------------
 .../internal/membership/gms/mgr/GMSMembershipManager.java         | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2a21b704/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index 56643ad..8ce5178 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -66,6 +66,7 @@ import com.gemstone.gemfire.distributed.internal.StartupMessage;
 import com.gemstone.gemfire.distributed.internal.ThrottlingMemLinkedQueueWithDMStats;
 import com.gemstone.gemfire.distributed.internal.direct.DirectChannel;
 import com.gemstone.gemfire.distributed.internal.direct.DirectChannelListener;
+import com.gemstone.gemfire.distributed.internal.direct.ShunnedMemberException;
 import com.gemstone.gemfire.distributed.internal.membership.DistributedMembershipListener;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
@@ -1765,7 +1766,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
         InternalDistributedMember member = (InternalDistributedMember)it_mem.next();
         Throwable th = (Throwable)it_causes.next();
         
-        if (!view.contains(member)) {
+        if (!view.contains(member) || (th instanceof ShunnedMemberException)) {
           continue;
         }
         logger.fatal(LocalizedMessage.create(


[22/33] incubator-geode git commit: Additional rat and git excludes

Posted by je...@apache.org.
Additional rat and git excludes

Excluding the following file types:
*.log
*.diff
*.patch
*.dat (locator data 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/7adbc2ad
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/7adbc2ad
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/7adbc2ad

Branch: refs/heads/feature/GEODE-14
Commit: 7adbc2ad57f000b5d8b5f92694cf90062c692a22
Parents: 94bce4d
Author: Dan Smith <up...@apache.org>
Authored: Tue Dec 22 10:49:50 2015 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Tue Dec 22 10:50:54 2015 -0800

----------------------------------------------------------------------
 .gitignore        | 4 ++++
 gradle/rat.gradle | 4 ++++
 2 files changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7adbc2ad/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 345a55a..0034762 100644
--- a/.gitignore
+++ b/.gitignore
@@ -17,5 +17,9 @@ build-eclipse/
 *.ipr
 *.iws
 *.swp
+*.log
+*.patch
+*.diff
+*.dat
 # bruce: my synch settings for windows->linux transfers
 transferhost.txt

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7adbc2ad/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index fb4648e..e59b50d 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -51,6 +51,9 @@ rat {
     '**/*.json',
     '**/*.tx0',
     '**/*.txo',
+    '**/*.log',
+    '**/*.patch',
+    '**/*.diff',
 
     // binary files
     '**/*.cer',
@@ -66,6 +69,7 @@ rat {
     '**/*.truststore',
     '**/*.xls',
     '**/publickeyfile',
+    '**/*.dat',
 
     // other text files
     'gemfire-spark-connector/project/plugins.sbt',


[32/33] incubator-geode git commit: GEODE-708: Add stats for Geode membership health monitor

Posted by je...@apache.org.
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/feature/GEODE-14
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();



[14/33] incubator-geode git commit: GEODE-660: Fixing inconsistencies and reliance on randomness in TableBuilderJUnitTest

Posted by je...@apache.org.
GEODE-660: Fixing inconsistencies and reliance on randomness in TableBuilderJUnitTest


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

Branch: refs/heads/feature/GEODE-14
Commit: c19c3cab01e2c8975b35a563adc84e07bd42af4e
Parents: 5787a48
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed Dec 16 16:19:53 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Dec 21 10:05:11 2015 -0800

----------------------------------------------------------------------
 gemfire-core/build.gradle                       |   8 +
 .../internal/cli/result/TableBuilder.java       |   9 +-
 .../internal/cli/result/TableBuilderHelper.java |  18 +-
 .../internal/cli/TableBuilderJUnitTest.java     | 307 +++++++++++++------
 gradle/dependency-versions.properties           |   2 +
 5 files changed, 246 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c19c3cab/gemfire-core/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-core/build.gradle b/gemfire-core/build.gradle
index 28fb3ba..3e3c19b 100755
--- a/gemfire-core/build.gradle
+++ b/gemfire-core/build.gradle
@@ -99,6 +99,14 @@ dependencies {
   testRuntime 'commons-io:commons-io:' + project.'commons-io.version'
   testCompile 'net.spy:spymemcached:' + project.'spymemcached.version'
   testCompile 'redis.clients:jedis:' + project.'jedis.version'
+
+  testCompile 'org.powermock:powermock-core:' + project.'powermock.version'
+  testCompile 'org.powermock:powermock-module-junit4:' + project.'powermock.version'
+  testCompile 'org.powermock:powermock-module-junit4-common:' + project.'powermock.version'
+  testCompile 'org.powermock:powermock-api-support:' + project.'powermock.version'
+  testCompile 'org.powermock:powermock-api-mockito:' + project.'powermock.version'
+  testRuntime 'org.powermock:powermock-reflect:' + project.'powermock.version'
+  testRuntime 'org.javassist:javassist:' + project.'javassist.version'
 }
 
 def generatedResources = "$buildDir/generated-resources/main"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c19c3cab/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/TableBuilder.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/TableBuilder.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/TableBuilder.java
index d2363de..dfa4b3e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/TableBuilder.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/TableBuilder.java
@@ -104,7 +104,14 @@ public class TableBuilder {
       RowGroup rowGroup = newRowGroup();
       rowGroup.newBlankRow();
     }
-    
+
+    public RowGroup getLastRowGroup() {
+      if (rowGroups.size() == 0) {
+        return null;
+      }
+      return rowGroups.get(rowGroups.size() - 1);
+    }
+
     /**
      * Computes total Max Row Length across table - for all row groups.
      */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c19c3cab/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/TableBuilderHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/TableBuilderHelper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/TableBuilderHelper.java
index 32fa048..aa4029f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/TableBuilderHelper.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/TableBuilderHelper.java
@@ -31,6 +31,8 @@ import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 
 public class TableBuilderHelper {
 
+  private static final int SCREEN_WIDTH_MARGIN_BUFFER = 5;
+
   public static class Column implements Comparable<Column>{
     int length;
     int originalIndex;
@@ -85,21 +87,19 @@ public class TableBuilderHelper {
       int totalExtra = 0;
       for (Column s : stringList) {
         int newLength = totalLength + s.length;
-        if (newLength > screenWidth) {
+        // Ensure that the spaceLeft is never < 2 which would prevent displaying a trimmed value
+        // even when there is space available on the screen.
+        if (newLength + SCREEN_WIDTH_MARGIN_BUFFER > screenWidth) {
           s.markForTrim = true;
           totalExtra += s.length;
-          if (spaceLeft == 0)
+          if (spaceLeft == 0) {
             spaceLeft = screenWidth - totalLength;
+          }
         }
         totalLength = newLength;
       }
 
-      Collections.sort(stringList, new Comparator<Column>() {
-        @Override
-        public int compare(Column o1, Column o2) {
-          return o1.originalIndex - o2.originalIndex;
-        }
-      });
+      Collections.sort(stringList, (o1, o2) -> o1.originalIndex - o2.originalIndex);
 
       //calculate trimmed width for columns marked for
       //distribute the trimming as per percentage
@@ -129,7 +129,7 @@ public class TableBuilderHelper {
           throw new TooManyColumnsException(
               "Computed ColSize="
                   + colSize
-                  + " Set RESULT_VIEWER to external (uses less commands for enabling horizontal scroll) to see wider results");
+                  + " Set RESULT_VIEWER to external. This uses the 'less' command (with horizontal scrolling) to see wider results");
         totalLength += colSize;
         index++;
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c19c3cab/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/TableBuilderJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/TableBuilderJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/TableBuilderJUnitTest.java
index e5f1d86..ad23427 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/TableBuilderJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/TableBuilderJUnitTest.java
@@ -21,163 +21,294 @@ import com.gemstone.gemfire.management.internal.cli.result.TableBuilder.Row;
 import com.gemstone.gemfire.management.internal.cli.result.TableBuilder.RowGroup;
 import com.gemstone.gemfire.management.internal.cli.result.TableBuilder.Table;
 import com.gemstone.gemfire.management.internal.cli.result.TableBuilderHelper;
-import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.Ignore;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
 
-import java.io.IOException;
-import java.util.Properties;
-import java.util.Random;
+import java.util.Arrays;
+import java.util.List;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
+import static org.powermock.api.mockito.PowerMockito.spy;
+import static org.powermock.api.mockito.PowerMockito.when;
 
 /**
- * TODO: fails when running integrationTest from gradle command-line or in Eclipse on Windows 7
- * <p>
- * com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest > testBasicScrapping FAILED
- * java.lang.AssertionError: Expected length < 100 is 101 at org.junit.Assert.fail(Assert.java:88) at
- * com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest.doTableBuilderTestUnit(TableBuilderJUnitTest.java:115)
- * at com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest.testBasicScrapping(TableBuilderJUnitTest.java:134)
- * <p>
- * com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest > testManyColumns FAILED java.lang.AssertionError:
- * Expected length < 100 is 101 at org.junit.Assert.fail(Assert.java:88) at com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest.doTableBuilderTestUnit(TableBuilderJUnitTest.java:115)
- * at com.gemstone.gemfire.management.internal.cli.TableBuilderJUnitTest.testManyColumns(TableBuilderJUnitTest.java:155)
+ * Testing TableBuilder and TableBuilderHelper using mocks for Gfsh
  *
  * @author tushark
+ * @author Jens Deppe
  */
 @Category(IntegrationTest.class)
+@RunWith(PowerMockRunner.class)
+@PowerMockIgnore("*.IntegrationTest")
+@PrepareForTest(TableBuilderHelper.class)
 public class TableBuilderJUnitTest {
 
   @Rule
   public TestName testName = new TestName();
 
-  private final Table createTable(int rows, int cols, int width, String separator) {
+  @Before
+  public void setUp() throws Exception {
+    // This sets up a partial mock for some static methods
+    spy(TableBuilderHelper.class);
+    when(TableBuilderHelper.class, "getScreenWidth").thenReturn(40);
+    when(TableBuilderHelper.class, "shouldTrimColumns").thenReturn(true);
+  }
+
+  private Table createTableStructure(int cols, String separator) {
+    String[] colNames = new String[cols];
+    for (int i = 0; i < cols; i++) {
+      colNames[i] = "Field";
+    }
+    return createTableStructure(cols, separator, colNames);
+  }
+
+  private Table createTableStructure(int cols, String separator, String... colNames) {
     Table resultTable = TableBuilder.newTable();
     resultTable.setTabularResult(true);
     resultTable.setColumnSeparator(separator);
 
     resultTable.newBlankRow();
-    resultTable.newRow().newLeftCol("Displaying all fields for member: ");
-    resultTable.newBlankRow();
     RowGroup rowGroup = resultTable.newRowGroup();
     Row row = rowGroup.newRow();
     for (int colIndex = 0; colIndex < cols; colIndex++) {
-      row.newCenterCol("Field" + colIndex);
+      row.newCenterCol(colNames[colIndex] + colIndex);
     }
 
     rowGroup.newRowSeparator('-', false);
 
-    int counter = rows;
-    for (int i = 0; i < counter; i++) {
-      row = rowGroup.newRow();
-      for (int k = 0; k < cols; k++) {
-        row.newLeftCol(getString(i, width / cols));
-      }
-    }
-    resultTable.newBlankRow();
-
     return resultTable;
   }
 
-  private Object getString(int i, int width) {
-    StringBuilder sb = new StringBuilder();
-    Random random = new Random();
-    int k = 0;
-    double d = random.nextDouble();
-    // .09 probability
-    if (d <= 0.9) {
-      k = random.nextInt(width);
-    } else {
-      k = width / 2 + random.nextInt(width);
-    }
-    random.nextInt(10);
-    for (int j = 0; j < k; j++) {
-      sb.append(i);
-      if (sb.length() > k) break;
-    }
-    return sb.toString();
-  }
-
-  private HeadlessGfsh createShell(Properties props) throws ClassNotFoundException, IOException {
-    String shellId = getClass().getSimpleName() + "_" + testName;
-    HeadlessGfsh shell = new HeadlessGfsh(shellId, 30, props);
-    return shell;
-  }
-
-  private void doTableBuilderTestUnit(int rows, int cols, String sep, boolean shouldTrim,
-      boolean expectTooManyColEx) throws ClassNotFoundException, IOException {
-    int width = Gfsh.getCurrentInstance().getTerminalWidth();
-    Table table = createTable(rows, cols, width, sep);
+  private List<String> validateTable(Table table, boolean shouldTrim) {
+    int screenWidth = TableBuilderHelper.getScreenWidth();
     String st = table.buildTable();
     System.out.println(st);
 
-    String[] array = st.split("\n");
+    List<String> lines = Arrays.asList(st.split(GfshParser.LINE_SEPARATOR));
 
     int line = 0;
-    for (String s : array) {
-      System.out.println("For line " + line++ + " length is " + s.length() + " isWider = " + (s.length() > width));
+    for (String s : lines) {
+      System.out.println("For line " + line++ + " length is " + s.length() + " isWider = " + (s.length() > screenWidth));
 
       if (shouldTrim) {
-        if (s.length() > width) {
-          fail("Expected length < " + width + " is " + s.length());
+        if (s.length() > screenWidth) {
+          fail("Expected length > screenWidth: " + s.length() + " > " + screenWidth);
         }
       } else {
-        if (s.length() > 50 && s.length() <= width) {
-          fail("Expected length <= " + width + " is " + s.length());
+        if (s.length() != 0 && s.length() <= screenWidth) {
+          fail("Expected length <= screenWidth: " + s.length() + " <= " + screenWidth);
         }
       }
-
     }
+
+    return lines;
   }
 
   /**
-   * Test Variations tablewide separator true false
+   * Test Variations table-wide separator true false
    */
   @Test
-  public void testBasicScraping() throws ClassNotFoundException, IOException {
-    Properties props = new Properties();
-    props.setProperty(Gfsh.ENV_APP_RESULT_VIEWER, Gfsh.DEFAULT_APP_RESULT_VIEWER);
-    createShell(props);
+  public void testSanity() throws Exception {
     assertTrue(TableBuilderHelper.shouldTrimColumns());
-    doTableBuilderTestUnit(15, 4, "|", true, false);
+
+    Table table = createTableStructure(3, "|");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("1")
+        .newLeftCol("1")
+        .newLeftCol("1");
+
+    List<String> result = validateTable(table, true);
+    // Check the last line
+    assertEquals("1     |1     |1", result.get(3));
   }
 
+  @Test
+  public void testLastColumnTruncated() throws Exception {
+    assertTrue(TableBuilderHelper.shouldTrimColumns());
+
+    Table table = createTableStructure(4, "|");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("1")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-123456789-12345");
+
+    List<String> result = validateTable(table, true);
+    // Check the last line
+    assertEquals("1     |123456789-|123456789-|123456789..", result.get(3));
+  }
 
   @Test
-  public void testSeparatorWithMultipleChars() throws ClassNotFoundException, IOException {
-    Properties props = new Properties();
-    props.setProperty(Gfsh.ENV_APP_RESULT_VIEWER, Gfsh.DEFAULT_APP_RESULT_VIEWER);
-    createShell(props);
+  public void testLongestColumnFirstTruncated() throws Exception {
     assertTrue(TableBuilderHelper.shouldTrimColumns());
-    doTableBuilderTestUnit(15, 4, " | ", true, false);
+
+    Table table = createTableStructure(4, "|");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("123456789-123456789-")
+        .newLeftCol("123456789-12345")
+        .newLeftCol("123456789-")
+        .newLeftCol("1");
+
+    List<String> result = validateTable(table, true);
+    // Check the last line
+    assertEquals("1234..|123456789-12345|123456789-|1", result.get(3));
+  }
+
+  @Test
+  public void testMultipleColumnsTruncated() throws Exception {
+    assertTrue(TableBuilderHelper.shouldTrimColumns());
+
+    Table table = createTableStructure(4, "|");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("1")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-123456789-123456789-")
+        .newLeftCol("123456789-123456789-12345");
+
+    List<String> result = validateTable(table, true);
+    // Check the last line
+    assertEquals("1     |123456789-|123456789..|1234567..", result.get(3));
+  }
+
+  @Test
+  public void testMultipleColumnsTruncatedLongestFirst() throws Exception {
+    assertTrue(TableBuilderHelper.shouldTrimColumns());
+
+    Table table = createTableStructure(4, "|");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("123456789-123456789-123456789-")
+        .newLeftCol("123456789-123456789-12345")
+        .newLeftCol("1")
+        .newLeftCol("123456789-");
+
+    List<String> result = validateTable(table, true);
+    // Check the last line
+    assertEquals("123456789..|1234567..|1     |123456789-", result.get(3));
+  }
+
+  @Test
+  public void testColumnsWithShortNames() throws Exception {
+    when(TableBuilderHelper.class, "getScreenWidth").thenReturn(9);
+    assertTrue(TableBuilderHelper.shouldTrimColumns());
+
+    Table table = createTableStructure(3, "|", "A", "A", "A");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("123")
+        .newLeftCol("123")
+        .newLeftCol("123");
+
+    List<String> result = validateTable(table, true);
+    // Check the last line
+    assertEquals("..|..|..", result.get(3));
+  }
+
+  @Test(expected = TableBuilderHelper.TooManyColumnsException.class)
+  public void testExceptionTooSmallWidth() throws Exception {
+    when(TableBuilderHelper.class, "getScreenWidth").thenReturn(7);
+    assertTrue(TableBuilderHelper.shouldTrimColumns());
+
+    Table table = createTableStructure(3, "|", "A", "A", "A");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("12")
+        .newLeftCol("12")
+        .newLeftCol("12");
+
+    // This should throw an exception
+    List<String> result = validateTable(table, true);
+  }
+
+  @Test
+  public void testTooLittleSpaceOnNextToLastColumn() throws Exception {
+    assertTrue(TableBuilderHelper.shouldTrimColumns());
+
+    Table table = createTableStructure(4, "|");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("1")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-123456789-123456789-")
+        .newLeftCol("123456789-123456789-12345");
+
+    List<String> result = validateTable(table, true);
+    // Check the last line
+    assertEquals("1     |123456789-|123456789..|1234567..", result.get(3));
+  }
+
+  @Test
+  public void testSeparatorWithMultipleChars() throws Exception {
+    assertTrue(TableBuilderHelper.shouldTrimColumns());
+
+    Table table = createTableStructure(4, "<|>");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("1")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-123456789-12345");
+
+    List<String> result = validateTable(table, true);
+    // Check the last line
+    assertEquals("1     <|>123456789-<|>123456789-<|>123..", result.get(3));
   }
 
   /**
    * multiple columns upto 8 : done
    */
   @Test
-  @Ignore("Bug 52051")
-  public void testManyColumns() throws ClassNotFoundException, IOException {
-    createShell(null);
+  public void testManyColumns() throws Exception {
     assertTrue(TableBuilderHelper.shouldTrimColumns());
-    doTableBuilderTestUnit(15, 6, "|", true, true);
+
+    Table table = createTableStructure(8, "|");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("123456789-")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-");
+
+    List<String> result = validateTable(table, true);
+    // Check the last line
+    assertEquals("123456789-|123456789-|..|..|..|..|..|..", result.get(3));
   }
 
   /**
    * set gfsh env property result_viewer to basic disable for external reader
    */
-  //
   @Test
-  public void testDisableColumnAdjustment() throws ClassNotFoundException, IOException {
-    createShell(null);
+  public void testDisableColumnAdjustment() throws Exception {
+    when(TableBuilderHelper.class, "shouldTrimColumns").thenReturn(false);
     assertFalse(TableBuilderHelper.shouldTrimColumns());
-    doTableBuilderTestUnit(15, 12, "|", false, false);
-  }
 
+    Table table = createTableStructure(5, "|");
+    RowGroup rowGroup = table.getLastRowGroup();
+    Row row1 = rowGroup.newRow();
+    row1.newLeftCol("1")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-")
+        .newLeftCol("123456789-123456789-12345")
+        .newLeftCol("1");
+
+    List<String> result = validateTable(table, false);
+    // Check the last line
+    assertEquals("1     |123456789-|123456789-|123456789-123456789-12345|1", result.get(3));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c19c3cab/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 71cfd43..30a7fd8 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -44,6 +44,7 @@ hbase.version = 0.94.27
 jackson.version = 2.2.0
 jackson-module-scala_2.10.version = 2.1.5
 jansi.version = 1.8
+javassist.version = 3.20.0-GA
 javax.mail-api.version = 1.4.5
 javax.resource-api.version = 1.7
 javax.servlet-api.version = 3.1.0
@@ -66,6 +67,7 @@ mx4j-tools.version = 3.0.1
 netty-all.version = 4.0.4.Final
 objenesis.version = 2.1
 paranamer.version = 2.3
+powermock.version = 1.6.4
 quartz.version = 2.2.1
 scala.version = 2.10.0
 slf4j-api.version = 1.7.7


[24/33] incubator-geode git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-geode into develop

Posted by je...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-geode into 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/39790d51
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/39790d51
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/39790d51

Branch: refs/heads/feature/GEODE-14
Commit: 39790d517d2cf53be3e6e2586ae5e3690b278405
Parents: a03e8b7 7adbc2a
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue Dec 22 10:56:43 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Dec 22 10:56:43 2015 -0800

----------------------------------------------------------------------
 .gitignore        | 4 ++++
 gradle/rat.gradle | 4 ++++
 2 files changed, 8 insertions(+)
----------------------------------------------------------------------



[12/33] incubator-geode git commit: GEODE-663: Dummy edit to close Github pull request.

Posted by je...@apache.org.
GEODE-663: Dummy edit to close Github pull request.

Closes #58


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

Branch: refs/heads/feature/GEODE-14
Commit: 3ff5d785131d50ab03ede22f10c7205798dda2bb
Parents: 3720666
Author: Jens Deppe <jd...@pivotal.io>
Authored: Mon Dec 21 09:24:04 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Dec 21 09:24:04 2015 -0800

----------------------------------------------------------------------
 .../gemstone/gemfire/security/ClientAuthenticationDUnitTest.java | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3ff5d785/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
index 2bd04e0..f3a6dd3 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
@@ -1,5 +1,3 @@
-package com.gemstone.gemfire.security;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -18,7 +16,7 @@ package com.gemstone.gemfire.security;
  * specific language governing permissions and limitations
  * under the License.
  */
-
+package com.gemstone.gemfire.security;
 
 import java.io.IOException;
 import java.util.Properties;


[17/33] incubator-geode git commit: fixing a test hook for a QueryMonitor unit test

Posted by je...@apache.org.
fixing a test hook for a QueryMonitor unit test

The affected test will be moved to open-source in the up-coming
migration of the continuous-query project to open-source.  The
test hook wasn't quite robust enough and allowed a query that was
canceled to appear, to the test at least, as if it had not been
canceled.


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

Branch: refs/heads/feature/GEODE-14
Commit: 4630e2ef4cd74c605c76a170a8c21173a919fef4
Parents: adacdd7
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Dec 21 12:18:10 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Dec 21 12:19:33 2015 -0800

----------------------------------------------------------------------
 .../gemstone/gemfire/cache/query/internal/QueryMonitor.java  | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4630e2ef/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/QueryMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/QueryMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/QueryMonitor.java
index 049b9ab..65a64a4 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/QueryMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/QueryMonitor.java
@@ -153,6 +153,12 @@ public class QueryMonitor implements Runnable {
             testException = new QueryExecutionTimeoutException("The Query completed sucessfully before it got canceled.");          
           }
         }
+        
+        if ((testException == null) && (query instanceof DefaultQuery)) {
+          if (((DefaultQuery)query).isCanceled()) {
+            testException = new QueryExecutionTimeoutException("The query task could not be found but the query is marked as having been canceled");
+          }
+        }
       }
       // END - DUnit Test purpose.
 
@@ -162,7 +168,7 @@ public class QueryMonitor implements Runnable {
     }
     
     if (logger.isDebugEnabled()) {
-      logger.debug("Removed thread from QueryMonitor. QueryMonitor size is:{}, Thread (id): thread is : {}", 
+      logger.debug("Removed thread from QueryMonitor. QueryMonitor size is:{}, Thread ID is: {}  thread is : {}", 
           queryThreads.size(), queryThread.getId(), queryThread);
     }
     


[03/33] incubator-geode git commit: GEODE-700: Make sure we're disconnected from the DS before starting the test

Posted by je...@apache.org.
GEODE-700: Make sure we're disconnected from the DS before starting the test


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

Branch: refs/heads/feature/GEODE-14
Commit: 341e29d1e7822e898d0be71ce439517379810919
Parents: e3d24d7
Author: Jens Deppe <jd...@pivotal.io>
Authored: Mon Dec 21 08:24:17 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Dec 21 08:24:56 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/341e29d1/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
index 54aed63..e9a4666 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
@@ -57,6 +57,11 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
     super(testName);
   }
 
+  public void setUp() throws Exception {
+    disconnectAllFromDS();
+    super.setUp();
+  }
+
   public void testAsyncEventQueue() throws IOException {
     final String queue1Name = "testAsyncEventQueue1";
     final String queue2Name = "testAsyncEventQueue2";


[07/33] incubator-geode git commit: GEM-164: move the security tests in gemfire-test module inside the com.gemstone.gemfire.security packages to the open side.

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/templates/security/CredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/CredentialGenerator.java b/gemfire-core/src/test/java/templates/security/CredentialGenerator.java
new file mode 100644
index 0000000..99d7e18
--- /dev/null
+++ b/gemfire-core/src/test/java/templates/security/CredentialGenerator.java
@@ -0,0 +1,343 @@
+
+package templates.security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import com.gemstone.gemfire.security.AuthInitialize;
+import com.gemstone.gemfire.security.Authenticator;
+
+/**
+ * Encapsulates obtaining valid and invalid credentials. Implementations will be
+ * for different kinds of authentication schemes.
+ * 
+ * @author sumedh
+ * @since 5.5
+ */
+public abstract class CredentialGenerator {
+
+  /**
+   * Enumeration for various {@link CredentialGenerator} implementations.
+   * 
+   * The following schemes are supported as of now:
+   * <code>DummyAuthenticator</code>, <code>LdapUserAuthenticator</code>,
+   * <code>PKCSAuthenticator</code>. In addition SSL socket mode with mutual
+   * authentication is also supported.
+   * 
+   * To add a new authentication scheme the following needs to be done:
+   * <ul>
+   * <li>Add implementations for {@link AuthInitialize} and
+   * {@link Authenticator} classes for clients/peers.</li>
+   * <li>Add a new enumeration value for the scheme in this class. Notice the
+   * size of <code>VALUES</code> array and increase that if it is getting
+   * overflowed. Note the methods and fields for existing schemes and add for
+   * the new one in a similar manner.</li>
+   * <li>Add an implementation for {@link CredentialGenerator}.</li>
+   * <li>Modify the {@link CredentialGenerator.Factory#create} method to add
+   * creation of an instance of the new implementation for the
+   * <code>ClassCode</code> enumeration value.</li>
+   * </ul>
+   * All security dunit tests will automagically start testing the new
+   * implementation after this.
+   * 
+   * @author sumedh
+   * @since 5.5
+   */
+  public static final class ClassCode {
+
+    private static final byte ID_DUMMY = 1;
+
+    private static final byte ID_LDAP = 2;
+
+    private static final byte ID_PKCS = 3;
+
+    private static final byte ID_SSL = 4;
+
+    private static byte nextOrdinal = 0;
+
+    private static final ClassCode[] VALUES = new ClassCode[10];
+
+    private static final Map CodeNameMap = new HashMap();
+
+    public static final ClassCode DUMMY = new ClassCode(
+        "templates.security.DummyAuthenticator.create", ID_DUMMY);
+
+    public static final ClassCode LDAP = new ClassCode(
+        "templates.security.LdapUserAuthenticator.create", ID_LDAP);
+
+    public static final ClassCode PKCS = new ClassCode(
+        "templates.security.PKCSAuthenticator.create", ID_PKCS);
+
+    public static final ClassCode SSL = new ClassCode("SSL", ID_SSL);
+
+    /** The name of this class. */
+    private final String name;
+
+    /** byte used as ordinal to represent this class */
+    private final byte ordinal;
+
+    /**
+     * One of the following: ID_DUMMY, ID_LDAP, ID_PKCS
+     */
+    private final byte classType;
+
+    /** Creates a new instance of class code. */
+    private ClassCode(String name, byte classType) {
+      this.name = name;
+      this.classType = classType;
+      this.ordinal = nextOrdinal++;
+      VALUES[this.ordinal] = this;
+      CodeNameMap.put(name, this);
+    }
+
+    public boolean isDummy() {
+      return (this.classType == ID_DUMMY);
+    }
+
+    public boolean isLDAP() {
+      return (this.classType == ID_LDAP);
+    }
+
+    public boolean isPKCS() {
+      return (this.classType == ID_PKCS);
+    }
+
+    public boolean isSSL() {
+      return (this.classType == ID_SSL);
+    }
+
+    /**
+     * Returns the <code>ClassCode</code> represented by specified ordinal.
+     */
+    public static ClassCode fromOrdinal(byte ordinal) {
+      return VALUES[ordinal];
+    }
+
+    /**
+     * Returns the <code>ClassCode</code> represented by specified string.
+     */
+    public static ClassCode parse(String operationName) {
+      return (ClassCode)CodeNameMap.get(operationName);
+    }
+
+    /**
+     * Returns all the possible values.
+     */
+    public static List getAll() {
+      List codes = new ArrayList();
+      Iterator iter = CodeNameMap.values().iterator();
+      while (iter.hasNext()) {
+        codes.add(iter.next());
+      }
+      return codes;
+    }
+
+    /**
+     * Returns the ordinal for this operation code.
+     * 
+     * @return the ordinal of this operation.
+     */
+    public byte toOrdinal() {
+      return this.ordinal;
+    }
+
+    /**
+     * Returns a string representation for this operation.
+     * 
+     * @return the name of this operation.
+     */
+    final public String toString() {
+      return this.name;
+    }
+
+    /**
+     * Indicates whether other object is same as this one.
+     * 
+     * @return true if other object is same as this one.
+     */
+    @Override
+    final public boolean equals(final Object obj) {
+      if (obj == this) {
+        return true;
+      }
+      if (!(obj instanceof ClassCode)) {
+        return false;
+      }
+      final ClassCode other = (ClassCode)obj;
+      return (other.ordinal == this.ordinal);
+    }
+
+    /**
+     * Indicates whether other <code>ClassCode</code> is same as this one.
+     * 
+     * @return true if other <code>ClassCode</code> is same as this one.
+     */
+    final public boolean equals(final ClassCode opCode) {
+      return (opCode != null && opCode.ordinal == this.ordinal);
+    }
+
+    /**
+     * Returns a hash code value for this <code>ClassCode</code> which is the
+     * same as its ordinal.
+     * 
+     * @return the ordinal of this operation.
+     */
+    @Override
+    final public int hashCode() {
+      return this.ordinal;
+    }
+
+  }
+
+  /**
+   * A set of properties that should be added to the Gemfire system properties
+   * before using the authentication module.
+   */
+  private Properties sysProps = null;
+
+  /**
+   * A set of properties that should be added to the java system properties
+   * before using the authentication module.
+   */
+  protected Properties javaProps = null;
+
+  /**
+   * A factory method to create a new instance of an {@link CredentialGenerator}
+   * for the given {@link ClassCode}. Caller is supposed to invoke
+   * {@link CredentialGenerator#init} immediately after obtaining the instance.
+   * 
+   * @param classCode
+   *                the <code>ClassCode</code> of the
+   *                <code>CredentialGenerator</code> implementation
+   * 
+   * @return an instance of <code>CredentialGenerator</code> for the given
+   *         class code
+   */
+  public static CredentialGenerator create(ClassCode classCode) {
+    switch (classCode.classType) {
+      // Removing dummy one to reduce test run times
+      // case ClassCode.ID_DUMMY:
+      // return new DummyCredentialGenerator();
+      case ClassCode.ID_LDAP:
+        return new LdapUserCredentialGenerator();
+        // case ClassCode.ID_SSL:ø
+        // return new SSLCredentialGenerator();
+      case ClassCode.ID_PKCS:
+        return new PKCSCredentialGenerator();
+      default:
+        return null;
+    }
+  }
+
+  /**
+   * Initialize the credential generator.
+   * 
+   * @throws IllegalArgumentException
+   *                 when there is a problem during initialization
+   */
+  public void init() throws IllegalArgumentException {
+    this.sysProps = initialize();
+  }
+
+  /**
+   * Initialize the credential generator. This is provided separately from the
+   * {@link #init} method for convenience of implementations so that they do not
+   * need to store in {@link #sysProps}. The latter is convenient for the users
+   * who do not need to store these properties rather can obtain it later by
+   * invoking {@link #getSystemProperties}
+   * 
+   * Required to be implemented by concrete classes that implement this abstract
+   * class.
+   * 
+   * @return A set of extra properties that should be added to Gemfire system
+   *         properties when not null.
+   * 
+   * @throws IllegalArgumentException
+   *                 when there is a problem during initialization
+   */
+  protected abstract Properties initialize() throws IllegalArgumentException;
+
+  /**
+   * 
+   * @return A set of extra properties that should be added to Gemfire system
+   *         properties when not null.
+   */
+  public Properties getSystemProperties() {
+    return this.sysProps;
+  }
+
+  /**
+   * 
+   * @return A set of extra properties that should be added to Gemfire system
+   *         properties when not null.
+   */
+  public Properties getJavaProperties() {
+    return this.javaProps;
+  }
+
+  /**
+   * The {@link ClassCode} of this particular implementation.
+   * 
+   * @return the <code>ClassCode</code>
+   */
+  public abstract ClassCode classCode();
+
+  /**
+   * The name of the {@link AuthInitialize} factory function that should be used
+   * in conjunction with the credentials generated by this generator.
+   * 
+   * @return name of the <code>AuthInitialize</code> factory function
+   */
+  public abstract String getAuthInit();
+
+  /**
+   * The name of the {@link Authenticator} factory function that should be used
+   * in conjunction with the credentials generated by this generator.
+   * 
+   * @return name of the <code>Authenticator</code> factory function
+   */
+  public abstract String getAuthenticator();
+
+  /**
+   * Get a set of valid credentials generated using the given index.
+   */
+  public abstract Properties getValidCredentials(int index);
+
+  /**
+   * Get a set of valid credentials for the given {@link Principal}.
+   * 
+   * @return credentials for the given <code>Principal</code> or null if none
+   *         possible.
+   */
+  public abstract Properties getValidCredentials(Principal principal);
+
+  /**
+   * Get a set of invalid credentials generated using the given index.
+   */
+  public abstract Properties getInvalidCredentials(int index);
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/templates/security/DummyAuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/DummyAuthzCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/DummyAuthzCredentialGenerator.java
new file mode 100644
index 0000000..5c27d60
--- /dev/null
+++ b/gemfire-core/src/test/java/templates/security/DummyAuthzCredentialGenerator.java
@@ -0,0 +1,142 @@
+
+package templates.security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+
+public class DummyAuthzCredentialGenerator extends AuthzCredentialGenerator {
+
+  public static final byte READER_ROLE = 1;
+
+  public static final byte WRITER_ROLE = 2;
+
+  public static final byte ADMIN_ROLE = 3;
+
+  private static Set readerOpsSet;
+
+  private static Set writerOpsSet;
+
+  static {
+
+    readerOpsSet = new HashSet();
+    for (int index = 0; index < DummyAuthorization.READER_OPS.length; index++) {
+      readerOpsSet.add(DummyAuthorization.READER_OPS[index]);
+    }
+    writerOpsSet = new HashSet();
+    for (int index = 0; index < DummyAuthorization.WRITER_OPS.length; index++) {
+      writerOpsSet.add(DummyAuthorization.WRITER_OPS[index]);
+    }
+  }
+
+  public DummyAuthzCredentialGenerator() {
+  }
+
+  protected Properties init() throws IllegalArgumentException {
+
+    if (!this.cGen.classCode().isDummy()) {
+      throw new IllegalArgumentException(
+          "DummyAuthorization module only works with DummyAuthenticator");
+    }
+    return null;
+  }
+
+  public ClassCode classCode() {
+    return ClassCode.DUMMY;
+  }
+
+  public String getAuthorizationCallback() {
+    return "templates.security.DummyAuthorization.create";
+  }
+
+  public static byte getRequiredRole(OperationCode[] opCodes) {
+
+    byte roleType = ADMIN_ROLE;
+    boolean requiresReader = true;
+    boolean requiresWriter = true;
+
+    for (int opNum = 0; opNum < opCodes.length; opNum++) {
+      if (requiresReader && !readerOpsSet.contains(opCodes[opNum])) {
+        requiresReader = false;
+      }
+      if (requiresWriter && !writerOpsSet.contains(opCodes[opNum])) {
+        requiresWriter = false;
+      }
+    }
+    if (requiresReader) {
+      roleType = READER_ROLE;
+    }
+    else if (requiresWriter) {
+      roleType = WRITER_ROLE;
+    }
+    return roleType;
+  }
+
+  private Principal getPrincipal(byte roleType, int index) {
+
+    String[] admins = new String[] { "root", "admin", "administrator" };
+    switch (roleType) {
+      case READER_ROLE:
+        return new UsernamePrincipal("reader" + index);
+      case WRITER_ROLE:
+        return new UsernamePrincipal("writer" + index);
+      default:
+        return new UsernamePrincipal(admins[index % admins.length]);
+    }
+  }
+
+  protected Principal getAllowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    byte roleType = getRequiredRole(opCodes);
+    return getPrincipal(roleType, index);
+  }
+
+  protected Principal getDisallowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    byte roleType = getRequiredRole(opCodes);
+    byte disallowedRoleType;
+    switch (roleType) {
+      case READER_ROLE:
+        disallowedRoleType = WRITER_ROLE;
+        break;
+      case WRITER_ROLE:
+        disallowedRoleType = READER_ROLE;
+        break;
+      default:
+        disallowedRoleType = READER_ROLE;
+        break;
+    }
+    return getPrincipal(disallowedRoleType, index);
+  }
+
+  protected int getNumPrincipalTries(OperationCode[] opCodes,
+      String[] regionNames) {
+    return 5;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/templates/security/DummyCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/DummyCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/DummyCredentialGenerator.java
new file mode 100644
index 0000000..af6c6eb
--- /dev/null
+++ b/gemfire-core/src/test/java/templates/security/DummyCredentialGenerator.java
@@ -0,0 +1,90 @@
+
+package templates.security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.Properties;
+
+public class DummyCredentialGenerator extends CredentialGenerator {
+
+  public DummyCredentialGenerator() {
+  }
+
+  protected Properties initialize() throws IllegalArgumentException {
+    return null;
+  }
+
+  public ClassCode classCode() {
+    return ClassCode.DUMMY;
+  }
+
+  public String getAuthInit() {
+    return "templates.security.UserPasswordAuthInit.create";
+  }
+
+  public String getAuthenticator() {
+    return "templates.security.DummyAuthenticator.create";
+  }
+
+  public Properties getValidCredentials(int index) {
+
+    String[] validGroups = new String[] { "admin", "user", "reader", "writer" };
+    String[] admins = new String[] { "root", "admin", "administrator" };
+
+    Properties props = new Properties();
+    int groupNum = (index % validGroups.length);
+    String userName;
+    if (groupNum == 0) {
+      userName = admins[index % admins.length];
+    }
+    else {
+      userName = validGroups[groupNum] + (index / validGroups.length);
+    }
+    props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
+    props.setProperty(UserPasswordAuthInit.PASSWORD, userName);
+    return props;
+  }
+
+  public Properties getValidCredentials(Principal principal) {
+
+    String userName = principal.getName();
+    if (DummyAuthenticator.testValidName(userName)) {
+      Properties props = new Properties();
+      props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
+      props.setProperty(UserPasswordAuthInit.PASSWORD, userName);
+      return props;
+    }
+    else {
+      throw new IllegalArgumentException("Dummy: [" + userName
+          + "] is not a valid user");
+    }
+  }
+
+  public Properties getInvalidCredentials(int index) {
+
+    Properties props = new Properties();
+    props.setProperty(UserPasswordAuthInit.USER_NAME, "invalid" + index);
+    props.setProperty(UserPasswordAuthInit.PASSWORD, "none");
+    return props;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/templates/security/LdapUserCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/LdapUserCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/LdapUserCredentialGenerator.java
new file mode 100644
index 0000000..816df63
--- /dev/null
+++ b/gemfire-core/src/test/java/templates/security/LdapUserCredentialGenerator.java
@@ -0,0 +1,158 @@
+
+package templates.security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.Properties;
+
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
+import com.gemstone.gemfire.util.test.TestUtil;
+
+import java.util.Random;
+
+public class LdapUserCredentialGenerator extends CredentialGenerator {
+
+  private static final String USER_PREFIX = "gemfire";
+
+  private static boolean enableServerAuthentication = false;
+
+  private boolean serverAuthEnabled = false;
+
+  private static final Random prng = new Random();
+
+  private static final String[] algos = new String[] { "", "DESede", "AES:128",
+      "Blowfish:128" };
+
+  public LdapUserCredentialGenerator() {
+    // Toggle server authentication enabled for each test
+    // This is done instead of running all the tests with both
+    // server auth enabled/disabled to reduce test run time.
+    enableServerAuthentication = !enableServerAuthentication;
+    serverAuthEnabled = enableServerAuthentication;
+  }
+
+  @Override
+  protected Properties initialize() throws IllegalArgumentException {
+
+    Properties extraProps = new Properties();
+    String ldapServer = System.getProperty("gf.ldap.server", "ldap");
+    String ldapBaseDN = System.getProperty("gf.ldap.basedn", "ou=ldapTesting,dc=pune,dc=gemstone,dc=com");
+    String ldapUseSSL = System.getProperty("gf.ldap.usessl");
+    extraProps.setProperty(LdapUserAuthenticator.LDAP_SERVER_NAME, ldapServer);
+    extraProps.setProperty(LdapUserAuthenticator.LDAP_BASEDN_NAME, ldapBaseDN);
+    if (ldapUseSSL != null && ldapUseSSL.length() > 0) {
+      extraProps.setProperty(LdapUserAuthenticator.LDAP_SSL_NAME, ldapUseSSL);
+    }
+    if (serverAuthEnabled) {
+      String keyStoreFile = TestUtil.getResourcePath(LdapUserCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/gemfire1.keystore");
+      extraProps.setProperty(HandShake.PRIVATE_KEY_FILE_PROP, keyStoreFile);
+      extraProps.setProperty(HandShake.PRIVATE_KEY_ALIAS_PROP, "gemfire1");
+      extraProps.setProperty(HandShake.PRIVATE_KEY_PASSWD_PROP, "gemfire");
+    }
+    return extraProps;
+  }
+
+  @Override
+  public ClassCode classCode() {
+    return ClassCode.LDAP;
+  }
+
+  @Override
+  public String getAuthInit() {
+    return "templates.security.UserPasswordAuthInit.create";
+  }
+
+  @Override
+  public String getAuthenticator() {
+    return "templates.security.LdapUserAuthenticator.create";
+  }
+
+  @Override
+  public Properties getValidCredentials(int index) {
+
+    Properties props = new Properties();
+    props.setProperty(UserPasswordAuthInit.USER_NAME, USER_PREFIX
+        + ((index % 10) + 1));
+    props.setProperty(UserPasswordAuthInit.PASSWORD, USER_PREFIX
+        + ((index % 10) + 1));
+    props.setProperty(DistributionConfig.SECURITY_CLIENT_DHALGO_NAME,
+        algos[prng.nextInt(algos.length)]);
+    if (serverAuthEnabled) {
+      String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/publickeyfile");
+      props.setProperty(HandShake.PUBLIC_KEY_FILE_PROP, keyStoreFile);
+      props.setProperty(HandShake.PUBLIC_KEY_PASSWD_PROP, "gemfire");
+    }
+    return props;
+  }
+
+  @Override
+  public Properties getValidCredentials(Principal principal) {
+
+    Properties props = null;
+    String userName = principal.getName();
+    if (userName != null && userName.startsWith(USER_PREFIX)) {
+      boolean isValid;
+      try {
+        int suffix = Integer.parseInt(userName.substring(USER_PREFIX.length()));
+        isValid = (suffix >= 1 && suffix <= 10);
+      }
+      catch (Exception ex) {
+        isValid = false;
+      }
+      if (isValid) {
+        props = new Properties();
+        props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
+        props.setProperty(UserPasswordAuthInit.PASSWORD, userName);
+      }
+    }
+    if (props == null) {
+      throw new IllegalArgumentException("LDAP: [" + userName
+          + "] not a valid user");
+    }
+    props.setProperty(DistributionConfig.SECURITY_CLIENT_DHALGO_NAME,
+        algos[prng.nextInt(algos.length)]);
+    if (serverAuthEnabled) {
+      String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/publickeyfile");
+      props.setProperty(HandShake.PUBLIC_KEY_FILE_PROP, keyStoreFile);
+      props.setProperty(HandShake.PUBLIC_KEY_PASSWD_PROP, "gemfire");
+    }
+    return props;
+  }
+
+  @Override
+  public Properties getInvalidCredentials(int index) {
+
+    Properties props = new Properties();
+    props.setProperty(UserPasswordAuthInit.USER_NAME, "invalid" + index);
+    props.setProperty(UserPasswordAuthInit.PASSWORD, "none");
+    props.setProperty(DistributionConfig.SECURITY_CLIENT_DHALGO_NAME,
+        algos[prng.nextInt(algos.length)]);
+    if (serverAuthEnabled) {
+      String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/publickeyfile");
+      props.setProperty(HandShake.PUBLIC_KEY_FILE_PROP, keyStoreFile);
+      props.setProperty(HandShake.PUBLIC_KEY_PASSWD_PROP, "gemfire");
+    }
+    return props;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/templates/security/PKCSCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/PKCSCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/PKCSCredentialGenerator.java
new file mode 100644
index 0000000..fa4bf04
--- /dev/null
+++ b/gemfire-core/src/test/java/templates/security/PKCSCredentialGenerator.java
@@ -0,0 +1,110 @@
+
+package templates.security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.security.Provider;
+import java.security.Security;
+import java.util.Properties;
+
+import com.gemstone.gemfire.util.test.TestUtil;
+
+/**
+ * @author kneeraj
+ * 
+ */
+public class PKCSCredentialGenerator extends CredentialGenerator {
+
+  public static String keyStoreDir = getKeyStoreDir();
+
+  public static boolean usesIBMJSSE;
+
+  // Checks if the current JVM uses only IBM JSSE providers.
+  private static boolean usesIBMProviders() {
+    Provider[] providers = Security.getProviders();
+    for (int index = 0; index < providers.length; ++index) {
+      if (!providers[index].getName().toLowerCase().startsWith("ibm")) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static String getKeyStoreDir() {
+    usesIBMJSSE = usesIBMProviders();
+    if (usesIBMJSSE) {
+      return "/lib/keys/ibm";
+    }
+    else {
+      return "/lib/keys";
+    }
+  }
+
+  public ClassCode classCode() {
+    return ClassCode.PKCS;
+  }
+
+  public String getAuthInit() {
+    return "templates.security.PKCSAuthInit.create";
+  }
+
+  public String getAuthenticator() {
+    return "templates.security.PKCSAuthenticator.create";
+  }
+
+  public Properties getInvalidCredentials(int index) {
+    Properties props = new Properties();
+    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + "/gemfire11.keystore");
+    props.setProperty(PKCSAuthInit.KEYSTORE_FILE_PATH, keyStoreFile);
+    props.setProperty(PKCSAuthInit.KEYSTORE_ALIAS, "gemfire11");
+    props.setProperty(PKCSAuthInit.KEYSTORE_PASSWORD, "gemfire");
+    return props;
+  }
+
+  public Properties getValidCredentials(int index) {
+    Properties props = new Properties();
+    int aliasnum = (index % 10) + 1;
+    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + "/gemfire" + aliasnum + ".keystore");
+    props.setProperty(PKCSAuthInit.KEYSTORE_FILE_PATH, keyStoreFile);
+    props.setProperty(PKCSAuthInit.KEYSTORE_ALIAS, "gemfire" + aliasnum);
+    props.setProperty(PKCSAuthInit.KEYSTORE_PASSWORD, "gemfire");
+    return props;
+  }
+
+  public Properties getValidCredentials(Principal principal) {
+    Properties props = new Properties();
+    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + principal.getName() + ".keystore");
+    props.setProperty(PKCSAuthInit.KEYSTORE_FILE_PATH, keyStoreFile);
+    props.setProperty(PKCSAuthInit.KEYSTORE_ALIAS, principal.getName());
+    props.setProperty(PKCSAuthInit.KEYSTORE_PASSWORD, "gemfire");
+    return props;
+  }
+
+  protected Properties initialize() throws IllegalArgumentException {
+    Properties props = new Properties();
+    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + "/publickeyfile");
+    props.setProperty(PKCSAuthenticator.PUBLIC_KEY_FILE, keyStoreFile);
+    props.setProperty(PKCSAuthenticator.PUBLIC_KEYSTORE_PASSWORD, "gemfire");
+    return props;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/templates/security/SSLCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/SSLCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/SSLCredentialGenerator.java
new file mode 100644
index 0000000..09367a2
--- /dev/null
+++ b/gemfire-core/src/test/java/templates/security/SSLCredentialGenerator.java
@@ -0,0 +1,116 @@
+
+package templates.security;
+
+/*
+ * 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.
+ */
+
+
+import java.io.File;
+import java.io.IOException;
+import java.security.Principal;
+import java.util.Properties;
+
+import com.gemstone.gemfire.security.AuthenticationFailedException;
+
+public class SSLCredentialGenerator extends CredentialGenerator {
+
+  private File findTrustedJKS() {
+    File ssldir = new File(System.getProperty("JTESTS") + "/ssl");
+    return new File(ssldir, "trusted.keystore");
+  }
+
+  private File findUntrustedJKS() {
+    File ssldir = new File(System.getProperty("JTESTS") + "/ssl");
+    return new File(ssldir, "untrusted.keystore");
+  }
+
+  private Properties getValidJavaSSLProperties() {
+    File jks = findTrustedJKS();
+    try {
+      Properties props = new Properties();
+      props.setProperty("javax.net.ssl.trustStore", jks.getCanonicalPath());
+      props.setProperty("javax.net.ssl.trustStorePassword", "password");
+      props.setProperty("javax.net.ssl.keyStore", jks.getCanonicalPath());
+      props.setProperty("javax.net.ssl.keyStorePassword", "password");
+      return props;
+    }
+    catch (IOException ex) {
+      throw new AuthenticationFailedException(
+          "SSL: Exception while opening the key store: " + ex);
+    }
+  }
+
+  private Properties getInvalidJavaSSLProperties() {
+    File jks = findUntrustedJKS();
+    try {
+      Properties props = new Properties();
+      props.setProperty("javax.net.ssl.trustStore", jks.getCanonicalPath());
+      props.setProperty("javax.net.ssl.trustStorePassword", "password");
+      props.setProperty("javax.net.ssl.keyStore", jks.getCanonicalPath());
+      props.setProperty("javax.net.ssl.keyStorePassword", "password");
+      return props;
+    }
+    catch (IOException ex) {
+      throw new AuthenticationFailedException(
+          "SSL: Exception while opening the key store: " + ex);
+    }
+  }
+
+  private Properties getSSLProperties() {
+    Properties props = new Properties();
+    props.setProperty("ssl-enabled", "true");
+    props.setProperty("ssl-require-authentication", "true");
+    props.setProperty("ssl-ciphers", "SSL_RSA_WITH_RC4_128_MD5");
+    props.setProperty("ssl-protocols", "TLSv1");
+    return props;
+  }
+
+  protected Properties initialize() throws IllegalArgumentException {
+    this.javaProps = getValidJavaSSLProperties();
+    return getSSLProperties();
+  }
+
+  public ClassCode classCode() {
+    return ClassCode.SSL;
+  }
+
+  public String getAuthInit() {
+    return null;
+  }
+
+  public String getAuthenticator() {
+    return null;
+  }
+
+  public Properties getValidCredentials(int index) {
+    this.javaProps = getValidJavaSSLProperties();
+    return getSSLProperties();
+  }
+
+  public Properties getValidCredentials(Principal principal) {
+    this.javaProps = getValidJavaSSLProperties();
+    return getSSLProperties();
+  }
+
+  public Properties getInvalidCredentials(int index) {
+    this.javaProps = getInvalidJavaSSLProperties();
+    return getSSLProperties();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/templates/security/UserPasswordWithExtraPropsAuthInit.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/UserPasswordWithExtraPropsAuthInit.java b/gemfire-core/src/test/java/templates/security/UserPasswordWithExtraPropsAuthInit.java
new file mode 100644
index 0000000..56dbf7d
--- /dev/null
+++ b/gemfire-core/src/test/java/templates/security/UserPasswordWithExtraPropsAuthInit.java
@@ -0,0 +1,76 @@
+
+package templates.security;
+
+/*
+ * 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.
+ */
+
+
+import java.util.Properties;
+import java.util.Iterator;
+
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.security.AuthInitialize;
+import com.gemstone.gemfire.security.AuthenticationFailedException;
+
+/**
+ * An {@link AuthInitialize} implementation that obtains the user name and
+ * password as the credentials from the given set of properties. If 
+ * keep-extra-props property exits, it will copy rest of the
+ * properties provided in getCredential props argument will also be 
+ * copied as new credentials.
+ * 
+ * @author Soubhik
+ * @since 5.5
+ */
+public class UserPasswordWithExtraPropsAuthInit extends UserPasswordAuthInit {
+
+  public static final String EXTRA_PROPS = "security-keep-extra-props";
+
+  public static final String SECURITY_PREFIX = "security-";
+  
+  public static AuthInitialize create() {
+    return new UserPasswordWithExtraPropsAuthInit();
+  }
+
+  public UserPasswordWithExtraPropsAuthInit() {
+    super();
+  }
+
+  public Properties getCredentials(Properties props, DistributedMember server,
+      boolean isPeer) throws AuthenticationFailedException {
+
+    Properties newProps = super.getCredentials(props, server, isPeer);
+    String extraProps = props.getProperty(EXTRA_PROPS);
+    if(extraProps != null) {
+    	for(Iterator it = props.keySet().iterator(); it.hasNext();) {
+    		String key = (String)it.next();
+    		if( key.startsWith(SECURITY_PREFIX) && 
+    		    key.equalsIgnoreCase(USER_NAME) == false &&
+    		    key.equalsIgnoreCase(PASSWORD) == false &&
+    		    key.equalsIgnoreCase(EXTRA_PROPS) == false) {
+    			newProps.setProperty(key, props.getProperty(key));
+    		}
+    	}
+    	this.securitylog.fine("got everything and now have: "
+          + newProps.keySet().toString());
+    }
+    return newProps;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/templates/security/XmlAuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/XmlAuthzCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/XmlAuthzCredentialGenerator.java
new file mode 100644
index 0000000..6b3f171
--- /dev/null
+++ b/gemfire-core/src/test/java/templates/security/XmlAuthzCredentialGenerator.java
@@ -0,0 +1,262 @@
+
+package templates.security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.util.test.TestUtil;
+
+public class XmlAuthzCredentialGenerator extends AuthzCredentialGenerator {
+
+  private static final String dummyXml = "authz-dummy.xml";
+
+  private static final String ldapXml = "authz-ldap.xml";
+
+  private static final String pkcsXml = "authz-pkcs.xml";
+
+  private static final String sslXml = "authz-ssl.xml";
+
+  private static final String[] QUERY_REGIONS = { "/Portfolios", "/Positions",
+      "/AuthRegion" };
+
+  public static OperationCode[] READER_OPS = { OperationCode.GET,
+      OperationCode.REGISTER_INTEREST, OperationCode.UNREGISTER_INTEREST,
+      OperationCode.KEY_SET, OperationCode.CONTAINS_KEY, OperationCode.EXECUTE_FUNCTION };
+
+  public static OperationCode[] WRITER_OPS = { OperationCode.PUT,
+      OperationCode.DESTROY, OperationCode.INVALIDATE, OperationCode.REGION_CLEAR };
+
+  public static OperationCode[] QUERY_OPS = { OperationCode.QUERY,
+      OperationCode.EXECUTE_CQ, OperationCode.STOP_CQ, OperationCode.CLOSE_CQ };
+
+  private static final byte READER_ROLE = 1;
+
+  private static final byte WRITER_ROLE = 2;
+
+  private static final byte QUERY_ROLE = 3;
+
+  private static final byte ADMIN_ROLE = 4;
+
+  private static Set readerOpsSet;
+
+  private static Set writerOpsSet;
+
+  private static Set queryOpsSet;
+
+  private static Set queryRegionSet;
+
+  static {
+
+    readerOpsSet = new HashSet();
+    for (int index = 0; index < READER_OPS.length; index++) {
+      readerOpsSet.add(READER_OPS[index]);
+    }
+    writerOpsSet = new HashSet();
+    for (int index = 0; index < WRITER_OPS.length; index++) {
+      writerOpsSet.add(WRITER_OPS[index]);
+    }
+    queryOpsSet = new HashSet();
+    for (int index = 0; index < QUERY_OPS.length; index++) {
+      queryOpsSet.add(QUERY_OPS[index]);
+    }
+    queryRegionSet = new HashSet();
+    for (int index = 0; index < QUERY_REGIONS.length; index++) {
+      queryRegionSet.add(QUERY_REGIONS[index]);
+    }
+  }
+
+  public XmlAuthzCredentialGenerator() {
+  }
+
+  protected Properties init() throws IllegalArgumentException {
+
+    Properties sysProps = new Properties();
+    String dirName = "/lib/";
+    if (this.cGen.classCode().isDummy()) {
+      String xmlFilename = TestUtil.getResourcePath(XmlAuthzCredentialGenerator.class, dirName + dummyXml);
+      sysProps.setProperty(XmlAuthorization.DOC_URI_PROP_NAME, xmlFilename);
+    }
+    else if (this.cGen.classCode().isLDAP()) {
+      String xmlFilename = TestUtil.getResourcePath(XmlAuthzCredentialGenerator.class, dirName + ldapXml);
+      sysProps.setProperty(XmlAuthorization.DOC_URI_PROP_NAME, xmlFilename);
+    }
+    // else if (this.cGen.classCode().isPKCS()) {
+    // sysProps
+    // .setProperty(XmlAuthorization.DOC_URI_PROP_NAME, dirName + pkcsXml);
+    // }
+    // else if (this.cGen.classCode().isSSL()) {
+    // sysProps
+    // .setProperty(XmlAuthorization.DOC_URI_PROP_NAME, dirName + sslXml);
+    // }
+    else {
+      throw new IllegalArgumentException(
+          "No XML defined for XmlAuthorization module to work with "
+              + this.cGen.getAuthenticator());
+    }
+    return sysProps;
+  }
+
+  public ClassCode classCode() {
+    return ClassCode.XML;
+  }
+
+  public String getAuthorizationCallback() {
+    return "templates.security.XmlAuthorization.create";
+  }
+
+  private Principal getDummyPrincipal(byte roleType, int index) {
+
+    String[] admins = new String[] { "root", "admin", "administrator" };
+    int numReaders = 3;
+    int numWriters = 3;
+
+    switch (roleType) {
+      case READER_ROLE:
+        return new UsernamePrincipal("reader" + (index % numReaders));
+      case WRITER_ROLE:
+        return new UsernamePrincipal("writer" + (index % numWriters));
+      case QUERY_ROLE:
+        return new UsernamePrincipal("reader" + ((index % 2) + 3));
+      default:
+        return new UsernamePrincipal(admins[index % admins.length]);
+    }
+  }
+
+  private Principal getLdapPrincipal(byte roleType, int index) {
+
+    final String userPrefix = "gemfire";
+    final int[] readerIndices = { 3, 4, 5 };
+    final int[] writerIndices = { 6, 7, 8 };
+    final int[] queryIndices = { 9, 10 };
+    final int[] adminIndices = { 1, 2 };
+
+    switch (roleType) {
+      case READER_ROLE:
+        int readerIndex = readerIndices[index % readerIndices.length];
+        return new UsernamePrincipal(userPrefix + readerIndex);
+      case WRITER_ROLE:
+        int writerIndex = writerIndices[index % writerIndices.length];
+        return new UsernamePrincipal(userPrefix + writerIndex);
+      case QUERY_ROLE:
+        int queryIndex = queryIndices[index % queryIndices.length];
+        return new UsernamePrincipal(userPrefix + queryIndex);
+      default:
+        int adminIndex = adminIndices[index % adminIndices.length];
+        return new UsernamePrincipal(userPrefix + adminIndex);
+    }
+  }
+
+  private byte getRequiredRole(OperationCode[] opCodes, String[] regionNames) {
+
+    byte roleType = ADMIN_ROLE;
+    boolean requiresReader = true;
+    boolean requiresWriter = true;
+    boolean requiresQuery = true;
+
+    for (int opNum = 0; opNum < opCodes.length; opNum++) {
+      OperationCode opCode = opCodes[opNum];
+      if (requiresReader && !readerOpsSet.contains(opCode)) {
+        requiresReader = false;
+      }
+      if (requiresWriter && !writerOpsSet.contains(opCode)) {
+        requiresWriter = false;
+      }
+      if (requiresQuery && !queryOpsSet.contains(opCode)) {
+        requiresQuery = false;
+      }
+    }
+    if (requiresReader) {
+      roleType = READER_ROLE;
+    }
+    else if (requiresWriter) {
+      roleType = WRITER_ROLE;
+    }
+    else if (requiresQuery) {
+      if (regionNames != null && regionNames.length > 0) {
+        for (int index = 0; index < regionNames.length; index++) {
+          String regionName = XmlAuthorization
+              .normalizeRegionName(regionNames[index]);
+          if (requiresQuery && !queryRegionSet.contains(regionName)) {
+            requiresQuery = false;
+            break;
+          }
+        }
+        if (requiresQuery) {
+          roleType = QUERY_ROLE;
+        }
+      }
+    }
+    return roleType;
+  }
+
+  protected Principal getAllowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    if (this.cGen.classCode().isDummy()) {
+      byte roleType = getRequiredRole(opCodes, regionNames);
+      return getDummyPrincipal(roleType, index);
+    }
+    else if (this.cGen.classCode().isLDAP()) {
+      byte roleType = getRequiredRole(opCodes, regionNames);
+      return getLdapPrincipal(roleType, index);
+    }
+    return null;
+  }
+
+  protected Principal getDisallowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    byte roleType = getRequiredRole(opCodes, regionNames);
+    byte disallowedRoleType = READER_ROLE;
+    switch (roleType) {
+      case READER_ROLE:
+        disallowedRoleType = WRITER_ROLE;
+        break;
+      case WRITER_ROLE:
+        disallowedRoleType = READER_ROLE;
+        break;
+      case QUERY_ROLE:
+        disallowedRoleType = READER_ROLE;
+        break;
+      case ADMIN_ROLE:
+        disallowedRoleType = READER_ROLE;
+        break;
+    }
+    if (this.cGen.classCode().isDummy()) {
+      return getDummyPrincipal(disallowedRoleType, index);
+    }
+    else if (this.cGen.classCode().isLDAP()) {
+      return getLdapPrincipal(disallowedRoleType, index);
+    }
+    return null;
+  }
+
+  protected int getNumPrincipalTries(OperationCode[] opCodes,
+      String[] regionNames) {
+    return 5;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/authz-dummy.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/authz-dummy.xml b/gemfire-core/src/test/resources/lib/authz-dummy.xml
new file mode 100644
index 0000000..7f73808
--- /dev/null
+++ b/gemfire-core/src/test/resources/lib/authz-dummy.xml
@@ -0,0 +1,126 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<!DOCTYPE acl PUBLIC "-//GemStone Systems, Inc.//GemFire XML Authorization 1.0//EN"
+              "authz6_0.dtd" >
+<acl>
+
+  <role name="reader">
+    <user>reader0</user>
+    <user>reader1</user>
+    <user>reader2</user>
+    <user>root</user>
+    <user>admin</user>
+    <user>administrator</user>
+  </role>
+
+  <role name="writer">
+    <user>writer0</user>
+    <user>writer1</user>
+    <user>writer2</user>
+    <user>root</user>
+    <user>admin</user>
+    <user>administrator</user>
+  </role>
+
+  <role name="cacheOps">
+    <user>root</user>
+    <user>admin</user>
+    <user>administrator</user>
+  </role>
+
+  <role name="queryRegions">
+    <user>reader3</user>
+    <user>reader4</user>
+  </role>
+
+  <role name="registerInterest">
+    <user>reader5</user>
+    <user>reader6</user>
+  </role>
+
+  <role name="unregisterInterest">
+    <user>reader5</user>
+    <user>reader7</user>
+  </role>
+  
+  <role name="onRegionFunctionExecutor">
+    <user>reader8</user>
+  </role>
+  
+  <role name="onServerFunctionExecutor">
+    <user>reader9</user>
+  </role>
+
+  <permission role="cacheOps">
+    <operation>QUERY</operation>
+    <operation>EXECUTE_CQ</operation>
+    <operation>STOP_CQ</operation>
+    <operation>CLOSE_CQ</operation>
+    <operation>REGION_CREATE</operation>
+    <operation>REGION_DESTROY</operation>
+  </permission>
+
+  <permission role="reader">
+    <operation>GET</operation>
+    <operation>REGISTER_INTEREST</operation>
+    <operation>UNREGISTER_INTEREST</operation>
+    <operation>KEY_SET</operation>
+    <operation>CONTAINS_KEY</operation>
+    <operation>EXECUTE_FUNCTION</operation>
+  </permission>
+
+  <permission role="writer">
+    <operation>PUT</operation>
+    <operation>PUTALL</operation>
+    <operation>DESTROY</operation>
+    <operation>INVALIDATE</operation>
+    <operation>REGION_CLEAR</operation>
+  </permission>
+
+  <permission role="queryRegions" regions="//Portfolios,/Positions/,AuthRegion">
+    <operation>QUERY</operation>
+    <operation>EXECUTE_CQ</operation>
+    <operation>STOP_CQ</operation>
+    <operation>CLOSE_CQ</operation>
+  </permission>
+  
+  <permission role="onRegionFunctionExecutor" regions="secureRegion,Positions">
+    <operation>PUT</operation>
+    <operation functionIds="SecureFunction,OptimizationFunction" optimizeForWrite="false" keySet="KEY-0,KEY-1">EXECUTE_FUNCTION</operation>
+  </permission>
+  
+  <permission role="onServerFunctionExecutor" >
+    <operation>PUT</operation>
+    <operation functionIds="SecureFunction,OptimizationFunction">EXECUTE_FUNCTION</operation>
+  </permission>
+
+  <permission role="registerInterest">
+    <operation>REGISTER_INTEREST</operation>
+    <operation>GET</operation>
+  </permission>
+
+  <permission role="unregisterInterest">
+    <operation>UNREGISTER_INTEREST</operation>
+    <operation>GET</operation>
+  </permission>
+
+</acl>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/authz-ldap.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/authz-ldap.xml b/gemfire-core/src/test/resources/lib/authz-ldap.xml
new file mode 100644
index 0000000..e63c23b
--- /dev/null
+++ b/gemfire-core/src/test/resources/lib/authz-ldap.xml
@@ -0,0 +1,85 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<!DOCTYPE acl PUBLIC "-//GemStone Systems, Inc.//GemFire XML Authorization 1.0//EN"
+              "authz5_5.dtd" >
+<acl>
+
+  <role name="reader">
+    <user>gemfire1</user>
+    <user>gemfire2</user>
+    <user>gemfire3</user>
+    <user>gemfire4</user>
+    <user>gemfire5</user>
+  </role>
+
+  <role name="writer">
+    <user>gemfire1</user>
+    <user>gemfire2</user>
+    <user>gemfire6</user>
+    <user>gemfire7</user>
+    <user>gemfire8</user>
+  </role>
+
+  <role name="cacheOps">
+    <user>gemfire1</user>
+    <user>gemfire2</user>
+  </role>
+
+  <role name="queryRegions">
+    <user>gemfire9</user>
+    <user>gemfire10</user>
+  </role>
+
+  <permission role="cacheOps">
+    <operation>QUERY</operation>
+    <operation>EXECUTE_CQ</operation>
+    <operation>STOP_CQ</operation>
+    <operation>CLOSE_CQ</operation>
+    <operation>REGION_CREATE</operation>
+    <operation>REGION_DESTROY</operation>
+  </permission>
+
+  <permission role="reader">
+    <operation>GET</operation>
+    <operation>REGISTER_INTEREST</operation>
+    <operation>UNREGISTER_INTEREST</operation>
+    <operation>KEY_SET</operation>
+    <operation>CONTAINS_KEY</operation>
+    <operation>EXECUTE_FUNCTION</operation>
+  </permission>
+
+  <permission role="writer">
+    <operation>PUT</operation>
+    <operation>PUTALL</operation>
+    <operation>DESTROY</operation>
+    <operation>INVALIDATE</operation>
+    <operation>REGION_CLEAR</operation>
+  </permission>
+
+  <permission role="queryRegions" regions="Portfolios,/Positions//,/AuthRegion">
+    <operation>QUERY</operation>
+    <operation>EXECUTE_CQ</operation>
+    <operation>STOP_CQ</operation>
+    <operation>CLOSE_CQ</operation>
+  </permission>
+
+</acl>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/authz-multiUser-dummy.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/authz-multiUser-dummy.xml b/gemfire-core/src/test/resources/lib/authz-multiUser-dummy.xml
new file mode 100644
index 0000000..0f3bbab
--- /dev/null
+++ b/gemfire-core/src/test/resources/lib/authz-multiUser-dummy.xml
@@ -0,0 +1,106 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<!DOCTYPE acl PUBLIC "-//GemStone Systems, Inc.//GemFire XML Authorization 1.0//EN"
+              "authz6_0.dtd" >
+<acl>
+
+  <role name="reader">
+    <user>user1</user>
+    <user>user2</user>
+    <user>root</user>
+    <user>admin</user>
+    <user>administrator</user>
+  </role>
+
+  <role name="writer">
+    <user>user3</user>
+    <user>user4</user>
+    <user>root</user>
+    <user>admin</user>
+    <user>administrator</user>
+  </role>
+  
+  <role name="cacheOps">
+    <user>user1</user>
+    <user>user2</user>
+    <user>root</user>
+    <user>admin</user>
+    <user>administrator</user>
+  </role>
+
+  <role name="queryRegions">
+    <user>user5</user>
+    <user>user6</user>
+  </role>
+
+  <role name="registerInterest">
+    <user>user7</user>
+    <user>user8</user>
+  </role>
+
+  <role name="unregisterInterest">
+    <user>user5</user>
+    <user>user7</user>
+  </role>
+  
+  <permission role="cacheOps">
+    <operation>QUERY</operation>
+    <operation>EXECUTE_CQ</operation>
+    <operation>STOP_CQ</operation>
+    <operation>CLOSE_CQ</operation>
+  </permission>
+
+  <permission role="reader">
+    <operation>GET</operation>
+    <operation>REGISTER_INTEREST</operation>
+    <operation>UNREGISTER_INTEREST</operation>
+    <operation>KEY_SET</operation>
+    <operation>CONTAINS_KEY</operation>
+    <operation>EXECUTE_FUNCTION</operation>
+  </permission>
+
+  <permission role="writer">
+    <operation>PUT</operation>
+    <operation>PUTALL</operation>
+    <operation>DESTROY</operation>
+    <operation>INVALIDATE</operation>
+    <operation>REGION_CLEAR</operation>
+  </permission>
+
+  <permission role="queryRegions" regions="//Portfolios,/Positions/,AuthRegion">
+    <operation>QUERY</operation>
+    <operation>EXECUTE_CQ</operation>
+    <operation>STOP_CQ</operation>
+    <operation>CLOSE_CQ</operation>
+  </permission>
+  
+  <permission role="registerInterest">
+    <operation>REGISTER_INTEREST</operation>
+    <operation>GET</operation>
+  </permission>
+
+  <permission role="unregisterInterest">
+    <operation>UNREGISTER_INTEREST</operation>
+    <operation>GET</operation>
+  </permission>
+
+</acl>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/authz-multiUser-ldap.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/authz-multiUser-ldap.xml b/gemfire-core/src/test/resources/lib/authz-multiUser-ldap.xml
new file mode 100644
index 0000000..a8e5392
--- /dev/null
+++ b/gemfire-core/src/test/resources/lib/authz-multiUser-ldap.xml
@@ -0,0 +1,83 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<!DOCTYPE acl PUBLIC "-//GemStone Systems, Inc.//GemFire XML Authorization 1.0//EN"
+              "authz5_5.dtd" >
+<acl>
+
+  <role name="reader">
+    <user>gemfire1</user>
+    <user>gemfire2</user>
+    <user>gemfire3</user>
+    <user>gemfire4</user>
+    <user>gemfire5</user>
+  </role>
+
+  <role name="writer">
+    <user>gemfire1</user>
+    <user>gemfire2</user>
+    <user>gemfire6</user>
+    <user>gemfire7</user>
+    <user>gemfire8</user>
+  </role>
+
+  <role name="cacheOps">
+    <user>gemfire1</user>
+    <user>gemfire2</user>
+  </role>
+
+  <role name="queryRegions">
+    <user>gemfire9</user>
+    <user>gemfire10</user>
+  </role>
+
+  <permission role="cacheOps">
+    <operation>QUERY</operation>
+    <operation>EXECUTE_CQ</operation>
+    <operation>STOP_CQ</operation>
+    <operation>CLOSE_CQ</operation>
+  </permission>
+
+  <permission role="reader">
+    <operation>GET</operation>
+    <operation>REGISTER_INTEREST</operation>
+    <operation>UNREGISTER_INTEREST</operation>
+    <operation>KEY_SET</operation>
+    <operation>CONTAINS_KEY</operation>
+    <operation>EXECUTE_FUNCTION</operation>
+  </permission>
+
+  <permission role="writer">
+    <operation>PUT</operation>
+    <operation>PUTALL</operation>
+    <operation>DESTROY</operation>
+    <operation>INVALIDATE</operation>
+    <operation>REGION_CLEAR</operation>
+  </permission>
+
+  <permission role="queryRegions" regions="Portfolios,/Positions//,/AuthRegion">
+    <operation>QUERY</operation>
+    <operation>EXECUTE_CQ</operation>
+    <operation>STOP_CQ</operation>
+    <operation>CLOSE_CQ</operation>
+  </permission>
+
+</acl>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire1.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire1.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire1.keystore
new file mode 100644
index 0000000..15270bb
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire1.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire10.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire10.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire10.keystore
new file mode 100644
index 0000000..bb6f827
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire10.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire11.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire11.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire11.keystore
new file mode 100644
index 0000000..6839c74
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire11.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire2.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire2.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire2.keystore
new file mode 100644
index 0000000..fcb7ab8
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire2.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire3.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire3.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire3.keystore
new file mode 100644
index 0000000..19afc4b
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire3.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire4.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire4.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire4.keystore
new file mode 100644
index 0000000..c65916a
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire4.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire5.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire5.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire5.keystore
new file mode 100644
index 0000000..d738cca
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire5.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire6.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire6.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire6.keystore
new file mode 100644
index 0000000..1fea2d3
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire6.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire7.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire7.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire7.keystore
new file mode 100644
index 0000000..7a3187c
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire7.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire8.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire8.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire8.keystore
new file mode 100644
index 0000000..a3bb886
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire8.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/gemfire9.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/gemfire9.keystore b/gemfire-core/src/test/resources/lib/keys/gemfire9.keystore
new file mode 100644
index 0000000..674b4e6
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/gemfire9.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire1.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire1.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire1.keystore
new file mode 100644
index 0000000..4f9120c
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire1.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire10.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire10.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire10.keystore
new file mode 100644
index 0000000..0bd97d7
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire10.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire11.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire11.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire11.keystore
new file mode 100644
index 0000000..62ae3c7
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire11.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire2.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire2.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire2.keystore
new file mode 100644
index 0000000..c65bc81
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire2.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire3.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire3.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire3.keystore
new file mode 100644
index 0000000..b0796e0
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire3.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire4.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire4.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire4.keystore
new file mode 100644
index 0000000..9c94018
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire4.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire5.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire5.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire5.keystore
new file mode 100644
index 0000000..33f6937
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire5.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire6.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire6.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire6.keystore
new file mode 100644
index 0000000..568f674
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire6.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire7.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire7.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire7.keystore
new file mode 100644
index 0000000..80e2d80
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire7.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire8.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire8.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire8.keystore
new file mode 100644
index 0000000..a15def5
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire8.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/gemfire9.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/gemfire9.keystore b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire9.keystore
new file mode 100644
index 0000000..72087f3
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/gemfire9.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/ibm/publickeyfile
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/ibm/publickeyfile b/gemfire-core/src/test/resources/lib/keys/ibm/publickeyfile
new file mode 100644
index 0000000..1b13872
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/ibm/publickeyfile differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/lib/keys/publickeyfile
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/lib/keys/publickeyfile b/gemfire-core/src/test/resources/lib/keys/publickeyfile
new file mode 100644
index 0000000..9c2daa3
Binary files /dev/null and b/gemfire-core/src/test/resources/lib/keys/publickeyfile differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/resources/ssl/untrusted.keystore
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/ssl/untrusted.keystore b/gemfire-core/src/test/resources/ssl/untrusted.keystore
new file mode 100755
index 0000000..aa73eeb
Binary files /dev/null and b/gemfire-core/src/test/resources/ssl/untrusted.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 2068b88..cb6bdfb 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -40,7 +40,7 @@ rat {
     '**/.classpath',
     '**/.settings/**',
     '**/build-eclipse/**',
-    '*.iml',
+    '**/*.iml',
     '.idea/**',
     '**/tags',
 


[18/33] incubator-geode git commit: Set TCP_NODELAY flag to help avoid delay in membership final check

Posted by je...@apache.org.
Set TCP_NODELAY flag to help avoid delay in membership final check


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

Branch: refs/heads/feature/GEODE-14
Commit: 625ea36a18cb2587d5b98ed673286d8d0d351de6
Parents: 4630e2e
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Mon Dec 21 16:24:24 2015 -0800
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Mon Dec 21 16:37:53 2015 -0800

----------------------------------------------------------------------
 .../distributed/internal/membership/gms/fd/GMSHealthMonitor.java   | 2 ++
 .../gemstone/gemfire/distributed/internal/tcpserver/TcpClient.java | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/625ea36a/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 33c7e76..005b0ed 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
@@ -274,6 +274,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
 
     public void run() {
       try {
+        socket.setTcpNoDelay(true);
         DataInputStream in = new DataInputStream(socket.getInputStream());
         OutputStream out = socket.getOutputStream();
         @SuppressWarnings("unused")
@@ -473,6 +474,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
       logger.debug("Checking member {} with TCP socket connection {}:{}.", suspectMember, suspectMember.getInetAddress(), port);
       clientSocket = SocketCreator.getDefaultInstance().connect(suspectMember.getInetAddress(), port,
           (int)memberTimeout, new ConnectTimeoutTask(services.getTimer(), memberTimeout), false, -1, false);
+      clientSocket.setTcpNoDelay(true);
       return doTCPCheckMember(suspectMember, clientSocket);
     }
     catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/625ea36a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpClient.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpClient.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpClient.java
index 8b9347e..47f50b3 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpClient.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpClient.java
@@ -110,7 +110,7 @@ public class TcpClient {
     long giveupTime = System.currentTimeMillis() + timeout;
     
     // Get the GemFire version of the TcpServer first, before sending any other request.
-    short serverVersion = getServerVersion(ipAddr, timeout/2).shortValue();
+    short serverVersion = getServerVersion(ipAddr, timeout).shortValue();
 
     if (serverVersion > Version.CURRENT_ORDINAL) {
       serverVersion = Version.CURRENT_ORDINAL;


[11/33] incubator-geode git commit: GEM-164: move the security tests in gemfire-test module inside the com.gemstone.gemfire.security packages to the open side.

Posted by je...@apache.org.
GEM-164: move the security tests in gemfire-test module inside the com.gemstone.gemfire.security packages to the open side.

GEM-164: remove the pivotal license header

GEM-164: add ASF license headers and have RAT ignore all subprojects' IDE files when checking license

GEM-164: remove the duplicate license


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

Branch: refs/heads/feature/GEODE-14
Commit: a622d6ec7f8228443bb99249e08766bd74337c10
Parents: 51ce64d
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Thu Dec 10 15:14:19 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Dec 21 08:51:59 2015 -0800

----------------------------------------------------------------------
 .../security/ClientAuthenticationDUnitTest.java |  969 +++++++++
 .../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 ++++++++++++++++++
 .../security/AuthzCredentialGenerator.java      |  465 +++++
 .../templates/security/CredentialGenerator.java |  343 ++++
 .../security/DummyAuthzCredentialGenerator.java |  142 ++
 .../security/DummyCredentialGenerator.java      |   90 +
 .../security/LdapUserCredentialGenerator.java   |  158 ++
 .../security/PKCSCredentialGenerator.java       |  110 +
 .../security/SSLCredentialGenerator.java        |  116 ++
 .../UserPasswordWithExtraPropsAuthInit.java     |   76 +
 .../security/XmlAuthzCredentialGenerator.java   |  262 +++
 .../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
 .../src/test/resources/ssl/untrusted.keystore   |  Bin 0 -> 1181 bytes
 gradle/rat.gradle                               |    2 +-
 48 files changed, 9309 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
new file mode 100644
index 0000000..2fdbc05
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
@@ -0,0 +1,969 @@
+package com.gemstone.gemfire.security;
+
+/*
+ * 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.
+ */
+
+
+import java.io.IOException;
+import java.util.Properties;
+
+import javax.net.ssl.SSLException;
+import javax.net.ssl.SSLHandshakeException;
+
+import templates.security.CredentialGenerator;
+import templates.security.CredentialGenerator.ClassCode;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePort;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.VM;
+import templates.security.DummyCredentialGenerator;
+
+/**
+ * Test for authentication from client to server. This tests for both valid and
+ * invalid credentials/modules. It also checks for authentication
+ * success/failure in case of failover and for the notification channel.
+ * 
+ * @author sumedh
+ * @since 5.5
+ */
+public class ClientAuthenticationDUnitTest extends DistributedTestCase {
+
+  /** constructor */
+  public ClientAuthenticationDUnitTest(String name) {
+    super(name);
+  }
+
+  private VM server1 = null;
+
+  private VM server2 = null;
+
+  private VM client1 = null;
+
+  private VM client2 = null;
+
+  private static final String[] serverExpectedExceptions = {
+      AuthenticationRequiredException.class.getName(),
+      AuthenticationFailedException.class.getName(),
+      GemFireSecurityException.class.getName(),
+      ClassNotFoundException.class.getName(), IOException.class.getName(),
+      SSLException.class.getName(), SSLHandshakeException.class.getName() };
+
+  private static final String[] clientExpectedExceptions = {
+      AuthenticationRequiredException.class.getName(),
+      AuthenticationFailedException.class.getName(),
+      SSLHandshakeException.class.getName() };
+
+  @Override
+  public void setUp() throws Exception {
+
+    super.setUp();
+    final Host host = Host.getHost(0);
+    server1 = host.getVM(0);
+    server2 = host.getVM(1);
+    client1 = host.getVM(2);
+    client2 = host.getVM(3);
+    
+    addExpectedException("Connection refused: connect");
+
+    server1.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { serverExpectedExceptions });
+    server2.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { serverExpectedExceptions });
+    client1.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { clientExpectedExceptions });
+    client2.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { clientExpectedExceptions });
+  }
+
+  // Region: Utility and static functions invoked by the tests
+
+  public static Integer createCacheServer(Object dsPort, Object locatorString,
+      Object authenticator, Object extraProps, Object javaProps) {
+
+    Properties authProps;
+    if (extraProps == null) {
+      authProps = new Properties();
+    }
+    else {
+      authProps = (Properties)extraProps;
+    }
+    if (authenticator != null) {
+      authProps.setProperty(
+          DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, authenticator
+              .toString());
+    }
+    return SecurityTestUtil.createCacheServer(authProps, javaProps,
+        (Integer)dsPort, (String)locatorString, null, new Integer(
+            SecurityTestUtil.NO_EXCEPTION));
+  }
+
+  public static void createCacheServer(Object dsPort, Object locatorString,
+      Integer serverPort, Object authenticator, Object extraProps,
+      Object javaProps) {
+
+    Properties authProps;
+    if (extraProps == null) {
+      authProps = new Properties();
+    }
+    else {
+      authProps = (Properties)extraProps;
+    }
+    if (authenticator != null) {
+      authProps.setProperty(
+          DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, authenticator
+              .toString());
+    }
+    SecurityTestUtil.createCacheServer(authProps, javaProps, (Integer)dsPort,
+        (String)locatorString, serverPort, new Integer(
+            SecurityTestUtil.NO_EXCEPTION));
+  }
+
+  private static void createCacheClient(Object authInit, Properties authProps,
+      Properties javaProps, Integer[] ports, Object numConnections,
+      Boolean multiUserMode, Boolean subscriptionEnabled, Integer expectedResult) {
+
+    String authInitStr = (authInit == null ? null : authInit.toString());
+    SecurityTestUtil.createCacheClient(authInitStr, authProps, javaProps,
+        ports, (Integer)numConnections, Boolean.FALSE,
+        multiUserMode.toString(), subscriptionEnabled, expectedResult);
+  }
+
+  public static void createCacheClient(Object authInit, Object authProps,
+      Object javaProps, Integer[] ports, Object numConnections,
+      Boolean multiUserMode, Integer expectedResult) {
+
+    createCacheClient(authInit, (Properties)authProps, (Properties)javaProps,
+        ports, numConnections, multiUserMode, Boolean.TRUE, expectedResult);
+  }
+
+  public static void createCacheClient(Object authInit, Object authProps,
+      Object javaProps, Integer port1, Object numConnections,
+      Integer expectedResult) {
+
+    createCacheClient(authInit, (Properties)authProps, (Properties)javaProps,
+        new Integer[] { port1 }, numConnections, Boolean.FALSE, Boolean.TRUE,
+        expectedResult);
+  }
+
+  public static void createCacheClient(Object authInit, Object authProps,
+      Object javaProps, Integer port1, Integer port2, Object numConnections,
+      Integer expectedResult) {
+    createCacheClient(authInit, authProps, javaProps, port1, port2,
+        numConnections, Boolean.FALSE, expectedResult);
+  }
+
+  public static void createCacheClient(Object authInit, Object authProps,
+      Object javaProps, Integer port1, Integer port2, Object numConnections,
+      Boolean multiUserMode, Integer expectedResult) {
+
+    createCacheClient(authInit, authProps, javaProps,
+        port1, port2, numConnections, multiUserMode, Boolean.TRUE,
+        expectedResult);
+  }
+
+  public static void createCacheClient(Object authInit, Object authProps,
+      Object javaProps, Integer port1, Integer port2, Object numConnections,
+      Boolean multiUserMode, Boolean subscriptionEnabled,
+      Integer expectedResult) {
+
+    createCacheClient(authInit, (Properties)authProps, (Properties)javaProps,
+        new Integer[] { port1, port2 }, numConnections, multiUserMode,
+        subscriptionEnabled, expectedResult);
+  }
+
+  public static void registerAllInterest() {
+
+    Region region = SecurityTestUtil.getCache().getRegion(
+        SecurityTestUtil.regionName);
+    assertNotNull(region);
+    region.registerInterestRegex(".*");
+  }
+
+  // End Region: Utility and static functions invoked by the tests
+
+  // Region: Tests
+
+  public void testValidCredentials() {
+    itestValidCredentials(Boolean.FALSE);
+  }
+
+  public void itestValidCredentials(Boolean multiUser) {
+      CredentialGenerator gen = new DummyCredentialGenerator();
+      Properties extraProps = gen.getSystemProperties();
+      Properties javaProps = gen.getJavaProperties();
+      String authenticator = gen.getAuthenticator();
+      String authInit = gen.getAuthInit();
+
+      getLogWriter().info(
+          "testValidCredentials: Using scheme: " + gen.classCode());
+      getLogWriter().info(
+          "testValidCredentials: Using authenticator: " + authenticator);
+      getLogWriter().info("testValidCredentials: Using authinit: " + authInit);
+
+      // Start the servers
+      Integer locPort1 = SecurityTestUtil.getLocatorPort();
+      Integer locPort2 = SecurityTestUtil.getLocatorPort();
+      String locString = SecurityTestUtil.getLocatorString();
+      Integer port1 = (Integer)server1.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort1, locString, authenticator, extraProps,
+              javaProps });
+      Integer port2 = (Integer)server2.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort2, locString, authenticator, extraProps,
+              javaProps });
+
+      // Start the clients with valid credentials
+      Properties credentials1 = gen.getValidCredentials(1);
+      Properties javaProps1 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testValidCredentials: For first client credentials: " + credentials1
+              + " : " + javaProps1);
+      Properties credentials2 = gen.getValidCredentials(2);
+      Properties javaProps2 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testValidCredentials: For second client credentials: "
+              + credentials2 + " : " + javaProps2);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials1, javaProps1, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials2, javaProps2, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts",
+          new Object[] { new Integer(2) });
+
+      // Verify that the puts succeeded
+      client2.invoke(SecurityTestUtil.class, "doGets",
+          new Object[] { new Integer(2) });
+      
+      if (multiUser) {
+        client1.invoke(SecurityTestUtil.class, "doProxyCacheClose");
+        client2.invoke(SecurityTestUtil.class, "doProxyCacheClose");
+        client1.invoke(SecurityTestUtil.class, "doSimplePut",
+            new Object[] {"CacheClosedException"});
+        client2.invoke(SecurityTestUtil.class, "doSimpleGet",
+            new Object[] {"CacheClosedException"});
+      }
+  }
+
+  public void testNoCredentials() {
+    itestNoCredentials(Boolean.FALSE);
+  }
+
+  public void itestNoCredentials(Boolean multiUser) {
+      CredentialGenerator gen = new DummyCredentialGenerator();
+      Properties extraProps = gen.getSystemProperties();
+      Properties javaProps = gen.getJavaProperties();
+      String authenticator = gen.getAuthenticator();
+      String authInit = gen.getAuthInit();
+
+      getLogWriter()
+          .info("testNoCredentials: Using scheme: " + gen.classCode());
+      getLogWriter().info(
+          "testNoCredentials: Using authenticator: " + authenticator);
+      getLogWriter().info("testNoCredentials: Using authinit: " + authInit);
+
+      // Start the servers
+      Integer locPort1 = SecurityTestUtil.getLocatorPort();
+      Integer locPort2 = SecurityTestUtil.getLocatorPort();
+      String locString = SecurityTestUtil.getLocatorString();
+      Integer port1 = ((Integer)server1.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort1, locString, authenticator, extraProps,
+              javaProps }));
+      Integer port2 = ((Integer)server2.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort2, locString, authenticator, extraProps,
+              javaProps }));
+
+      // Start first client with valid credentials
+      Properties credentials1 = gen.getValidCredentials(1);
+      Properties javaProps1 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testNoCredentials: For first client credentials: " + credentials1
+              + " : " + javaProps1);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials1, javaProps1, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts",
+          new Object[] { new Integer(2) });
+
+      // Trying to create the region on client2 
+      if (gen.classCode().equals(ClassCode.SSL)) {
+        // For SSL the exception may not come since the server can close socket
+        // before handshake message is sent from client. However exception
+        // should come in any region operations.
+        client2
+            .invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+                new Object[] { null, null, null, port1, port2, null, multiUser,
+                    new Integer(SecurityTestUtil.NO_EXCEPTION) });
+        client2.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+            new Integer(2), new Integer(SecurityTestUtil.OTHER_EXCEPTION) });
+      }
+      else {
+        client2.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { null, null, null, port1, port2,
+                null, multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+      }
+  }
+
+  public void testInvalidCredentials() {
+    itestInvalidCredentials(Boolean.FALSE);
+  }
+
+  public void itestInvalidCredentials(Boolean multiUser) {
+
+
+      CredentialGenerator gen = new DummyCredentialGenerator();
+      Properties extraProps = gen.getSystemProperties();
+      Properties javaProps = gen.getJavaProperties();
+      String authenticator = gen.getAuthenticator();
+      String authInit = gen.getAuthInit();
+
+      getLogWriter().info(
+          "testInvalidCredentials: Using scheme: " + gen.classCode());
+      getLogWriter().info(
+          "testInvalidCredentials: Using authenticator: " + authenticator);
+      getLogWriter()
+          .info("testInvalidCredentials: Using authinit: " + authInit);
+
+      // Start the servers
+      Integer locPort1 = SecurityTestUtil.getLocatorPort();
+      Integer locPort2 = SecurityTestUtil.getLocatorPort();
+      String locString = SecurityTestUtil.getLocatorString();
+      Integer port1 = ((Integer)server1.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort1, locString, authenticator, extraProps,
+              javaProps }));
+      Integer port2 = ((Integer)server2.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort2, locString, authenticator, extraProps,
+              javaProps }));
+
+      // Start first client with valid credentials
+      Properties credentials1 = gen.getValidCredentials(1);
+      Properties javaProps1 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testInvalidCredentials: For first client credentials: "
+              + credentials1 + " : " + javaProps1);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials1, javaProps1, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts",
+          new Object[] { new Integer(2) });
+
+      // Start second client with invalid credentials
+      // Trying to create the region on client2 should throw a security
+      // exception
+      Properties credentials2 = gen.getInvalidCredentials(1);
+      Properties javaProps2 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testInvalidCredentials: For second client credentials: "
+              + credentials2 + " : " + javaProps2);
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials2, javaProps2, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) });
+  }
+
+  public void testInvalidAuthInit() {
+    itestInvalidAuthInit(Boolean.FALSE);
+  }
+
+  public void itestInvalidAuthInit(Boolean multiUser) {
+
+      CredentialGenerator gen = new DummyCredentialGenerator();
+      Properties extraProps = gen.getSystemProperties();
+      Properties javaProps = gen.getJavaProperties();
+      String authenticator = gen.getAuthenticator();
+
+      getLogWriter().info(
+          "testInvalidAuthInit: Using scheme: " + gen.classCode());
+      getLogWriter().info(
+          "testInvalidAuthInit: Using authenticator: " + authenticator);
+
+      // Start the server
+      Integer locPort1 = SecurityTestUtil.getLocatorPort();
+      String locString = SecurityTestUtil.getLocatorString();
+      Integer port1 = ((Integer)server1.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort1, locString, authenticator, extraProps,
+              javaProps }));
+
+      Properties credentials = gen.getValidCredentials(1);
+      javaProps = gen.getJavaProperties();
+      getLogWriter().info(
+          "testInvalidAuthInit: For first client credentials: " + credentials
+              + " : " + javaProps);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { "com.gemstone.none", credentials, javaProps,
+            new Integer[] { port1 }, null, multiUser,
+            Integer.valueOf(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+  }
+
+  public void testNoAuthInitWithCredentials() {
+    itestNoAuthInitWithCredentials(Boolean.FALSE);
+  }
+
+  public void itestNoAuthInitWithCredentials(Boolean multiUser) {
+
+      CredentialGenerator gen = new DummyCredentialGenerator();
+      Properties extraProps = gen.getSystemProperties();
+      Properties javaProps = gen.getJavaProperties();
+      String authenticator = gen.getAuthenticator();
+
+
+      getLogWriter().info(
+          "testNoAuthInitWithCredentials: Using scheme: " + gen.classCode());
+      getLogWriter().info(
+          "testNoAuthInitWithCredentials: Using authenticator: "
+              + authenticator);
+
+      // Start the servers
+      Integer locPort1 = SecurityTestUtil.getLocatorPort();
+      Integer locPort2 = SecurityTestUtil.getLocatorPort();
+      String locString = SecurityTestUtil.getLocatorString();
+      Integer port1 = ((Integer)server1.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort1, locString, authenticator, extraProps,
+              javaProps }));
+      Integer port2 = ((Integer)server2.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort2, locString, authenticator, extraProps,
+              javaProps }));
+
+      // Start the clients with valid credentials
+      Properties credentials1 = gen.getValidCredentials(1);
+      Properties javaProps1 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testNoAuthInitWithCredentials: For first client credentials: "
+              + credentials1 + " : " + javaProps1);
+      Properties credentials2 = gen.getValidCredentials(2);
+      Properties javaProps2 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testNoAuthInitWithCredentials: For second client credentials: "
+              + credentials2 + " : " + javaProps2);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { null, credentials1, javaProps1, port1, port2, null,
+          multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { null, credentials2, javaProps2, port1, port2, null,
+          multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+      client2.invoke(SecurityTestUtil.class, "closeCache");
+      
+
+      // Now also try with invalid credentials
+      credentials2 = gen.getInvalidCredentials(5);
+      javaProps2 = gen.getJavaProperties();
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { null, credentials2, javaProps2, port1, port2, null,
+          multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+  }
+
+  public void testInvalidAuthenticator() {
+    itestInvalidAuthenticator(Boolean.FALSE);
+  }
+
+  public void itestInvalidAuthenticator(Boolean multiUser) {
+
+      CredentialGenerator gen = new DummyCredentialGenerator();
+      Properties extraProps = gen.getSystemProperties();
+      Properties javaProps = gen.getJavaProperties();
+      String authInit = gen.getAuthInit();
+
+      getLogWriter().info(
+          "testInvalidAuthenticator: Using scheme: " + gen.classCode());
+      getLogWriter().info(
+          "testInvalidAuthenticator: Using authinit: " + authInit);
+
+      // Start the server with invalid authenticator
+      Integer locPort1 = SecurityTestUtil.getLocatorPort();
+      String locString = SecurityTestUtil.getLocatorString();
+      Integer port1 = (Integer)server1.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort1, locString, "com.gemstone.gemfire.none",
+              extraProps, javaProps });
+
+      // Trying to create the region on client should throw a security exception
+      Properties credentials = gen.getValidCredentials(1);
+      javaProps = gen.getJavaProperties();
+      getLogWriter().info(
+          "testInvalidAuthenticator: For first client credentials: "
+              + credentials + " : " + javaProps);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials, javaProps, port1, null,
+              new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) });
+      client1.invoke(SecurityTestUtil.class, "closeCache");
+      
+
+      // Also test with invalid credentials
+      credentials = gen.getInvalidCredentials(1);
+      javaProps = gen.getJavaProperties();
+      getLogWriter().info(
+          "testInvalidAuthenticator: For first client credentials: "
+              + credentials + " : " + javaProps);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials, javaProps, port1, null,
+              new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) });
+  }
+
+  public void testNoAuthenticatorWithCredentials() {
+    itestNoAuthenticatorWithCredentials(Boolean.FALSE);
+  }
+
+  public void itestNoAuthenticatorWithCredentials(Boolean multiUser) {
+
+      CredentialGenerator gen = new DummyCredentialGenerator();
+      Properties extraProps = gen.getSystemProperties();
+      Properties javaProps = gen.getJavaProperties();
+      String authenticator = gen.getAuthenticator();
+      String authInit = gen.getAuthInit();
+
+      getLogWriter().info(
+          "testNoAuthenticatorWithCredentials: Using scheme: "
+              + gen.classCode());
+      getLogWriter().info(
+          "testNoAuthenticatorWithCredentials: Using authinit: " + authInit);
+
+      // Start the servers with no authenticator
+      Integer locPort1 = SecurityTestUtil.getLocatorPort();
+      Integer locPort2 = SecurityTestUtil.getLocatorPort();
+      String locString = SecurityTestUtil.getLocatorString();
+      Integer port1 = (Integer)server1.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort1, locString, null, extraProps, javaProps });
+      Integer port2 = (Integer)server2.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort2, locString, null, extraProps, javaProps });
+
+      // Clients should connect successfully and work properly with
+      // valid/invalid credentials when none are required on the server side
+      Properties credentials1 = gen.getValidCredentials(3);
+      Properties javaProps1 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testNoAuthenticatorWithCredentials: For first client credentials: "
+              + credentials1 + " : " + javaProps1);
+      Properties credentials2 = gen.getInvalidCredentials(5);
+      Properties javaProps2 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testNoAuthenticatorWithCredentials: For second client credentials: "
+              + credentials2 + " : " + javaProps2);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials1, javaProps1, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials2, javaProps2, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts",
+          new Object[] { new Integer(2) });
+
+      // Verify that the puts succeeded
+      client2.invoke(SecurityTestUtil.class, "doGets",
+          new Object[] { new Integer(2) });
+  }
+
+  public void testCredentialsWithFailover() {
+    itestCredentialsWithFailover(Boolean.FALSE);
+  }
+
+  public void itestCredentialsWithFailover(Boolean multiUser) {
+      CredentialGenerator gen = new DummyCredentialGenerator();
+      Properties extraProps = gen.getSystemProperties();
+      Properties javaProps = gen.getJavaProperties();
+      String authenticator = gen.getAuthenticator();
+      String authInit = gen.getAuthInit();
+
+      getLogWriter().info(
+          "testCredentialsWithFailover: Using scheme: " + gen.classCode());
+      getLogWriter().info(
+          "testCredentialsWithFailover: Using authenticator: " + authenticator);
+      getLogWriter().info(
+          "testCredentialsWithFailover: Using authinit: " + authInit);
+
+      // Start the first server
+      Integer locPort1 = SecurityTestUtil.getLocatorPort();
+      Integer locPort2 = SecurityTestUtil.getLocatorPort();
+      String locString = SecurityTestUtil.getLocatorString();
+      Integer port1 = (Integer)server1.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort1, locString, authenticator, extraProps,
+              javaProps });
+      // Get a port for second server but do not start it
+      // This forces the clients to connect to the first server
+      Integer port2 = new Integer(AvailablePort
+          .getRandomAvailablePort(AvailablePort.SOCKET));
+
+      // Start the clients with valid credentials
+      Properties credentials1 = gen.getValidCredentials(5);
+      Properties javaProps1 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testCredentialsWithFailover: For first client credentials: "
+              + credentials1 + " : " + javaProps1);
+      Properties credentials2 = gen.getValidCredentials(6);
+      Properties javaProps2 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testCredentialsWithFailover: For second client credentials: "
+              + credentials2 + " : " + javaProps2);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials1, javaProps1, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials2, javaProps2, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts",
+          new Object[] { new Integer(2) });
+      // Verify that the puts succeeded
+      client2.invoke(SecurityTestUtil.class, "doGets",
+          new Object[] { new Integer(2) });
+
+      // start the second one and stop the first server to force a failover
+      server2.invoke(ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort2, locString, port2, authenticator, extraProps,
+              javaProps });
+      server1.invoke(SecurityTestUtil.class, "closeCache");
+
+      // Perform some create/update operations from client1
+      client1.invoke(SecurityTestUtil.class, "doNPuts",
+          new Object[] { new Integer(4) });
+      // Verify that the creates/updates succeeded
+      client2.invoke(SecurityTestUtil.class, "doNGets",
+          new Object[] { new Integer(4) });
+
+      // Try to connect client2 with no credentials
+      // Verify that the creation of region throws security exception
+      if (gen.classCode().equals(ClassCode.SSL)) {
+        // For SSL the exception may not come since the server can close socket
+        // before handshake message is sent from client. However exception
+        // should come in any region operations.
+        client2
+            .invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+                new Object[] { null, null, null, port1, port2, null, multiUser,
+                    new Integer(SecurityTestUtil.NOFORCE_AUTHREQ_EXCEPTION) });
+        client2.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+            new Integer(2), new Integer(SecurityTestUtil.OTHER_EXCEPTION) });
+      }
+      else {
+        client2.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { null, null, null, port1, port2,
+                null, multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+      }
+
+      // Now try to connect client1 with invalid credentials
+      // Verify that the creation of region throws security exception
+      credentials1 = gen.getInvalidCredentials(7);
+      javaProps1 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testCredentialsWithFailover: For first client invalid credentials: "
+              + credentials1 + " : " + javaProps1);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials1, javaProps1, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) });
+
+      if (multiUser) {
+        client1.invoke(SecurityTestUtil.class, "doProxyCacheClose");
+        client2.invoke(SecurityTestUtil.class, "doProxyCacheClose");
+        client1.invoke(SecurityTestUtil.class, "doSimplePut",
+            new Object[] {"CacheClosedException"});
+        client2.invoke(SecurityTestUtil.class, "doSimpleGet",
+            new Object[] {"CacheClosedException"});
+      }
+  }
+
+  public void testCredentialsForNotifications() {
+    itestCredentialsForNotifications(Boolean.FALSE);
+  }
+
+  public void itestCredentialsForNotifications(Boolean multiUser) {
+      CredentialGenerator gen = new DummyCredentialGenerator();
+      Properties extraProps = gen.getSystemProperties();
+      Properties javaProps = gen.getJavaProperties();
+      String authenticator = gen.getAuthenticator();
+      String authInit = gen.getAuthInit();
+
+      getLogWriter().info(
+          "testCredentialsForNotifications: Using scheme: " + gen.classCode());
+      getLogWriter().info(
+          "testCredentialsForNotifications: Using authenticator: "
+              + authenticator);
+      getLogWriter().info(
+          "testCredentialsForNotifications: Using authinit: " + authInit);
+
+      // Start the first server
+      Integer locPort1 = SecurityTestUtil.getLocatorPort();
+      Integer locPort2 = SecurityTestUtil.getLocatorPort();
+      String locString = SecurityTestUtil.getLocatorString();
+      Integer port1 = (Integer)server1.invoke(
+          ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort1, locString, authenticator, extraProps,
+              javaProps });
+      // Get a port for second server but do not start it
+      // This forces the clients to connect to the first server
+      Integer port2 = new Integer(AvailablePort
+          .getRandomAvailablePort(AvailablePort.SOCKET));
+
+      // Start the clients with valid credentials
+      Properties credentials1 = gen.getValidCredentials(3);
+      Properties javaProps1 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testCredentialsForNotifications: For first client credentials: "
+              + credentials1 + " : " + javaProps1);
+      Properties credentials2 = gen.getValidCredentials(4);
+      Properties javaProps2 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testCredentialsForNotifications: For second client credentials: "
+              + credentials2 + " : " + javaProps2);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials1, javaProps1, port1, port2,
+              null, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+      // Set up zero forward connections to check notification handshake only
+      Object zeroConns = new Integer(0);
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials2, javaProps2, port1, port2,
+              zeroConns, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Register interest on all keys on second client
+      client2
+          .invoke(ClientAuthenticationDUnitTest.class, "registerAllInterest");
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts",
+          new Object[] { new Integer(2) });
+
+      // Verify that the puts succeeded
+      client2.invoke(SecurityTestUtil.class, "doLocalGets",
+          new Object[] { new Integer(2) });
+
+      // start the second one and stop the first server to force a failover
+      server2.invoke(ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort2, locString, port2, authenticator, extraProps,
+              javaProps });
+      server1.invoke(SecurityTestUtil.class, "closeCache");
+
+      // Wait for failover to complete
+      pause(500);
+
+      // Perform some create/update operations from client1
+      client1.invoke(SecurityTestUtil.class, "doNPuts",
+          new Object[] { new Integer(4) });
+      // Verify that the creates/updates succeeded
+      client2.invoke(SecurityTestUtil.class, "doNLocalGets",
+          new Object[] { new Integer(4) });
+
+      // Try to connect client1 with no credentials
+      // Verify that the creation of region throws security exception
+      server1.invoke(ClientAuthenticationDUnitTest.class, "createCacheServer",
+          new Object[] { locPort1, locString, port1, authenticator, extraProps,
+              javaProps });
+      if (gen.classCode().equals(ClassCode.SSL)) {
+        // For SSL the exception may not come since the server can close socket
+        // before handshake message is sent from client. However exception
+        // should come in any region operations.
+        client1.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { null, null, null, port1, port2,
+                zeroConns, multiUser,
+                new Integer(SecurityTestUtil.NOFORCE_AUTHREQ_EXCEPTION) });
+        client1.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+            new Integer(2), new Integer(SecurityTestUtil.OTHER_EXCEPTION) });
+      }
+      else {
+        client1.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { null, null, null, port1, port2,
+                zeroConns, multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+      }
+
+      // Now try to connect client2 with invalid credentials
+      // Verify that the creation of region throws security exception
+      credentials2 = gen.getInvalidCredentials(3);
+      javaProps2 = gen.getJavaProperties();
+      getLogWriter().info(
+          "testCredentialsForNotifications: For second client invalid credentials: "
+              + credentials2 + " : " + javaProps2);
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, credentials2, javaProps2, port1, port2,
+              zeroConns, multiUser, new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) });
+
+      // Now try to connect client2 with invalid auth-init method
+      // Trying to create the region on client with valid credentials should
+      // throw a security exception
+      client2
+          .invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+              new Object[] { "com.gemstone.none", credentials1, javaProps1,
+                  port1, port2, zeroConns, multiUser, 
+                  new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+
+      // Now start the servers with invalid authenticator method.
+      // Skip this test for a scheme which does not have an authInit in the
+      // first place (e.g. SSL) since that will fail with AuthReqEx before
+      // authenticator is even invoked.
+      if (authInit != null && authInit.length() > 0) {
+        server1.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheServer", new Object[] { locPort1, locString, port1,
+                "com.gemstone.gemfire.none", extraProps, javaProps });
+        server2.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheServer", new Object[] { locPort2, locString, port2,
+                "com.gemstone.gemfire.none", extraProps, javaProps });
+
+        // Trying to create the region on client with valid/invalid credentials
+        // should throw a security exception
+        client2.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { authInit, credentials1,
+                javaProps1, port1, port2, zeroConns, multiUser,
+                new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) });
+        client1.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { authInit, credentials2,
+                javaProps2, port1, port2, zeroConns, multiUser,
+                new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) });
+      }
+      else {
+        getLogWriter().info(
+            "testCredentialsForNotifications: Skipping invalid authenticator for scheme ["
+                + gen.classCode() + "] which has no authInit");
+      }
+
+      // Try connection with null auth-init on clients.
+      // Skip this test for a scheme which does not have an authInit in the
+      // first place (e.g. SSL).
+      if (authInit != null && authInit.length() > 0) {
+        server1.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheServer", new Object[] { locPort1, locString, port1,
+                authenticator, extraProps, javaProps });
+        server2.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheServer", new Object[] { locPort2, locString, port2,
+                authenticator, extraProps, javaProps });
+        client1.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { null, credentials1, javaProps1,
+                port1, port2, null, multiUser,
+                new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+        client2.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { null, credentials2, javaProps2,
+                port1, port2, zeroConns, multiUser,
+                new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+
+        // Now also try with invalid credentials on client2
+        client2.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { null, credentials2, javaProps2,
+                port1, port2, zeroConns, multiUser,
+                new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) });
+      }
+      else {
+        getLogWriter().info(
+            "testCredentialsForNotifications: Skipping null authInit for scheme ["
+                + gen.classCode() + "] which has no authInit");
+      }
+
+      // Try connection with null authenticator on server and sending
+      // valid/invalid credentials.
+      // If the scheme does not have an authenticator in the first place (e.g.
+      // SSL) then skip it since this test is useless.
+      if (authenticator != null && authenticator.length() > 0) {
+        server1.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheServer", new Object[] { locPort1, locString, port1,
+                null, extraProps, javaProps });
+        server2.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheServer", new Object[] { locPort2, locString, port2,
+                null, extraProps, javaProps });
+        client1.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { authInit, credentials1,
+                javaProps1, port1, port2, null, multiUser,
+                new Integer(SecurityTestUtil.NO_EXCEPTION) });
+        client2.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { authInit, credentials2,
+                javaProps2, port1, port2, zeroConns, multiUser,
+                new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+        // Register interest on all keys on second client
+        client2.invoke(ClientAuthenticationDUnitTest.class,
+            "registerAllInterest");
+
+        // Perform some put operations from client1
+        client1.invoke(SecurityTestUtil.class, "doPuts",
+            new Object[] { new Integer(4) });
+
+        // Verify that the puts succeeded
+        client2.invoke(SecurityTestUtil.class, "doLocalGets",
+            new Object[] { new Integer(4) });
+
+        // Now also try with valid credentials on client2
+        client1.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { authInit, credentials2,
+                javaProps2, port1, port2, null, multiUser,
+                new Integer(SecurityTestUtil.NO_EXCEPTION) });
+        client2.invoke(ClientAuthenticationDUnitTest.class,
+            "createCacheClient", new Object[] { authInit, credentials1,
+                javaProps1, port1, port2, zeroConns, multiUser,
+                new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+        // Register interest on all keys on second client
+        client2.invoke(ClientAuthenticationDUnitTest.class,
+            "registerAllInterest");
+
+        // Perform some put operations from client1
+        client1.invoke(SecurityTestUtil.class, "doNPuts",
+            new Object[] { new Integer(4) });
+
+        // Verify that the puts succeeded
+        client2.invoke(SecurityTestUtil.class, "doNLocalGets",
+            new Object[] { new Integer(4) });
+      }
+      else {
+        getLogWriter().info(
+            "testCredentialsForNotifications: Skipping scheme ["
+                + gen.classCode() + "] which has no authenticator");
+      }
+  }
+
+  //////////////////////////////////////////////////////////////////////////////
+  // Tests for MULTI_USER_MODE start here
+  //////////////////////////////////////////////////////////////////////////////
+
+  public void xtestValidCredentialsForMultipleUsers() {
+    itestValidCredentials(Boolean.TRUE);
+  }
+
+  //////////////////////////////////////////////////////////////////////////////
+  // Tests for MULTI_USER_MODE end here
+  //////////////////////////////////////////////////////////////////////////////
+  
+  @Override
+  public void tearDown2() throws Exception {
+
+    super.tearDown2();
+    // close the clients first
+    client1.invoke(SecurityTestUtil.class, "closeCache");
+    client2.invoke(SecurityTestUtil.class, "closeCache");
+    // then close the servers
+    server1.invoke(SecurityTestUtil.class, "closeCache");
+    server2.invoke(SecurityTestUtil.class, "closeCache");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
new file mode 100644
index 0000000..138114a
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
@@ -0,0 +1,88 @@
+
+package com.gemstone.gemfire.security;
+
+/*
+ * 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.
+ */
+
+
+/**
+ * this class contains test methods that used to be in its superclass but
+ * that test started taking too long and caused dunit runs to hang
+ */
+public class ClientAuthenticationPart2DUnitTest extends
+    ClientAuthenticationDUnitTest {
+
+  /** constructor */
+  public ClientAuthenticationPart2DUnitTest(String name) {
+    super(name);
+  }
+
+  // override inherited tests so they aren't executed again
+  
+  @Override
+  public void testValidCredentials() {  }
+  @Override
+  public void testNoCredentials() {  }
+  @Override
+  public void testInvalidCredentials() {  }
+  @Override
+  public void testInvalidAuthInit() {  }
+  @Override
+  public void testNoAuthInitWithCredentials() {  }
+  @Override
+  public void testInvalidAuthenticator() {  }
+  @Override
+  public void testNoAuthenticatorWithCredentials() {  }
+  @Override
+  public void testCredentialsWithFailover() {  }
+  @Override
+  public void testCredentialsForNotifications() {  }
+  //@Override
+  public void testValidCredentialsForMultipleUsers() {  }
+
+
+  
+  
+  
+  public void testNoCredentialsForMultipleUsers() {
+    itestNoCredentials(Boolean.TRUE);
+  }
+  public void testInvalidCredentialsForMultipleUsers() {
+    itestInvalidCredentials(Boolean.TRUE);
+  }
+  public void testInvalidAuthInitForMultipleUsers() {
+    itestInvalidAuthInit(Boolean.TRUE);
+  }
+  public void testNoAuthInitWithCredentialsForMultipleUsers() {
+    itestNoAuthInitWithCredentials(Boolean.TRUE);
+  }
+  public void testInvalidAuthenitcatorForMultipleUsers() {
+    itestInvalidAuthenticator(Boolean.TRUE);
+  }
+  public void testNoAuthenticatorWithCredentialsForMultipleUsers() {
+    itestNoAuthenticatorWithCredentials(Boolean.TRUE);
+  }
+  public void disabled_testCredentialsWithFailoverForMultipleUsers() {
+    itestCredentialsWithFailover(Boolean.TRUE);
+  }
+  public void __testCredentialsForNotificationsForMultipleUsers() {
+    itestCredentialsForNotifications(Boolean.TRUE);
+  }
+
+}


[05/33] incubator-geode git commit: GEODE-663: add aditional security tests to the open source

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/templates/security/AuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/AuthzCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/AuthzCredentialGenerator.java
deleted file mode 100644
index 0e1ccf6..0000000
--- a/gemfire-core/src/test/java/templates/security/AuthzCredentialGenerator.java
+++ /dev/null
@@ -1,465 +0,0 @@
-
-package templates.security;
-
-/*
- * 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.
- */
-
-
-import java.security.Principal;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import com.gemstone.gemfire.security.AccessControl;
-
-/**
- * Encapsulates obtaining authorized and unauthorized credentials for a given
- * operation in a region. Implementations will be for different kinds of
- * authorization scheme and authentication scheme combos.
- * 
- * @author sumedh
- * @since 5.5
- */
-public abstract class AuthzCredentialGenerator {
-
-  /**
-   * Enumeration for various {@link AuthzCredentialGenerator} implementations.
-   * 
-   * The following schemes are supported as of now:
-   * <ul>
-   * <li><code>DummyAuthorization</code> with <code>DummyAuthenticator</code></li>
-   * <li><code>XMLAuthorization</code> with <code>DummyAuthenticator</code></li>
-   * <li><code>XMLAuthorization</code> with <code>LDAPAuthenticator</code></li>
-   * <li><code>XMLAuthorization</code> with <code>PKCSAuthenticator</code></li>
-   * <li><code>XMLAuthorization</code> when using SSL sockets</li>
-   * </ul>
-   * 
-   * To add a new authorization scheme the following needs to be done:
-   * <ul>
-   * <li>Add implementation for {@link AccessControl}.</li>
-   * <li>Choose the authentication schemes that it shall work with from
-   * {@link CredentialGenerator.ClassCode}</li>
-   * <li>Add a new enumeration value for the scheme in this class. Notice the
-   * size of <code>VALUES</code> array and increase that if it is getting
-   * overflowed. Note the methods and fields for existing schemes and add for
-   * the new one in a similar manner.</li>
-   * <li>Add an implementation for {@link AuthzCredentialGenerator}. Note the
-   * {@link AuthzCredentialGenerator#init} method where different authentication
-   * schemes can be passed and initialize differently for the authentication
-   * schemes that shall be handled.</li>
-   * <li>Modify the {@link AuthzCredentialGenerator#create} method to add
-   * creation of an instance of the new implementation for the
-   * <code>ClassCode</code> enumeration value.</li>
-   * </ul>
-   * All dunit tests will automagically start testing the new implementation
-   * after this.
-   * 
-   * @author sumedh
-   * @since 5.5
-   */
-  public static final class ClassCode {
-
-    private static final byte ID_DUMMY = 1;
-
-    private static final byte ID_XML = 2;
-
-    private static byte nextOrdinal = 0;
-
-    private static final ClassCode[] VALUES = new ClassCode[10];
-
-    private static final Map CodeNameMap = new HashMap();
-
-    public static final ClassCode DUMMY = new ClassCode(
-        "templates.security.DummyAuthorization.create", ID_DUMMY);
-
-    public static final ClassCode XML = new ClassCode(
-        "templates.security.XmlAuthorization.create", ID_XML);
-
-    /** The name of this class. */
-    private final String name;
-
-    /** byte used as ordinal to represent this class */
-    private final byte ordinal;
-
-    /**
-     * One of the following: ID_DUMMY, ID_LDAP, ID_PKI
-     */
-    private final byte classType;
-
-    /** Creates a new instance of class code. */
-    private ClassCode(String name, byte classType) {
-      this.name = name;
-      this.classType = classType;
-      this.ordinal = nextOrdinal++;
-      VALUES[this.ordinal] = this;
-      CodeNameMap.put(name, this);
-    }
-
-    public boolean isDummy() {
-      return (this.classType == ID_DUMMY);
-    }
-
-    public boolean isXml() {
-      return (this.classType == ID_XML);
-    }
-
-    /**
-     * Returns the <code>ClassCode</code> represented by specified ordinal.
-     */
-    public static ClassCode fromOrdinal(byte ordinal) {
-      return VALUES[ordinal];
-    }
-
-    /**
-     * Returns the <code>ClassCode</code> represented by specified string.
-     */
-    public static ClassCode parse(String operationName) {
-      return (ClassCode)CodeNameMap.get(operationName);
-    }
-
-    /**
-     * Returns all the possible values.
-     */
-    public static List getAll() {
-      List codes = new ArrayList();
-      Iterator iter = CodeNameMap.values().iterator();
-      while (iter.hasNext()) {
-        codes.add(iter.next());
-      }
-      return codes;
-    }
-
-    /**
-     * Returns the ordinal for this class code.
-     * 
-     * @return the ordinal of this class code.
-     */
-    public byte toOrdinal() {
-      return this.ordinal;
-    }
-
-    /**
-     * Returns a string representation for this class code.
-     * 
-     * @return the name of this class code.
-     */
-    final public String toString() {
-      return this.name;
-    }
-
-    /**
-     * Indicates whether other object is same as this one.
-     * 
-     * @return true if other object is same as this one.
-     */
-    @Override
-    final public boolean equals(final Object obj) {
-      if (obj == this) {
-        return true;
-      }
-      if (!(obj instanceof ClassCode)) {
-        return false;
-      }
-      final ClassCode other = (ClassCode)obj;
-      return (other.ordinal == this.ordinal);
-    }
-
-    /**
-     * Indicates whether other <code>ClassCode</code> is same as this one.
-     * 
-     * @return true if other <code>ClassCode</code> is same as this one.
-     */
-    final public boolean equals(final ClassCode opCode) {
-      return (opCode != null && opCode.ordinal == this.ordinal);
-    }
-
-    /**
-     * Returns a hash code value for this <code>ClassCode</code> which is the
-     * same as its ordinal.
-     * 
-     * @return the ordinal of this <code>ClassCode</code>.
-     */
-    @Override
-    final public int hashCode() {
-      return this.ordinal;
-    }
-
-  }
-
-  /**
-   * The {@link CredentialGenerator} being used.
-   */
-  protected CredentialGenerator cGen;
-
-  /**
-   * A set of system properties that should be added to the gemfire system
-   * properties before using the authorization module.
-   */
-  private Properties sysProps;
-
-  /**
-   * A factory method to create a new instance of an
-   * {@link AuthzCredentialGenerator} for the given {@link ClassCode}. Caller
-   * is supposed to invoke {@link AuthzCredentialGenerator#init} immediately
-   * after obtaining the instance.
-   * 
-   * @param classCode
-   *                the <code>ClassCode</code> of the
-   *                <code>AuthzCredentialGenerator</code> implementation
-   * 
-   * @return an instance of <code>AuthzCredentialGenerator</code> for the
-   *         given class code
-   */
-  public static AuthzCredentialGenerator create(ClassCode classCode) {
-    switch (classCode.classType) {
-      case ClassCode.ID_DUMMY:
-        return new DummyAuthzCredentialGenerator();
-      case ClassCode.ID_XML:
-        return new XmlAuthzCredentialGenerator();
-      default:
-        return null;
-    }
-  }
-
-  /**
-   * Initialize the authorized credential generator.
-   * 
-   * @param cGen
-   *                an instance of {@link CredentialGenerator} of the credential
-   *                implementation for which to obtain authorized/unauthorized
-   *                credentials.
-   * 
-   * @return false when the given {@link CredentialGenerator} is incompatible
-   *         with this authorization module.
-   */
-  public boolean init(CredentialGenerator cGen) {
-    this.cGen = cGen;
-    try {
-      this.sysProps = init();
-    }
-    catch (IllegalArgumentException ex) {
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * 
-   * @return A set of extra properties that should be added to Gemfire system
-   *         properties when not null.
-   */
-  public Properties getSystemProperties() {
-    return this.sysProps;
-  }
-
-  /**
-   * Get the {@link CredentialGenerator} being used by this instance.
-   */
-  public CredentialGenerator getCredentialGenerator() {
-    return this.cGen;
-  }
-
-  /**
-   * The {@link ClassCode} of the particular implementation.
-   * 
-   * @return the <code>ClassCode</code>
-   */
-  public abstract ClassCode classCode();
-
-  /**
-   * The name of the {@link AccessControl} factory function that should be used
-   * as the authorization module on the server side.
-   * 
-   * @return name of the <code>AccessControl</code> factory function
-   */
-  public abstract String getAuthorizationCallback();
-
-  /**
-   * Get a set of credentials generated using the given index allowed to perform
-   * the given {@link OperationCode}s for the given regions.
-   * 
-   * @param opCodes
-   *                the list of {@link OperationCode}s of the operations
-   *                requiring authorization; should not be null
-   * @param regionNames
-   *                list of the region names requiring authorization; a value of
-   *                null indicates all regions
-   * @param index
-   *                used to generate multiple such credentials by passing
-   *                different values for this
-   * 
-   * @return the set of credentials authorized to perform the given operation in
-   *         the given regions
-   */
-  public Properties getAllowedCredentials(OperationCode[] opCodes,
-      String[] regionNames, int index) {
-
-    int numTries = getNumPrincipalTries(opCodes, regionNames);
-    if (numTries <= 0) {
-      numTries = 1;
-    }
-    for (int tries = 0; tries < numTries; tries++) {
-      Principal principal = getAllowedPrincipal(opCodes, regionNames,
-          (index + tries) % numTries);
-      try {
-        return this.cGen.getValidCredentials(principal);
-      }
-      catch (IllegalArgumentException ex) {
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Get a set of credentials generated using the given index not allowed to
-   * perform the given {@link OperationCode}s for the given regions. The
-   * credentials are required to be valid for authentication.
-   * 
-   * @param opCode
-   *                the {@link OperationCode}s of the operations requiring
-   *                authorization failure; should not be null
-   * @param regionNames
-   *                list of the region names requiring authorization failure; a
-   *                value of null indicates all regions
-   * @param index
-   *                used to generate multiple such credentials by passing
-   *                different values for this
-   * 
-   * @return the set of credentials that are not authorized to perform the given
-   *         operation in the given region
-   */
-  public Properties getDisallowedCredentials(OperationCode[] opCodes,
-      String[] regionNames, int index) {
-
-    // This may not be very correct since we use the value of
-    // getNumPrincipalTries() but is used to avoid adding another method.
-    // Also something like getNumDisallowedPrincipals() will be normally always
-    // infinite, and the number here is just to perform some number of tries
-    // before giving up.
-    int numTries = getNumPrincipalTries(opCodes, regionNames);
-    if (numTries <= 0) {
-      numTries = 1;
-    }
-    for (int tries = 0; tries < numTries; tries++) {
-      Principal principal = getDisallowedPrincipal(opCodes, regionNames,
-          (index + tries) % numTries);
-      try {
-        return this.cGen.getValidCredentials(principal);
-      }
-      catch (IllegalArgumentException ex) {
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Initialize the authorized credential generator.
-   * 
-   * Required to be implemented by concrete classes that implement this abstract
-   * class.
-   * 
-   * @return A set of extra properties that should be added to Gemfire system
-   *         properties when not null.
-   * 
-   * @throws IllegalArgumentException
-   *                 when the {@link CredentialGenerator} is incompatible with
-   *                 this authorization module.
-   */
-  protected abstract Properties init() throws IllegalArgumentException;
-
-  /**
-   * Get the number of tries to be done for obtaining valid credentials for the
-   * given operations in the given region. It is required that
-   * {@link #getAllowedPrincipal} method returns valid principals for values of
-   * <code>index</code> from 0 through (n-1) where <code>n</code> is the
-   * value returned by this method. It is recommended that the principals so
-   * returned be unique for efficiency.
-   * 
-   * This will be used by {@link #getAllowedCredentials} to step through
-   * different principals and obtain a set of valid credentials.
-   * 
-   * Required to be implemented by concrete classes that implement this abstract
-   * class.
-   * 
-   * @param opCodes
-   *                the {@link OperationCode}s of the operations requiring
-   *                authorization
-   * @param regionNames
-   *                list of the region names requiring authorization; a value of
-   *                null indicates all regions
-   * @param index
-   *                used to generate multiple such credentials by passing
-   *                different values for this
-   * 
-   * @return the number of principals allowed to perform the given operation in
-   *         the given region
-   */
-  protected abstract int getNumPrincipalTries(OperationCode[] opCodes,
-      String[] regionNames);
-
-  /**
-   * Get a {@link Principal} generated using the given index allowed to perform
-   * the given {@link OperationCode}s for the given region.
-   * 
-   * Required to be implemented by concrete classes that implement this abstract
-   * class.
-   * 
-   * @param opCodes
-   *                the {@link OperationCode}s of the operations requiring
-   *                authorization
-   * @param regionNames
-   *                list of the region names requiring authorization; a value of
-   *                null indicates all regions
-   * @param index
-   *                used to generate multiple such principals by passing
-   *                different values for this
-   * 
-   * @return the {@link Principal} authorized to perform the given operation in
-   *         the given region
-   */
-  protected abstract Principal getAllowedPrincipal(OperationCode[] opCodes,
-      String[] regionNames, int index);
-
-  /**
-   * Get a {@link Principal} generated using the given index not allowed to
-   * perform the given {@link OperationCode}s for the given region.
-   * 
-   * Required to be implemented by concrete classes that implement this abstract
-   * class.
-   * 
-   * @param opCodes
-   *                the {@link OperationCode}s of the operations requiring
-   *                authorization failure
-   * @param regionNames
-   *                list of the region names requiring authorization failure; a
-   *                value of null indicates all regions
-   * @param index
-   *                used to generate multiple such principals by passing
-   *                different values for this
-   * 
-   * @return a {@link Principal} not authorized to perform the given operation
-   *         in the given region
-   */
-  protected abstract Principal getDisallowedPrincipal(OperationCode[] opCodes,
-      String[] regionNames, int index);
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/templates/security/CredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/CredentialGenerator.java b/gemfire-core/src/test/java/templates/security/CredentialGenerator.java
deleted file mode 100644
index 99d7e18..0000000
--- a/gemfire-core/src/test/java/templates/security/CredentialGenerator.java
+++ /dev/null
@@ -1,343 +0,0 @@
-
-package templates.security;
-
-/*
- * 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.
- */
-
-
-import java.security.Principal;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import com.gemstone.gemfire.security.AuthInitialize;
-import com.gemstone.gemfire.security.Authenticator;
-
-/**
- * Encapsulates obtaining valid and invalid credentials. Implementations will be
- * for different kinds of authentication schemes.
- * 
- * @author sumedh
- * @since 5.5
- */
-public abstract class CredentialGenerator {
-
-  /**
-   * Enumeration for various {@link CredentialGenerator} implementations.
-   * 
-   * The following schemes are supported as of now:
-   * <code>DummyAuthenticator</code>, <code>LdapUserAuthenticator</code>,
-   * <code>PKCSAuthenticator</code>. In addition SSL socket mode with mutual
-   * authentication is also supported.
-   * 
-   * To add a new authentication scheme the following needs to be done:
-   * <ul>
-   * <li>Add implementations for {@link AuthInitialize} and
-   * {@link Authenticator} classes for clients/peers.</li>
-   * <li>Add a new enumeration value for the scheme in this class. Notice the
-   * size of <code>VALUES</code> array and increase that if it is getting
-   * overflowed. Note the methods and fields for existing schemes and add for
-   * the new one in a similar manner.</li>
-   * <li>Add an implementation for {@link CredentialGenerator}.</li>
-   * <li>Modify the {@link CredentialGenerator.Factory#create} method to add
-   * creation of an instance of the new implementation for the
-   * <code>ClassCode</code> enumeration value.</li>
-   * </ul>
-   * All security dunit tests will automagically start testing the new
-   * implementation after this.
-   * 
-   * @author sumedh
-   * @since 5.5
-   */
-  public static final class ClassCode {
-
-    private static final byte ID_DUMMY = 1;
-
-    private static final byte ID_LDAP = 2;
-
-    private static final byte ID_PKCS = 3;
-
-    private static final byte ID_SSL = 4;
-
-    private static byte nextOrdinal = 0;
-
-    private static final ClassCode[] VALUES = new ClassCode[10];
-
-    private static final Map CodeNameMap = new HashMap();
-
-    public static final ClassCode DUMMY = new ClassCode(
-        "templates.security.DummyAuthenticator.create", ID_DUMMY);
-
-    public static final ClassCode LDAP = new ClassCode(
-        "templates.security.LdapUserAuthenticator.create", ID_LDAP);
-
-    public static final ClassCode PKCS = new ClassCode(
-        "templates.security.PKCSAuthenticator.create", ID_PKCS);
-
-    public static final ClassCode SSL = new ClassCode("SSL", ID_SSL);
-
-    /** The name of this class. */
-    private final String name;
-
-    /** byte used as ordinal to represent this class */
-    private final byte ordinal;
-
-    /**
-     * One of the following: ID_DUMMY, ID_LDAP, ID_PKCS
-     */
-    private final byte classType;
-
-    /** Creates a new instance of class code. */
-    private ClassCode(String name, byte classType) {
-      this.name = name;
-      this.classType = classType;
-      this.ordinal = nextOrdinal++;
-      VALUES[this.ordinal] = this;
-      CodeNameMap.put(name, this);
-    }
-
-    public boolean isDummy() {
-      return (this.classType == ID_DUMMY);
-    }
-
-    public boolean isLDAP() {
-      return (this.classType == ID_LDAP);
-    }
-
-    public boolean isPKCS() {
-      return (this.classType == ID_PKCS);
-    }
-
-    public boolean isSSL() {
-      return (this.classType == ID_SSL);
-    }
-
-    /**
-     * Returns the <code>ClassCode</code> represented by specified ordinal.
-     */
-    public static ClassCode fromOrdinal(byte ordinal) {
-      return VALUES[ordinal];
-    }
-
-    /**
-     * Returns the <code>ClassCode</code> represented by specified string.
-     */
-    public static ClassCode parse(String operationName) {
-      return (ClassCode)CodeNameMap.get(operationName);
-    }
-
-    /**
-     * Returns all the possible values.
-     */
-    public static List getAll() {
-      List codes = new ArrayList();
-      Iterator iter = CodeNameMap.values().iterator();
-      while (iter.hasNext()) {
-        codes.add(iter.next());
-      }
-      return codes;
-    }
-
-    /**
-     * Returns the ordinal for this operation code.
-     * 
-     * @return the ordinal of this operation.
-     */
-    public byte toOrdinal() {
-      return this.ordinal;
-    }
-
-    /**
-     * Returns a string representation for this operation.
-     * 
-     * @return the name of this operation.
-     */
-    final public String toString() {
-      return this.name;
-    }
-
-    /**
-     * Indicates whether other object is same as this one.
-     * 
-     * @return true if other object is same as this one.
-     */
-    @Override
-    final public boolean equals(final Object obj) {
-      if (obj == this) {
-        return true;
-      }
-      if (!(obj instanceof ClassCode)) {
-        return false;
-      }
-      final ClassCode other = (ClassCode)obj;
-      return (other.ordinal == this.ordinal);
-    }
-
-    /**
-     * Indicates whether other <code>ClassCode</code> is same as this one.
-     * 
-     * @return true if other <code>ClassCode</code> is same as this one.
-     */
-    final public boolean equals(final ClassCode opCode) {
-      return (opCode != null && opCode.ordinal == this.ordinal);
-    }
-
-    /**
-     * Returns a hash code value for this <code>ClassCode</code> which is the
-     * same as its ordinal.
-     * 
-     * @return the ordinal of this operation.
-     */
-    @Override
-    final public int hashCode() {
-      return this.ordinal;
-    }
-
-  }
-
-  /**
-   * A set of properties that should be added to the Gemfire system properties
-   * before using the authentication module.
-   */
-  private Properties sysProps = null;
-
-  /**
-   * A set of properties that should be added to the java system properties
-   * before using the authentication module.
-   */
-  protected Properties javaProps = null;
-
-  /**
-   * A factory method to create a new instance of an {@link CredentialGenerator}
-   * for the given {@link ClassCode}. Caller is supposed to invoke
-   * {@link CredentialGenerator#init} immediately after obtaining the instance.
-   * 
-   * @param classCode
-   *                the <code>ClassCode</code> of the
-   *                <code>CredentialGenerator</code> implementation
-   * 
-   * @return an instance of <code>CredentialGenerator</code> for the given
-   *         class code
-   */
-  public static CredentialGenerator create(ClassCode classCode) {
-    switch (classCode.classType) {
-      // Removing dummy one to reduce test run times
-      // case ClassCode.ID_DUMMY:
-      // return new DummyCredentialGenerator();
-      case ClassCode.ID_LDAP:
-        return new LdapUserCredentialGenerator();
-        // case ClassCode.ID_SSL:ø
-        // return new SSLCredentialGenerator();
-      case ClassCode.ID_PKCS:
-        return new PKCSCredentialGenerator();
-      default:
-        return null;
-    }
-  }
-
-  /**
-   * Initialize the credential generator.
-   * 
-   * @throws IllegalArgumentException
-   *                 when there is a problem during initialization
-   */
-  public void init() throws IllegalArgumentException {
-    this.sysProps = initialize();
-  }
-
-  /**
-   * Initialize the credential generator. This is provided separately from the
-   * {@link #init} method for convenience of implementations so that they do not
-   * need to store in {@link #sysProps}. The latter is convenient for the users
-   * who do not need to store these properties rather can obtain it later by
-   * invoking {@link #getSystemProperties}
-   * 
-   * Required to be implemented by concrete classes that implement this abstract
-   * class.
-   * 
-   * @return A set of extra properties that should be added to Gemfire system
-   *         properties when not null.
-   * 
-   * @throws IllegalArgumentException
-   *                 when there is a problem during initialization
-   */
-  protected abstract Properties initialize() throws IllegalArgumentException;
-
-  /**
-   * 
-   * @return A set of extra properties that should be added to Gemfire system
-   *         properties when not null.
-   */
-  public Properties getSystemProperties() {
-    return this.sysProps;
-  }
-
-  /**
-   * 
-   * @return A set of extra properties that should be added to Gemfire system
-   *         properties when not null.
-   */
-  public Properties getJavaProperties() {
-    return this.javaProps;
-  }
-
-  /**
-   * The {@link ClassCode} of this particular implementation.
-   * 
-   * @return the <code>ClassCode</code>
-   */
-  public abstract ClassCode classCode();
-
-  /**
-   * The name of the {@link AuthInitialize} factory function that should be used
-   * in conjunction with the credentials generated by this generator.
-   * 
-   * @return name of the <code>AuthInitialize</code> factory function
-   */
-  public abstract String getAuthInit();
-
-  /**
-   * The name of the {@link Authenticator} factory function that should be used
-   * in conjunction with the credentials generated by this generator.
-   * 
-   * @return name of the <code>Authenticator</code> factory function
-   */
-  public abstract String getAuthenticator();
-
-  /**
-   * Get a set of valid credentials generated using the given index.
-   */
-  public abstract Properties getValidCredentials(int index);
-
-  /**
-   * Get a set of valid credentials for the given {@link Principal}.
-   * 
-   * @return credentials for the given <code>Principal</code> or null if none
-   *         possible.
-   */
-  public abstract Properties getValidCredentials(Principal principal);
-
-  /**
-   * Get a set of invalid credentials generated using the given index.
-   */
-  public abstract Properties getInvalidCredentials(int index);
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/templates/security/DummyAuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/DummyAuthzCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/DummyAuthzCredentialGenerator.java
deleted file mode 100644
index 5c27d60..0000000
--- a/gemfire-core/src/test/java/templates/security/DummyAuthzCredentialGenerator.java
+++ /dev/null
@@ -1,142 +0,0 @@
-
-package templates.security;
-
-/*
- * 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.
- */
-
-
-import java.security.Principal;
-import java.util.HashSet;
-import java.util.Properties;
-import java.util.Set;
-
-import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-
-public class DummyAuthzCredentialGenerator extends AuthzCredentialGenerator {
-
-  public static final byte READER_ROLE = 1;
-
-  public static final byte WRITER_ROLE = 2;
-
-  public static final byte ADMIN_ROLE = 3;
-
-  private static Set readerOpsSet;
-
-  private static Set writerOpsSet;
-
-  static {
-
-    readerOpsSet = new HashSet();
-    for (int index = 0; index < DummyAuthorization.READER_OPS.length; index++) {
-      readerOpsSet.add(DummyAuthorization.READER_OPS[index]);
-    }
-    writerOpsSet = new HashSet();
-    for (int index = 0; index < DummyAuthorization.WRITER_OPS.length; index++) {
-      writerOpsSet.add(DummyAuthorization.WRITER_OPS[index]);
-    }
-  }
-
-  public DummyAuthzCredentialGenerator() {
-  }
-
-  protected Properties init() throws IllegalArgumentException {
-
-    if (!this.cGen.classCode().isDummy()) {
-      throw new IllegalArgumentException(
-          "DummyAuthorization module only works with DummyAuthenticator");
-    }
-    return null;
-  }
-
-  public ClassCode classCode() {
-    return ClassCode.DUMMY;
-  }
-
-  public String getAuthorizationCallback() {
-    return "templates.security.DummyAuthorization.create";
-  }
-
-  public static byte getRequiredRole(OperationCode[] opCodes) {
-
-    byte roleType = ADMIN_ROLE;
-    boolean requiresReader = true;
-    boolean requiresWriter = true;
-
-    for (int opNum = 0; opNum < opCodes.length; opNum++) {
-      if (requiresReader && !readerOpsSet.contains(opCodes[opNum])) {
-        requiresReader = false;
-      }
-      if (requiresWriter && !writerOpsSet.contains(opCodes[opNum])) {
-        requiresWriter = false;
-      }
-    }
-    if (requiresReader) {
-      roleType = READER_ROLE;
-    }
-    else if (requiresWriter) {
-      roleType = WRITER_ROLE;
-    }
-    return roleType;
-  }
-
-  private Principal getPrincipal(byte roleType, int index) {
-
-    String[] admins = new String[] { "root", "admin", "administrator" };
-    switch (roleType) {
-      case READER_ROLE:
-        return new UsernamePrincipal("reader" + index);
-      case WRITER_ROLE:
-        return new UsernamePrincipal("writer" + index);
-      default:
-        return new UsernamePrincipal(admins[index % admins.length]);
-    }
-  }
-
-  protected Principal getAllowedPrincipal(OperationCode[] opCodes,
-      String[] regionNames, int index) {
-
-    byte roleType = getRequiredRole(opCodes);
-    return getPrincipal(roleType, index);
-  }
-
-  protected Principal getDisallowedPrincipal(OperationCode[] opCodes,
-      String[] regionNames, int index) {
-
-    byte roleType = getRequiredRole(opCodes);
-    byte disallowedRoleType;
-    switch (roleType) {
-      case READER_ROLE:
-        disallowedRoleType = WRITER_ROLE;
-        break;
-      case WRITER_ROLE:
-        disallowedRoleType = READER_ROLE;
-        break;
-      default:
-        disallowedRoleType = READER_ROLE;
-        break;
-    }
-    return getPrincipal(disallowedRoleType, index);
-  }
-
-  protected int getNumPrincipalTries(OperationCode[] opCodes,
-      String[] regionNames) {
-    return 5;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/templates/security/DummyCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/DummyCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/DummyCredentialGenerator.java
deleted file mode 100644
index af6c6eb..0000000
--- a/gemfire-core/src/test/java/templates/security/DummyCredentialGenerator.java
+++ /dev/null
@@ -1,90 +0,0 @@
-
-package templates.security;
-
-/*
- * 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.
- */
-
-
-import java.security.Principal;
-import java.util.Properties;
-
-public class DummyCredentialGenerator extends CredentialGenerator {
-
-  public DummyCredentialGenerator() {
-  }
-
-  protected Properties initialize() throws IllegalArgumentException {
-    return null;
-  }
-
-  public ClassCode classCode() {
-    return ClassCode.DUMMY;
-  }
-
-  public String getAuthInit() {
-    return "templates.security.UserPasswordAuthInit.create";
-  }
-
-  public String getAuthenticator() {
-    return "templates.security.DummyAuthenticator.create";
-  }
-
-  public Properties getValidCredentials(int index) {
-
-    String[] validGroups = new String[] { "admin", "user", "reader", "writer" };
-    String[] admins = new String[] { "root", "admin", "administrator" };
-
-    Properties props = new Properties();
-    int groupNum = (index % validGroups.length);
-    String userName;
-    if (groupNum == 0) {
-      userName = admins[index % admins.length];
-    }
-    else {
-      userName = validGroups[groupNum] + (index / validGroups.length);
-    }
-    props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
-    props.setProperty(UserPasswordAuthInit.PASSWORD, userName);
-    return props;
-  }
-
-  public Properties getValidCredentials(Principal principal) {
-
-    String userName = principal.getName();
-    if (DummyAuthenticator.testValidName(userName)) {
-      Properties props = new Properties();
-      props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
-      props.setProperty(UserPasswordAuthInit.PASSWORD, userName);
-      return props;
-    }
-    else {
-      throw new IllegalArgumentException("Dummy: [" + userName
-          + "] is not a valid user");
-    }
-  }
-
-  public Properties getInvalidCredentials(int index) {
-
-    Properties props = new Properties();
-    props.setProperty(UserPasswordAuthInit.USER_NAME, "invalid" + index);
-    props.setProperty(UserPasswordAuthInit.PASSWORD, "none");
-    return props;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/templates/security/LdapUserCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/LdapUserCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/LdapUserCredentialGenerator.java
deleted file mode 100644
index 816df63..0000000
--- a/gemfire-core/src/test/java/templates/security/LdapUserCredentialGenerator.java
+++ /dev/null
@@ -1,158 +0,0 @@
-
-package templates.security;
-
-/*
- * 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.
- */
-
-
-import java.security.Principal;
-import java.util.Properties;
-
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
-import com.gemstone.gemfire.util.test.TestUtil;
-
-import java.util.Random;
-
-public class LdapUserCredentialGenerator extends CredentialGenerator {
-
-  private static final String USER_PREFIX = "gemfire";
-
-  private static boolean enableServerAuthentication = false;
-
-  private boolean serverAuthEnabled = false;
-
-  private static final Random prng = new Random();
-
-  private static final String[] algos = new String[] { "", "DESede", "AES:128",
-      "Blowfish:128" };
-
-  public LdapUserCredentialGenerator() {
-    // Toggle server authentication enabled for each test
-    // This is done instead of running all the tests with both
-    // server auth enabled/disabled to reduce test run time.
-    enableServerAuthentication = !enableServerAuthentication;
-    serverAuthEnabled = enableServerAuthentication;
-  }
-
-  @Override
-  protected Properties initialize() throws IllegalArgumentException {
-
-    Properties extraProps = new Properties();
-    String ldapServer = System.getProperty("gf.ldap.server", "ldap");
-    String ldapBaseDN = System.getProperty("gf.ldap.basedn", "ou=ldapTesting,dc=pune,dc=gemstone,dc=com");
-    String ldapUseSSL = System.getProperty("gf.ldap.usessl");
-    extraProps.setProperty(LdapUserAuthenticator.LDAP_SERVER_NAME, ldapServer);
-    extraProps.setProperty(LdapUserAuthenticator.LDAP_BASEDN_NAME, ldapBaseDN);
-    if (ldapUseSSL != null && ldapUseSSL.length() > 0) {
-      extraProps.setProperty(LdapUserAuthenticator.LDAP_SSL_NAME, ldapUseSSL);
-    }
-    if (serverAuthEnabled) {
-      String keyStoreFile = TestUtil.getResourcePath(LdapUserCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/gemfire1.keystore");
-      extraProps.setProperty(HandShake.PRIVATE_KEY_FILE_PROP, keyStoreFile);
-      extraProps.setProperty(HandShake.PRIVATE_KEY_ALIAS_PROP, "gemfire1");
-      extraProps.setProperty(HandShake.PRIVATE_KEY_PASSWD_PROP, "gemfire");
-    }
-    return extraProps;
-  }
-
-  @Override
-  public ClassCode classCode() {
-    return ClassCode.LDAP;
-  }
-
-  @Override
-  public String getAuthInit() {
-    return "templates.security.UserPasswordAuthInit.create";
-  }
-
-  @Override
-  public String getAuthenticator() {
-    return "templates.security.LdapUserAuthenticator.create";
-  }
-
-  @Override
-  public Properties getValidCredentials(int index) {
-
-    Properties props = new Properties();
-    props.setProperty(UserPasswordAuthInit.USER_NAME, USER_PREFIX
-        + ((index % 10) + 1));
-    props.setProperty(UserPasswordAuthInit.PASSWORD, USER_PREFIX
-        + ((index % 10) + 1));
-    props.setProperty(DistributionConfig.SECURITY_CLIENT_DHALGO_NAME,
-        algos[prng.nextInt(algos.length)]);
-    if (serverAuthEnabled) {
-      String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/publickeyfile");
-      props.setProperty(HandShake.PUBLIC_KEY_FILE_PROP, keyStoreFile);
-      props.setProperty(HandShake.PUBLIC_KEY_PASSWD_PROP, "gemfire");
-    }
-    return props;
-  }
-
-  @Override
-  public Properties getValidCredentials(Principal principal) {
-
-    Properties props = null;
-    String userName = principal.getName();
-    if (userName != null && userName.startsWith(USER_PREFIX)) {
-      boolean isValid;
-      try {
-        int suffix = Integer.parseInt(userName.substring(USER_PREFIX.length()));
-        isValid = (suffix >= 1 && suffix <= 10);
-      }
-      catch (Exception ex) {
-        isValid = false;
-      }
-      if (isValid) {
-        props = new Properties();
-        props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
-        props.setProperty(UserPasswordAuthInit.PASSWORD, userName);
-      }
-    }
-    if (props == null) {
-      throw new IllegalArgumentException("LDAP: [" + userName
-          + "] not a valid user");
-    }
-    props.setProperty(DistributionConfig.SECURITY_CLIENT_DHALGO_NAME,
-        algos[prng.nextInt(algos.length)]);
-    if (serverAuthEnabled) {
-      String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/publickeyfile");
-      props.setProperty(HandShake.PUBLIC_KEY_FILE_PROP, keyStoreFile);
-      props.setProperty(HandShake.PUBLIC_KEY_PASSWD_PROP, "gemfire");
-    }
-    return props;
-  }
-
-  @Override
-  public Properties getInvalidCredentials(int index) {
-
-    Properties props = new Properties();
-    props.setProperty(UserPasswordAuthInit.USER_NAME, "invalid" + index);
-    props.setProperty(UserPasswordAuthInit.PASSWORD, "none");
-    props.setProperty(DistributionConfig.SECURITY_CLIENT_DHALGO_NAME,
-        algos[prng.nextInt(algos.length)]);
-    if (serverAuthEnabled) {
-      String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/publickeyfile");
-      props.setProperty(HandShake.PUBLIC_KEY_FILE_PROP, keyStoreFile);
-      props.setProperty(HandShake.PUBLIC_KEY_PASSWD_PROP, "gemfire");
-    }
-    return props;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/templates/security/PKCSCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/PKCSCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/PKCSCredentialGenerator.java
deleted file mode 100644
index fa4bf04..0000000
--- a/gemfire-core/src/test/java/templates/security/PKCSCredentialGenerator.java
+++ /dev/null
@@ -1,110 +0,0 @@
-
-package templates.security;
-
-/*
- * 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.
- */
-
-
-import java.security.Principal;
-import java.security.Provider;
-import java.security.Security;
-import java.util.Properties;
-
-import com.gemstone.gemfire.util.test.TestUtil;
-
-/**
- * @author kneeraj
- * 
- */
-public class PKCSCredentialGenerator extends CredentialGenerator {
-
-  public static String keyStoreDir = getKeyStoreDir();
-
-  public static boolean usesIBMJSSE;
-
-  // Checks if the current JVM uses only IBM JSSE providers.
-  private static boolean usesIBMProviders() {
-    Provider[] providers = Security.getProviders();
-    for (int index = 0; index < providers.length; ++index) {
-      if (!providers[index].getName().toLowerCase().startsWith("ibm")) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private static String getKeyStoreDir() {
-    usesIBMJSSE = usesIBMProviders();
-    if (usesIBMJSSE) {
-      return "/lib/keys/ibm";
-    }
-    else {
-      return "/lib/keys";
-    }
-  }
-
-  public ClassCode classCode() {
-    return ClassCode.PKCS;
-  }
-
-  public String getAuthInit() {
-    return "templates.security.PKCSAuthInit.create";
-  }
-
-  public String getAuthenticator() {
-    return "templates.security.PKCSAuthenticator.create";
-  }
-
-  public Properties getInvalidCredentials(int index) {
-    Properties props = new Properties();
-    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + "/gemfire11.keystore");
-    props.setProperty(PKCSAuthInit.KEYSTORE_FILE_PATH, keyStoreFile);
-    props.setProperty(PKCSAuthInit.KEYSTORE_ALIAS, "gemfire11");
-    props.setProperty(PKCSAuthInit.KEYSTORE_PASSWORD, "gemfire");
-    return props;
-  }
-
-  public Properties getValidCredentials(int index) {
-    Properties props = new Properties();
-    int aliasnum = (index % 10) + 1;
-    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + "/gemfire" + aliasnum + ".keystore");
-    props.setProperty(PKCSAuthInit.KEYSTORE_FILE_PATH, keyStoreFile);
-    props.setProperty(PKCSAuthInit.KEYSTORE_ALIAS, "gemfire" + aliasnum);
-    props.setProperty(PKCSAuthInit.KEYSTORE_PASSWORD, "gemfire");
-    return props;
-  }
-
-  public Properties getValidCredentials(Principal principal) {
-    Properties props = new Properties();
-    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + principal.getName() + ".keystore");
-    props.setProperty(PKCSAuthInit.KEYSTORE_FILE_PATH, keyStoreFile);
-    props.setProperty(PKCSAuthInit.KEYSTORE_ALIAS, principal.getName());
-    props.setProperty(PKCSAuthInit.KEYSTORE_PASSWORD, "gemfire");
-    return props;
-  }
-
-  protected Properties initialize() throws IllegalArgumentException {
-    Properties props = new Properties();
-    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + "/publickeyfile");
-    props.setProperty(PKCSAuthenticator.PUBLIC_KEY_FILE, keyStoreFile);
-    props.setProperty(PKCSAuthenticator.PUBLIC_KEYSTORE_PASSWORD, "gemfire");
-    return props;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/templates/security/SSLCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/SSLCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/SSLCredentialGenerator.java
deleted file mode 100644
index 09367a2..0000000
--- a/gemfire-core/src/test/java/templates/security/SSLCredentialGenerator.java
+++ /dev/null
@@ -1,116 +0,0 @@
-
-package templates.security;
-
-/*
- * 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.
- */
-
-
-import java.io.File;
-import java.io.IOException;
-import java.security.Principal;
-import java.util.Properties;
-
-import com.gemstone.gemfire.security.AuthenticationFailedException;
-
-public class SSLCredentialGenerator extends CredentialGenerator {
-
-  private File findTrustedJKS() {
-    File ssldir = new File(System.getProperty("JTESTS") + "/ssl");
-    return new File(ssldir, "trusted.keystore");
-  }
-
-  private File findUntrustedJKS() {
-    File ssldir = new File(System.getProperty("JTESTS") + "/ssl");
-    return new File(ssldir, "untrusted.keystore");
-  }
-
-  private Properties getValidJavaSSLProperties() {
-    File jks = findTrustedJKS();
-    try {
-      Properties props = new Properties();
-      props.setProperty("javax.net.ssl.trustStore", jks.getCanonicalPath());
-      props.setProperty("javax.net.ssl.trustStorePassword", "password");
-      props.setProperty("javax.net.ssl.keyStore", jks.getCanonicalPath());
-      props.setProperty("javax.net.ssl.keyStorePassword", "password");
-      return props;
-    }
-    catch (IOException ex) {
-      throw new AuthenticationFailedException(
-          "SSL: Exception while opening the key store: " + ex);
-    }
-  }
-
-  private Properties getInvalidJavaSSLProperties() {
-    File jks = findUntrustedJKS();
-    try {
-      Properties props = new Properties();
-      props.setProperty("javax.net.ssl.trustStore", jks.getCanonicalPath());
-      props.setProperty("javax.net.ssl.trustStorePassword", "password");
-      props.setProperty("javax.net.ssl.keyStore", jks.getCanonicalPath());
-      props.setProperty("javax.net.ssl.keyStorePassword", "password");
-      return props;
-    }
-    catch (IOException ex) {
-      throw new AuthenticationFailedException(
-          "SSL: Exception while opening the key store: " + ex);
-    }
-  }
-
-  private Properties getSSLProperties() {
-    Properties props = new Properties();
-    props.setProperty("ssl-enabled", "true");
-    props.setProperty("ssl-require-authentication", "true");
-    props.setProperty("ssl-ciphers", "SSL_RSA_WITH_RC4_128_MD5");
-    props.setProperty("ssl-protocols", "TLSv1");
-    return props;
-  }
-
-  protected Properties initialize() throws IllegalArgumentException {
-    this.javaProps = getValidJavaSSLProperties();
-    return getSSLProperties();
-  }
-
-  public ClassCode classCode() {
-    return ClassCode.SSL;
-  }
-
-  public String getAuthInit() {
-    return null;
-  }
-
-  public String getAuthenticator() {
-    return null;
-  }
-
-  public Properties getValidCredentials(int index) {
-    this.javaProps = getValidJavaSSLProperties();
-    return getSSLProperties();
-  }
-
-  public Properties getValidCredentials(Principal principal) {
-    this.javaProps = getValidJavaSSLProperties();
-    return getSSLProperties();
-  }
-
-  public Properties getInvalidCredentials(int index) {
-    this.javaProps = getInvalidJavaSSLProperties();
-    return getSSLProperties();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/templates/security/UserPasswordWithExtraPropsAuthInit.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/UserPasswordWithExtraPropsAuthInit.java b/gemfire-core/src/test/java/templates/security/UserPasswordWithExtraPropsAuthInit.java
deleted file mode 100644
index 56dbf7d..0000000
--- a/gemfire-core/src/test/java/templates/security/UserPasswordWithExtraPropsAuthInit.java
+++ /dev/null
@@ -1,76 +0,0 @@
-
-package templates.security;
-
-/*
- * 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.
- */
-
-
-import java.util.Properties;
-import java.util.Iterator;
-
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.security.AuthInitialize;
-import com.gemstone.gemfire.security.AuthenticationFailedException;
-
-/**
- * An {@link AuthInitialize} implementation that obtains the user name and
- * password as the credentials from the given set of properties. If 
- * keep-extra-props property exits, it will copy rest of the
- * properties provided in getCredential props argument will also be 
- * copied as new credentials.
- * 
- * @author Soubhik
- * @since 5.5
- */
-public class UserPasswordWithExtraPropsAuthInit extends UserPasswordAuthInit {
-
-  public static final String EXTRA_PROPS = "security-keep-extra-props";
-
-  public static final String SECURITY_PREFIX = "security-";
-  
-  public static AuthInitialize create() {
-    return new UserPasswordWithExtraPropsAuthInit();
-  }
-
-  public UserPasswordWithExtraPropsAuthInit() {
-    super();
-  }
-
-  public Properties getCredentials(Properties props, DistributedMember server,
-      boolean isPeer) throws AuthenticationFailedException {
-
-    Properties newProps = super.getCredentials(props, server, isPeer);
-    String extraProps = props.getProperty(EXTRA_PROPS);
-    if(extraProps != null) {
-    	for(Iterator it = props.keySet().iterator(); it.hasNext();) {
-    		String key = (String)it.next();
-    		if( key.startsWith(SECURITY_PREFIX) && 
-    		    key.equalsIgnoreCase(USER_NAME) == false &&
-    		    key.equalsIgnoreCase(PASSWORD) == false &&
-    		    key.equalsIgnoreCase(EXTRA_PROPS) == false) {
-    			newProps.setProperty(key, props.getProperty(key));
-    		}
-    	}
-    	this.securitylog.fine("got everything and now have: "
-          + newProps.keySet().toString());
-    }
-    return newProps;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/templates/security/XmlAuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/XmlAuthzCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/XmlAuthzCredentialGenerator.java
deleted file mode 100644
index 6b3f171..0000000
--- a/gemfire-core/src/test/java/templates/security/XmlAuthzCredentialGenerator.java
+++ /dev/null
@@ -1,262 +0,0 @@
-
-package templates.security;
-
-/*
- * 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.
- */
-
-
-import java.security.Principal;
-import java.util.HashSet;
-import java.util.Properties;
-import java.util.Set;
-
-import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import com.gemstone.gemfire.util.test.TestUtil;
-
-public class XmlAuthzCredentialGenerator extends AuthzCredentialGenerator {
-
-  private static final String dummyXml = "authz-dummy.xml";
-
-  private static final String ldapXml = "authz-ldap.xml";
-
-  private static final String pkcsXml = "authz-pkcs.xml";
-
-  private static final String sslXml = "authz-ssl.xml";
-
-  private static final String[] QUERY_REGIONS = { "/Portfolios", "/Positions",
-      "/AuthRegion" };
-
-  public static OperationCode[] READER_OPS = { OperationCode.GET,
-      OperationCode.REGISTER_INTEREST, OperationCode.UNREGISTER_INTEREST,
-      OperationCode.KEY_SET, OperationCode.CONTAINS_KEY, OperationCode.EXECUTE_FUNCTION };
-
-  public static OperationCode[] WRITER_OPS = { OperationCode.PUT,
-      OperationCode.DESTROY, OperationCode.INVALIDATE, OperationCode.REGION_CLEAR };
-
-  public static OperationCode[] QUERY_OPS = { OperationCode.QUERY,
-      OperationCode.EXECUTE_CQ, OperationCode.STOP_CQ, OperationCode.CLOSE_CQ };
-
-  private static final byte READER_ROLE = 1;
-
-  private static final byte WRITER_ROLE = 2;
-
-  private static final byte QUERY_ROLE = 3;
-
-  private static final byte ADMIN_ROLE = 4;
-
-  private static Set readerOpsSet;
-
-  private static Set writerOpsSet;
-
-  private static Set queryOpsSet;
-
-  private static Set queryRegionSet;
-
-  static {
-
-    readerOpsSet = new HashSet();
-    for (int index = 0; index < READER_OPS.length; index++) {
-      readerOpsSet.add(READER_OPS[index]);
-    }
-    writerOpsSet = new HashSet();
-    for (int index = 0; index < WRITER_OPS.length; index++) {
-      writerOpsSet.add(WRITER_OPS[index]);
-    }
-    queryOpsSet = new HashSet();
-    for (int index = 0; index < QUERY_OPS.length; index++) {
-      queryOpsSet.add(QUERY_OPS[index]);
-    }
-    queryRegionSet = new HashSet();
-    for (int index = 0; index < QUERY_REGIONS.length; index++) {
-      queryRegionSet.add(QUERY_REGIONS[index]);
-    }
-  }
-
-  public XmlAuthzCredentialGenerator() {
-  }
-
-  protected Properties init() throws IllegalArgumentException {
-
-    Properties sysProps = new Properties();
-    String dirName = "/lib/";
-    if (this.cGen.classCode().isDummy()) {
-      String xmlFilename = TestUtil.getResourcePath(XmlAuthzCredentialGenerator.class, dirName + dummyXml);
-      sysProps.setProperty(XmlAuthorization.DOC_URI_PROP_NAME, xmlFilename);
-    }
-    else if (this.cGen.classCode().isLDAP()) {
-      String xmlFilename = TestUtil.getResourcePath(XmlAuthzCredentialGenerator.class, dirName + ldapXml);
-      sysProps.setProperty(XmlAuthorization.DOC_URI_PROP_NAME, xmlFilename);
-    }
-    // else if (this.cGen.classCode().isPKCS()) {
-    // sysProps
-    // .setProperty(XmlAuthorization.DOC_URI_PROP_NAME, dirName + pkcsXml);
-    // }
-    // else if (this.cGen.classCode().isSSL()) {
-    // sysProps
-    // .setProperty(XmlAuthorization.DOC_URI_PROP_NAME, dirName + sslXml);
-    // }
-    else {
-      throw new IllegalArgumentException(
-          "No XML defined for XmlAuthorization module to work with "
-              + this.cGen.getAuthenticator());
-    }
-    return sysProps;
-  }
-
-  public ClassCode classCode() {
-    return ClassCode.XML;
-  }
-
-  public String getAuthorizationCallback() {
-    return "templates.security.XmlAuthorization.create";
-  }
-
-  private Principal getDummyPrincipal(byte roleType, int index) {
-
-    String[] admins = new String[] { "root", "admin", "administrator" };
-    int numReaders = 3;
-    int numWriters = 3;
-
-    switch (roleType) {
-      case READER_ROLE:
-        return new UsernamePrincipal("reader" + (index % numReaders));
-      case WRITER_ROLE:
-        return new UsernamePrincipal("writer" + (index % numWriters));
-      case QUERY_ROLE:
-        return new UsernamePrincipal("reader" + ((index % 2) + 3));
-      default:
-        return new UsernamePrincipal(admins[index % admins.length]);
-    }
-  }
-
-  private Principal getLdapPrincipal(byte roleType, int index) {
-
-    final String userPrefix = "gemfire";
-    final int[] readerIndices = { 3, 4, 5 };
-    final int[] writerIndices = { 6, 7, 8 };
-    final int[] queryIndices = { 9, 10 };
-    final int[] adminIndices = { 1, 2 };
-
-    switch (roleType) {
-      case READER_ROLE:
-        int readerIndex = readerIndices[index % readerIndices.length];
-        return new UsernamePrincipal(userPrefix + readerIndex);
-      case WRITER_ROLE:
-        int writerIndex = writerIndices[index % writerIndices.length];
-        return new UsernamePrincipal(userPrefix + writerIndex);
-      case QUERY_ROLE:
-        int queryIndex = queryIndices[index % queryIndices.length];
-        return new UsernamePrincipal(userPrefix + queryIndex);
-      default:
-        int adminIndex = adminIndices[index % adminIndices.length];
-        return new UsernamePrincipal(userPrefix + adminIndex);
-    }
-  }
-
-  private byte getRequiredRole(OperationCode[] opCodes, String[] regionNames) {
-
-    byte roleType = ADMIN_ROLE;
-    boolean requiresReader = true;
-    boolean requiresWriter = true;
-    boolean requiresQuery = true;
-
-    for (int opNum = 0; opNum < opCodes.length; opNum++) {
-      OperationCode opCode = opCodes[opNum];
-      if (requiresReader && !readerOpsSet.contains(opCode)) {
-        requiresReader = false;
-      }
-      if (requiresWriter && !writerOpsSet.contains(opCode)) {
-        requiresWriter = false;
-      }
-      if (requiresQuery && !queryOpsSet.contains(opCode)) {
-        requiresQuery = false;
-      }
-    }
-    if (requiresReader) {
-      roleType = READER_ROLE;
-    }
-    else if (requiresWriter) {
-      roleType = WRITER_ROLE;
-    }
-    else if (requiresQuery) {
-      if (regionNames != null && regionNames.length > 0) {
-        for (int index = 0; index < regionNames.length; index++) {
-          String regionName = XmlAuthorization
-              .normalizeRegionName(regionNames[index]);
-          if (requiresQuery && !queryRegionSet.contains(regionName)) {
-            requiresQuery = false;
-            break;
-          }
-        }
-        if (requiresQuery) {
-          roleType = QUERY_ROLE;
-        }
-      }
-    }
-    return roleType;
-  }
-
-  protected Principal getAllowedPrincipal(OperationCode[] opCodes,
-      String[] regionNames, int index) {
-
-    if (this.cGen.classCode().isDummy()) {
-      byte roleType = getRequiredRole(opCodes, regionNames);
-      return getDummyPrincipal(roleType, index);
-    }
-    else if (this.cGen.classCode().isLDAP()) {
-      byte roleType = getRequiredRole(opCodes, regionNames);
-      return getLdapPrincipal(roleType, index);
-    }
-    return null;
-  }
-
-  protected Principal getDisallowedPrincipal(OperationCode[] opCodes,
-      String[] regionNames, int index) {
-
-    byte roleType = getRequiredRole(opCodes, regionNames);
-    byte disallowedRoleType = READER_ROLE;
-    switch (roleType) {
-      case READER_ROLE:
-        disallowedRoleType = WRITER_ROLE;
-        break;
-      case WRITER_ROLE:
-        disallowedRoleType = READER_ROLE;
-        break;
-      case QUERY_ROLE:
-        disallowedRoleType = READER_ROLE;
-        break;
-      case ADMIN_ROLE:
-        disallowedRoleType = READER_ROLE;
-        break;
-    }
-    if (this.cGen.classCode().isDummy()) {
-      return getDummyPrincipal(disallowedRoleType, index);
-    }
-    else if (this.cGen.classCode().isLDAP()) {
-      return getLdapPrincipal(disallowedRoleType, index);
-    }
-    return null;
-  }
-
-  protected int getNumPrincipalTries(OperationCode[] opCodes,
-      String[] regionNames) {
-    return 5;
-  }
-
-}


[02/33] incubator-geode git commit: Disabled SimpleMemoryAllocatorJUnitTest.testClosed until it is fixed

Posted by je...@apache.org.
Disabled SimpleMemoryAllocatorJUnitTest.testClosed until it is fixed

This is causing the test run to crash when it fails.


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

Branch: refs/heads/feature/GEODE-14
Commit: e3d24d77150382f363f334b3e2d6622c6e02e8bc
Parents: af654ca
Author: Sai Boorlagadda <sb...@pivotal.io>
Authored: Fri Dec 18 14:11:12 2015 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Fri Dec 18 16:13:53 2015 -0800

----------------------------------------------------------------------
 .../internal/offheap/SimpleMemoryAllocatorJUnitTest.java       | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e3d24d77/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 1477764..4dda498 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
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -437,6 +438,11 @@ public class SimpleMemoryAllocatorJUnitTest {
     }
   }
 
+  /* This test fails intermittently.
+   * Disabling it until this test case is fixed. GEODE-701
+   * Sonar test coverage job has failed due to this test failure.
+   */
+  @Ignore("Disabled for GEODE-701")
   @Test
   public void testClose() {
     UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024*1024);


[25/33] incubator-geode git commit: GEODE-608: Have rat produce only textual output. Prevents malformed xml

Posted by je...@apache.org.
GEODE-608: Have rat produce only textual output. Prevents malformed xml


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

Branch: refs/heads/feature/GEODE-14
Commit: 05b7ed151e8a218eb4bd492f886dac6176564bc7
Parents: 39790d5
Author: Jens Deppe <jd...@pivotal.io>
Authored: Tue Dec 22 13:14:04 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Tue Dec 22 13:14:04 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/05b7ed15/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index e59b50d..abda034 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -17,6 +17,10 @@
 apply plugin: "org.nosphere.apache.rat"
 
 rat {
+  xmlOutput = false
+  htmlOutput = false
+  plainOutput = true
+
   excludes = [
     // git
     '.git/**',


[26/33] incubator-geode git commit: GEODE-473: Capture status file lines in finally block

Posted by je...@apache.org.
GEODE-473: Capture status file lines in finally block

Change FileProcessController status handler to capture lines read from
status file in finally block.

* Add new UnitTest for LocatorState

* Add new IntegrationTest for FileProcessController


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

Branch: refs/heads/feature/GEODE-14
Commit: 121710cdd9558442ba6ee4ecdaea2fc2172a241e
Parents: 05b7ed1
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Dec 21 13:02:11 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Dec 22 14:26:28 2015 -0800

----------------------------------------------------------------------
 .../internal/process/FileProcessController.java |  39 +++-
 .../distributed/LocatorStateJUnitTest.java      | 208 +++++++++++++++++++
 ...leProcessControllerIntegrationJUnitTest.java | 155 ++++++++++++++
 3 files changed, 393 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/121710cd/gemfire-core/src/main/java/com/gemstone/gemfire/internal/process/FileProcessController.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/process/FileProcessController.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/process/FileProcessController.java
index 471877f..f635bfb 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/process/FileProcessController.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/process/FileProcessController.java
@@ -20,6 +20,7 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -37,28 +38,44 @@ import com.gemstone.gemfire.lang.AttachAPINotFoundException;
  * @author Kirk Lund
  * @since 8.0
  */
-public final class FileProcessController implements ProcessController {
+public class FileProcessController implements ProcessController {
   private static final Logger logger = LogService.getLogger();
 
   public static final String STATUS_TIMEOUT_PROPERTY = "gemfire.FileProcessController.STATUS_TIMEOUT";
-  private final long statusTimeout = Long.getLong(STATUS_TIMEOUT_PROPERTY, 60*1000);
   
+  private final long statusTimeoutMillis;
   private final FileControllerParameters arguments;
   private final int pid;
 
   /**
    * Constructs an instance for controlling a local process.
    * 
-   * @param pid process id identifying the process to attach to
+   * @param arguments details about the controllable process
+   * @param pid process id identifying the process to control
    * 
    * @throws IllegalArgumentException if pid is not a positive integer
    */
   public FileProcessController(final FileControllerParameters arguments, final int pid) {
+    this(arguments, pid, Long.getLong(STATUS_TIMEOUT_PROPERTY, 60*1000), TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Constructs an instance for controlling a local process.
+   * 
+   * @param arguments details about the controllable process
+   * @param pid process id identifying the process to control
+   * @param timeout the timeout that operations must complete within
+   * @param units the units of the timeout
+   * 
+   * @throws IllegalArgumentException if pid is not a positive integer
+   */
+  public FileProcessController(final FileControllerParameters arguments, final int pid, final long timeout, final TimeUnit units) {
     if (pid < 1) {
       throw new IllegalArgumentException("Invalid pid '" + pid + "' specified");
     }
     this.pid = pid;
     this.arguments = arguments;
+    this.statusTimeoutMillis = units.toMillis(timeout);
   }
 
   @Override
@@ -98,14 +115,14 @@ public final class FileProcessController implements ProcessController {
       public void handleRequest() throws IOException {
         // read the statusFile
         final BufferedReader reader = new BufferedReader(new FileReader(statusFile));
+        final StringBuilder lines = new StringBuilder();
         try {
-          final StringBuilder lines = new StringBuilder();
           String line = null;
           while ((line = reader.readLine()) != null) {
             lines.append(line);
           }
-          statusRef.set(lines.toString());
         } finally {
+          statusRef.set(lines.toString());
           reader.close();
         }
       }
@@ -122,8 +139,8 @@ public final class FileProcessController implements ProcessController {
     // if timeout invoke stop and then throw TimeoutException
     final long start = System.currentTimeMillis();
     while (statusFileWatchdog.isAlive()) {
-      Thread.sleep(100);
-      if (System.currentTimeMillis() >= start + this.statusTimeout) {
+      Thread.sleep(10);
+      if (System.currentTimeMillis() >= start + this.statusTimeoutMillis) {
         final TimeoutException te = new TimeoutException("Timed out waiting for process to create " + statusFile);
         try {
           statusFileWatchdog.stop();
@@ -135,7 +152,11 @@ public final class FileProcessController implements ProcessController {
         throw te;
       }
     }
-    assert statusRef.get() != null;
-    return statusRef.get();
+    
+    final String lines = statusRef.get();
+    if (null == lines || lines.trim().isEmpty()) {
+      throw new IllegalStateException("Failed to read status file");
+    }
+    return lines;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/121710cd/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorStateJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorStateJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorStateJUnitTest.java
new file mode 100755
index 0000000..248c39f
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorStateJUnitTest.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import static com.googlecode.catchexception.CatchException.*;
+import static org.assertj.core.api.Assertions.*;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.distributed.AbstractLauncher.ServiceState;
+import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
+import com.gemstone.gemfire.distributed.LocatorLauncher.LocatorState;
+import com.gemstone.gemfire.management.internal.cli.json.GfJsonException;
+import com.gemstone.gemfire.management.internal.cli.json.GfJsonObject;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for LocatorLauncher.LocatorState
+ */
+@Category(UnitTest.class)
+public class LocatorStateJUnitTest {
+
+  @Test
+  public void fromJsonWithEmptyStringThrowsIllegalArgumentException() throws Exception {
+    // given: empty string
+    String emptyString = "";
+    
+    // when: passed to fromJson
+    verifyException(this).fromJson(emptyString);
+    
+    // then: throws IllegalArgumentException with cause of GfJsonException
+    assertThat((Exception)caughtException())
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasCauseInstanceOf(GfJsonException.class);
+    
+    assertThat(caughtException().getCause())
+        .isInstanceOf(GfJsonException.class)
+        .hasNoCause();
+  }
+  
+  @Test
+  public void fromJsonWithWhiteSpaceStringThrowsIllegalArgumentException() throws Exception {
+    // given: white space string
+    String whiteSpaceString = "      ";
+    
+    // when: passed to fromJson
+    verifyException(this).fromJson(whiteSpaceString);
+
+    // then: throws IllegalArgumentException with cause of GfJsonException
+    assertThat((Exception)caughtException())
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasCauseInstanceOf(GfJsonException.class);
+    
+    assertThat(caughtException().getCause())
+        .isInstanceOf(GfJsonException.class)
+        .hasNoCause();
+  }
+  
+  @Test
+  public void fromJsonWithNullStringThrowsNullPointerException() throws Exception {
+    // given: null string
+    String nullString = null;
+    
+    // when: passed to fromJson
+    verifyException(this).fromJson(nullString);
+    
+    // then: throws NullPointerException
+    assertThat((Exception)caughtException())
+        .isInstanceOf(NullPointerException.class)
+        .hasNoCause();
+  }
+  
+  @Test
+  public void fromJsonWithValidJsonStringReturnsLocatorState() throws Exception {
+    // given: valid json string
+    String jsonString = createStatusJson();
+    
+    // when: passed to fromJson
+    LocatorState value = fromJson(jsonString);
+    
+    // then: return valid instance of LocatorState
+    assertThat(value).isInstanceOf(LocatorState.class);
+    
+    assertThat(value.getClasspath()).isEqualTo(getClasspath());
+    assertThat(value.getGemFireVersion()).isEqualTo(getGemFireVersion());
+    assertThat(value.getHost()).isEqualTo(getHost());
+    assertThat(value.getJavaVersion()).isEqualTo(getJavaVersion());
+    assertThat(value.getJvmArguments()).isEqualTo(getJvmArguments());
+    assertThat(value.getServiceLocation()).isEqualTo(getServiceLocation());
+    assertThat(value.getLogFile()).isEqualTo(getLogFile());
+    assertThat(value.getMemberName()).isEqualTo(getMemberName());
+    assertThat(value.getPid()).isEqualTo(getPid());
+    assertThat(value.getPort()).isEqualTo(getPort());
+    assertThat(value.getStatus().getDescription()).isEqualTo(getStatusDescription());
+    assertThat(value.getStatusMessage()).isEqualTo(getStatusMessage());
+    assertThat(value.getTimestamp().getTime()).isEqualTo(getTimestampTime());
+    assertThat(value.getUptime()).isEqualTo(getUptime());
+    assertThat(value.getWorkingDirectory()).isEqualTo(getWorkingDirectory());
+  }
+  
+  protected LocatorState fromJson(final String value) {
+    return LocatorState.fromJson(value);
+  }
+
+  private String classpath = "test_classpath";
+  private String gemFireVersion = "test_gemfireversion";
+  private String host = "test_host";
+  private String javaVersion = "test_javaversion";
+  private String jvmArguments = "test_jvmarguments";
+  private String serviceLocation = "test_location";
+  private String logFile = "test_logfile";
+  private String memberName = "test_membername";
+  private Integer pid = 6396;
+  private String port = "test_port";
+  private String statusDescription = Status.NOT_RESPONDING.getDescription();
+  private String statusMessage = "test_statusmessage";
+  private Long timestampTime = 1450728233024L;
+  private Long uptime = 1629L;
+  private String workingDirectory = "test_workingdirectory";
+  
+  private String getClasspath() {
+    return this.classpath;
+  }
+  private String getGemFireVersion() {
+    return this.gemFireVersion;
+  }
+  private String getHost() {
+    return this.host;
+  }
+  private String getJavaVersion() {
+    return this.javaVersion;
+  }
+  private List<String> getJvmArguments() {
+    List<String> list = new ArrayList<String>();
+    list.add(this.jvmArguments);
+    return list;
+  }
+  private String getServiceLocation() {
+    return this.serviceLocation;
+  }
+  private String getLogFile() {
+    return this.logFile;
+  }
+  private String getMemberName() {
+    return this.memberName;
+  }
+  private Integer getPid() {
+    return this.pid;
+  }
+  private String getPort() {
+    return this.port;
+  }
+  private String getStatusDescription() {
+    return this.statusDescription;
+  }
+  private String getStatusMessage() {
+    return this.statusMessage;
+  }
+  private Long getTimestampTime() {
+    return this.timestampTime;
+  }
+  private Long getUptime() {
+    return this.uptime;
+  }
+  private String getWorkingDirectory() {
+    return this.workingDirectory;
+  }
+
+  private String createStatusJson() {
+    final Map<String, Object> map = new HashMap<String, Object>();
+    map.put(ServiceState.JSON_CLASSPATH, getClasspath());
+    map.put(ServiceState.JSON_GEMFIREVERSION, getGemFireVersion());
+    map.put(ServiceState.JSON_HOST, getHost());
+    map.put(ServiceState.JSON_JAVAVERSION, getJavaVersion());
+    map.put(ServiceState.JSON_JVMARGUMENTS, getJvmArguments());
+    map.put(ServiceState.JSON_LOCATION, getServiceLocation());
+    map.put(ServiceState.JSON_LOGFILE, getLogFile());
+    map.put(ServiceState.JSON_MEMBERNAME, getMemberName());
+    map.put(ServiceState.JSON_PID, getPid());
+    map.put(ServiceState.JSON_PORT, getPort());
+    map.put(ServiceState.JSON_STATUS, getStatusDescription());
+    map.put(ServiceState.JSON_STATUSMESSAGE, getStatusMessage());
+    map.put(ServiceState.JSON_TIMESTAMP, getTimestampTime());
+    map.put(ServiceState.JSON_UPTIME, getUptime());
+    map.put(ServiceState.JSON_WORKINGDIRECTORY, getWorkingDirectory());
+    return new GfJsonObject(map).toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/121710cd/gemfire-core/src/test/java/com/gemstone/gemfire/internal/process/FileProcessControllerIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/process/FileProcessControllerIntegrationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/process/FileProcessControllerIntegrationJUnitTest.java
new file mode 100755
index 0000000..6255af1
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/process/FileProcessControllerIntegrationJUnitTest.java
@@ -0,0 +1,155 @@
+/*
+ * 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.process;
+
+import static com.googlecode.catchexception.CatchException.*;
+import static com.jayway.awaitility.Awaitility.*;
+import static java.util.concurrent.TimeUnit.*;
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+import static org.hamcrest.Matchers.*;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+
+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 org.junit.rules.TestName;
+
+import com.gemstone.gemfire.distributed.LocatorLauncher;
+import com.gemstone.gemfire.distributed.LocatorStateJUnitTest;
+import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
+import com.gemstone.gemfire.distributed.LocatorLauncher.Builder;
+import com.gemstone.gemfire.distributed.LocatorLauncher.LocatorState;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Integration tests for FileProcessController.
+ */
+@Category(IntegrationTest.class)
+public class FileProcessControllerIntegrationJUnitTest {
+  
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  
+  @Rule
+  public TestName testName = new TestName();
+  
+  private ProcessType processType;
+  private ExecutorService executor;
+  
+  @Before
+  public void setUp() throws Exception {
+    this.processType = ProcessType.LOCATOR;
+  }
+  
+  @After
+  public void tearDown() throws Exception {
+    if (this.executor != null) {
+      this.executor.shutdownNow();
+    }
+  }
+  
+  @Test
+  public void statusShouldAwaitTimeoutWhileFileIsEmpty() throws Exception {
+    // given: FileProcessController with empty pidFile
+    int pid = ProcessUtils.identifyPid();
+    File emptyPidFile = this.temporaryFolder.newFile(this.processType.getPidFileName());
+    FileControllerParameters params = mock(FileControllerParameters.class);
+    when(params.getPidFile()).thenReturn(emptyPidFile);
+    when(params.getProcessId()).thenReturn(pid);
+    when(params.getProcessType()).thenReturn(this.processType);
+    when(params.getWorkingDirectory()).thenReturn(this.temporaryFolder.getRoot());
+    
+    FileProcessController controller = new FileProcessController(params, 1, 10, MILLISECONDS);
+    
+    // when
+    verifyException(controller).status();
+
+    // then: we expect TimeoutException to be thrown
+    assertThat((Exception)caughtException())
+            .isInstanceOf(TimeoutException.class)
+            .hasMessageContaining("Timed out waiting for process to create")
+            .hasNoCause();
+  }
+  
+  @Test
+  public void statusShouldReturnJsonFromStatusFile() throws Exception {
+    // given: FileProcessController with pidFile containing real pid
+    int pid = ProcessUtils.identifyPid();
+    File pidFile = this.temporaryFolder.newFile(this.processType.getPidFileName());
+    writeToFile(pidFile, String.valueOf(pid));
+    
+    FileControllerParameters params = mock(FileControllerParameters.class);
+    when(params.getPidFile()).thenReturn(pidFile);
+    when(params.getProcessId()).thenReturn(pid);
+    when(params.getProcessType()).thenReturn(this.processType);
+    when(params.getWorkingDirectory()).thenReturn(this.temporaryFolder.getRoot());
+    
+    FileProcessController controller = new FileProcessController(params, pid, 1, MINUTES);
+    
+    // when: status is called in one thread and json is written to the file
+    AtomicReference<String> status = new AtomicReference<String>();
+    AtomicReference<Exception> exception = new AtomicReference<Exception>();
+    ExecutorService executor = Executors.newSingleThreadExecutor();
+    executor.execute(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          status.set(controller.status());
+        } catch (Exception e) {
+          exception.set(e);
+        }
+      }
+    });
+    
+    // write status
+    String statusJson = generateStatusJson();
+    File statusFile = this.temporaryFolder.newFile(this.processType.getStatusFileName());
+    writeToFile(statusFile, statusJson);
+    
+    // then: returned status should be the json in the file
+    assertThat(exception.get()).isNull();
+    with().pollInterval(10, MILLISECONDS).await().atMost(2, MINUTES).untilAtomic(status, equalTo(statusJson));
+    assertThat(status.get()).isEqualTo(statusJson);
+    System.out.println(statusJson);
+  }
+
+  private static void writeToFile(final File file, final String value) throws IOException {
+    final FileWriter writer = new FileWriter(file);
+    writer.write(value);
+    writer.flush();
+    writer.close();
+  }
+  
+  private static String generateStatusJson() {
+    Builder builder = new Builder();
+    LocatorLauncher defaultLauncher = builder.build();
+    Status status = Status.ONLINE;
+    LocatorState locatorState = new LocatorState(defaultLauncher, status);
+    return locatorState.toJson();
+  }
+}


[33/33] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-14

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


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

Branch: refs/heads/feature/GEODE-14
Commit: f23e6fd727475463b717944ca892cce6390a17b4
Parents: b15ddfd 91b4389
Author: Jens Deppe <jd...@pivotal.io>
Authored: Tue Dec 29 16:09:08 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Tue Dec 29 16:09:08 2015 -0800

----------------------------------------------------------------------
 .gitignore                                      |    7 +-
 gemfire-core/build.gradle                       |   10 +-
 .../cache/query/internal/QueryMonitor.java      |    8 +-
 .../gemfire/distributed/internal/DMStats.java   |   80 +
 .../distributed/internal/DistributionStats.java |  237 +++
 .../distributed/internal/InternalLocator.java   |    2 +-
 .../internal/LonerDistributionManager.java      |   64 +
 .../internal/SharedConfiguration.java           |    1 +
 .../membership/gms/fd/GMSHealthMonitor.java     |   95 +-
 .../gms/mgr/GMSMembershipManager.java           |   24 +-
 .../internal/tcpserver/TcpClient.java           |    2 +-
 .../internal/tcpserver/TcpServer.java           |    2 +-
 .../gemfire/internal/cache/LocalRegion.java     |   45 +-
 .../internal/cache/PartitionedRegion.java       |   75 -
 .../internal/process/FileProcessController.java |   39 +-
 .../management/internal/cli/CliUtil.java        |    2 +-
 .../management/internal/cli/Launcher.java       |    4 +-
 .../internal/cli/commands/ShellCommands.java    |    4 +-
 .../internal/cli/result/TableBuilder.java       |    9 +-
 .../internal/cli/result/TableBuilderHelper.java |   18 +-
 .../management/internal/cli/shell/Gfsh.java     |   15 +-
 .../internal/cli/shell/jline/ANSIBuffer.java    |  433 ++++
 .../internal/cli/shell/jline/ANSIHandler.java   |    5 +-
 .../cli/shell/jline/CygwinMinttyTerminal.java   |  137 +-
 .../internal/cli/shell/jline/GfshHistory.java   |   13 +-
 .../shell/jline/GfshUnsupportedTerminal.java    |    2 +-
 .../internal/cli/util/CLIConsoleBufferUtil.java |    8 +-
 .../gemfire/cache30/SearchAndLoadDUnitTest.java |  177 +-
 .../gemfire/codeAnalysis/decode/cp/Cp.java      |   14 +-
 .../codeAnalysis/decode/cp/CpInvokeDynamic.java |   33 +
 .../codeAnalysis/decode/cp/CpMethodHandle.java  |   33 +
 .../codeAnalysis/decode/cp/CpMethodType.java    |   31 +
 .../AbstractLauncherJUnitTestCase.java          |    2 +-
 .../gemfire/distributed/LocatorDUnitTest.java   |    3 +-
 .../distributed/LocatorStateJUnitTest.java      |  208 ++
 .../ServerLauncherRemoteJUnitTest.java          |    2 +
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |   42 +-
 .../PersistentPartitionedRegionDUnitTest.java   |    6 +
 .../PersistentPartitionedRegionTestBase.java    |    2 +-
 .../offheap/SimpleMemoryAllocatorJUnitTest.java |    6 +
 ...leProcessControllerIntegrationJUnitTest.java |  155 ++
 .../internal/process/PidFileJUnitTest.java      |    2 +-
 .../management/internal/cli/HeadlessGfsh.java   |    8 +-
 .../internal/cli/TableBuilderJUnitTest.java     |  307 ++-
 .../cli/commands/QueueCommandsDUnitTest.java    |    5 +
 .../cli/commands/ShellCommandsDUnitTest.java    |    2 +-
 .../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 ++++++++++++++++++
 .../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 +++
 .../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
 .../src/test/resources/ssl/untrusted.keystore   |  Bin 0 -> 1181 bytes
 gradle/dependency-versions.properties           |    6 +-
 gradle/rat.gradle                               |   10 +
 95 files changed, 11293 insertions(+), 423 deletions(-)
----------------------------------------------------------------------


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

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


[04/33] incubator-geode git commit: GEODE-568: SharedConfiguration region scope needs to be DISTRIBUTED_ACK to make sure update events will fire in the correct order.

Posted by je...@apache.org.
GEODE-568: SharedConfiguration region scope needs to be DISTRIBUTED_ACK to make sure update events will fire in the correct order.

Closes #63


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

Branch: refs/heads/feature/GEODE-14
Commit: 51ce64d81e0f2c9fdf6975b4b23aec5138078e00
Parents: 341e29d
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Dec 18 14:58:35 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Dec 21 08:50:24 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/51ce64d8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/SharedConfiguration.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/SharedConfiguration.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/SharedConfiguration.java
index 4b7da5c..30a1da7 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/SharedConfiguration.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/SharedConfiguration.java
@@ -646,6 +646,7 @@ public class SharedConfiguration {
         regionAttrsFactory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
         regionAttrsFactory.setCacheListener(new ConfigurationChangeListener(this));
         regionAttrsFactory.setDiskStoreName(CLUSTER_CONFIG_DISK_STORE_NAME);
+        regionAttrsFactory.setScope(Scope.DISTRIBUTED_ACK);
         InternalRegionArguments internalArgs = new InternalRegionArguments();
         internalArgs.setIsUsedForMetaRegion(true);
         internalArgs.setMetaRegionWithTransactions(false);


[09/33] incubator-geode git commit: GEM-164: move the security tests in gemfire-test module inside the com.gemstone.gemfire.security packages to the open side.

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
new file mode 100644
index 0000000..ceeb989
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
@@ -0,0 +1,537 @@
+
+package com.gemstone.gemfire.security;
+
+/*
+ * 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.
+ */
+
+
+import java.util.Iterator;
+import java.util.Properties;
+
+import templates.security.AuthzCredentialGenerator;
+import templates.security.CredentialGenerator;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.execute.PRClientServerTestBase;
+import com.gemstone.gemfire.internal.cache.functions.TestFunction;
+
+import dunit.Host;
+
+public class ClientMultiUserAuthzDUnitTest extends ClientAuthorizationTestBase {
+
+  /** constructor */
+  public ClientMultiUserAuthzDUnitTest(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception {
+
+    super.setUp();
+    final Host host = Host.getHost(0);
+    server1 = host.getVM(0);
+    server2 = host.getVM(1);
+    client1 = host.getVM(2);
+    client2 = host.getVM(3);
+
+    server1.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { serverExpectedExceptions });
+    server2.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { serverExpectedExceptions });
+    client2.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { clientExpectedExceptions });
+    SecurityTestUtil.registerExpectedExceptions(clientExpectedExceptions);
+  }
+
+  // Tests with one user authorized to do puts/gets/containsKey/destroys and
+  // another not authorized for the same.
+  public void testOps1() throws Exception {
+    Iterator iter = getDummyGeneratorCombos().iterator();
+    while (iter.hasNext()) {
+      AuthzCredentialGenerator gen = (AuthzCredentialGenerator)iter.next();
+      CredentialGenerator cGen = gen.getCredentialGenerator();
+      Properties extraAuthProps = cGen.getSystemProperties();
+      Properties javaProps = cGen.getJavaProperties();
+      Properties extraAuthzProps = gen.getSystemProperties();
+      String authenticator = cGen.getAuthenticator();
+      String authInit = cGen.getAuthInit();
+      String accessor = gen.getAuthorizationCallback();
+
+      getLogWriter().info("testOps1: Using authinit: " + authInit);
+      getLogWriter().info(
+          "testOps1: Using authenticator: " + authenticator);
+      getLogWriter().info("testOps1: Using accessor: " + accessor);
+
+      // Start servers with all required properties
+      Properties serverProps = buildProperties(authenticator, accessor, false,
+          extraAuthProps, extraAuthzProps);
+      Integer port1 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+      Integer port2 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+      server1.invoke(ClientAuthorizationTestBase.class, "createCacheServer",
+          new Object[] {SecurityTestUtil.getLocatorPort(), port1, serverProps,
+              javaProps});
+      server2.invoke(ClientAuthorizationTestBase.class, "createCacheServer",
+          new Object[] {SecurityTestUtil.getLocatorPort(), port2, serverProps,
+              javaProps});
+
+      if (!prepareClientsForOps(gen, cGen, new OperationCode[] {
+          OperationCode.PUT, OperationCode.PUT}, new OperationCode[] {
+          OperationCode.GET, OperationCode.GET}, javaProps, authInit, port1,
+          port2)) {
+        continue;
+      }
+      verifyPutsGets();
+
+      if (!prepareClientsForOps(gen, cGen, new OperationCode[] {
+          OperationCode.PUT, OperationCode.CONTAINS_KEY}, new OperationCode[] {
+          OperationCode.DESTROY, OperationCode.DESTROY},
+          javaProps, authInit, port1, port2)) {
+        continue;
+      }
+      verifyContainsKeyDestroys();
+
+      if (!prepareClientsForOps(gen, cGen, new OperationCode[] {
+          OperationCode.PUT, OperationCode.CONTAINS_KEY}, new OperationCode[] {
+          OperationCode.INVALIDATE, OperationCode.INVALIDATE},
+          javaProps, authInit, port1, port2)) {
+        continue;
+      }
+      verifyContainsKeyInvalidates();
+
+      if (!prepareClientsForOps(gen, cGen, new OperationCode[] {
+          OperationCode.GET, OperationCode.GET}, new OperationCode[] {
+          OperationCode.REGION_DESTROY, OperationCode.REGION_DESTROY},
+          javaProps, authInit, port1, port2)) {
+        continue;
+      }
+      verifyGetAllInTX();
+      verifyGetAllRegionDestroys();
+    }
+  }
+
+  private boolean prepareClientsForOps(AuthzCredentialGenerator gen,
+      CredentialGenerator cGen, OperationCode[] client1OpCodes,
+      OperationCode[] client2OpCodes, Properties javaProps, String authInit,
+      Integer port1, Integer port2) {
+    return prepareClientsForOps(gen, cGen, client1OpCodes, client2OpCodes,
+        javaProps, authInit, port1, port2, Boolean.TRUE /*
+                                                         * both clients in
+                                                         * multiuser mode
+                                                         */, Boolean.FALSE /* unused */);
+  }
+
+  private boolean prepareClientsForOps(AuthzCredentialGenerator gen,
+      CredentialGenerator cGen, OperationCode[] client1OpCodes,
+      OperationCode[] client2OpCodes, Properties javaProps, String authInit,
+      Integer port1, Integer port2, Boolean bothClientsInMultiuserMode,
+      Boolean allowOp) {
+    // Start client1 with valid/invalid client1OpCodes credentials
+    Properties[] client1Credentials = new Properties[] {
+        gen.getAllowedCredentials(client1OpCodes, new String[] {regionName}, 1),
+        gen.getDisallowedCredentials(new OperationCode[] {client1OpCodes[1]},
+            new String[] {regionName}, 1)};
+    if (client1Credentials[0] == null || client1Credentials[0].size() == 0) {
+      getLogWriter().info(
+          "testOps1: Unable to obtain valid credentials with "
+              + client1OpCodes[0].toString()
+              + " permission; skipping this combination.");
+      return false;
+    }
+    if (client1Credentials[1] == null || client1Credentials[1].size() == 0) {
+      getLogWriter().info(
+          "testOps1: Unable to obtain valid credentials with no "
+              + client1OpCodes[0].toString()
+              + " permission; skipping this combination.");
+      return false;
+    }
+    javaProps = cGen.getJavaProperties();
+    getLogWriter().info(
+        "testOps1: For first client credentials: " + client1Credentials[0]
+            + "\n" + client1Credentials[1]);
+    client1.invoke(SecurityTestUtil.class, "createCacheClientForMultiUserMode",
+        new Object[] {Integer.valueOf(2), authInit, client1Credentials,
+            javaProps, new Integer[] {port1, port2}, null, Boolean.FALSE,
+            SecurityTestUtil.NO_EXCEPTION});
+
+    // Start client2 with valid/invalid client2OpCodes credentials
+    Properties[] client2Credentials = new Properties[] {
+        gen.getAllowedCredentials(client2OpCodes,
+            new String[] {regionName}, 2),
+        gen.getDisallowedCredentials(client2OpCodes,
+            new String[] {regionName}, 9)};
+    if (client2Credentials[0] == null || client2Credentials[0].size() == 0) {
+      getLogWriter().info(
+          "testOps1: Unable to obtain valid credentials with "
+              + client2OpCodes[0].toString()
+              + " permission; skipping this combination.");
+      return false;
+    }
+    if (client2Credentials[1] == null || client2Credentials[1].size() == 0) {
+      getLogWriter().info(
+          "testOps1: Unable to obtain valid credentials with no "
+              + client2OpCodes[0].toString()
+              + " permission; skipping this combination.");
+      return false;
+    }
+    javaProps = cGen.getJavaProperties();
+    getLogWriter().info(
+        "testOps1: For second client credentials: " + client2Credentials[0]
+            + "\n" + client2Credentials[1]);
+    if (bothClientsInMultiuserMode) {
+      client2.invoke(SecurityTestUtil.class,
+          "createCacheClientForMultiUserMode", new Object[] {
+              Integer.valueOf(2), authInit, client2Credentials, javaProps,
+              new Integer[] {port1, port2}, null, Boolean.FALSE,
+              SecurityTestUtil.NO_EXCEPTION});
+    } else {
+      int credentialsIndex = allowOp ? 0 : 1;
+      client2.invoke(SecurityTestUtil.class, "createCacheClient", new Object[] {
+          authInit, client2Credentials[credentialsIndex], javaProps,
+          new Integer[] {port1, port2}, null, Boolean.FALSE, "false",
+          SecurityTestUtil.NO_EXCEPTION});
+    }
+    return true;
+  }
+
+  private void verifyPutsGets() throws Exception {
+    verifyPutsGets(true, false /*unused */);
+  }
+
+  private void verifyPutsGets(Boolean isMultiuser, Boolean opAllowed)
+      throws Exception {
+    // Perform some put operations from client1
+    client1.invoke(SecurityTestUtil.class, "doMultiUserPuts", new Object[] {
+        Integer.valueOf(2),
+        Integer.valueOf(2),
+        new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+            SecurityTestUtil.NOTAUTHZ_EXCEPTION}});
+
+    // Verify that the gets succeed/fail
+    if (isMultiuser) {
+    client2.invoke(SecurityTestUtil.class, "doMultiUserGets", new Object[] {
+        Integer.valueOf(2),
+        Integer.valueOf(2),
+        new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+            SecurityTestUtil.NOTAUTHZ_EXCEPTION}});
+    } else {
+      int expectedResult = (opAllowed) ? SecurityTestUtil.NO_EXCEPTION
+          : SecurityTestUtil.NOTAUTHZ_EXCEPTION;
+      client2.invoke(SecurityTestUtil.class, "doMultiUserGets", new Object[] {
+          Integer.valueOf(1), Integer.valueOf(1),
+          new Integer[] {expectedResult}});
+    }
+  }
+
+  private void verifyContainsKeyDestroys() throws Exception {
+    verifyContainsKeyDestroys(true, false /* unused */);
+  }
+
+  private void verifyContainsKeyDestroys(Boolean isMultiuser, Boolean opAllowed)
+      throws Exception {
+    // Do puts before verifying containsKey
+    client1.invoke(SecurityTestUtil.class, "doMultiUserPuts", new Object[] {
+        Integer.valueOf(2),
+        Integer.valueOf(2),
+        new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+            SecurityTestUtil.NO_EXCEPTION}});
+    client1.invoke(SecurityTestUtil.class, "doMultiUserContainsKeys",
+        new Object[] {
+            Integer.valueOf(1),
+            Integer.valueOf(2),
+            new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+                SecurityTestUtil.NOTAUTHZ_EXCEPTION},
+            new Boolean[] {Boolean.TRUE, Boolean.FALSE}});
+
+    // Verify that the destroys succeed/fail
+    if (isMultiuser) {
+      client2.invoke(SecurityTestUtil.class, "doMultiUserDestroys",
+          new Object[] {
+              Integer.valueOf(2),
+              Integer.valueOf(2),
+              new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+                  SecurityTestUtil.NOTAUTHZ_EXCEPTION}});
+    } else {
+      int expectedResult = (opAllowed) ? SecurityTestUtil.NO_EXCEPTION
+          : SecurityTestUtil.NOTAUTHZ_EXCEPTION;
+      client2.invoke(SecurityTestUtil.class, "doMultiUserDestroys",
+          new Object[] {Integer.valueOf(1), Integer.valueOf(1),
+              new Integer[] {expectedResult}});
+    }
+  }
+
+  private void verifyContainsKeyInvalidates() throws Exception {
+    verifyContainsKeyInvalidates(true, false /* unused */);
+  }
+
+  private void verifyContainsKeyInvalidates(Boolean isMultiuser, Boolean opAllowed)
+      throws Exception {
+    // Do puts before verifying containsKey
+    client1.invoke(SecurityTestUtil.class, "doMultiUserPuts", new Object[] {
+        Integer.valueOf(2),
+        Integer.valueOf(2),
+        new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+            SecurityTestUtil.NO_EXCEPTION}});
+    client1.invoke(SecurityTestUtil.class, "doMultiUserContainsKeys",
+        new Object[] {
+            Integer.valueOf(1),
+            Integer.valueOf(2),
+            new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+                SecurityTestUtil.NOTAUTHZ_EXCEPTION},
+            new Boolean[] {Boolean.TRUE, Boolean.FALSE}});
+
+    // Verify that the invalidates succeed/fail
+    if (isMultiuser) {
+      client2.invoke(SecurityTestUtil.class, "doMultiUserInvalidates",
+          new Object[] {
+              Integer.valueOf(2),
+              Integer.valueOf(2),
+              new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+                  SecurityTestUtil.NOTAUTHZ_EXCEPTION}});
+    } else {
+      int expectedResult = (opAllowed) ? SecurityTestUtil.NO_EXCEPTION
+          : SecurityTestUtil.NOTAUTHZ_EXCEPTION;
+      client2.invoke(SecurityTestUtil.class, "doMultiUserInvalidates",
+          new Object[] {Integer.valueOf(1), Integer.valueOf(1),
+              new Integer[] {expectedResult}});
+    }
+  }
+
+  private void verifyGetAllInTX() {
+    server1.invoke(ClientMultiUserAuthzDUnitTest.class, "doPuts");
+    client1.invoke(SecurityTestUtil.class, "doMultiUserGetAll", new Object[] {
+      Integer.valueOf(2),
+      new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+          SecurityTestUtil.NOTAUTHZ_EXCEPTION}, Boolean.TRUE/*use TX*/});
+  }
+
+  private void verifyGetAllRegionDestroys() {
+    server1.invoke(ClientMultiUserAuthzDUnitTest.class, "doPuts");
+    client1.invoke(SecurityTestUtil.class, "doMultiUserGetAll", new Object[] {
+      Integer.valueOf(2),
+      new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+          SecurityTestUtil.NOTAUTHZ_EXCEPTION}});
+
+    // Verify that the region destroys succeed/fail
+    client2.invoke(SecurityTestUtil.class, "doMultiUserRegionDestroys",
+        new Object[] {
+            Integer.valueOf(2),
+            new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+                SecurityTestUtil.NOTAUTHZ_EXCEPTION}});
+  }
+  
+  public static void doPuts() {
+    Region region = GemFireCacheImpl.getInstance().getRegion(SecurityTestUtil.regionName);
+    region.put("key1", "value1");
+    region.put("key2", "value2");
+  }
+
+  // Test query/function execute
+  public void testOps2() throws Exception {
+      AuthzCredentialGenerator gen = getXmlAuthzGenerator();
+      CredentialGenerator cGen = gen.getCredentialGenerator();
+      Properties extraAuthProps = cGen.getSystemProperties();
+      Properties javaProps = cGen.getJavaProperties();
+      Properties extraAuthzProps = gen.getSystemProperties();
+      String authenticator = cGen.getAuthenticator();
+      String authInit = cGen.getAuthInit();
+      String accessor = gen.getAuthorizationCallback();
+
+      getLogWriter().info("testOps2: Using authinit: " + authInit);
+      getLogWriter().info("testOps2: Using authenticator: " + authenticator);
+      getLogWriter().info("testOps2: Using accessor: " + accessor);
+
+      // Start servers with all required properties
+      Properties serverProps = buildProperties(authenticator, accessor, false,
+          extraAuthProps, extraAuthzProps);
+      Integer port1 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+      Integer port2 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+      server1.invoke(ClientAuthorizationTestBase.class, "createCacheServer",
+          new Object[] {SecurityTestUtil.getLocatorPort(), port1, serverProps,
+              javaProps});
+      server2.invoke(ClientAuthorizationTestBase.class, "createCacheServer",
+          new Object[] {SecurityTestUtil.getLocatorPort(), port2, serverProps,
+              javaProps});
+
+      // Start client1 with valid/invalid QUERY credentials
+      Properties[] client1Credentials = new Properties[] {
+          gen.getAllowedCredentials(
+                  new OperationCode[] {OperationCode.PUT, OperationCode.QUERY},
+                  new String[] {regionName},
+                  1),
+          gen.getDisallowedCredentials(
+                  new OperationCode[] {OperationCode.PUT, OperationCode.QUERY},
+                  new String[] {regionName},
+                  1)
+      };
+
+      javaProps = cGen.getJavaProperties();
+      getLogWriter().info(
+          "testOps2: For first client credentials: " + client1Credentials[0]
+              + "\n" + client1Credentials[1]);
+      client1.invoke(SecurityTestUtil.class,
+          "createCacheClientForMultiUserMode", new Object[] {
+              Integer.valueOf(2), authInit, client1Credentials, javaProps,
+              new Integer[] {port1, port2}, null, Boolean.FALSE,
+              SecurityTestUtil.NO_EXCEPTION});
+
+      // Start client2 with valid/invalid EXECUTE_FUNCTION credentials
+      Properties[] client2Credentials = new Properties[] {
+          gen.getAllowedCredentials(new OperationCode[] {OperationCode.EXECUTE_FUNCTION},
+              new String[] {regionName}, 2),
+          gen.getDisallowedCredentials(new OperationCode[] {OperationCode.EXECUTE_FUNCTION},
+              new String[] {regionName}, 9)};
+
+      javaProps = cGen.getJavaProperties();
+      getLogWriter().info(
+          "testOps2: For second client credentials: " + client2Credentials[0]
+              + "\n" + client2Credentials[1]);
+      client2.invoke(SecurityTestUtil.class,
+          "createCacheClientForMultiUserMode", new Object[] {
+              Integer.valueOf(2), authInit, client2Credentials, javaProps,
+              new Integer[] {port1, port2}, null, Boolean.FALSE,
+              SecurityTestUtil.NO_EXCEPTION});
+      Function function = new TestFunction(true,TestFunction.TEST_FUNCTION1);
+      server1.invoke(PRClientServerTestBase.class,
+          "registerFunction", new Object []{function});
+      
+      server2.invoke(PRClientServerTestBase.class,
+          "registerFunction", new Object []{function});
+      
+      // Perform some put operations before verifying queries
+      client1.invoke(SecurityTestUtil.class, "doMultiUserPuts", new Object[] {
+          Integer.valueOf(4),
+          Integer.valueOf(2),
+          new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+              SecurityTestUtil.NOTAUTHZ_EXCEPTION}});
+      client1.invoke(SecurityTestUtil.class, "doMultiUserQueries",
+          new Object[] {
+              Integer.valueOf(2),
+              new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+                  SecurityTestUtil.NOTAUTHZ_EXCEPTION}, Integer.valueOf(4)});
+      client1.invoke(SecurityTestUtil.class, "doMultiUserQueryExecute",
+          new Object[] {
+              Integer.valueOf(2),
+              new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+                  SecurityTestUtil.NOTAUTHZ_EXCEPTION}, Integer.valueOf(4)});
+
+      // Verify that the FE succeeds/fails
+      client2.invoke(SecurityTestUtil.class, "doMultiUserFE", new Object[] {
+          Integer.valueOf(2),
+          function,
+          new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+              SecurityTestUtil.NOTAUTHZ_EXCEPTION}, new Object[] {null, null},
+          Boolean.FALSE});
+
+      // Failover
+      server1.invoke(SecurityTestUtil.class, "closeCache");
+      Thread.sleep(2000);
+
+      client1.invoke(SecurityTestUtil.class, "doMultiUserPuts", new Object[] {
+          Integer.valueOf(4),
+          Integer.valueOf(2),
+          new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+              SecurityTestUtil.NOTAUTHZ_EXCEPTION}});
+
+      client1.invoke(SecurityTestUtil.class, "doMultiUserQueries",
+          new Object[] {
+              Integer.valueOf(2),
+              new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+                  SecurityTestUtil.NOTAUTHZ_EXCEPTION}, Integer.valueOf(4)});
+      client1.invoke(SecurityTestUtil.class, "doMultiUserQueryExecute",
+          new Object[] {
+              Integer.valueOf(2),
+              new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+                  SecurityTestUtil.NOTAUTHZ_EXCEPTION}, Integer.valueOf(4)});
+
+      // Verify that the FE succeeds/fails
+      client2.invoke(SecurityTestUtil.class, "doMultiUserFE", new Object[] {
+          Integer.valueOf(2),
+          function,
+          new Integer[] {SecurityTestUtil.NO_EXCEPTION,
+              SecurityTestUtil.NOTAUTHZ_EXCEPTION}, new Object[] {null, null},
+          Boolean.TRUE});
+
+
+  }
+
+  public void testOpsWithClientsInDifferentModes() throws Exception {
+    Iterator iter = getDummyGeneratorCombos().iterator();
+    while (iter.hasNext()) {
+      AuthzCredentialGenerator gen = (AuthzCredentialGenerator)iter.next();
+      CredentialGenerator cGen = gen.getCredentialGenerator();
+      Properties extraAuthProps = cGen.getSystemProperties();
+      Properties javaProps = cGen.getJavaProperties();
+      Properties extraAuthzProps = gen.getSystemProperties();
+      String authenticator = cGen.getAuthenticator();
+      String authInit = cGen.getAuthInit();
+      String accessor = gen.getAuthorizationCallback();
+
+      getLogWriter().info("testOpsWithClientsInDifferentModes: Using authinit: " + authInit);
+      getLogWriter().info(
+          "testOpsWithClientsInDifferentModes: Using authenticator: " + authenticator);
+      getLogWriter().info("testOpsWithClientsInDifferentModes: Using accessor: " + accessor);
+
+      // Start servers with all required properties
+      Properties serverProps = buildProperties(authenticator, accessor, false,
+          extraAuthProps, extraAuthzProps);
+      Integer port1 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+      Integer port2 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+      server1.invoke(ClientAuthorizationTestBase.class, "createCacheServer",
+          new Object[] {SecurityTestUtil.getLocatorPort(), port1, serverProps,
+              javaProps});
+      server2.invoke(ClientAuthorizationTestBase.class, "createCacheServer",
+          new Object[] {SecurityTestUtil.getLocatorPort(), port2, serverProps,
+              javaProps});
+
+      if (!prepareClientsForOps(gen, cGen, new OperationCode[] {
+          OperationCode.PUT, OperationCode.PUT}, new OperationCode[] {
+          OperationCode.GET, OperationCode.GET}, javaProps, authInit, port1,
+          port2, Boolean.FALSE, Boolean.TRUE)) {
+        continue;
+      }
+      verifyPutsGets(false, true);
+
+      if (!prepareClientsForOps(gen, cGen, new OperationCode[] {
+          OperationCode.PUT, OperationCode.CONTAINS_KEY}, new OperationCode[] {
+          OperationCode.DESTROY, OperationCode.DESTROY},
+          javaProps, authInit, port1, port2, Boolean.FALSE, Boolean.FALSE)) {
+        continue;
+      }
+      verifyContainsKeyDestroys(false, false);
+    }
+  }
+
+  // End Region: Tests
+
+  public void tearDown2() throws Exception {
+
+    super.tearDown2();
+    // close the clients first
+    client1.invoke(SecurityTestUtil.class, "closeCache");
+    client2.invoke(SecurityTestUtil.class, "closeCache");
+    SecurityTestUtil.closeCache();
+    // then close the servers
+    server1.invoke(SecurityTestUtil.class, "closeCache");
+    server2.invoke(SecurityTestUtil.class, "closeCache");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
new file mode 100644
index 0000000..16573c2
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
@@ -0,0 +1,336 @@
+
+package com.gemstone.gemfire.security;
+
+/*
+ * 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.
+ */
+
+
+import java.util.Properties;
+
+import templates.security.AuthzCredentialGenerator;
+import templates.security.CredentialGenerator;
+
+import com.gemstone.gemfire.DeltaTestImpl;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.client.NoAvailableServersException;
+import com.gemstone.gemfire.cache.client.ServerConnectivityException;
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.internal.cache.PartitionedRegionLocalMaxMemoryDUnitTest.TestObject1;
+
+import dunit.Host;
+
+/**
+ * @since 6.1
+ */
+public class DeltaClientAuthorizationDUnitTest extends
+    ClientAuthorizationTestBase {
+
+  protected static final DeltaTestImpl[] deltas = new DeltaTestImpl[8];
+
+  static {
+    for (int i = 0; i < 8; i++) {
+      deltas[i] = new DeltaTestImpl(0, "0", new Double(0), new byte[0],
+          new TestObject1("0", 0));
+    }
+    deltas[1].setIntVar(5);
+    deltas[2].setIntVar(5);
+    deltas[3].setIntVar(5);
+    deltas[4].setIntVar(5);
+    deltas[5].setIntVar(5);
+    deltas[6].setIntVar(5);
+    deltas[7].setIntVar(5);
+
+    deltas[2].resetDeltaStatus();
+    deltas[2].setByteArr(new byte[] { 1, 2, 3, 4, 5 });
+    deltas[3].setByteArr(new byte[] { 1, 2, 3, 4, 5 });
+    deltas[4].setByteArr(new byte[] { 1, 2, 3, 4, 5 });
+    deltas[5].setByteArr(new byte[] { 1, 2, 3, 4, 5 });
+    //deltas[6].setByteArr(new byte[] { 1, 2, 3, 4, 5 });
+    //deltas[7].setByteArr(new byte[] { 1, 2, 3, 4, 5 });
+
+    deltas[3].resetDeltaStatus();
+    deltas[3].setDoubleVar(new Double(5));
+    deltas[4].setDoubleVar(new Double(5));
+    deltas[5].setDoubleVar(new Double(5));
+    deltas[6].setDoubleVar(new Double(5));
+    deltas[7].setDoubleVar(new Double(5));
+
+    deltas[4].resetDeltaStatus();
+    deltas[4].setStr("str changed");
+    deltas[5].setStr("str changed");
+    deltas[6].setStr("str changed");
+    //deltas[7].setStr("str changed");
+
+    deltas[5].resetDeltaStatus();
+    deltas[5].setIntVar(100);
+    deltas[5].setTestObj(new TestObject1("CHANGED", 100));
+    deltas[6].setTestObj(new TestObject1("CHANGED", 100));
+    deltas[7].setTestObj(new TestObject1("CHANGED", 100));
+
+    deltas[6].resetDeltaStatus();
+    deltas[6].setByteArr(new byte[] { 1, 2, 3 });
+    deltas[7].setByteArr(new byte[] { 1, 2, 3 });
+
+    deltas[7].resetDeltaStatus();
+    deltas[7].setStr("delta string");
+    
+  }
+
+  /** constructor */
+  public DeltaClientAuthorizationDUnitTest(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception {
+    super.setUp();
+    final Host host = Host.getHost(0);
+    server1 = host.getVM(0);
+    server2 = host.getVM(1);
+    client1 = host.getVM(2);
+    client2 = host.getVM(3);
+
+    server1.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { serverExpectedExceptions });
+    server2.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { serverExpectedExceptions });
+    client2.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { clientExpectedExceptions });
+    SecurityTestUtil.registerExpectedExceptions(clientExpectedExceptions);
+  }
+
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+    // close the clients first
+    client1.invoke(SecurityTestUtil.class, "closeCache");
+    client2.invoke(SecurityTestUtil.class, "closeCache");
+    SecurityTestUtil.closeCache();
+    // then close the servers
+    server1.invoke(SecurityTestUtil.class, "closeCache");
+    server2.invoke(SecurityTestUtil.class, "closeCache");
+  }
+
+  public void testAllowPutsGets() throws Exception {
+      AuthzCredentialGenerator gen = this.getXmlAuthzGenerator();
+      CredentialGenerator cGen = gen.getCredentialGenerator();
+      Properties extraAuthProps = cGen.getSystemProperties();
+      Properties javaProps = cGen.getJavaProperties();
+      Properties extraAuthzProps = gen.getSystemProperties();
+      String authenticator = cGen.getAuthenticator();
+      String authInit = cGen.getAuthInit();
+      String accessor = gen.getAuthorizationCallback();
+
+      getLogWriter().info("testAllowPutsGets: Using authinit: " + authInit);
+      getLogWriter().info(
+          "testAllowPutsGets: Using authenticator: " + authenticator);
+      getLogWriter().info("testAllowPutsGets: Using accessor: " + accessor);
+
+      // Start servers with all required properties
+      Properties serverProps = buildProperties(authenticator, accessor, false,
+          extraAuthProps, extraAuthzProps);
+      Integer port1 = ((Integer)server1.invoke(
+          ClientAuthorizationTestBase.class, "createCacheServer", new Object[] {
+              SecurityTestUtil.getLocatorPort(), serverProps, javaProps }));
+      Integer port2 = ((Integer)server2.invoke(
+          ClientAuthorizationTestBase.class, "createCacheServer", new Object[] {
+              SecurityTestUtil.getLocatorPort(), serverProps, javaProps }));
+
+      // Start client1 with valid CREATE credentials
+      Properties createCredentials = gen.getAllowedCredentials(
+          new OperationCode[] { OperationCode.PUT },
+          new String[] { regionName }, 1);
+      javaProps = cGen.getJavaProperties();
+      getLogWriter().info(
+          "testAllowPutsGets: For first client credentials: "
+              + createCredentials);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, createCredentials, javaProps, port1, port2,
+              null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Start client2 with valid GET credentials
+      Properties getCredentials = gen.getAllowedCredentials(
+          new OperationCode[] { OperationCode.GET },
+          new String[] { regionName }, 2);
+      javaProps = cGen.getJavaProperties();
+      getLogWriter()
+          .info(
+              "testAllowPutsGets: For second client credentials: "
+                  + getCredentials);
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, getCredentials, javaProps, port1, port2,
+              null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Perform some put operations from client1
+      client1.invoke(DeltaClientAuthorizationDUnitTest.class, "doPuts", new Object[] {
+          new Integer(2), new Integer(SecurityTestUtil.NO_EXCEPTION), Boolean.FALSE });
+      Thread.sleep(5000);
+      assertTrue("Delta feature NOT used", (Boolean)client1.invoke(DeltaTestImpl.class, "toDeltaFeatureUsed"));
+
+      // Verify that the gets succeed
+      client2.invoke(DeltaClientAuthorizationDUnitTest.class, "doGets", new Object[] {
+          new Integer(2), new Integer(SecurityTestUtil.NO_EXCEPTION), Boolean.FALSE  });
+  }
+
+  public static void doPuts(Integer num, Integer expectedResult,
+      boolean newVals) {
+
+    assertTrue(num.intValue() <= SecurityTestUtil.keys.length);
+    Region region = null;
+    try {
+      region = SecurityTestUtil.getCache().getRegion(regionName);
+      assertNotNull(region);
+    }
+    catch (Exception ex) {
+      if (expectedResult.intValue() == SecurityTestUtil.OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing puts: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when doing puts", ex);
+      }
+    }
+    for (int index = 0; index < num.intValue(); ++index) {
+      region.put(SecurityTestUtil.keys[index], deltas[0]);
+    }
+    for (int index = 0; index < num.intValue(); ++index) {
+      try {
+        region.put(SecurityTestUtil.keys[index], deltas[index]);
+        if (expectedResult.intValue() != SecurityTestUtil.NO_EXCEPTION) {
+          fail("Expected a NotAuthorizedException while doing puts");
+        }
+      }
+      catch (NoAvailableServersException ex) {
+        if (expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+          getLogWriter().info(
+              "Got expected NoAvailableServers when doing puts: "
+                  + ex.getCause());
+          continue;
+        }
+        else {
+          fail("Got unexpected exception when doing puts", ex);
+        }
+      }
+      catch (ServerConnectivityException ex) {
+        if ((expectedResult.intValue() == SecurityTestUtil.NOTAUTHZ_EXCEPTION)
+            && (ex.getCause() instanceof NotAuthorizedException)) {
+          getLogWriter().info(
+              "Got expected NotAuthorizedException when doing puts: "
+                  + ex.getCause());
+          continue;
+        }
+        if ((expectedResult.intValue() == SecurityTestUtil.AUTHREQ_EXCEPTION)
+            && (ex.getCause() instanceof AuthenticationRequiredException)) {
+          getLogWriter().info(
+              "Got expected AuthenticationRequiredException when doing puts: "
+                  + ex.getCause());
+          continue;
+        }
+        if ((expectedResult.intValue() == SecurityTestUtil.AUTHFAIL_EXCEPTION)
+            && (ex.getCause() instanceof AuthenticationFailedException)) {
+          getLogWriter().info(
+              "Got expected AuthenticationFailedException when doing puts: "
+                  + ex.getCause());
+          continue;
+        }
+        else if (expectedResult.intValue() == SecurityTestUtil.OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing puts: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing puts", ex);
+        }
+      }
+      catch (Exception ex) {
+        if (expectedResult.intValue() == SecurityTestUtil.OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing puts: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing puts", ex);
+        }
+      }
+    }
+  }
+
+  public static void doGets(Integer num, Integer expectedResult,
+      boolean newVals) {
+
+    assertTrue(num.intValue() <= SecurityTestUtil.keys.length);
+    Region region = null;
+    try {
+      region = SecurityTestUtil.getCache().getRegion(regionName);
+      assertNotNull(region);
+    }
+    catch (Exception ex) {
+      if (expectedResult.intValue() == SecurityTestUtil.OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing gets: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when doing gets", ex);
+      }
+    }
+    for (int index = 0; index < num.intValue(); ++index) {
+      Object value = null;
+      try {
+        try {
+          region.localInvalidate(SecurityTestUtil.keys[index]);
+        }
+        catch (Exception ex) {
+        }
+        value = region.get(SecurityTestUtil.keys[index]);
+        if (expectedResult.intValue() != SecurityTestUtil.NO_EXCEPTION) {
+          fail("Expected a NotAuthorizedException while doing gets");
+        }
+      }
+      catch(NoAvailableServersException ex) {
+        if(expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+          getLogWriter().info(
+              "Got expected NoAvailableServers when doing puts: "
+              + ex.getCause());
+          continue;
+        }
+        else {
+          fail("Got unexpected exception when doing puts", ex);
+        }
+      }
+      catch (ServerConnectivityException ex) {
+        if ((expectedResult.intValue() == SecurityTestUtil.NOTAUTHZ_EXCEPTION)
+            && (ex.getCause() instanceof NotAuthorizedException)) {
+          getLogWriter().info(
+              "Got expected NotAuthorizedException when doing gets: "
+                  + ex.getCause());
+          continue;
+        }
+        else if (expectedResult.intValue() == SecurityTestUtil.OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing gets: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing gets", ex);
+        }
+      }
+      catch (Exception ex) {
+        if (expectedResult.intValue() == SecurityTestUtil.OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing gets: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing gets", ex);
+        }
+      }
+      assertNotNull(value);
+      assertEquals(deltas[index], value);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
new file mode 100644
index 0000000..7cfa23d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
@@ -0,0 +1,541 @@
+
+package com.gemstone.gemfire.security;
+
+/*
+ * 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.
+ */
+
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+
+import templates.security.AuthzCredentialGenerator;
+import templates.security.CredentialGenerator;
+
+import com.gemstone.gemfire.DeltaTestImpl;
+import com.gemstone.gemfire.cache.InterestResultPolicy;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.client.ServerConnectivityException;
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.cache.query.CqException;
+import com.gemstone.gemfire.cache.query.QueryInvocationTargetException;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.util.Callable;
+
+import dunit.Host;
+import dunit.VM;
+
+/**
+ * @since 6.1
+ * 
+ */
+public class DeltaClientPostAuthorizationDUnitTest extends
+    ClientAuthorizationTestBase {
+  private static final int PAUSE = 5 * 1000;
+
+  /** constructor */
+  public DeltaClientPostAuthorizationDUnitTest(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception {
+
+    super.setUp();
+    final Host host = Host.getHost(0);
+    server1 = host.getVM(0);
+    server2 = host.getVM(1);
+    client1 = host.getVM(2);
+    client2 = host.getVM(3);
+
+    server1.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { serverExpectedExceptions });
+    server2.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { serverExpectedExceptions });
+    client2.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { clientExpectedExceptions });
+    SecurityTestUtil.registerExpectedExceptions(clientExpectedExceptions);
+  }
+
+  public void tearDown2() throws Exception {
+
+    super.tearDown2();
+    // close the clients first
+    client1.invoke(SecurityTestUtil.class, "closeCache");
+    client2.invoke(SecurityTestUtil.class, "closeCache");
+    SecurityTestUtil.closeCache();
+    // then close the servers
+    server1.invoke(SecurityTestUtil.class, "closeCache");
+    server2.invoke(SecurityTestUtil.class, "closeCache");
+  }
+
+  public void testPutPostOpNotifications() throws Exception {
+    addExpectedException("Unexpected IOException");
+    addExpectedException("SocketException");
+
+    OperationWithAction[] allOps = {
+        // Test CREATE and verify with a GET
+        new OperationWithAction(OperationCode.REGISTER_INTEREST,
+            OperationCode.GET, 2, OpFlags.USE_REGEX
+                | OpFlags.REGISTER_POLICY_NONE, 8),
+        new OperationWithAction(OperationCode.REGISTER_INTEREST,
+            OperationCode.GET, 3, OpFlags.USE_REGEX
+                | OpFlags.REGISTER_POLICY_NONE | OpFlags.USE_NOTAUTHZ, 8),
+        new OperationWithAction(OperationCode.PUT),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP, 4),
+        new OperationWithAction(OperationCode.GET, 3, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP | OpFlags.CHECK_FAIL, 4),
+
+        // OPBLOCK_END indicates end of an operation block that needs to
+        // be executed on each server when doing failover
+        OperationWithAction.OPBLOCK_END,
+
+        // Test UPDATE and verify with a GET
+        new OperationWithAction(OperationCode.REGISTER_INTEREST,
+            OperationCode.GET, 2, OpFlags.USE_REGEX
+                | OpFlags.REGISTER_POLICY_NONE, 8),
+        new OperationWithAction(OperationCode.REGISTER_INTEREST,
+            OperationCode.GET, 3, OpFlags.USE_REGEX
+                | OpFlags.REGISTER_POLICY_NONE | OpFlags.USE_NOTAUTHZ, 8),
+        new OperationWithAction(OperationCode.PUT, 1, OpFlags.USE_OLDCONN
+            | OpFlags.USE_NEWVAL, 4),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP | OpFlags.USE_NEWVAL, 4),
+        new OperationWithAction(OperationCode.GET, 3, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP | OpFlags.USE_NEWVAL | OpFlags.CHECK_FAIL, 4),
+
+        OperationWithAction.OPBLOCK_END };
+
+      AuthzCredentialGenerator gen = this.getXmlAuthzGenerator();
+      CredentialGenerator cGen = gen.getCredentialGenerator();
+      Properties extraAuthProps = cGen.getSystemProperties();
+      Properties javaProps = cGen.getJavaProperties();
+      Properties extraAuthzProps = gen.getSystemProperties();
+      String authenticator = cGen.getAuthenticator();
+      String authInit = cGen.getAuthInit();
+      String accessor = gen.getAuthorizationCallback();
+      TestAuthzCredentialGenerator tgen = new TestAuthzCredentialGenerator(gen);
+
+      getLogWriter().info(
+          "testAllOpsNotifications: Using authinit: " + authInit);
+      getLogWriter().info(
+          "testAllOpsNotifications: Using authenticator: " + authenticator);
+      getLogWriter().info(
+          "testAllOpsNotifications: Using accessor: " + accessor);
+
+      // Start servers with all required properties
+      Properties serverProps = buildProperties(authenticator, accessor, true,
+          extraAuthProps, extraAuthzProps);
+      // Get ports for the servers
+      Integer port1 = new Integer(AvailablePort
+          .getRandomAvailablePort(AvailablePort.SOCKET));
+      Integer port2 = new Integer(AvailablePort
+          .getRandomAvailablePort(AvailablePort.SOCKET));
+
+      // Perform all the ops on the clients
+      List opBlock = new ArrayList();
+      Random rnd = new Random();
+      for (int opNum = 0; opNum < allOps.length; ++opNum) {
+        // Start client with valid credentials as specified in
+        // OperationWithAction
+        OperationWithAction currentOp = allOps[opNum];
+        if (currentOp.equals(OperationWithAction.OPBLOCK_END)
+            || currentOp.equals(OperationWithAction.OPBLOCK_NO_FAILOVER)) {
+          // End of current operation block; execute all the operations
+          // on the servers with failover
+          if (opBlock.size() > 0) {
+            // Start the first server and execute the operation block
+            server1.invoke(ClientAuthorizationTestBase.class,
+                "createCacheServer", new Object[] {
+                    SecurityTestUtil.getLocatorPort(), port1, serverProps,
+                    javaProps });
+            server2.invoke(SecurityTestUtil.class, "closeCache");
+            executeOpBlock(opBlock, port1, port2, authInit, extraAuthProps,
+                extraAuthzProps, tgen, rnd);
+            if (!currentOp.equals(OperationWithAction.OPBLOCK_NO_FAILOVER)) {
+              // Failover to the second server and run the block again
+              server2.invoke(ClientAuthorizationTestBase.class,
+                  "createCacheServer", new Object[] {
+                      SecurityTestUtil.getLocatorPort(), port2, serverProps,
+                      javaProps });
+              server1.invoke(SecurityTestUtil.class, "closeCache");
+              executeOpBlock(opBlock, port1, port2, authInit, extraAuthProps,
+                  extraAuthzProps, tgen, rnd);
+            }
+            opBlock.clear();
+          }
+        }
+        else {
+          currentOp.setOpNum(opNum);
+          opBlock.add(currentOp);
+        }
+      }
+  }
+
+  protected void executeOpBlock(List opBlock, Integer port1, Integer port2,
+      String authInit, Properties extraAuthProps, Properties extraAuthzProps,
+      TestCredentialGenerator gen, Random rnd) {
+    Iterator opIter = opBlock.iterator();
+    while (opIter.hasNext()) {
+      // Start client with valid credentials as specified in
+      // OperationWithAction
+      OperationWithAction currentOp = (OperationWithAction)opIter.next();
+      OperationCode opCode = currentOp.getOperationCode();
+      int opFlags = currentOp.getFlags();
+      int clientNum = currentOp.getClientNum();
+      VM clientVM = null;
+      boolean useThisVM = false;
+      switch (clientNum) {
+        case 1:
+          clientVM = client1;
+          break;
+        case 2:
+          clientVM = client2;
+          break;
+        case 3:
+          useThisVM = true;
+          break;
+        default:
+          fail("executeOpBlock: Unknown client number " + clientNum);
+          break;
+      }
+      getLogWriter().info(
+          "executeOpBlock: performing operation number ["
+              + currentOp.getOpNum() + "]: " + currentOp);
+      if ((opFlags & OpFlags.USE_OLDCONN) == 0) {
+        Properties opCredentials;
+        int newRnd = rnd.nextInt(100) + 1;
+        String currentRegionName = '/' + regionName;
+        if ((opFlags & OpFlags.USE_SUBREGION) > 0) {
+          currentRegionName += ('/' + subregionName);
+        }
+        String credentialsTypeStr;
+        OperationCode authOpCode = currentOp.getAuthzOperationCode();
+        int[] indices = currentOp.getIndices();
+        CredentialGenerator cGen = gen.getCredentialGenerator();
+        Properties javaProps = null;
+        if ((opFlags & OpFlags.CHECK_NOTAUTHZ) > 0
+            || (opFlags & OpFlags.USE_NOTAUTHZ) > 0) {
+          opCredentials = gen.getDisallowedCredentials(
+              new OperationCode[] { authOpCode },
+              new String[] { currentRegionName }, indices, newRnd);
+          credentialsTypeStr = " unauthorized " + authOpCode;
+        }
+        else {
+          opCredentials = gen.getAllowedCredentials(new OperationCode[] {
+              opCode, authOpCode }, new String[] { currentRegionName },
+              indices, newRnd);
+          credentialsTypeStr = " authorized " + authOpCode;
+        }
+        if (cGen != null) {
+          javaProps = cGen.getJavaProperties();
+        }
+        Properties clientProps = SecurityTestUtil
+            .concatProperties(new Properties[] { opCredentials, extraAuthProps,
+                extraAuthzProps });
+        // Start the client with valid credentials but allowed or disallowed to
+        // perform an operation
+        getLogWriter().info(
+            "executeOpBlock: For client" + clientNum + credentialsTypeStr
+                + " credentials: " + opCredentials);
+        boolean setupDynamicRegionFactory = (opFlags & OpFlags.ENABLE_DRF) > 0;
+        if (useThisVM) {
+          createCacheClient(authInit, clientProps, javaProps, new Integer[] {
+              port1, port2 }, null, Boolean.valueOf(setupDynamicRegionFactory),
+              new Integer(SecurityTestUtil.NO_EXCEPTION));
+        }
+        else {
+          clientVM.invoke(ClientAuthorizationTestBase.class,
+              "createCacheClient", new Object[] { authInit, clientProps,
+                  javaProps, new Integer[] { port1, port2 }, null,
+                  Boolean.valueOf(setupDynamicRegionFactory),
+                  new Integer(SecurityTestUtil.NO_EXCEPTION) });
+        }
+      }
+      int expectedResult;
+      if ((opFlags & OpFlags.CHECK_NOTAUTHZ) > 0) {
+        expectedResult = SecurityTestUtil.NOTAUTHZ_EXCEPTION;
+      }
+      else if ((opFlags & OpFlags.CHECK_EXCEPTION) > 0) {
+        expectedResult = SecurityTestUtil.OTHER_EXCEPTION;
+      }
+      else {
+        expectedResult = SecurityTestUtil.NO_EXCEPTION;
+      }
+
+      // Perform the operation from selected client
+      if (useThisVM) {
+        doOp(new Byte(opCode.toOrdinal()), currentOp.getIndices(), new Integer(
+            opFlags), new Integer(expectedResult));
+      }
+      else {
+        clientVM.invoke(DeltaClientPostAuthorizationDUnitTest.class, "doOp",
+            new Object[] { new Byte(opCode.toOrdinal()),
+                currentOp.getIndices(), new Integer(opFlags),
+                new Integer(expectedResult) });
+      }
+    }
+  }
+
+  private static Region createSubregion(Region region) {
+
+    Region subregion = getSubregion();
+    if (subregion == null) {
+      subregion = region.createSubregion(subregionName, region.getAttributes());
+    }
+    return subregion;
+  }
+
+  public static void doOp(Byte opCode, int[] indices, Integer flagsI,
+      Integer expectedResult) {
+
+    OperationCode op = OperationCode.fromOrdinal(opCode.byteValue());
+    boolean operationOmitted = false;
+    final int flags = flagsI.intValue();
+    Region region = getRegion();
+//    for (int i = 0; i < indices.length; i++) {
+//      region.put(SecurityTestUtil.keys[i],
+//          DeltaClientAuthorizationDUnitTest.deltas[i]);
+//    }
+    if ((flags & OpFlags.USE_SUBREGION) > 0) {
+      assertNotNull(region);
+      Region subregion = null;
+      if ((flags & OpFlags.NO_CREATE_SUBREGION) > 0) {
+        if ((flags & OpFlags.CHECK_NOREGION) > 0) {
+          // Wait for some time for DRF update to come
+          SecurityTestUtil.waitForCondition(new Callable() {
+            public Object call() throws Exception {
+              return Boolean.valueOf(getSubregion() == null);
+            }
+          });
+          subregion = getSubregion();
+          assertNull(subregion);
+          return;
+        }
+        else {
+          // Wait for some time for DRF update to come
+          SecurityTestUtil.waitForCondition(new Callable() {
+            public Object call() throws Exception {
+              return Boolean.valueOf(getSubregion() != null);
+            }
+          });
+          subregion = getSubregion();
+          assertNotNull(subregion);
+        }
+      }
+      else {
+        subregion = createSubregion(region);
+      }
+      assertNotNull(subregion);
+      region = subregion;
+    }
+    else if ((flags & OpFlags.CHECK_NOREGION) > 0) {
+      // Wait for some time for region destroy update to come
+      SecurityTestUtil.waitForCondition(new Callable() {
+        public Object call() throws Exception {
+          return Boolean.valueOf(getRegion() == null);
+        }
+      });
+      region = getRegion();
+      assertNull(region);
+      return;
+    }
+    else {
+      assertNotNull(region);
+    }
+    final String[] keys = SecurityTestUtil.keys;
+    final DeltaTestImpl[] vals;
+    if ((flags & OpFlags.USE_NEWVAL) > 0) {
+      vals = DeltaClientAuthorizationDUnitTest.deltas;
+    }
+    else {
+      vals = DeltaClientAuthorizationDUnitTest.deltas;
+    }
+    InterestResultPolicy policy = InterestResultPolicy.KEYS_VALUES;
+    if ((flags & OpFlags.REGISTER_POLICY_NONE) > 0) {
+      policy = InterestResultPolicy.NONE;
+    }
+    final int numOps = indices.length;
+    getLogWriter().info(
+        "Got doOp for op: " + op.toString() + ", numOps: " + numOps
+            + ", indices: " + indicesToString(indices) + ", expect: " + expectedResult);
+    boolean exceptionOccured = false;
+    boolean breakLoop = false;
+    if (op.isGet()) {
+      try {
+        Thread.sleep(PAUSE);
+      }
+      catch (InterruptedException e) {
+        fail("interrupted");
+      }
+    }
+    for (int indexIndex = 0; indexIndex < numOps; ++indexIndex) {
+      if (breakLoop) {
+        break;
+      }
+      int index = indices[indexIndex];
+      try {
+        final Object key = keys[index];
+        final Object expectedVal = vals[index];
+        if (op.isGet()) {
+          Object value = null;
+          // this is the case for testing GET_ALL
+          if ((flags & OpFlags.USE_ALL_KEYS) > 0) {
+            breakLoop = true;
+            List keyList = new ArrayList(numOps);
+            Object searchKey;
+            for (int keyNumIndex = 0; keyNumIndex < numOps; ++keyNumIndex) {
+              int keyNum = indices[keyNumIndex];
+              searchKey = keys[keyNum];
+              keyList.add(searchKey);
+              // local invalidate some keys to force fetch of those keys from
+              // server
+              if ((flags & OpFlags.CHECK_NOKEY) > 0) {
+                assertFalse(region.containsKey(searchKey));
+              }
+              else {
+                if (keyNumIndex % 2 == 1) {
+                  assertTrue(region.containsKey(searchKey));
+                  region.localInvalidate(searchKey);
+                }
+              }
+            }
+            Map entries = region.getAll(keyList);
+            for (int keyNumIndex = 0; keyNumIndex < numOps; ++keyNumIndex) {
+              int keyNum = indices[keyNumIndex];
+              searchKey = keys[keyNum];
+              if ((flags & OpFlags.CHECK_FAIL) > 0) {
+                assertFalse(entries.containsKey(searchKey));
+              }
+              else {
+                assertTrue(entries.containsKey(searchKey));
+                value = entries.get(searchKey);
+                assertEquals(vals[keyNum], value);
+              }
+            }
+            break;
+          }
+          if ((flags & OpFlags.LOCAL_OP) > 0) {
+            Callable cond = new Callable() {
+              private Region region;
+
+              public Object call() throws Exception {
+                Object value = SecurityTestUtil.getLocalValue(region, key);
+                return Boolean
+                    .valueOf((flags & OpFlags.CHECK_FAIL) > 0 ? !expectedVal
+                        .equals(value) : expectedVal.equals(value));
+              }
+
+              public Callable init(Region region) {
+                this.region = region;
+                return this;
+              }
+            }.init(region);
+            SecurityTestUtil.waitForCondition(cond);
+            value = SecurityTestUtil.getLocalValue(region, key);
+          }
+          else {
+            if ((flags & OpFlags.CHECK_NOKEY) > 0) {
+              assertFalse(region.containsKey(key));
+            }
+            else {
+              assertTrue(region.containsKey(key));
+              region.localInvalidate(key);
+            }
+            value = region.get(key);
+          }
+          if ((flags & OpFlags.CHECK_FAIL) > 0) {
+            assertFalse(expectedVal.equals(value));
+          }
+          else {
+            assertNotNull(value);
+            assertEquals(expectedVal, value);
+          }
+        }
+        else if (op.isPut()) {
+          region.put(key, expectedVal);
+        }
+        else if (op.isRegisterInterest()) {
+          if ((flags & OpFlags.USE_LIST) > 0) {
+            breakLoop = true;
+            // Register interest list in this case
+            List keyList = new ArrayList(numOps);
+            for (int keyNumIndex = 0; keyNumIndex < numOps; ++keyNumIndex) {
+              int keyNum = indices[keyNumIndex];
+              keyList.add(keys[keyNum]);
+            }
+            region.registerInterest(keyList, policy);
+          }
+          else if ((flags & OpFlags.USE_REGEX) > 0) {
+            breakLoop = true;
+            region.registerInterestRegex("key[1-" + numOps + ']', policy);
+          }
+          else if ((flags & OpFlags.USE_ALL_KEYS) > 0) {
+            breakLoop = true;
+            region.registerInterest("ALL_KEYS", policy);
+          }
+          else {
+            region.registerInterest(key, policy);
+          }
+        }
+        else {
+          fail("doOp: Unhandled operation " + op);
+        }
+        if (expectedResult.intValue() != SecurityTestUtil.NO_EXCEPTION) {
+          if (!operationOmitted && !op.isUnregisterInterest()) {
+            fail("Expected an exception while performing operation op =" + op +
+                "flags = " + OpFlags.description(flags));
+          }
+        }
+      }
+      catch (Exception ex) {
+        exceptionOccured = true;
+        if ((ex instanceof ServerConnectivityException
+            || ex instanceof QueryInvocationTargetException || ex instanceof CqException)
+            && (expectedResult.intValue() == SecurityTestUtil.NOTAUTHZ_EXCEPTION)
+            && (ex.getCause() instanceof NotAuthorizedException)) {
+          getLogWriter().info(
+              "doOp: Got expected NotAuthorizedException when doing operation ["
+                  + op + "] with flags " + OpFlags.description(flags) 
+                  + ": " + ex.getCause());
+          continue;
+        }
+        else if (expectedResult.intValue() == SecurityTestUtil.OTHER_EXCEPTION) {
+          getLogWriter().info(
+              "doOp: Got expected exception when doing operation: "
+                  + ex.toString());
+          continue;
+        }
+        else {
+          fail("doOp: Got unexpected exception when doing operation. Policy = " 
+              + policy + " flags = " + OpFlags.description(flags), ex);
+        }
+      }
+    }
+    if (!exceptionOccured && !operationOmitted
+        && expectedResult.intValue() != SecurityTestUtil.NO_EXCEPTION) {
+      fail("Expected an exception while performing operation: " + op + 
+          " flags = " + OpFlags.description(flags));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
new file mode 100644
index 0000000..0f83050
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
@@ -0,0 +1,622 @@
+
+package com.gemstone.gemfire.security;
+
+/*
+ * 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.
+ */
+
+
+import java.io.File;
+import java.util.Properties;
+
+import javax.net.ssl.SSLHandshakeException;
+
+import templates.security.CredentialGenerator;
+import templates.security.DummyCredentialGenerator;
+import templates.security.LdapUserCredentialGenerator;
+
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
+import com.gemstone.gemfire.distributed.internal.membership.gms.MembershipManagerHelper;
+import com.gemstone.gemfire.internal.AvailablePort;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.VM;
+
+/**
+ * Tests peer to peer authentication in Gemfire
+ * 
+ * @author Yogesh Mahajan
+ * @since 5.5
+ */
+public class P2PAuthenticationDUnitTest extends DistributedTestCase {
+
+  private static VM locatorVM = null;
+
+  public static final String USER_NAME = "security-username";
+
+  public static final String PASSWORD = "security-password";
+
+  private static final String[] expectedExceptions = {
+      AuthenticationRequiredException.class.getName(),
+      AuthenticationFailedException.class.getName(),
+      GemFireSecurityException.class.getName(),
+      SSLHandshakeException.class.getName(),
+      ClassNotFoundException.class.getName(),
+      "Authentication failed for",
+      "Failed to obtain credentials"};
+
+  public P2PAuthenticationDUnitTest(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception {
+
+    super.setUp();
+    final Host host = Host.getHost(0);
+    locatorVM = host.getVM(0);
+  }
+
+  private void setProperty(Properties props, String key, String value) {
+
+    if (key != null && value != null) {
+      props.setProperty(key, value);
+    }
+  }
+
+  /**
+   * Check that mcast-port setting for discovery or with locator are
+   * incompatible with security
+   */
+  public void testIllegalPropertyCombos() throws Exception {
+
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    File logFile = new File(getUniqueName() + "-locator" + port + ".log");
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "26753");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, 
+                      DistributedTestCase.getIPLiteral() + "[" + port + "]");
+    props.setProperty(DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+        "templates.security.UserPasswordAuthInit.create");
+    props.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
+
+    try {
+      Locator.startLocatorAndDS(port, logFile, null, props);
+      fail("Expected an IllegalArgumentException while starting locator");
+    }
+    catch (IllegalArgumentException ex) {
+      // success
+    }
+
+    // Also try setting the authenticator
+    props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "26753");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, 
+                      DistributedTestCase.getIPLiteral() +"[" + port + "]");
+    props.setProperty(DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+        "templates.security.LdapUserAuthenticator.create");
+    props.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
+    try {
+      Locator.startLocatorAndDS(port, logFile, null, props);
+      fail("Expected an IllegalArgumentException while starting locator");
+    }
+    catch (IllegalArgumentException ex) {
+      // success
+    }
+
+    props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "26753");
+    props.setProperty(DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+        "templates.security.UserPasswordAuthInit.create");
+    try {
+      getSystem(props);
+      fail("Expected an IllegalArgumentException while connection to DS");
+    }
+    catch (IllegalArgumentException ex) {
+      // success
+    }
+
+    // Also try setting the authenticator
+    props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "26753");
+    props.setProperty(DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+        "templates.security.LdapUserAuthenticator.create");
+    try {
+      getSystem(props);
+      fail("Expected an IllegalArgumentException while connection to DS");
+    }
+    catch (IllegalArgumentException ex) {
+      // success
+    }
+  }
+
+  // AuthInitialize is incorrect
+  public void testP2PAuthenticationWithInvalidAuthInitialize() throws Exception {
+
+    disconnectAllFromDS();
+    CredentialGenerator gen = new DummyCredentialGenerator();
+    Properties props = gen.getSystemProperties();
+    Properties javaProps = gen.getJavaProperties();
+    String authenticator = gen.getAuthenticator();
+    if (props == null) {
+      props = new Properties();
+    }
+    String authInit = " Incorrect_AuthInitialize";
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    final String locators = DistributedTestCase.getIPLiteral() + "[" + port + "]";
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+            authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+            authenticator);
+    locatorVM.invoke(SecurityTestUtil.class, "startLocator", new Object[]{
+            getUniqueName(), new Integer(port), props, javaProps,
+            expectedExceptions});
+
+    LogWriter dsLogger = createLogWriter(props);
+    SecurityTestUtil.addExpectedExceptions(expectedExceptions, dsLogger);
+    try {
+      new SecurityTestUtil("tmp").createSystem(props, null);
+      fail("AuthenticationFailedException was expected as the AuthInitialize object passed is incorrect");
+    } catch (AuthenticationFailedException expected) {
+      // success
+    } finally {
+      SecurityTestUtil.removeExpectedExceptions(expectedExceptions, dsLogger);
+      locatorVM.invoke(SecurityTestUtil.class, "stopLocator", new Object[]{
+              new Integer(port), expectedExceptions});
+    }
+
+  }
+
+  // Authenticator is incorrect
+  public void testP2PAuthenticationWithInvalidAuthenticator() throws Exception {
+    disconnectAllFromDS();
+    CredentialGenerator gen = new DummyCredentialGenerator();
+    Properties props = gen.getSystemProperties();
+    Properties javaProps = gen.getJavaProperties();
+    String authenticator = "xyz";
+    String authInit = gen.getAuthInit();
+    if (props == null) {
+      props = new Properties();
+    }
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    final String locators = DistributedTestCase.getIPLiteral() +"["+port+"]";
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+            authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+            authenticator);
+    locatorVM.invoke(SecurityTestUtil.class, "startLocator", new Object[] {
+            getUniqueName(), new Integer(port), props, javaProps,
+            expectedExceptions });
+
+    LogWriter dsLogger = createLogWriter(props);
+    SecurityTestUtil.addExpectedExceptions(expectedExceptions, dsLogger);
+    try {
+      new SecurityTestUtil("tmp").createSystem(props, javaProps);
+      fail("AuthenticationFailedException was expected as the Authenticator object passed is incorrect");
+    }
+    catch (AuthenticationFailedException expected) {
+      // success
+    }
+    finally {
+      SecurityTestUtil.removeExpectedExceptions(expectedExceptions, dsLogger);
+      locatorVM.invoke(SecurityTestUtil.class, "stopLocator", new Object[] {
+              new Integer(port), expectedExceptions });
+    }
+  }
+
+  public void testP2PAuthenticationWithNoCredentials() throws Exception {
+
+    disconnectAllFromDS();
+
+    CredentialGenerator gen = new DummyCredentialGenerator();
+    Properties props = gen.getSystemProperties();
+    Properties javaProps = gen.getJavaProperties();
+    String authenticator = gen.getAuthenticator();
+    String authInit = gen.getAuthInit();
+    if (props == null) {
+      props = new Properties();
+    }
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    final String locators = DistributedTestCase.getIPLiteral() +"["+port+"]";
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+            authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+            authenticator);
+    locatorVM.invoke(SecurityTestUtil.class, "startLocator", new Object[] {
+            getUniqueName(), new Integer(port), props, javaProps,
+            expectedExceptions });
+
+    LogWriter dsLogger = createLogWriter(props);
+    SecurityTestUtil.addExpectedExceptions(expectedExceptions, dsLogger);
+    try {
+      new SecurityTestUtil("tmp").createSystem(props, null);
+      fail("AuthenticationFailedException was expected as no credentials are set");
+    }
+    catch (AuthenticationFailedException expected) {
+      // success
+    }
+    finally {
+      SecurityTestUtil.removeExpectedExceptions(expectedExceptions, dsLogger);
+      locatorVM.invoke(SecurityTestUtil.class, "stopLocator", new Object[] {
+              new Integer(port), expectedExceptions });
+    }
+  }
+
+  public void testP2PAuthenticationWithValidCredentials() throws Exception {
+
+    disconnectAllFromDS();
+    CredentialGenerator gen = new DummyCredentialGenerator();
+    Properties props = gen.getSystemProperties();
+    String authenticator = gen.getAuthenticator();
+    String authInit = gen.getAuthInit();
+    if (props == null) {
+      props = new Properties();
+    }
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    final String locators = DistributedTestCase.getIPLiteral() +"["+port+"]";
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+            authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+            authenticator);
+    Properties credentials = gen.getValidCredentials(1);
+    Properties javaProps = gen.getJavaProperties();
+    props.putAll(credentials);
+    locatorVM.invoke(SecurityTestUtil.class, "startLocator", new Object[] {
+            getUniqueName(), new Integer(port), props, javaProps,
+            expectedExceptions });
+    try {
+      createDS(props, javaProps);
+      verifyMembers(new Integer(2));
+      disconnectFromDS();
+
+    } finally {
+      locatorVM.invoke(SecurityTestUtil.class, "stopLocator", new Object[] {
+              new Integer(port), expectedExceptions });
+    }
+  }
+
+  public void testP2PAuthenticationWithBothValidAndInValidCredentials()
+      throws Exception {
+
+    disconnectAllFromDS();
+    addExpectedException("Authentication failed");
+
+    CredentialGenerator gen = new DummyCredentialGenerator();
+    Properties props = gen.getSystemProperties();
+    String authenticator = gen.getAuthenticator();
+    String authInit = gen.getAuthInit();
+    if (props == null) {
+      props = new Properties();
+    }
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    final String locators = DistributedTestCase.getIPLiteral() +"["+port+"]";
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+            authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+            authenticator);
+    // valid credentials for locator
+    Properties credentials = gen.getValidCredentials(1);
+    Properties javaProps = gen.getJavaProperties();
+    props.putAll(credentials);
+    locatorVM.invoke(SecurityTestUtil.class, "startLocator", new Object[] {
+            getUniqueName(), new Integer(port), props, javaProps,
+            expectedExceptions });
+    try {
+      // invalid credentials for the peer
+      credentials = gen.getInvalidCredentials(1);
+      javaProps = gen.getJavaProperties();
+      props.putAll(credentials);
+
+      LogWriter dsLogger = createLogWriter(props);
+      SecurityTestUtil.addExpectedExceptions(expectedExceptions, dsLogger);
+      try {
+        new SecurityTestUtil("tmp").createSystem(props, javaProps);
+        fail("AuthenticationFailedException was expected as wrong credentials were passed");
+      }
+      catch (AuthenticationFailedException expected) {
+        // success
+      }
+      finally {
+        SecurityTestUtil.removeExpectedExceptions(expectedExceptions, dsLogger);
+      }
+
+      credentials = gen.getValidCredentials(3);
+      javaProps = gen.getJavaProperties();
+      props.putAll(credentials);
+      createDS(props, javaProps);
+      verifyMembers(new Integer(2));
+      disconnectFromDS();
+
+    } finally {
+      locatorVM.invoke(SecurityTestUtil.class, "stopLocator", new Object[] {
+              new Integer(port), expectedExceptions });
+    }
+  }
+
+  /**
+   * The strategy is to test view change reject by having two different
+   * authenticators on different VMs.
+   * 
+   * Here locator will accept the credentials from peer2 but the first peer will
+   * reject them due to different authenticator. Hence the number of members
+   * reported by the first peer should be only two while others will report as
+   * three.
+   */
+  public void disabled_testP2PViewChangeReject() throws Exception {
+
+    disconnectAllFromDS();
+    final Host host = Host.getHost(0);
+    final VM peer2 = host.getVM(1);
+    final VM peer3 = host.getVM(2);
+
+    CredentialGenerator gen = new LdapUserCredentialGenerator();
+    gen.init();
+    Properties extraProps = gen.getSystemProperties();
+    String authenticator = gen.getAuthenticator();
+    String authInit = gen.getAuthInit();
+    if (extraProps == null) {
+      extraProps = new Properties();
+    }
+
+    CredentialGenerator gen2 = new DummyCredentialGenerator();
+    gen2.init();
+    Properties extraProps2 = gen2.getSystemProperties();
+    String authenticator2 = gen2.getAuthenticator();
+    if (extraProps2 == null) {
+      extraProps2 = new Properties();
+    }
+
+    // Start the locator with the LDAP authenticator
+    Properties props = new Properties();
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    final String locators = DistributedTestCase.getIPLiteral() +"["+port+"]";
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+        authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+        authenticator);
+    Properties credentials = gen.getValidCredentials(1);
+    Properties javaProps = gen.getJavaProperties();
+    props.putAll(credentials);
+    props.putAll(extraProps);
+    locatorVM.invoke(SecurityTestUtil.class, "startLocator", new Object[] {
+        getUniqueName(), new Integer(port), props, javaProps,
+        expectedExceptions });
+    try {
+
+    // Start the first peer with different authenticator
+    props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+        authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+        authenticator2);
+    credentials = gen.getValidCredentials(3);
+    Properties javaProps2 = gen2.getJavaProperties();
+    props.putAll(credentials);
+    props.putAll(extraProps2);
+    createDS(props, javaProps2);
+
+    // Start the second peer with the same authenticator as locator
+    props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+        authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+        authenticator);
+    credentials = gen.getValidCredentials(7);
+    javaProps = gen.getJavaProperties();
+    props.putAll(credentials);
+    props.putAll(extraProps);
+    peer2.invoke(P2PAuthenticationDUnitTest.class, "createDS", new Object[] {
+        props, javaProps });
+
+    // Start the third peer with the same authenticator as locator
+    peer3.invoke(P2PAuthenticationDUnitTest.class, "createDS", new Object[] {
+        props, javaProps });
+
+    // wait for view propagation
+    pause(2000);
+    // Verify the number of members on all peers and locator
+    locatorVM.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(4) });
+    verifyMembers(new Integer(2));
+    peer2.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(4) });
+    peer3.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(4) });
+
+    // Disconnect the first peer and check again
+    disconnectFromDS();
+    pause(2000);
+    locatorVM.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(3) });
+    peer2.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(3) });
+    peer3.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(3) });
+
+    // Disconnect the second peer and check again
+    peer2.invoke(DistributedTestCase.class, "disconnectFromDS");
+    pause(2000);
+    locatorVM.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(2) });
+    peer3.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(2) });
+
+    // Same for last peer
+    peer3.invoke(DistributedTestCase.class, "disconnectFromDS");
+    pause(2000);
+    locatorVM.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(1) });
+
+    } finally {
+    locatorVM.invoke(SecurityTestUtil.class, "stopLocator", new Object[] {
+        new Integer(port), expectedExceptions });
+    }
+  }
+
+  /**
+   * The strategy is to test credential size greater than UDP datagram size.
+   * 
+   * @see Bug # 38570.
+   * 
+   * Here locator will accept the credentials from peer2 and the large credential
+   * from the first peer. Number of members in the DS
+   * should be four
+   */
+  public void testP2PLargeCredentialSucceeds() throws Exception {
+
+    disconnectAllFromDS();
+    final Host host = Host.getHost(0);
+    final VM peer2 = host.getVM(1);
+    final VM peer3 = host.getVM(2);
+
+    CredentialGenerator gen = new DummyCredentialGenerator();
+    gen.init();
+    Properties extraProps = gen.getSystemProperties();
+    String authenticator = gen.getAuthenticator();
+    String authInit = "templates.security.UserPasswordWithExtraPropsAuthInit.create";
+    if (extraProps == null) {
+      extraProps = new Properties();
+    }
+
+    // Start the locator with the Dummy authenticator
+    Properties props = new Properties();
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    final String locators = DistributedTestCase.getIPLiteral() +"["+port+"]";
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+        authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+        authenticator);
+    Properties credentials = gen.getValidCredentials(1);
+    Properties javaProps = gen.getJavaProperties();
+    props.putAll(credentials);
+    props.putAll(extraProps);
+    locatorVM.invoke(SecurityTestUtil.class, "startLocator", new Object[] {
+        getUniqueName(), new Integer(port), props, javaProps,
+        expectedExceptions });
+    try {
+
+    // Start the first peer with huge credentials
+    props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+        authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+        authenticator);
+    credentials = gen.getValidCredentials(3);
+    javaProps = gen.getJavaProperties();
+    String hugeStr = "20KString";
+    for (int i = 0; i <= 20000; i++) {
+      hugeStr += "A";
+    }
+    credentials.setProperty("security-keep-extra-props", "-");
+    credentials.setProperty("security-hugeentryone", hugeStr);
+    credentials.setProperty("security-hugeentrytwo", hugeStr);
+    credentials.setProperty("security-hugeentrythree", hugeStr);
+
+    props.putAll(credentials);
+    props.putAll(extraProps);
+
+    LogWriter dsLogger = createLogWriter(props);
+    SecurityTestUtil.addExpectedExceptions(
+        new String[] { IllegalArgumentException.class.getName() }, dsLogger);
+    try {
+      createDS(props, javaProps);
+//      fail("AuthenticationFailedException was expected as credentials were passed beyond 50k");
+    }
+    finally {
+      SecurityTestUtil.removeExpectedExceptions(
+          new String[] { IllegalArgumentException.class.getName() }, dsLogger);
+    }
+
+    // Start the second peer with the same authenticator as locator
+    props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
+        authInit);
+    setProperty(props, DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
+        authenticator);
+    credentials = gen.getValidCredentials(7);
+    javaProps = gen.getJavaProperties();
+    props.putAll(credentials);
+    props.putAll(extraProps);
+    peer2.invoke(P2PAuthenticationDUnitTest.class, "createDS", new Object[] {
+        props, javaProps });
+
+    // Start the third peer with the same authenticator as locator
+    peer3.invoke(P2PAuthenticationDUnitTest.class, "createDS", new Object[] {
+        props, javaProps });
+
+    // wait for view propagation
+    pause(2000);
+    // Verify the number of members on all peers and locator
+    locatorVM.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(4) });
+    peer2.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(4) });
+    peer3.invoke(P2PAuthenticationDUnitTest.class, "verifyMembers",
+        new Object[] { new Integer(4) });
+
+
+    // Disconnect the peers
+    disconnectFromDS();
+    peer2.invoke(DistributedTestCase.class, "disconnectFromDS");
+    peer3.invoke(DistributedTestCase.class, "disconnectFromDS");
+
+    } finally {
+    // Stopping the locator
+    locatorVM.invoke(SecurityTestUtil.class, "stopLocator", new Object[] {
+        new Integer(port), expectedExceptions });
+    }
+  }
+
+  public static void createDS(Properties props, Object javaProps) {
+
+    SecurityTestUtil tmpUtil = new SecurityTestUtil("tmp");
+    tmpUtil.createSystem(props, (Properties)javaProps);
+  }
+
+  public static void verifyMembers(Integer numExpectedMembers) {
+
+    DistributedSystem ds = InternalDistributedSystem.getAnyInstance();
+    MembershipManager mgr = MembershipManagerHelper
+        .getMembershipManager(ds);
+    assertEquals(numExpectedMembers.intValue(), mgr.getView().size());
+  }
+
+}


[21/33] incubator-geode git commit: GEODE-649: add more logging to see the reason of future failure.

Posted by je...@apache.org.
GEODE-649: add more logging to see the reason of future failure.


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

Branch: refs/heads/feature/GEODE-14
Commit: 94bce4d394b24cceb2f183db0b35aa7688679fb5
Parents: 1a02572
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue Dec 22 08:38:29 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Dec 22 10:47:49 2015 -0800

----------------------------------------------------------------------
 .../gemfire/distributed/AbstractLauncherJUnitTestCase.java         | 2 +-
 .../gemfire/distributed/ServerLauncherRemoteJUnitTest.java         | 2 ++
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94bce4d3/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTestCase.java
index 5e79bef..38accda 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTestCase.java
@@ -52,7 +52,7 @@ import com.gemstone.gemfire.internal.util.StopWatch;
  * @since 8.0
  */
 public abstract class AbstractLauncherJUnitTestCase {
-  private static final Logger logger = LogService.getLogger();
+  protected static final Logger logger = LogService.getLogger();
   
   protected static final int WAIT_FOR_PROCESS_TO_DIE_TIMEOUT = 5 * 60 * 1000; // 5 minutes
   protected static final int TIMEOUT_MILLISECONDS = WAIT_FOR_PROCESS_TO_DIE_TIMEOUT;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94bce4d3/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
index 5833bd4..cbcfb77 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
@@ -1346,9 +1346,11 @@ public class ServerLauncherRemoteJUnitTest extends AbstractServerLauncherJUnitTe
           }
           final ServerState serverState = launcher.status();
           assertNotNull(serverState);
+          logger.info("serverState: "+serverState);
           return Status.ONLINE.equals(serverState.getStatus());
         }
         catch (RuntimeException e) {
+          logger.error(e, e);
           return false;
         }
       }


[29/33] incubator-geode git commit: fixing a possible memory leak

Posted by je...@apache.org.
fixing a possible memory leak

GMSMembershipManager thought it was destroying artifacts for shunned
members who had expired but it wasn't.


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

Branch: refs/heads/feature/GEODE-14
Commit: 093ac12e5106388c2ee33e2cca140856f1e4aa91
Parents: 2a21b70
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Wed Dec 23 10:34:10 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Wed Dec 23 10:34:10 2015 -0800

----------------------------------------------------------------------
 .../gms/mgr/GMSMembershipManager.java           | 21 +++++++++++++++++---
 1 file changed, 18 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/093ac12e/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index 8ce5178..7fe55e5 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -2160,14 +2160,18 @@ public class GMSMembershipManager implements MembershipManager, Manager
     // the iterator.
     Set oldMembers = new HashSet(shunnedMembers.entrySet());
     
+    Set removedMembers = new HashSet();
+    
     Iterator it = oldMembers.iterator();
     while (it.hasNext()) {
       Map.Entry e = (Map.Entry)it.next();
       
       // Key is the member.  Value is the time to remove it.
       long ll = ((Long)e.getValue()).longValue(); 
-      if (ll >= deathTime)
+      if (ll >= deathTime) {
         continue; // too new.
+      }
+      
       InternalDistributedMember mm = (InternalDistributedMember)e.getKey();
 
       if (latestView.contains(mm)) {
@@ -2179,10 +2183,21 @@ public class GMSMembershipManager implements MembershipManager, Manager
         // will depart on its own accord, but we force the issue here.)
         destroyMember(mm, true, "shunned but never disconnected");
       }
-      if (logger.isDebugEnabled())
+      if (logger.isDebugEnabled()) {
         logger.debug("Membership: finally removed shunned member entry <{}>", mm);
-    } // while
+      }
+      
+      removedMembers.add(mm);
+    }
     
+    // removed timed-out entries from the shunned-members collections
+    if (removedMembers.size() > 0) {
+      it = removedMembers.iterator();
+      while (it.hasNext()) {
+        InternalDistributedMember idm = (InternalDistributedMember)it.next();
+        endShun(idm);
+      }
+    }
   }
   
   


[10/33] incubator-geode git commit: GEM-164: move the security tests in gemfire-test module inside the com.gemstone.gemfire.security packages to the open side.

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
new file mode 100644
index 0000000..e89e0d3
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
@@ -0,0 +1,798 @@
+
+package com.gemstone.gemfire.security;
+
+/*
+ * 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.
+ */
+
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+
+import templates.security.AuthzCredentialGenerator;
+import templates.security.CredentialGenerator;
+import templates.security.DummyCredentialGenerator;
+import templates.security.XmlAuthzCredentialGenerator;
+
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.internal.AvailablePort;
+import templates.security.UserPasswordAuthInit;
+
+import dunit.Host;
+import dunit.VM;
+
+/**
+ * Tests for authorization from client to server. This tests for authorization
+ * of all operations with both valid and invalid credentials/modules with
+ * pre-operation callbacks. It also checks for authorization in case of
+ * failover.
+ * 
+ * @author sumedh
+ * @since 5.5
+ */
+public class ClientAuthorizationDUnitTest extends ClientAuthorizationTestBase {
+
+  /** constructor */
+  public ClientAuthorizationDUnitTest(String name) {
+    super(name);
+  }
+
+  @Override
+  public void setUp() throws Exception {
+
+    super.setUp();
+    final Host host = Host.getHost(0);
+    server1 = host.getVM(0);
+    server2 = host.getVM(1);
+    client1 = host.getVM(2);
+    client2 = host.getVM(3);
+
+    server1.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { serverExpectedExceptions });
+    server2.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { serverExpectedExceptions });
+    client1.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { clientExpectedExceptions });
+    client2.invoke(SecurityTestUtil.class, "registerExpectedExceptions",
+        new Object[] { clientExpectedExceptions });
+    SecurityTestUtil.registerExpectedExceptions(clientExpectedExceptions);
+  }
+
+  private Properties getUserPassword(String userName) {
+
+    Properties props = new Properties();
+    props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
+    props.setProperty(UserPasswordAuthInit.PASSWORD, userName);
+    return props;
+  }
+
+  private void executeRIOpBlock(List opBlock, Integer port1, Integer port2,
+      String authInit, Properties extraAuthProps, Properties extraAuthzProps,
+      Properties javaProps) {
+
+    Iterator opIter = opBlock.iterator();
+    while (opIter.hasNext()) {
+      // Start client with valid credentials as specified in
+      // OperationWithAction
+      OperationWithAction currentOp = (OperationWithAction)opIter.next();
+      OperationCode opCode = currentOp.getOperationCode();
+      int opFlags = currentOp.getFlags();
+      int clientNum = currentOp.getClientNum();
+      VM clientVM = null;
+      boolean useThisVM = false;
+      switch (clientNum) {
+        case 1:
+          clientVM = client1;
+          break;
+        case 2:
+          clientVM = client2;
+          break;
+        case 3:
+          useThisVM = true;
+          break;
+        default:
+          fail("executeRIOpBlock: Unknown client number " + clientNum);
+          break;
+      }
+      getLogWriter().info(
+          "executeRIOpBlock: performing operation number ["
+              + currentOp.getOpNum() + "]: " + currentOp);
+      if ((opFlags & OpFlags.USE_OLDCONN) == 0) {
+        Properties opCredentials = null;
+        String currentRegionName = '/' + regionName;
+        if ((opFlags & OpFlags.USE_SUBREGION) > 0) {
+          currentRegionName += ('/' + subregionName);
+        }
+        String credentialsTypeStr;
+        OperationCode authOpCode = currentOp.getAuthzOperationCode();
+        if ((opFlags & OpFlags.CHECK_NOTAUTHZ) > 0
+            || (opFlags & OpFlags.USE_NOTAUTHZ) > 0
+            || !authOpCode.equals(opCode)) {
+          credentialsTypeStr = " unauthorized " + authOpCode;
+          if (authOpCode.isRegisterInterest()) {
+            opCredentials = getUserPassword("reader7");
+          }
+          else if (authOpCode.isUnregisterInterest()) {
+            opCredentials = getUserPassword("reader6");
+          }
+          else {
+            fail("executeRIOpBlock: cannot determine credentials for"
+                + credentialsTypeStr);
+          }
+        }
+        else {
+          credentialsTypeStr = " authorized " + authOpCode;
+          if (authOpCode.isRegisterInterest()
+              || authOpCode.isUnregisterInterest()) {
+            opCredentials = getUserPassword("reader5");
+          }
+          else if (authOpCode.isPut()) {
+            opCredentials = getUserPassword("writer1");
+          }
+          else if (authOpCode.isGet()) {
+            opCredentials = getUserPassword("reader1");
+          }
+          else {
+            fail("executeRIOpBlock: cannot determine credentials for"
+                + credentialsTypeStr);
+          }
+        }
+        Properties clientProps = SecurityTestUtil
+            .concatProperties(new Properties[] { opCredentials, extraAuthProps,
+                extraAuthzProps });
+        // Start the client with valid credentials but allowed or disallowed to
+        // perform an operation
+        getLogWriter().info(
+            "executeRIOpBlock: For client" + clientNum + credentialsTypeStr
+                + " credentials: " + opCredentials);
+        if (useThisVM) {
+          createCacheClient(authInit, clientProps, javaProps, new Integer[] {
+              port1, port2 }, null, Boolean.valueOf(false), new Integer(
+              SecurityTestUtil.NO_EXCEPTION));
+        }
+        else {
+          clientVM.invoke(ClientAuthorizationTestBase.class,
+              "createCacheClient", new Object[] { authInit, clientProps,
+                  javaProps, new Integer[] { port1, port2 }, null,
+                  Boolean.valueOf(false),
+                  new Integer(SecurityTestUtil.NO_EXCEPTION) });
+        }
+      }
+      int expectedResult;
+      if ((opFlags & OpFlags.CHECK_NOTAUTHZ) > 0) {
+        expectedResult = SecurityTestUtil.NOTAUTHZ_EXCEPTION;
+      }
+      else if ((opFlags & OpFlags.CHECK_EXCEPTION) > 0) {
+        expectedResult = SecurityTestUtil.OTHER_EXCEPTION;
+      }
+      else {
+        expectedResult = SecurityTestUtil.NO_EXCEPTION;
+      }
+
+      // Perform the operation from selected client
+      if (useThisVM) {
+        doOp(new Byte(opCode.toOrdinal()), currentOp.getIndices(), new Integer(
+            opFlags), new Integer(expectedResult));
+      }
+      else {
+        clientVM.invoke(ClientAuthorizationTestBase.class, "doOp",
+            new Object[] { new Byte(opCode.toOrdinal()),
+                currentOp.getIndices(), new Integer(opFlags),
+                new Integer(expectedResult) });
+      }
+    }
+  }
+
+  // Region: Tests
+
+  public void testAllowPutsGets() {
+      AuthzCredentialGenerator gen = getXmlAuthzGenerator();
+      CredentialGenerator cGen = gen.getCredentialGenerator();
+      Properties extraAuthProps = cGen.getSystemProperties();
+      Properties javaProps = cGen.getJavaProperties();
+      Properties extraAuthzProps = gen.getSystemProperties();
+      String authenticator = cGen.getAuthenticator();
+      String authInit = cGen.getAuthInit();
+      String accessor = gen.getAuthorizationCallback();
+
+      getLogWriter().info("testAllowPutsGets: Using authinit: " + authInit);
+      getLogWriter().info(
+          "testAllowPutsGets: Using authenticator: " + authenticator);
+      getLogWriter().info("testAllowPutsGets: Using accessor: " + accessor);
+
+      // Start servers with all required properties
+      Properties serverProps = buildProperties(authenticator, accessor, false,
+          extraAuthProps, extraAuthzProps);
+      Integer port1 = ((Integer)server1.invoke(
+          ClientAuthorizationTestBase.class, "createCacheServer", new Object[] {
+              SecurityTestUtil.getLocatorPort(), serverProps, javaProps }));
+      Integer port2 = ((Integer)server2.invoke(
+          ClientAuthorizationTestBase.class, "createCacheServer", new Object[] {
+              SecurityTestUtil.getLocatorPort(), serverProps, javaProps }));
+
+      // Start client1 with valid CREATE credentials
+      Properties createCredentials = gen.getAllowedCredentials(
+          new OperationCode[] { OperationCode.PUT },
+          new String[] { regionName }, 1);
+      javaProps = cGen.getJavaProperties();
+      getLogWriter().info(
+          "testAllowPutsGets: For first client credentials: "
+              + createCredentials);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, createCredentials, javaProps, port1, port2,
+              null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Start client2 with valid GET credentials
+      Properties getCredentials = gen.getAllowedCredentials(
+          new OperationCode[] { OperationCode.GET },
+          new String[] { regionName }, 2);
+      javaProps = cGen.getJavaProperties();
+      getLogWriter()
+          .info(
+              "testAllowPutsGets: For second client credentials: "
+                  + getCredentials);
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, getCredentials, javaProps, port1, port2,
+              null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+          new Integer(2), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Verify that the gets succeed
+      client2.invoke(SecurityTestUtil.class, "doGets", new Object[] {
+          new Integer(2), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+  }
+
+  public void testDisallowPutsGets() {
+
+      AuthzCredentialGenerator gen = getXmlAuthzGenerator();
+      CredentialGenerator cGen = gen.getCredentialGenerator();
+      Properties extraAuthProps = cGen.getSystemProperties();
+      Properties javaProps = cGen.getJavaProperties();
+      Properties extraAuthzProps = gen.getSystemProperties();
+      String authenticator = cGen.getAuthenticator();
+      String authInit = cGen.getAuthInit();
+      String accessor = gen.getAuthorizationCallback();
+
+      getLogWriter().info("testDisallowPutsGets: Using authinit: " + authInit);
+      getLogWriter().info(
+          "testDisallowPutsGets: Using authenticator: " + authenticator);
+      getLogWriter().info("testDisallowPutsGets: Using accessor: " + accessor);
+
+      // Check that we indeed can obtain valid credentials not allowed to do
+      // gets
+      Properties createCredentials = gen.getAllowedCredentials(
+          new OperationCode[] { OperationCode.PUT },
+          new String[] { regionName }, 1);
+      Properties createJavaProps = cGen.getJavaProperties();
+      getLogWriter().info(
+          "testDisallowPutsGets: For first client credentials: "
+              + createCredentials);
+      Properties getCredentials = gen.getDisallowedCredentials(
+          new OperationCode[] { OperationCode.GET },
+          new String[] { regionName }, 2);
+      Properties getJavaProps = cGen.getJavaProperties();
+
+      getLogWriter().info(
+          "testDisallowPutsGets: For second client disallowed GET credentials: "
+              + getCredentials);
+
+      // Start servers with all required properties
+      Properties serverProps = buildProperties(authenticator, accessor, false,
+          extraAuthProps, extraAuthzProps);
+      Integer port1 = ((Integer)server1.invoke(
+          ClientAuthorizationTestBase.class, "createCacheServer", new Object[] {
+              SecurityTestUtil.getLocatorPort(), serverProps, javaProps }));
+      Integer port2 = ((Integer)server2.invoke(
+          ClientAuthorizationTestBase.class, "createCacheServer", new Object[] {
+              SecurityTestUtil.getLocatorPort(), serverProps, javaProps }));
+
+      // Start client1 with valid CREATE credentials
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, createCredentials, createJavaProps, port1,
+              port2, null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Start client2 with invalid GET credentials
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, getCredentials, getJavaProps, port1, port2,
+              null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+          new Integer(2), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Gets as normal user should throw exception
+      client2.invoke(SecurityTestUtil.class, "doGets", new Object[] {
+          new Integer(2), new Integer(SecurityTestUtil.NOTAUTHZ_EXCEPTION) });
+
+      // Try to connect client2 with reader credentials
+      getCredentials = gen.getAllowedCredentials(
+          new OperationCode[] { OperationCode.GET },
+          new String[] { regionName }, 5);
+      getJavaProps = cGen.getJavaProperties();
+      getLogWriter().info(
+          "testDisallowPutsGets: For second client with GET credentials: "
+              + getCredentials);
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, getCredentials, getJavaProps, port1, port2,
+              null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Verify that the gets succeed
+      client2.invoke(SecurityTestUtil.class, "doGets", new Object[] {
+          new Integer(2), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Verify that the puts throw exception
+      client2.invoke(SecurityTestUtil.class, "doNPuts", new Object[] {
+          new Integer(2), new Integer(SecurityTestUtil.NOTAUTHZ_EXCEPTION) });
+  }
+
+  public void testInvalidAccessor() {
+      AuthzCredentialGenerator gen = getXmlAuthzGenerator();
+      CredentialGenerator cGen = gen.getCredentialGenerator();
+      Properties extraAuthProps = cGen.getSystemProperties();
+      Properties javaProps = cGen.getJavaProperties();
+      Properties extraAuthzProps = gen.getSystemProperties();
+      String authenticator = cGen.getAuthenticator();
+      String authInit = cGen.getAuthInit();
+      String accessor = gen.getAuthorizationCallback();
+
+      getLogWriter().info("testInvalidAccessor: Using authinit: " + authInit);
+      getLogWriter().info(
+          "testInvalidAccessor: Using authenticator: " + authenticator);
+
+      // Start server1 with invalid accessor
+      Properties serverProps = buildProperties(authenticator,
+          "com.gemstone.none", false, extraAuthProps, extraAuthzProps);
+      Integer port1 = ((Integer)server1.invoke(
+          ClientAuthorizationTestBase.class, "createCacheServer", new Object[] {
+              SecurityTestUtil.getLocatorPort(), serverProps, javaProps }));
+      Integer port2 = new Integer(AvailablePort
+          .getRandomAvailablePort(AvailablePort.SOCKET));
+
+      // Client creation should throw exceptions
+      Properties createCredentials = gen.getAllowedCredentials(
+          new OperationCode[] { OperationCode.PUT },
+          new String[] { regionName }, 3);
+      Properties createJavaProps = cGen.getJavaProperties();
+      getLogWriter().info(
+          "testInvalidAccessor: For first client CREATE credentials: "
+              + createCredentials);
+      Properties getCredentials = gen.getAllowedCredentials(
+          new OperationCode[] { OperationCode.GET },
+          new String[] { regionName }, 7);
+      Properties getJavaProps = cGen.getJavaProperties();
+      getLogWriter().info(
+          "testInvalidAccessor: For second client GET credentials: "
+              + getCredentials);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, createCredentials, createJavaProps, port1,
+              port2, null, Boolean.FALSE, Boolean.FALSE,
+              Integer.valueOf(SecurityTestUtil.NO_EXCEPTION) });
+      client1.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+          new Integer(1), new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) });
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, getCredentials, getJavaProps, port1, port2,
+              null, Boolean.FALSE, Boolean.FALSE,
+              Integer.valueOf(SecurityTestUtil.NO_EXCEPTION) });
+      client2.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+          new Integer(1), new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) });
+
+      // Now start server2 that has valid accessor
+      getLogWriter().info("testInvalidAccessor: Using accessor: " + accessor);
+      serverProps = buildProperties(authenticator, accessor, false,
+          extraAuthProps, extraAuthzProps);
+      server2.invoke(ClientAuthorizationTestBase.class, "createCacheServer",
+          new Object[] { SecurityTestUtil.getLocatorPort(), port2, serverProps,
+              javaProps });
+      server1.invoke(SecurityTestUtil.class, "closeCache");
+
+      // Client creation should be successful now
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, createCredentials, createJavaProps, port1,
+              port2, null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, getCredentials, getJavaProps, port1, port2,
+              null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Now perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+          new Integer(4), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Verify that the gets succeed
+      client2.invoke(SecurityTestUtil.class, "doGets", new Object[] {
+          new Integer(4), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+  }
+
+  public void testPutsGetsWithFailover() {
+      AuthzCredentialGenerator gen = getXmlAuthzGenerator();
+      CredentialGenerator cGen = gen.getCredentialGenerator();
+      Properties extraAuthProps = cGen.getSystemProperties();
+      Properties javaProps = cGen.getJavaProperties();
+      Properties extraAuthzProps = gen.getSystemProperties();
+      String authenticator = cGen.getAuthenticator();
+      String authInit = cGen.getAuthInit();
+      String accessor = gen.getAuthorizationCallback();
+
+      getLogWriter().info(
+          "testPutsGetsWithFailover: Using authinit: " + authInit);
+      getLogWriter().info(
+          "testPutsGetsWithFailover: Using authenticator: " + authenticator);
+      getLogWriter().info(
+          "testPutsGetsWithFailover: Using accessor: " + accessor);
+
+      // Start servers with all required properties
+      Properties serverProps = buildProperties(authenticator, accessor, false,
+          extraAuthProps, extraAuthzProps);
+      Integer port1 = ((Integer)server1.invoke(
+          ClientAuthorizationTestBase.class, "createCacheServer", new Object[] {
+              SecurityTestUtil.getLocatorPort(), serverProps, javaProps }));
+      // Get a port for second server but do not start it
+      // This forces the clients to connect to the first server
+      Integer port2 = new Integer(AvailablePort
+          .getRandomAvailablePort(AvailablePort.SOCKET));
+
+      // Start client1 with valid CREATE credentials
+      Properties createCredentials = gen.getAllowedCredentials(
+          new OperationCode[] { OperationCode.PUT },
+          new String[] { regionName }, 1);
+      Properties createJavaProps = cGen.getJavaProperties();
+      getLogWriter().info(
+          "testPutsGetsWithFailover: For first client credentials: "
+              + createCredentials);
+      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, createCredentials, createJavaProps, port1,
+              port2, null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Start client2 with valid GET credentials
+      Properties getCredentials = gen.getAllowedCredentials(
+          new OperationCode[] { OperationCode.GET },
+          new String[] { regionName }, 5);
+      Properties getJavaProps = cGen.getJavaProperties();
+      getLogWriter().info(
+          "testPutsGetsWithFailover: For second client credentials: "
+              + getCredentials);
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, getCredentials, getJavaProps, port1, port2,
+              null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+          new Integer(2), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+      // Verify that the puts succeeded
+      client2.invoke(SecurityTestUtil.class, "doGets", new Object[] {
+          new Integer(2), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // start the second one and stop the first server to force a failover
+      server2.invoke(ClientAuthorizationTestBase.class, "createCacheServer",
+          new Object[] { SecurityTestUtil.getLocatorPort(), port2, serverProps,
+              javaProps });
+      server1.invoke(SecurityTestUtil.class, "closeCache");
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doNPuts", new Object[] {
+          new Integer(4), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+      // Verify that the puts succeeded
+      client2.invoke(SecurityTestUtil.class, "doNGets", new Object[] {
+          new Integer(4), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Now re-connect with credentials not allowed to do gets
+      Properties noGetCredentials = gen.getDisallowedCredentials(
+          new OperationCode[] { OperationCode.GET },
+          new String[] { regionName }, 9);
+      getJavaProps = cGen.getJavaProperties();
+
+      getLogWriter().info(
+          "testPutsGetsWithFailover: For second client disallowed GET credentials: "
+              + noGetCredentials);
+
+      // Re-connect client2 with invalid GET credentials
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, noGetCredentials, getJavaProps, port1,
+              port2, null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+          new Integer(4), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+      // Gets as normal user should throw exception
+      client2.invoke(SecurityTestUtil.class, "doGets", new Object[] {
+          new Integer(4), new Integer(SecurityTestUtil.NOTAUTHZ_EXCEPTION) });
+
+      // force a failover and do the drill again
+      server1.invoke(ClientAuthorizationTestBase.class, "createCacheServer",
+          new Object[] { SecurityTestUtil.getLocatorPort(), port1, serverProps,
+              javaProps });
+      server2.invoke(SecurityTestUtil.class, "closeCache");
+
+      // Perform some put operations from client1
+      client1.invoke(SecurityTestUtil.class, "doNPuts", new Object[] {
+          new Integer(4), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+      // Gets as normal user should throw exception
+      client2.invoke(SecurityTestUtil.class, "doNGets", new Object[] {
+          new Integer(4), new Integer(SecurityTestUtil.NOTAUTHZ_EXCEPTION) });
+
+      // Try to connect client2 with reader credentials
+      client2.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
+          new Object[] { authInit, getCredentials, getJavaProps, port1, port2,
+              null, new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Verify that the gets succeed
+      client2.invoke(SecurityTestUtil.class, "doNGets", new Object[] {
+          new Integer(4), new Integer(SecurityTestUtil.NO_EXCEPTION) });
+
+      // Verify that the puts throw exception
+      client2.invoke(SecurityTestUtil.class, "doPuts", new Object[] {
+          new Integer(4), new Integer(SecurityTestUtil.NOTAUTHZ_EXCEPTION) });
+  }
+
+  public void testUnregisterInterestWithFailover() {
+
+    OperationWithAction[] unregisterOps = {
+        // Register interest in all keys using one key at a time
+        new OperationWithAction(OperationCode.REGISTER_INTEREST,
+            OperationCode.UNREGISTER_INTEREST, 3, OpFlags.NONE, 4),
+        new OperationWithAction(OperationCode.REGISTER_INTEREST, 2),
+        // UPDATE and test with GET
+        new OperationWithAction(OperationCode.PUT),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP, 4),
+
+        // Unregister interest in all keys using one key at a time
+        new OperationWithAction(OperationCode.UNREGISTER_INTEREST, 3,
+            OpFlags.USE_OLDCONN | OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.UNREGISTER_INTEREST, 2,
+            OpFlags.USE_OLDCONN, 4),
+        // UPDATE and test with GET for no updates
+        new OperationWithAction(OperationCode.PUT, 1, OpFlags.USE_OLDCONN
+            | OpFlags.USE_NEWVAL, 4),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP, 4),
+
+        OperationWithAction.OPBLOCK_END,
+
+        // Register interest in all keys using list
+        new OperationWithAction(OperationCode.REGISTER_INTEREST,
+            OperationCode.UNREGISTER_INTEREST, 3, OpFlags.USE_LIST, 4),
+        new OperationWithAction(OperationCode.REGISTER_INTEREST, 1,
+            OpFlags.USE_LIST, 4),
+        // UPDATE and test with GET
+        new OperationWithAction(OperationCode.PUT, 2),
+        new OperationWithAction(OperationCode.GET, 1, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP, 4),
+
+        // Unregister interest in all keys using list
+        new OperationWithAction(OperationCode.UNREGISTER_INTEREST, 3,
+            OpFlags.USE_OLDCONN | OpFlags.USE_LIST | OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.UNREGISTER_INTEREST, 1,
+            OpFlags.USE_OLDCONN | OpFlags.USE_LIST, 4),
+        // UPDATE and test with GET for no updates
+        new OperationWithAction(OperationCode.PUT, 2, OpFlags.USE_OLDCONN
+            | OpFlags.USE_NEWVAL, 4),
+        new OperationWithAction(OperationCode.GET, 1, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP, 4),
+
+        OperationWithAction.OPBLOCK_END,
+
+        // Register interest in all keys using regular expression
+        new OperationWithAction(OperationCode.REGISTER_INTEREST,
+            OperationCode.UNREGISTER_INTEREST, 3, OpFlags.USE_REGEX, 4),
+        new OperationWithAction(OperationCode.REGISTER_INTEREST, 2,
+            OpFlags.USE_REGEX, 4),
+        // UPDATE and test with GET
+        new OperationWithAction(OperationCode.PUT),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP, 4),
+
+        // Unregister interest in all keys using regular expression
+        new OperationWithAction(OperationCode.UNREGISTER_INTEREST, 3,
+            OpFlags.USE_OLDCONN | OpFlags.USE_REGEX | OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.UNREGISTER_INTEREST, 2,
+            OpFlags.USE_OLDCONN | OpFlags.USE_REGEX, 4),
+        // UPDATE and test with GET for no updates
+        new OperationWithAction(OperationCode.PUT, 1, OpFlags.USE_OLDCONN
+            | OpFlags.USE_NEWVAL, 4),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP, 4),
+
+        OperationWithAction.OPBLOCK_END };
+
+    AuthzCredentialGenerator gen = new XmlAuthzCredentialGenerator();
+    CredentialGenerator cGen = new DummyCredentialGenerator();
+    cGen.init();
+    gen.init(cGen);
+    Properties extraAuthProps = cGen.getSystemProperties();
+    Properties javaProps = cGen.getJavaProperties();
+    Properties extraAuthzProps = gen.getSystemProperties();
+    String authenticator = cGen.getAuthenticator();
+    String authInit = cGen.getAuthInit();
+    String accessor = gen.getAuthorizationCallback();
+
+    getLogWriter().info("testAllOpsWithFailover: Using authinit: " + authInit);
+    getLogWriter().info(
+        "testAllOpsWithFailover: Using authenticator: " + authenticator);
+    getLogWriter().info("testAllOpsWithFailover: Using accessor: " + accessor);
+
+    // Start servers with all required properties
+    Properties serverProps = buildProperties(authenticator, accessor, false,
+        extraAuthProps, extraAuthzProps);
+    // Get ports for the servers
+    Integer port1 = new Integer(AvailablePort
+        .getRandomAvailablePort(AvailablePort.SOCKET));
+    Integer port2 = new Integer(AvailablePort
+        .getRandomAvailablePort(AvailablePort.SOCKET));
+
+    // Perform all the ops on the clients
+    List opBlock = new ArrayList();
+    for (int opNum = 0; opNum < unregisterOps.length; ++opNum) {
+      // Start client with valid credentials as specified in
+      // OperationWithAction
+      OperationWithAction currentOp = unregisterOps[opNum];
+      if (currentOp.equals(OperationWithAction.OPBLOCK_END)
+          || currentOp.equals(OperationWithAction.OPBLOCK_NO_FAILOVER)) {
+        // End of current operation block; execute all the operations
+        // on the servers with/without failover
+        if (opBlock.size() > 0) {
+          // Start the first server and execute the operation block
+          server1.invoke(ClientAuthorizationTestBase.class,
+              "createCacheServer", new Object[] {
+                  SecurityTestUtil.getLocatorPort(), port1, serverProps,
+                  javaProps });
+          server2.invoke(SecurityTestUtil.class, "closeCache");
+          executeRIOpBlock(opBlock, port1, port2, authInit, extraAuthProps,
+              extraAuthzProps, javaProps);
+          if (!currentOp.equals(OperationWithAction.OPBLOCK_NO_FAILOVER)) {
+            // Failover to the second server and run the block again
+            server2.invoke(ClientAuthorizationTestBase.class,
+                "createCacheServer", new Object[] {
+                    SecurityTestUtil.getLocatorPort(), port2, serverProps,
+                    javaProps });
+            server1.invoke(SecurityTestUtil.class, "closeCache");
+            executeRIOpBlock(opBlock, port1, port2, authInit, extraAuthProps,
+                extraAuthzProps, javaProps);
+          }
+          opBlock.clear();
+        }
+      }
+      else {
+        currentOp.setOpNum(opNum);
+        opBlock.add(currentOp);
+      }
+    }
+  }
+
+  
+  public void testAllOpsWithFailover() {
+    addExpectedException("Read timed out");
+
+    OperationWithAction[] allOps = {
+        // Test CREATE and verify with a GET
+        new OperationWithAction(OperationCode.PUT, 3, OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.PUT),
+        new OperationWithAction(OperationCode.GET, 3, OpFlags.CHECK_NOKEY
+            | OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.CHECK_NOKEY, 4),
+
+        // OPBLOCK_END indicates end of an operation block; the above block of
+        // three operations will be first executed on server1 and then on
+        // server2 after failover
+        OperationWithAction.OPBLOCK_END,
+
+        // Test PUTALL and verify with GETs
+        new OperationWithAction(OperationCode.PUTALL, 3, OpFlags.USE_NEWVAL
+            | OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.PUTALL, 1, OpFlags.USE_NEWVAL, 4),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.USE_NEWVAL, 4),
+        OperationWithAction.OPBLOCK_END,
+        
+        // Test UPDATE and verify with a GET
+        new OperationWithAction(OperationCode.PUT, 3, OpFlags.USE_NEWVAL
+            | OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.PUT, 1, OpFlags.USE_NEWVAL, 4),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.USE_NEWVAL, 4),
+
+        OperationWithAction.OPBLOCK_END,
+
+        // Test DESTROY and verify with a GET and that key should not exist
+        new OperationWithAction(OperationCode.DESTROY, 3, OpFlags.USE_NEWVAL
+            | OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.DESTROY),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.CHECK_FAIL, 4), // bruce: added check_nokey because we now bring tombstones to the client in 8.0
+        // Repopulate the region
+        new OperationWithAction(OperationCode.PUT, 1, OpFlags.USE_NEWVAL, 4),
+
+        OperationWithAction.OPBLOCK_END,
+
+        // Check CONTAINS_KEY
+        new OperationWithAction(OperationCode.CONTAINS_KEY, 3,
+            OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.CONTAINS_KEY),
+        // Destroy the keys and check for failure in CONTAINS_KEY
+        new OperationWithAction(OperationCode.DESTROY, 2),
+        new OperationWithAction(OperationCode.CONTAINS_KEY, 3,
+            OpFlags.CHECK_FAIL | OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.CONTAINS_KEY, 1,
+            OpFlags.USE_OLDCONN | OpFlags.CHECK_FAIL, 4),
+        // Repopulate the region
+        new OperationWithAction(OperationCode.PUT),
+
+        OperationWithAction.OPBLOCK_END,
+
+        // Check KEY_SET
+        new OperationWithAction(OperationCode.KEY_SET, 3,
+            OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.KEY_SET, 2),
+
+        OperationWithAction.OPBLOCK_END,
+
+        // Check QUERY
+        new OperationWithAction(OperationCode.QUERY, 3, OpFlags.CHECK_NOTAUTHZ,
+            4),
+        new OperationWithAction(OperationCode.QUERY),
+
+        OperationWithAction.OPBLOCK_END,
+
+        // Register interest in all keys using one key at a time
+        new OperationWithAction(OperationCode.REGISTER_INTEREST, 3,
+            OpFlags.CHECK_NOTAUTHZ, 4),
+        new OperationWithAction(OperationCode.REGISTER_INTEREST, 2),
+        // UPDATE and test with GET
+        new OperationWithAction(OperationCode.PUT),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP, 4),
+
+        // Unregister interest in all keys using one key at a time
+        new OperationWithAction(OperationCode.UNREGISTER_INTEREST, 2,
+            OpFlags.USE_OLDCONN, 4),
+        // UPDATE and test with GET for no updates
+        new OperationWithAction(OperationCode.PUT, 1, OpFlags.USE_OLDCONN
+            | OpFlags.USE_NEWVAL, 4),
+        new OperationWithAction(OperationCode.GET, 2, OpFlags.USE_OLDCONN
+            | OpFlags.LOCAL_OP, 4),
+
+        OperationWithAction.OPBLOCK_END,
+
+        // Test GET_ENTRY inside a TX, see #49951
+        new OperationWithAction(OperationCode.GET, 2,
+            OpFlags.USE_GET_ENTRY_IN_TX | OpFlags.CHECK_FAIL, 4),
+
+        OperationWithAction.OPBLOCK_END };
+
+    runOpsWithFailover(allOps, "testAllOpsWithFailover");
+  }
+
+  // End Region: Tests
+
+  @Override
+  public void tearDown2() throws Exception {
+
+    super.tearDown2();
+    // close the clients first
+    client1.invoke(SecurityTestUtil.class, "closeCache");
+    client2.invoke(SecurityTestUtil.class, "closeCache");
+    SecurityTestUtil.closeCache();
+    // then close the servers
+    server1.invoke(SecurityTestUtil.class, "closeCache");
+    server2.invoke(SecurityTestUtil.class, "closeCache");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestBase.java
new file mode 100644
index 0000000..ad6526a
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestBase.java
@@ -0,0 +1,1384 @@
+
+package com.gemstone.gemfire.security;
+
+/*
+ * 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.
+ */
+
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+
+import templates.security.AuthzCredentialGenerator;
+import templates.security.CredentialGenerator;
+import templates.security.AuthzCredentialGenerator.ClassCode;
+
+import com.gemstone.gemfire.cache.DynamicRegionFactory;
+import com.gemstone.gemfire.cache.InterestResultPolicy;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.Region.Entry;
+import com.gemstone.gemfire.cache.RegionDestroyedException;
+import com.gemstone.gemfire.cache.client.ServerConnectivityException;
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.cache.query.CqAttributes;
+import com.gemstone.gemfire.cache.query.CqAttributesFactory;
+import com.gemstone.gemfire.cache.query.CqEvent;
+import com.gemstone.gemfire.cache.query.CqException;
+import com.gemstone.gemfire.cache.query.CqListener;
+import com.gemstone.gemfire.cache.query.CqQuery;
+import com.gemstone.gemfire.cache.query.QueryInvocationTargetException;
+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.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.AvailablePort.Keeper;
+import com.gemstone.gemfire.internal.cache.AbstractRegionEntry;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.util.Callable;
+
+import dunit.DistributedTestCase;
+import dunit.VM;
+import templates.security.DummyCredentialGenerator;
+import templates.security.XmlAuthzCredentialGenerator;
+
+/**
+ * Base class for tests for authorization from client to server. It contains
+ * utility functions for the authorization tests from client to server.
+ * 
+ * @author sumedh
+ * @since 5.5
+ */
+public class ClientAuthorizationTestBase extends DistributedTestCase {
+
+  /** constructor */
+  public ClientAuthorizationTestBase(String name) {
+    super(name);
+  }
+
+  protected static VM server1 = null;
+
+  protected static VM server2 = null;
+
+  protected static VM client1 = null;
+
+  protected static VM client2 = null;
+
+  protected static final String regionName = SecurityTestUtil.regionName;
+
+  protected static final String subregionName = "AuthSubregion";
+
+  protected static final String[] serverExpectedExceptions = {
+      "Connection refused",
+      AuthenticationRequiredException.class.getName(),
+      AuthenticationFailedException.class.getName(),
+      NotAuthorizedException.class.getName(),
+      GemFireSecurityException.class.getName(),
+      RegionDestroyedException.class.getName(),
+      ClassNotFoundException.class.getName() };
+
+  protected static final String[] clientExpectedExceptions = {
+      AuthenticationFailedException.class.getName(),
+      NotAuthorizedException.class.getName(),
+      RegionDestroyedException.class.getName() };
+
+  protected static Properties buildProperties(String authenticator,
+      String accessor, boolean isAccessorPP, Properties extraAuthProps,
+      Properties extraAuthzProps) {
+
+    Properties authProps = new Properties();
+    if (authenticator != null) {
+      authProps.setProperty(
+          DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, authenticator);
+    }
+    if (accessor != null) {
+      if (isAccessorPP) {
+        authProps.setProperty(
+            DistributionConfig.SECURITY_CLIENT_ACCESSOR_PP_NAME, accessor);
+      }
+      else {
+        authProps.setProperty(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME,
+            accessor);
+      }
+    }
+    return SecurityTestUtil.concatProperties(new Properties[] { authProps,
+        extraAuthProps, extraAuthzProps });
+  }
+
+  public static Integer createCacheServer(Integer locatorPort, Object authProps,
+      Object javaProps) {
+
+    if (locatorPort == null) {
+      locatorPort = new Integer(AvailablePort
+          .getRandomAvailablePort(AvailablePort.SOCKET));
+    }
+    return SecurityTestUtil.createCacheServer((Properties)authProps, javaProps,
+        locatorPort, null, null, Boolean.TRUE, new Integer(
+            SecurityTestUtil.NO_EXCEPTION));
+  }
+
+  public static void createCacheServer(Integer locatorPort, Integer serverPort,
+      Object authProps, Object javaProps) {
+    if (locatorPort == null) {
+      locatorPort = new Integer(AvailablePort
+          .getRandomAvailablePort(AvailablePort.SOCKET));
+    }
+    SecurityTestUtil.createCacheServer((Properties)authProps, javaProps,
+        locatorPort, null, serverPort, Boolean.TRUE, new Integer(
+            SecurityTestUtil.NO_EXCEPTION));
+  }
+
+  public static void createCacheClient(Object authInit, Object authProps,
+      Object javaProps, Integer[] ports, Integer numConnections,
+      Boolean setupDynamicRegionFactory, Integer expectedResult) {
+
+    String authInitStr = (authInit == null ? null : authInit.toString());
+    if (authProps == null) {
+      authProps = new Properties();
+    }
+    SecurityTestUtil.createCacheClient(authInitStr, (Properties)authProps,
+        (Properties)javaProps, ports, numConnections,
+        setupDynamicRegionFactory, expectedResult);
+  }
+
+  protected static Region getRegion() {
+    return SecurityTestUtil.getCache().getRegion(regionName);
+  }
+
+  protected static Region getSubregion() {
+    return SecurityTestUtil.getCache().getRegion(
+        regionName + '/' + subregionName);
+  }
+
+  private static Region createSubregion(Region region) {
+
+    Region subregion = getSubregion();
+    if (subregion == null) {
+      subregion = region.createSubregion(subregionName, region.getAttributes());
+    }
+    return subregion;
+  }
+
+  protected static String indicesToString(int[] indices) {
+
+    String str = "";
+    if (indices != null && indices.length > 0) {
+      str += indices[0];
+      for (int index = 1; index < indices.length; ++index) {
+        str += ",";
+        str += indices[index];
+      }
+    }
+    return str;
+  }
+
+  private static final int PAUSE = 5 * 1000;
+  
+  public static void doOp(Byte opCode, int[] indices, Integer flagsI,
+      Integer expectedResult) {
+
+    OperationCode op = OperationCode.fromOrdinal(opCode.byteValue());
+    boolean operationOmitted = false;
+    final int flags = flagsI.intValue();
+    Region region = getRegion();
+    if ((flags & OpFlags.USE_SUBREGION) > 0) {
+      assertNotNull(region);
+      Region subregion = null;
+      if ((flags & OpFlags.NO_CREATE_SUBREGION) > 0) {
+        if ((flags & OpFlags.CHECK_NOREGION) > 0) {
+          // Wait for some time for DRF update to come
+          SecurityTestUtil.waitForCondition(new Callable() {
+            public Object call() throws Exception {
+              return Boolean.valueOf(getSubregion() == null);
+            }
+          });
+          subregion = getSubregion();
+          assertNull(subregion);
+          return;
+        }
+        else {
+          // Wait for some time for DRF update to come
+          SecurityTestUtil.waitForCondition(new Callable() {
+            public Object call() throws Exception {
+              return Boolean.valueOf(getSubregion() != null);
+            }
+          });
+          subregion = getSubregion();
+          assertNotNull(subregion);
+        }
+      }
+      else {
+        subregion = createSubregion(region);
+      }
+      assertNotNull(subregion);
+      region = subregion;
+    }
+    else if ((flags & OpFlags.CHECK_NOREGION) > 0) {
+      // Wait for some time for region destroy update to come
+      SecurityTestUtil.waitForCondition(new Callable() {
+        public Object call() throws Exception {
+          return Boolean.valueOf(getRegion() == null);
+        }
+      });
+      region = getRegion();
+      assertNull(region);
+      return;
+    }
+    else {
+      assertNotNull(region);
+    }
+    final String[] keys = SecurityTestUtil.keys;
+    final String[] vals;
+    if ((flags & OpFlags.USE_NEWVAL) > 0) {
+      vals = SecurityTestUtil.nvalues;
+    }
+    else {
+      vals = SecurityTestUtil.values;
+    }
+    InterestResultPolicy policy = InterestResultPolicy.KEYS_VALUES;
+    if ((flags & OpFlags.REGISTER_POLICY_NONE) > 0) {
+      policy = InterestResultPolicy.NONE;
+    }
+    final int numOps = indices.length;
+    getLogWriter().info(
+        "Got doOp for op: " + op.toString() + ", numOps: " + numOps
+            + ", indices: " + indicesToString(indices) + ", expect: " + expectedResult);
+    boolean exceptionOccured = false;
+    boolean breakLoop = false;
+    if (op.isGet() || 
+        op.isContainsKey() || 
+        op.isKeySet() || 
+        op.isQuery() || 
+        op.isExecuteCQ()) {
+      try {
+        Thread.sleep(PAUSE);
+      }
+      catch (InterruptedException e) {
+        fail("interrupted");
+      }
+    }
+    for (int indexIndex = 0; indexIndex < numOps; ++indexIndex) {
+      if (breakLoop) {
+        break;
+      }
+      int index = indices[indexIndex];
+      try {
+        final Object key = keys[index];
+        final Object expectedVal = vals[index];
+        if (op.isGet()) {
+          Object value = null;
+          // this is the case for testing GET_ALL
+          if ((flags & OpFlags.USE_ALL_KEYS) > 0) {
+            breakLoop = true;
+            List keyList = new ArrayList(numOps);
+            Object searchKey;
+            for (int keyNumIndex = 0; keyNumIndex < numOps; ++keyNumIndex) {
+              int keyNum = indices[keyNumIndex];
+              searchKey = keys[keyNum];
+              keyList.add(searchKey);
+              // local invalidate some keys to force fetch of those keys from
+              // server
+              if ((flags & OpFlags.CHECK_NOKEY) > 0) {
+                AbstractRegionEntry entry = (AbstractRegionEntry)((LocalRegion)region).getRegionEntry(searchKey);
+                getLogWriter().info(""+keyNum+": key is " + searchKey + " and entry is " + entry);
+                assertFalse(region.containsKey(searchKey));
+              }
+              else {
+                if (keyNumIndex % 2 == 1) {
+                  assertTrue(region.containsKey(searchKey));
+                  region.localInvalidate(searchKey);
+                }
+              }
+            }
+            Map entries = region.getAll(keyList);
+            for (int keyNumIndex = 0; keyNumIndex < numOps; ++keyNumIndex) {
+              int keyNum = indices[keyNumIndex];
+              searchKey = keys[keyNum];
+              if ((flags & OpFlags.CHECK_FAIL) > 0) {
+                assertFalse(entries.containsKey(searchKey));
+              }
+              else {
+                assertTrue(entries.containsKey(searchKey));
+                value = entries.get(searchKey);
+                assertEquals(vals[keyNum], value);
+              }
+            }
+            break;
+          }
+          if ((flags & OpFlags.LOCAL_OP) > 0) {
+            Callable cond = new Callable() {
+              private Region region;
+
+              public Object call() throws Exception {
+                Object value = SecurityTestUtil.getLocalValue(region, key);
+                return Boolean
+                    .valueOf((flags & OpFlags.CHECK_FAIL) > 0 ? !expectedVal
+                        .equals(value) : expectedVal.equals(value));
+              }
+
+              public Callable init(Region region) {
+                this.region = region;
+                return this;
+              }
+            }.init(region);
+            SecurityTestUtil.waitForCondition(cond);
+            value = SecurityTestUtil.getLocalValue(region, key);
+          }
+          else if ((flags & OpFlags.USE_GET_ENTRY_IN_TX) > 0) {
+            SecurityTestUtil.getCache().getCacheTransactionManager().begin();
+            Entry e = region.getEntry(key);
+            // Also, check getAll()
+            ArrayList a = new ArrayList();
+            a.addAll(a);
+            region.getAll(a);
+
+            SecurityTestUtil.getCache().getCacheTransactionManager().commit();
+            value = e.getValue();
+          }
+          else {
+            if ((flags & OpFlags.CHECK_NOKEY) > 0) {
+              assertFalse(region.containsKey(key));
+            }
+            else {
+              assertTrue(region.containsKey(key) || ((LocalRegion)region).getRegionEntry(key).isTombstone());
+              region.localInvalidate(key);
+            }
+            value = region.get(key);
+          }
+          if ((flags & OpFlags.CHECK_FAIL) > 0) {
+            assertFalse(expectedVal.equals(value));
+          }
+          else {
+            assertNotNull(value);
+            assertEquals(expectedVal, value);
+          }
+        }
+        else if (op.isPut()) {
+          region.put(key, expectedVal);
+        }
+        else if (op.isPutAll()) {
+          HashMap map = new HashMap();
+          for (int i=0; i<indices.length; i++) {
+            map.put(keys[indices[i]], vals[indices[i]]);
+          }
+          region.putAll(map);
+          breakLoop = true;
+        }
+        else if (op.isDestroy()) {
+          // if (!region.containsKey(key)) {
+          // // Since DESTROY will fail unless the value is present
+          // // in the local cache, this is a workaround for two cases:
+          // // 1. When the operation is supposed to succeed then in
+          // // the current AuthzCredentialGenerators the clients having
+          // // DESTROY permission also has CREATE/UPDATE permission
+          // // so that calling region.put() will work for that case.
+          // // 2. When the operation is supposed to fail with
+          // // NotAuthorizedException then in the current
+          // // AuthzCredentialGenerators the clients not
+          // // having DESTROY permission are those with reader role that have
+          // // GET permission.
+          // //
+          // // If either of these assumptions fails, then this has to be
+          // // adjusted or reworked accordingly.
+          // if ((flags & OpFlags.CHECK_NOTAUTHZ) > 0) {
+          // Object value = region.get(key);
+          // assertNotNull(value);
+          // assertEquals(vals[index], value);
+          // }
+          // else {
+          // region.put(key, vals[index]);
+          // }
+          // }
+          if ((flags & OpFlags.LOCAL_OP) > 0) {
+            region.localDestroy(key);
+          }
+          else {
+            region.destroy(key);
+          }
+        }
+        else if (op.isInvalidate()) {
+          if (region.containsKey(key)) {
+            if ((flags & OpFlags.LOCAL_OP) > 0) {
+              region.localInvalidate(key);
+            }
+            else {
+              region.invalidate(key);
+            }
+          }
+        }
+        else if (op.isContainsKey()) {
+          boolean result;
+          if ((flags & OpFlags.LOCAL_OP) > 0) {
+            result = region.containsKey(key);
+          }
+          else {
+            result = region.containsKeyOnServer(key);
+          }
+          if ((flags & OpFlags.CHECK_FAIL) > 0) {
+            assertFalse(result);
+          }
+          else {
+            assertTrue(result);
+          }
+        }
+        else if (op.isRegisterInterest()) {
+          if ((flags & OpFlags.USE_LIST) > 0) {
+            breakLoop = true;
+            // Register interest list in this case
+            List keyList = new ArrayList(numOps);
+            for (int keyNumIndex = 0; keyNumIndex < numOps; ++keyNumIndex) {
+              int keyNum = indices[keyNumIndex];
+              keyList.add(keys[keyNum]);
+            }
+            region.registerInterest(keyList, policy);
+          }
+          else if ((flags & OpFlags.USE_REGEX) > 0) {
+            breakLoop = true;
+            region.registerInterestRegex("key[1-" + numOps + ']', policy);
+          }
+          else if ((flags & OpFlags.USE_ALL_KEYS) > 0) {
+            breakLoop = true;
+            region.registerInterest("ALL_KEYS", policy);
+          }
+          else {
+            region.registerInterest(key, policy);
+          }
+        }
+        else if (op.isUnregisterInterest()) {
+          if ((flags & OpFlags.USE_LIST) > 0) {
+            breakLoop = true;
+            // Register interest list in this case
+            List keyList = new ArrayList(numOps);
+            for (int keyNumIndex = 0; keyNumIndex < numOps; ++keyNumIndex) {
+              int keyNum = indices[keyNumIndex];
+              keyList.add(keys[keyNum]);
+            }
+            region.unregisterInterest(keyList);
+          }
+          else if ((flags & OpFlags.USE_REGEX) > 0) {
+            breakLoop = true;
+            region.unregisterInterestRegex("key[1-" + numOps + ']');
+          }
+          else if ((flags & OpFlags.USE_ALL_KEYS) > 0) {
+            breakLoop = true;
+            region.unregisterInterest("ALL_KEYS");
+          }
+          else {
+            region.unregisterInterest(key);
+          }
+        }
+        else if (op.isKeySet()) {
+          breakLoop = true;
+          Set keySet;
+          if ((flags & OpFlags.LOCAL_OP) > 0) {
+            keySet = region.keySet();
+          }
+          else {
+            keySet = region.keySetOnServer();
+          }
+          assertNotNull(keySet);
+          if ((flags & OpFlags.CHECK_FAIL) == 0) {
+            assertEquals(numOps, keySet.size());
+          }
+          for (int keyNumIndex = 0; keyNumIndex < numOps; ++keyNumIndex) {
+            int keyNum = indices[keyNumIndex];
+            if ((flags & OpFlags.CHECK_FAIL) > 0) {
+              assertFalse(keySet.contains(keys[keyNum]));
+            }
+            else {
+              assertTrue(keySet.contains(keys[keyNum]));
+            }
+          }
+        }
+        else if (op.isQuery()) {
+          breakLoop = true;
+          SelectResults queryResults = region.query("SELECT DISTINCT * FROM "
+              + region.getFullPath());
+          assertNotNull(queryResults);
+          Set queryResultSet = queryResults.asSet();
+          if ((flags & OpFlags.CHECK_FAIL) == 0) {
+            assertEquals(numOps, queryResultSet.size());
+          }
+          for (int keyNumIndex = 0; keyNumIndex < numOps; ++keyNumIndex) {
+            int keyNum = indices[keyNumIndex];
+            if ((flags & OpFlags.CHECK_FAIL) > 0) {
+              assertFalse(queryResultSet.contains(vals[keyNum]));
+            }
+            else {
+              assertTrue(queryResultSet.contains(vals[keyNum]));
+            }
+          }
+        }
+        else if (op.isExecuteCQ()) {
+          breakLoop = true;
+          QueryService queryService = SecurityTestUtil.getCache()
+              .getQueryService();
+          CqQuery cqQuery;
+          if ((cqQuery = queryService.getCq("cq1")) == null) {
+            CqAttributesFactory cqFact = new CqAttributesFactory();
+            cqFact.addCqListener(new AuthzCqListener());
+            CqAttributes cqAttrs = cqFact.create();
+            cqQuery = queryService.newCq("cq1", "SELECT * FROM "
+                + region.getFullPath(), cqAttrs);
+          }
+          if ((flags & OpFlags.LOCAL_OP) > 0) {
+            // Interpret this as testing results using CqListener
+            final AuthzCqListener listener = (AuthzCqListener)cqQuery
+                .getCqAttributes().getCqListener();
+            WaitCriterion ev = new WaitCriterion() {
+              public boolean done() {
+                if ((flags & OpFlags.CHECK_FAIL) > 0) {
+                  return 0 == listener.getNumUpdates();
+                }
+                else {
+                  return numOps == listener.getNumUpdates();
+                }
+              }
+              public String description() {
+                return null;
+              }
+            };
+            DistributedTestCase.waitForCriterion(ev, 3 * 1000, 200, true);
+            if ((flags & OpFlags.CHECK_FAIL) > 0) {
+              assertEquals(0, listener.getNumUpdates());
+            }
+            else {
+              assertEquals(numOps, listener.getNumUpdates());
+              listener.checkPuts(vals, indices);
+            }
+            assertEquals(0, listener.getNumCreates());
+            assertEquals(0, listener.getNumDestroys());
+            assertEquals(0, listener.getNumOtherOps());
+            assertEquals(0, listener.getNumErrors());
+          }
+          else {
+            SelectResults cqResults = cqQuery.executeWithInitialResults();
+            assertNotNull(cqResults);
+            Set cqResultValues = new HashSet();
+            for (Object o : cqResults.asList()) {
+              Struct s = (Struct)o;
+              cqResultValues.add(s.get("value"));
+            }
+            Set cqResultSet = cqResults.asSet();
+            if ((flags & OpFlags.CHECK_FAIL) == 0) {
+              assertEquals(numOps, cqResultSet.size());
+            }
+            for (int keyNumIndex = 0; keyNumIndex < numOps; ++keyNumIndex) {
+              int keyNum = indices[keyNumIndex];
+              if ((flags & OpFlags.CHECK_FAIL) > 0) {
+                assertFalse(cqResultValues.contains(vals[keyNum]));
+              }
+              else {
+                assertTrue(cqResultValues.contains(vals[keyNum]));
+              }
+            }
+          }
+        }
+        else if (op.isStopCQ()) {
+          breakLoop = true;
+          CqQuery cqQuery = SecurityTestUtil.getCache().getQueryService()
+              .getCq("cq1");
+          ((AuthzCqListener)cqQuery.getCqAttributes().getCqListener()).reset();
+          cqQuery.stop();
+        }
+        else if (op.isCloseCQ()) {
+          breakLoop = true;
+          CqQuery cqQuery = SecurityTestUtil.getCache().getQueryService()
+              .getCq("cq1");
+          ((AuthzCqListener)cqQuery.getCqAttributes().getCqListener()).reset();
+          cqQuery.close();
+        }
+        else if (op.isRegionClear()) {
+          breakLoop = true;
+          if ((flags & OpFlags.LOCAL_OP) > 0) {
+            region.localClear();
+          }
+          else {
+            region.clear();
+          }
+        }
+        else if (op.isRegionCreate()) {
+          breakLoop = true;
+          // Region subregion = createSubregion(region);
+          // subregion.createRegionOnServer();
+          // Create region on server using the DynamicRegionFactory
+          // Assume it has been already initialized
+          DynamicRegionFactory drf = DynamicRegionFactory.get();
+          Region subregion = drf.createDynamicRegion(regionName, subregionName);
+          assertEquals('/' + regionName + '/' + subregionName, subregion
+              .getFullPath());
+        }
+        else if (op.isRegionDestroy()) {
+          breakLoop = true;
+          if ((flags & OpFlags.LOCAL_OP) > 0) {
+            region.localDestroyRegion();
+          }
+          else {
+            if ((flags & OpFlags.USE_SUBREGION) > 0) {
+              try {
+                DynamicRegionFactory.get().destroyDynamicRegion(
+                    region.getFullPath());
+              }
+              catch (RegionDestroyedException ex) {
+                // harmless to ignore this
+                getLogWriter().info(
+                    "doOp: sub-region " + region.getFullPath()
+                        + " already destroyed");
+                operationOmitted = true;
+              }
+            }
+            else {
+              region.destroyRegion();
+            }
+          }
+        }
+        else {
+          fail("doOp: Unhandled operation " + op);
+        }
+        if (expectedResult.intValue() != SecurityTestUtil.NO_EXCEPTION) {
+          if (!operationOmitted && !op.isUnregisterInterest()) {
+            fail("Expected an exception while performing operation op =" + op +
+                "flags = " + OpFlags.description(flags));
+          }
+        }
+      }
+      catch (Exception ex) {
+        exceptionOccured = true;
+        if ((ex instanceof ServerConnectivityException
+            || ex instanceof QueryInvocationTargetException || ex instanceof CqException)
+            && (expectedResult.intValue() == SecurityTestUtil.NOTAUTHZ_EXCEPTION)
+            && (ex.getCause() instanceof NotAuthorizedException)) {
+          getLogWriter().info(
+              "doOp: Got expected NotAuthorizedException when doing operation ["
+                  + op + "] with flags " + OpFlags.description(flags) 
+                  + ": " + ex.getCause());
+          continue;
+        }
+        else if (expectedResult.intValue() == SecurityTestUtil.OTHER_EXCEPTION) {
+          getLogWriter().info(
+              "doOp: Got expected exception when doing operation: "
+                  + ex.toString());
+          continue;
+        }
+        else {
+          fail("doOp: Got unexpected exception when doing operation. Policy = " 
+              + policy + " flags = " + OpFlags.description(flags), ex);
+        }
+      }
+    }
+    if (!exceptionOccured && !operationOmitted
+        && expectedResult.intValue() != SecurityTestUtil.NO_EXCEPTION) {
+      fail("Expected an exception while performing operation: " + op + 
+          " flags = " + OpFlags.description(flags));
+    }
+  }
+
+  protected void executeOpBlock(List opBlock, Integer port1, Integer port2,
+      String authInit, Properties extraAuthProps, Properties extraAuthzProps,
+      TestCredentialGenerator gen, Random rnd) {
+
+    Iterator opIter = opBlock.iterator();
+    while (opIter.hasNext()) {
+      // Start client with valid credentials as specified in
+      // OperationWithAction
+      OperationWithAction currentOp = (OperationWithAction)opIter.next();
+      OperationCode opCode = currentOp.getOperationCode();
+      int opFlags = currentOp.getFlags();
+      int clientNum = currentOp.getClientNum();
+      VM clientVM = null;
+      boolean useThisVM = false;
+      switch (clientNum) {
+        case 1:
+          clientVM = client1;
+          break;
+        case 2:
+          clientVM = client2;
+          break;
+        case 3:
+          useThisVM = true;
+          break;
+        default:
+          fail("executeOpBlock: Unknown client number " + clientNum);
+          break;
+      }
+      getLogWriter().info(
+          "executeOpBlock: performing operation number ["
+              + currentOp.getOpNum() + "]: " + currentOp);
+      if ((opFlags & OpFlags.USE_OLDCONN) == 0) {
+        Properties opCredentials;
+        int newRnd = rnd.nextInt(100) + 1;
+        String currentRegionName = '/' + regionName;
+        if ((opFlags & OpFlags.USE_SUBREGION) > 0) {
+          currentRegionName += ('/' + subregionName);
+        }
+        String credentialsTypeStr;
+        OperationCode authOpCode = currentOp.getAuthzOperationCode();
+        int[] indices = currentOp.getIndices();
+        CredentialGenerator cGen = gen.getCredentialGenerator();
+        Properties javaProps = null;
+        if ((opFlags & OpFlags.CHECK_NOTAUTHZ) > 0
+            || (opFlags & OpFlags.USE_NOTAUTHZ) > 0) {
+          opCredentials = gen.getDisallowedCredentials(
+              new OperationCode[] { authOpCode },
+              new String[] { currentRegionName }, indices, newRnd);
+          credentialsTypeStr = " unauthorized " + authOpCode;
+        }
+        else {
+          opCredentials = gen.getAllowedCredentials(new OperationCode[] {
+              opCode, authOpCode }, new String[] { currentRegionName },
+              indices, newRnd);
+          credentialsTypeStr = " authorized " + authOpCode;
+        }
+        if (cGen != null) {
+          javaProps = cGen.getJavaProperties();
+        }
+        Properties clientProps = SecurityTestUtil
+            .concatProperties(new Properties[] { opCredentials, extraAuthProps,
+                extraAuthzProps });
+        // Start the client with valid credentials but allowed or disallowed to
+        // perform an operation
+        getLogWriter().info(
+            "executeOpBlock: For client" + clientNum + credentialsTypeStr
+                + " credentials: " + opCredentials);
+        boolean setupDynamicRegionFactory = (opFlags & OpFlags.ENABLE_DRF) > 0;
+        if (useThisVM) {
+          createCacheClient(authInit, clientProps, javaProps, new Integer[] {
+              port1, port2 }, null, Boolean.valueOf(setupDynamicRegionFactory),
+              new Integer(SecurityTestUtil.NO_EXCEPTION));
+        }
+        else {
+          clientVM.invoke(ClientAuthorizationTestBase.class,
+              "createCacheClient", new Object[] { authInit, clientProps,
+                  javaProps, new Integer[] { port1, port2 }, null,
+                  Boolean.valueOf(setupDynamicRegionFactory),
+                  new Integer(SecurityTestUtil.NO_EXCEPTION) });
+        }
+      }
+      int expectedResult;
+      if ((opFlags & OpFlags.CHECK_NOTAUTHZ) > 0) {
+        expectedResult = SecurityTestUtil.NOTAUTHZ_EXCEPTION;
+      }
+      else if ((opFlags & OpFlags.CHECK_EXCEPTION) > 0) {
+        expectedResult = SecurityTestUtil.OTHER_EXCEPTION;
+      }
+      else {
+        expectedResult = SecurityTestUtil.NO_EXCEPTION;
+      }
+
+      // Perform the operation from selected client
+      if (useThisVM) {
+        doOp(new Byte(opCode.toOrdinal()), currentOp.getIndices(), new Integer(
+            opFlags), new Integer(expectedResult));
+      }
+      else {
+        clientVM.invoke(ClientAuthorizationTestBase.class, "doOp",
+            new Object[] { new Byte(opCode.toOrdinal()),
+                currentOp.getIndices(), new Integer(opFlags),
+                new Integer(expectedResult) });
+      }
+    }
+  }
+
+  protected AuthzCredentialGenerator getXmlAuthzGenerator(){
+    AuthzCredentialGenerator authzGen = new XmlAuthzCredentialGenerator();
+    CredentialGenerator cGen = new DummyCredentialGenerator();
+    cGen.init();
+    authzGen.init(cGen);
+    return authzGen;
+  }
+
+  protected List getDummyGeneratorCombos() {
+    List generators = new ArrayList();
+    Iterator authzCodeIter = AuthzCredentialGenerator.ClassCode.getAll()
+            .iterator();
+    while (authzCodeIter.hasNext()) {
+      ClassCode authzClassCode = (ClassCode) authzCodeIter.next();
+      AuthzCredentialGenerator authzGen = AuthzCredentialGenerator
+              .create(authzClassCode);
+      if (authzGen != null) {
+        CredentialGenerator cGen = new DummyCredentialGenerator();
+        cGen.init();
+        if (authzGen.init(cGen)) {
+          generators.add(authzGen);
+        }
+      }
+    }
+
+    assertTrue(generators.size() > 0);
+    return generators;
+  }
+
+
+  protected void runOpsWithFailover(OperationWithAction[] opCodes,
+      String testName) {
+      AuthzCredentialGenerator gen = getXmlAuthzGenerator();
+      CredentialGenerator cGen = gen.getCredentialGenerator();
+      Properties extraAuthProps = cGen.getSystemProperties();
+      Properties javaProps = cGen.getJavaProperties();
+      Properties extraAuthzProps = gen.getSystemProperties();
+      String authenticator = cGen.getAuthenticator();
+      String authInit = cGen.getAuthInit();
+      String accessor = gen.getAuthorizationCallback();
+      TestAuthzCredentialGenerator tgen = new TestAuthzCredentialGenerator(gen);
+
+      getLogWriter().info(testName + ": Using authinit: " + authInit);
+      getLogWriter().info(testName + ": Using authenticator: " + authenticator);
+      getLogWriter().info(testName + ": Using accessor: " + accessor);
+
+      // Start servers with all required properties
+      Properties serverProps = buildProperties(authenticator, accessor, false,
+          extraAuthProps, extraAuthzProps);
+      // Get ports for the servers
+      Keeper port1Keeper = AvailablePort.getRandomAvailablePortKeeper(AvailablePort.SOCKET);
+      Keeper port2Keeper = AvailablePort.getRandomAvailablePortKeeper(AvailablePort.SOCKET);
+      int port1 = port1Keeper.getPort();
+      int port2 = port2Keeper.getPort();
+
+      // Perform all the ops on the clients
+      List opBlock = new ArrayList();
+      Random rnd = new Random();
+      for (int opNum = 0; opNum < opCodes.length; ++opNum) {
+        // Start client with valid credentials as specified in
+        // OperationWithAction
+        OperationWithAction currentOp = opCodes[opNum];
+        if (currentOp.equals(OperationWithAction.OPBLOCK_END)
+            || currentOp.equals(OperationWithAction.OPBLOCK_NO_FAILOVER)) {
+          // End of current operation block; execute all the operations
+          // on the servers with/without failover
+          if (opBlock.size() > 0) {
+            port1Keeper.release();
+            // Start the first server and execute the operation block
+            server1.invoke(ClientAuthorizationTestBase.class,
+                "createCacheServer", new Object[] {
+                    SecurityTestUtil.getLocatorPort(), port1, serverProps,
+                    javaProps });
+            server2.invoke(SecurityTestUtil.class, "closeCache");
+            executeOpBlock(opBlock, port1, port2, authInit, extraAuthProps,
+                extraAuthzProps, tgen, rnd);
+            if (!currentOp.equals(OperationWithAction.OPBLOCK_NO_FAILOVER)) {
+              // Failover to the second server and run the block again
+              port2Keeper.release();
+              server2.invoke(ClientAuthorizationTestBase.class,
+                  "createCacheServer", new Object[] {
+                      SecurityTestUtil.getLocatorPort(), port2, serverProps,
+                      javaProps });
+              server1.invoke(SecurityTestUtil.class, "closeCache");
+              executeOpBlock(opBlock, port1, port2, authInit, extraAuthProps,
+                  extraAuthzProps, tgen, rnd);
+            }
+            opBlock.clear();
+          }
+        }
+        else {
+          currentOp.setOpNum(opNum);
+          opBlock.add(currentOp);
+        }
+      }
+  }
+
+  /**
+   * Implements the {@link CqListener} interface and counts the number of
+   * different operations and also queues up the received updates to precise
+   * checking of each update.
+   * 
+   * @author sumedh
+   * @since 5.5
+   */
+  public static class AuthzCqListener implements CqListener {
+
+    private List eventList;
+
+    private int numCreates;
+
+    private int numUpdates;
+
+    private int numDestroys;
+
+    private int numOtherOps;
+
+    private int numErrors;
+
+    public AuthzCqListener() {
+      this.eventList = new ArrayList();
+      reset();
+    }
+
+    public void reset() {
+      this.eventList.clear();
+      this.numCreates = 0;
+      this.numUpdates = 0;
+      this.numErrors = 0;
+    }
+
+    public void onEvent(CqEvent aCqEvent) {
+      Operation op = aCqEvent.getBaseOperation();
+      if (op.isCreate()) {
+        ++this.numCreates;
+      }
+      else if (op.isUpdate()) {
+        ++this.numUpdates;
+      }
+      else if (op.isDestroy()) {
+        ++this.numDestroys;
+      }
+      else {
+        ++this.numOtherOps;
+      }
+      eventList.add(aCqEvent);
+    }
+
+    public void onError(CqEvent aCqEvent) {
+      ++this.numErrors;
+    }
+
+    public void close() {
+      this.eventList.clear();
+    }
+
+    public int getNumCreates() {
+      return this.numCreates;
+    }
+
+    public int getNumUpdates() {
+      return this.numUpdates;
+    }
+
+    public int getNumDestroys() {
+      return this.numDestroys;
+    }
+
+    public int getNumOtherOps() {
+      return this.numOtherOps;
+    }
+
+    public int getNumErrors() {
+      return this.numErrors;
+    }
+
+    public void checkPuts(String[] vals, int[] indices) {
+      for (int indexIndex = 0; indexIndex < indices.length; ++indexIndex) {
+        int index = indices[indexIndex];
+        Iterator eventIter = this.eventList.iterator();
+        boolean foundKey = false;
+        while (eventIter.hasNext()) {
+          CqEvent event = (CqEvent)eventIter.next();
+          if (SecurityTestUtil.keys[index].equals(event.getKey())) {
+            assertEquals(vals[index], event.getNewValue());
+            foundKey = true;
+            break;
+          }
+        }
+        assertTrue(foundKey);
+      }
+    }
+  }
+
+  /**
+   * This class specifies flags that can be used to alter the behaviour of
+   * operations being performed by the <code>doOp</code> function.
+   * 
+   * @author sumedh
+   * @since 5.5
+   */
+  public static class OpFlags {
+
+    /**
+     * Default behaviour.
+     */
+    public static final int NONE = 0x0;
+
+    /**
+     * Check that the operation should fail.
+     */
+    public static final int CHECK_FAIL = 0x1;
+
+    /**
+     * Check that the operation should throw <code>NotAuthorizedException</code>.
+     */
+    public static final int CHECK_NOTAUTHZ = 0x2;
+
+    /**
+     * Check that the region should not be available.
+     */
+    public static final int CHECK_NOREGION = 0x4;
+
+    /**
+     * Check that the operation should throw an exception other than the
+     * <code>NotAuthorizedException</code>.
+     */
+    public static final int CHECK_EXCEPTION = 0x8;
+
+    /**
+     * Check for nvalues[] instead of values[].
+     */
+    public static final int USE_NEWVAL = 0x10;
+
+    /**
+     * Register all keys. For GET operations indicates using getAll().
+     */
+    public static final int USE_ALL_KEYS = 0x20;
+
+    /**
+     * Register a regular expression.
+     */
+    public static final int USE_REGEX = 0x40;
+
+    /**
+     * Register a list of keys.
+     */
+    public static final int USE_LIST = 0x80;
+
+    /**
+     * Perform the local version of the operation.
+     */
+    public static final int LOCAL_OP = 0x100;
+
+    /**
+     * Check that the key for the operation should not be present.
+     */
+    public static final int CHECK_NOKEY = 0x200;
+
+    /**
+     * Use the sub-region for performing the operation.
+     */
+    public static final int USE_SUBREGION = 0x400;
+
+    /**
+     * Do not try to create the sub-region.
+     */
+    public static final int NO_CREATE_SUBREGION = 0x800;
+
+    /**
+     * Do not re-connect using new credentials rather use the previous
+     * connection.
+     */
+    public static final int USE_OLDCONN = 0x1000;
+
+    /**
+     * Do the connection with unauthorized credentials but do not check that the
+     * operation throws <code>NotAuthorizedException</code>.
+     */
+    public static final int USE_NOTAUTHZ = 0x2000;
+
+    /**
+     * Enable {@link DynamicRegionFactory} on the client.
+     */
+    public static final int ENABLE_DRF = 0x4000;
+
+    /**
+     * Use the {@link InterestResultPolicy#NONE} for register interest.
+     */
+    public static final int REGISTER_POLICY_NONE = 0x8000;
+    
+    /**
+     * Use the {@link LocalRegion#getEntry} under transaction.
+     */
+    public static final int USE_GET_ENTRY_IN_TX = 0x10000;
+    
+    static public String description(int f) {
+      StringBuffer sb = new StringBuffer();
+      sb.append("[");
+      if ((f & CHECK_FAIL) != 0) {
+        sb.append("CHECK_FAIL,");
+      }
+      if ((f & CHECK_NOTAUTHZ) != 0) {
+        sb.append("CHECK_NOTAUTHZ,");
+      }
+      if ((f & CHECK_NOREGION) != 0) {
+        sb.append("CHECK_NOREGION,");
+      }
+      if ((f & CHECK_EXCEPTION) != 0) {
+        sb.append("CHECK_EXCEPTION,");
+      }
+      if ((f & USE_NEWVAL) != 0) {
+        sb.append("USE_NEWVAL,");
+      }
+      if ((f & USE_ALL_KEYS) != 0) {
+        sb.append("USE_ALL_KEYS,");
+      }
+      if ((f & USE_REGEX) != 0) {
+        sb.append("USE_REGEX,");
+      }
+      if ((f & USE_LIST) != 0) {
+        sb.append("USE_LIST,");
+      }
+      if ((f & LOCAL_OP) != 0) {
+        sb.append("LOCAL_OP,");
+      }
+      if ((f & CHECK_NOKEY) != 0) {
+        sb.append("CHECK_NOKEY,");
+      }
+      if ((f & USE_SUBREGION) != 0) {
+        sb.append("USE_SUBREGION,");
+      }
+      if ((f & NO_CREATE_SUBREGION) != 0) {
+        sb.append("NO_CREATE_SUBREGION,");
+      }
+      if ((f & USE_OLDCONN) != 0) {
+        sb.append("USE_OLDCONN,");
+      }
+      if ((f & USE_NOTAUTHZ) != 0) {
+        sb.append("USE_NOTAUTHZ");
+      }
+      if ((f & ENABLE_DRF) != 0) {
+        sb.append("ENABLE_DRF,");
+      }
+      if ((f & REGISTER_POLICY_NONE) != 0) {
+        sb.append("REGISTER_POLICY_NONE,");
+      }
+      sb.append("]");
+      return sb.toString();
+    }
+  }
+
+  /**
+   * This class encapsulates an {@link OperationCode} with associated flags, the
+   * client to perform the operation, and the number of operations to perform.
+   * 
+   * @author sumedh
+   * @since 5.5
+   */
+  public static class OperationWithAction {
+
+    /**
+     * The operation to be performed.
+     */
+    private OperationCode opCode;
+
+    /**
+     * The operation for which authorized or unauthorized credentials have to be
+     * generated. This is the same as {@link #opCode} when not specified.
+     */
+    private OperationCode authzOpCode;
+
+    /**
+     * The client number on which the operation has to be performed.
+     */
+    private int clientNum;
+
+    /**
+     * Bitwise or'd {@link OpFlags} integer to change/specify the behaviour of
+     * the operations.
+     */
+    private int flags;
+
+    /**
+     * Indices of the keys array to be used for operations.
+     */
+    private int[] indices;
+
+    /**
+     * An index for the operation used for logging.
+     */
+    private int opNum;
+
+    /**
+     * Indicates end of an operation block which can be used for testing with
+     * failover
+     */
+    public static final OperationWithAction OPBLOCK_END = new OperationWithAction(
+        null, 4);
+
+    /**
+     * Indicates end of an operation block which should not be used for testing
+     * with failover
+     */
+    public static final OperationWithAction OPBLOCK_NO_FAILOVER = new OperationWithAction(
+        null, 5);
+
+    private void setIndices(int numOps) {
+
+      this.indices = new int[numOps];
+      for (int index = 0; index < numOps; ++index) {
+        this.indices[index] = index;
+      }
+    }
+
+    public OperationWithAction(OperationCode opCode) {
+
+      this.opCode = opCode;
+      this.authzOpCode = opCode;
+      this.clientNum = 1;
+      this.flags = OpFlags.NONE;
+      setIndices(4);
+      this.opNum = 0;
+    }
+
+    public OperationWithAction(OperationCode opCode, int clientNum) {
+
+      this.opCode = opCode;
+      this.authzOpCode = opCode;
+      this.clientNum = clientNum;
+      this.flags = OpFlags.NONE;
+      setIndices(4);
+      this.opNum = 0;
+    }
+
+    public OperationWithAction(OperationCode opCode, int clientNum, int flags,
+        int numOps) {
+
+      this.opCode = opCode;
+      this.authzOpCode = opCode;
+      this.clientNum = clientNum;
+      this.flags = flags;
+      setIndices(numOps);
+      this.opNum = 0;
+    }
+
+    public OperationWithAction(OperationCode opCode,
+        OperationCode deniedOpCode, int clientNum, int flags, int numOps) {
+
+      this.opCode = opCode;
+      this.authzOpCode = deniedOpCode;
+      this.clientNum = clientNum;
+      this.flags = flags;
+      setIndices(numOps);
+      this.opNum = 0;
+    }
+
+    public OperationWithAction(OperationCode opCode, int clientNum, int flags,
+        int[] indices) {
+
+      this.opCode = opCode;
+      this.authzOpCode = opCode;
+      this.clientNum = clientNum;
+      this.flags = flags;
+      this.indices = indices;
+      this.opNum = 0;
+    }
+
+    public OperationWithAction(OperationCode opCode,
+        OperationCode deniedOpCode, int clientNum, int flags, int[] indices) {
+
+      this.opCode = opCode;
+      this.authzOpCode = deniedOpCode;
+      this.clientNum = clientNum;
+      this.flags = flags;
+      this.indices = indices;
+      this.opNum = 0;
+    }
+
+    public OperationCode getOperationCode() {
+      return this.opCode;
+    }
+
+    public OperationCode getAuthzOperationCode() {
+      return this.authzOpCode;
+    }
+
+    public int getClientNum() {
+      return this.clientNum;
+    }
+
+    public int getFlags() {
+      return this.flags;
+    }
+
+    public int[] getIndices() {
+      return this.indices;
+    }
+
+    public int getOpNum() {
+      return this.opNum;
+    }
+
+    public void setOpNum(int opNum) {
+      this.opNum = opNum;
+    }
+
+    public String toString() {
+      return "opCode:" + this.opCode + ",authOpCode:" + this.authzOpCode
+          + ",clientNum:" + this.clientNum + ",flags:" + this.flags
+          + ",numOps:" + this.indices.length + ",indices:"
+          + indicesToString(this.indices);
+    }
+  }
+
+  /**
+   * Simple interface to generate credentials with authorization based on key
+   * indices also. This is utilized by the post-operation authorization tests
+   * where authorization is based on key indices.
+   * 
+   * @author sumedh
+   * @since 5.5
+   */
+  public interface TestCredentialGenerator {
+
+    /**
+     * Get allowed credentials for the given set of operations in the given
+     * regions and indices of keys in the <code>keys</code> array
+     */
+    public Properties getAllowedCredentials(OperationCode[] opCodes,
+        String[] regionNames, int[] keyIndices, int num);
+
+    /**
+     * Get disallowed credentials for the given set of operations in the given
+     * regions and indices of keys in the <code>keys</code> array
+     */
+    public Properties getDisallowedCredentials(OperationCode[] opCodes,
+        String[] regionNames, int[] keyIndices, int num);
+
+    /**
+     * Get the {@link CredentialGenerator} if any.
+     */
+    public CredentialGenerator getCredentialGenerator();
+  }
+
+  /**
+   * Contains a {@link AuthzCredentialGenerator} and implements the
+   * {@link TestCredentialGenerator} interface.
+   * 
+   * @author sumedh
+   * @since 5.5
+   */
+  protected static class TestAuthzCredentialGenerator implements
+      TestCredentialGenerator {
+
+    private AuthzCredentialGenerator authzGen;
+
+    public TestAuthzCredentialGenerator(AuthzCredentialGenerator authzGen) {
+      this.authzGen = authzGen;
+    }
+
+    public Properties getAllowedCredentials(OperationCode[] opCodes,
+        String[] regionNames, int[] keyIndices, int num) {
+
+      return this.authzGen.getAllowedCredentials(opCodes, regionNames, num);
+    }
+
+    public Properties getDisallowedCredentials(OperationCode[] opCodes,
+        String[] regionNames, int[] keyIndices, int num) {
+
+      return this.authzGen.getDisallowedCredentials(opCodes, regionNames, num);
+    }
+
+    public CredentialGenerator getCredentialGenerator() {
+
+      return authzGen.getCredentialGenerator();
+    }
+  }
+
+}


[08/33] incubator-geode git commit: GEM-164: move the security tests in gemfire-test module inside the com.gemstone.gemfire.security packages to the open side.

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtil.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtil.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtil.java
new file mode 100644
index 0000000..6cb515a
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtil.java
@@ -0,0 +1,1871 @@
+package com.gemstone.gemfire.security;
+
+/*
+ * 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.
+ */
+
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import javax.net.ServerSocketFactory;
+import javax.net.SocketFactory;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLContextSpi;
+import javax.net.ssl.SSLServerSocketFactory;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.DynamicRegionFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.client.NoAvailableServersException;
+import com.gemstone.gemfire.cache.client.Pool;
+import com.gemstone.gemfire.cache.client.PoolFactory;
+import com.gemstone.gemfire.cache.client.PoolManager;
+import com.gemstone.gemfire.cache.client.ServerConnectivityException;
+import com.gemstone.gemfire.cache.client.ServerOperationException;
+import com.gemstone.gemfire.cache.client.ServerRefusedConnectionException;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.cache.client.internal.ProxyCache;
+import com.gemstone.gemfire.cache.execute.Execution;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.FunctionException;
+import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.cache.query.Query;
+import com.gemstone.gemfire.cache.query.QueryInvocationTargetException;
+import com.gemstone.gemfire.cache.query.SelectResults;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.logging.InternalLogWriter;
+import com.gemstone.gemfire.internal.logging.PureLogWriter;
+import com.gemstone.gemfire.internal.util.Callable;
+
+import dunit.DistributedTestCase;
+
+/**
+ * Contains utility methods for setting up servers/clients for authentication
+ * and authorization tests.
+ * 
+ * @author sumedh
+ * @since 5.5
+ */
+public class SecurityTestUtil extends DistributedTestCase {
+
+  public SecurityTestUtil(String name) {
+    super(name);
+  }
+
+  private static Locator locator = null;
+
+  private static Cache cache = null;
+
+  private static Properties currentJavaProps = null;
+
+  private static String locatorString = null;
+
+  private static Integer mcastPort = null;
+
+  public static final int NO_EXCEPTION = 0;
+
+  public static final int AUTHREQ_EXCEPTION = 1;
+
+  public static final int AUTHFAIL_EXCEPTION = 2;
+
+  public static final int CONNREFUSED_EXCEPTION = 3;
+
+  public static final int NOTAUTHZ_EXCEPTION = 4;
+
+  public static final int OTHER_EXCEPTION = 5;
+  
+  public static final int NO_AVAILABLE_SERVERS = 6;
+
+  // Indicates that AuthReqException may not necessarily be thrown
+  public static final int NOFORCE_AUTHREQ_EXCEPTION = 16;
+
+  protected static final String regionName = "AuthRegion";
+
+  protected static final String[] keys = { "key1", "key2", "key3", "key4",
+      "key5", "key6", "key7", "key8" };
+
+  protected static final String[] values = { "value1", "value2", "value3",
+      "value4", "value5", "value6", "value7", "value8" };
+
+  protected static final String[] nvalues = { "nvalue1", "nvalue2", "nvalue3",
+      "nvalue4", "nvalue5", "nvalue6", "nvalue7", "nvalue8" };
+
+  static String[] expectedExceptions = null;
+
+  private static Pool pool = null;
+
+  private static boolean multiUserAuthMode = false;
+
+  private static final int numberOfUsers = 1;
+
+  static ProxyCache[] proxyCaches = new ProxyCache[numberOfUsers];
+
+  private static Region regionRef = null;
+
+  public static void addExpectedExceptions(String[] expectedExceptions,
+      LogWriter logger) {
+    if (expectedExceptions != null) {
+      for (int index = 0; index < expectedExceptions.length; index++) {
+        logger.info("<ExpectedException action=add>"
+            + expectedExceptions[index] + "</ExpectedException>");
+      }
+    }
+  }
+
+  public static void removeExpectedExceptions(String[] expectedExceptions,
+      LogWriter logger) {
+    if (expectedExceptions != null) {
+      for (int index = 0; index < expectedExceptions.length; index++) {
+        logger.info("<ExpectedException action=remove>"
+            + expectedExceptions[index] + "</ExpectedException>");
+      }
+    }
+  }
+
+  public static void setJavaProps(Properties javaProps) {
+
+    removeJavaProperties(currentJavaProps);
+    addJavaProperties(javaProps);
+    currentJavaProps = javaProps;
+  }
+
+  public DistributedSystem createSystem(Properties sysProps, Properties javaProps) {
+
+    closeCache();
+    clearStaticSSLContext();
+    setJavaProps(javaProps);
+
+    DistributedSystem dsys = getSystem(sysProps);
+    assertNotNull(dsys);
+    addExpectedExceptions(SecurityTestUtil.expectedExceptions, system
+        .getLogWriter());
+    return dsys;
+  }
+
+  void openCache() {
+
+    assertNotNull(system);
+    assertTrue(system.isConnected());
+    cache = CacheFactory.create(system);
+    assertNotNull(cache);
+  }
+
+  private static void initClientDynamicRegionFactory(String poolName) {
+
+    DynamicRegionFactory.get().open(
+        new DynamicRegionFactory.Config(null, poolName, false,true));
+  }
+
+  public static void initDynamicRegionFactory() {
+
+    DynamicRegionFactory.get().open(
+        new DynamicRegionFactory.Config(null, null, false, true));
+  }
+
+  public static Integer getLocatorPort() {
+
+    Integer locatorPort = new Integer(AvailablePort
+        .getRandomAvailablePort(AvailablePort.SOCKET));
+    String addr = DistributedTestCase.getIPLiteral();
+    if (locatorString == null) {
+      locatorString = addr + "[" + locatorPort + ']';
+    }
+    else {
+      locatorString += "," + addr + "[" + locatorPort + ']';
+    }
+    return locatorPort;
+  }
+
+  /**
+   * Note that this clears the string after returning for convenience in reusing
+   * for other tests. Hence it should normally be invoked only once for a test.
+   */
+  public static String getLocatorString() {
+
+    String locString = locatorString;
+    locatorString = null;
+    return locString;
+  }
+
+  public static Properties concatProperties(Properties[] propsList) {
+
+    Properties props = new Properties();
+    for (int index = 0; index < propsList.length; ++index) {
+      if (propsList[index] != null) {
+        props.putAll(propsList[index]);
+      }
+    }
+    return props;
+  }
+
+  public static void registerExpectedExceptions(String[] expectedExceptions) {
+    SecurityTestUtil.expectedExceptions = expectedExceptions;
+  }
+
+  private static void addJavaProperties(Properties javaProps) {
+
+    if (javaProps != null) {
+      Iterator iter = javaProps.entrySet().iterator();
+      while (iter.hasNext()) {
+        Map.Entry entry = (Map.Entry)iter.next();
+        System.setProperty((String)entry.getKey(), (String)entry.getValue());
+      }
+    }
+  }
+
+  private static void removeJavaProperties(Properties javaProps) {
+
+    if (javaProps != null) {
+      Properties props = System.getProperties();
+      Iterator iter = javaProps.keySet().iterator();
+      while (iter.hasNext()) {
+        props.remove(iter.next());
+      }
+      System.setProperties(props);
+    }
+  }
+
+  public static Integer createCacheServer(Properties authProps,
+      Object javaProps, Integer dsPort, String locatorString,
+      Integer serverPort, Integer expectedResult) {
+
+    return createCacheServer(authProps, javaProps, dsPort, locatorString,
+        serverPort, Boolean.FALSE, expectedResult);
+  }
+
+  public static Integer createCacheServer(Properties authProps,
+      Object javaProps, Integer locatorPort, String locatorString,
+      Integer serverPort, Boolean setupDynamicRegionFactory,
+      Integer expectedResult) {
+
+    if (authProps == null) {
+      authProps = new Properties();
+    }
+    authProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    if (locatorString != null && locatorString.length() > 0) {
+      authProps.setProperty(DistributionConfig.LOCATORS_NAME, locatorString);
+      if (locatorPort != null) {
+        authProps.setProperty(DistributionConfig.START_LOCATOR_NAME,
+            DistributedTestCase.getIPLiteral() + "[" + locatorPort.toString() + ']');
+      }
+    } else {
+      authProps.setProperty("locators", "localhost["+getDUnitLocatorPort()+"]");
+    }
+    authProps.setProperty(DistributionConfig.SECURITY_LOG_LEVEL_NAME, "finest");
+    getLogWriter().info("Set the server properties to: " + authProps);
+    getLogWriter().info("Set the java properties to: " + javaProps);
+
+    SecurityTestUtil tmpInstance = new SecurityTestUtil("temp");
+    try {
+      tmpInstance.createSystem(authProps, (Properties)javaProps);
+      if (expectedResult.intValue() != NO_EXCEPTION) {
+        fail("Expected a security exception when starting peer");
+      }
+    }
+    catch (AuthenticationRequiredException ex) {
+      if (expectedResult.intValue() == AUTHREQ_EXCEPTION) {
+        getLogWriter().info("Got expected exception when starting peer: " + ex);
+        return new Integer(0);
+      }
+      else {
+        fail("Got unexpected exception when starting peer", ex);
+      }
+    }
+    catch (AuthenticationFailedException ex) {
+      if (expectedResult.intValue() == AUTHFAIL_EXCEPTION) {
+        getLogWriter().info("Got expected exception when starting peer: " + ex);
+        return new Integer(0);
+      }
+      else {
+        fail("Got unexpected exception when starting peer", ex);
+      }
+    }
+    catch (Exception ex) {
+      fail("Got unexpected exception when starting peer", ex);
+    }
+
+    if (setupDynamicRegionFactory.booleanValue()) {
+      initDynamicRegionFactory();
+    }
+    tmpInstance.openCache();
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setDataPolicy(DataPolicy.REPLICATE);
+    RegionAttributes attrs = factory.create();
+    cache.createRegion(regionName, attrs);
+    int port;
+    if (serverPort == null || serverPort.intValue() <= 0) {
+      port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    }
+    else {
+      port = serverPort.intValue();
+    }
+    CacheServer server1 = cache.addCacheServer();
+    server1.setPort(port);
+    server1.setNotifyBySubscription(true);
+    try {
+      server1.start();
+    }
+    catch (Exception ex) {
+      fail("Got unexpected exception when starting CacheServer", ex);
+    }
+    return new Integer(server1.getPort());
+  }
+
+  public static void createCacheClient(String authInitModule,
+      Properties authProps, Properties javaProps, Integer[] ports,
+      Object numConnections, Integer expectedResult) {
+    createCacheClient(authInitModule, authProps, javaProps, ports,
+        numConnections, "false", expectedResult);
+  }
+
+  public static void createCacheClient(String authInitModule,
+      Properties authProps, Properties javaProps, Integer[] ports,
+      Object numConnections, String multiUserMode, Integer expectedResult) {
+    createCacheClient(authInitModule, authProps, javaProps, ports,
+        (Integer)numConnections, Boolean.FALSE, multiUserMode, expectedResult);
+  }
+
+  public static void createCacheClient(String authInitModule,
+      Properties authProps, Properties javaProps, Integer[] ports,
+      Integer numConnections, Boolean setupDynamicRegionFactory,
+      Integer expectedResult) {
+    createCacheClient(authInitModule, authProps, javaProps, ports,
+        numConnections, setupDynamicRegionFactory, "false", expectedResult);
+  }
+
+  public static void createCacheClient(String authInitModule,
+      Properties authProps, Properties javaProps, Integer[] ports,
+      Integer numConnections, Boolean setupDynamicRegionFactory,
+      String multiUserMode, Integer expectedResult) {
+    createCacheClient(authInitModule, authProps, javaProps, ports,
+        numConnections, setupDynamicRegionFactory, multiUserMode, Boolean.TRUE,
+        expectedResult);
+  }
+
+  public static void createCacheClient(String authInitModule,
+      Properties authProps, Properties javaProps, Integer[] ports,
+      Integer numConnections, Boolean setupDynamicRegionFactory,
+      String multiUserMode, Boolean subscriptionEnabled,
+      Integer expectedResult) {
+
+    multiUserAuthMode = Boolean.valueOf(multiUserMode);
+    if (authProps == null) {
+      authProps = new Properties();
+    }
+    authProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    authProps.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    authProps.setProperty(DistributionConfig.SECURITY_LOG_LEVEL_NAME, "finest");
+    // TODO (ashetkar) Add " && (!multiUserAuthMode)" below.
+    if (authInitModule != null) {
+      authProps.setProperty(DistributionConfig.SECURITY_CLIENT_AUTH_INIT_NAME,
+          authInitModule);
+    }
+
+    SecurityTestUtil tmpInstance = new SecurityTestUtil("temp");
+    tmpInstance.createSystem(authProps, javaProps);
+    AttributesFactory factory = new AttributesFactory();
+    int[] portsI = new int[ports.length];
+    for(int z=0;z<ports.length;z++) {
+      portsI[z] = ports[z].intValue();
+    }
+   
+    try {
+      PoolFactory poolFactory = PoolManager.createFactory();
+      poolFactory.setRetryAttempts(200);
+      if (multiUserAuthMode) {
+        poolFactory.setMultiuserAuthentication(multiUserAuthMode);
+        // [sumedh] Why is this false here only to be overridden in
+        // ClientServerTestCase.configureConnectionPoolWithNameAndFactory below?
+        // Actually setting it to false causes MultiuserAPIDUnitTest to fail.
+        //poolFactory.setSubscriptionEnabled(false);
+      }
+      pool = ClientServerTestCase.configureConnectionPoolWithNameAndFactory(factory,
+          DistributedTestCase.getIPLiteral(), portsI, subscriptionEnabled, 0,
+          numConnections == null ? -1 : numConnections.intValue(), null, null,
+          poolFactory);
+
+      if (setupDynamicRegionFactory.booleanValue()) {
+        initClientDynamicRegionFactory(pool.getName());
+      }
+      tmpInstance.openCache();
+      try {
+        getLogWriter().info("multi-user mode " + multiUserAuthMode);
+        proxyCaches[0] = (ProxyCache)((PoolImpl) pool).createAuthenticatedCacheView(authProps);
+        if (!multiUserAuthMode) {
+          fail("Expected a UnsupportedOperationException but got none in single-user mode");
+        }
+      } catch (UnsupportedOperationException uoe) {
+        if (!multiUserAuthMode) {
+          getLogWriter().info("Got expected UnsupportedOperationException in single-user mode");
+        }
+        else {
+          fail("Got unexpected exception in multi-user mode ", uoe);
+        }
+      }
+
+      factory.setScope(Scope.LOCAL);
+      if (multiUserAuthMode) {
+        factory.setDataPolicy(DataPolicy.EMPTY);
+      }
+      RegionAttributes attrs = factory.create();
+      cache.createRegion(regionName, attrs);
+
+      if (expectedResult.intValue() != NO_EXCEPTION
+          && expectedResult.intValue() != NOFORCE_AUTHREQ_EXCEPTION) {
+        if (!multiUserAuthMode) {
+          fail("Expected an exception when starting client");
+        }
+      }
+    }
+    catch (AuthenticationRequiredException ex) {
+      if (expectedResult.intValue() == AUTHREQ_EXCEPTION
+          || expectedResult.intValue() == NOFORCE_AUTHREQ_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when starting client: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when starting client", ex);
+      }
+    }
+    catch (AuthenticationFailedException ex) {
+      if (expectedResult.intValue() == AUTHFAIL_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when starting client: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when starting client", ex);
+      }
+    }
+    catch (ServerRefusedConnectionException ex) {
+      if (expectedResult.intValue() == CONNREFUSED_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when starting client: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when starting client", ex);
+      }
+    }
+    catch (Exception ex) {
+      fail("Got unexpected exception when starting client", ex);
+    }
+  }
+
+  public static void createCacheClientForMultiUserMode(Integer numOfUsers,
+      String authInitModule, Properties[] authProps, Properties javaProps,
+      Integer[] ports, Integer numConnections,
+      Boolean setupDynamicRegionFactory, Integer expectedResult) {
+    createCacheClientForMultiUserMode(numOfUsers, authInitModule, authProps,
+        javaProps, ports, numConnections, setupDynamicRegionFactory, null,
+        expectedResult);
+  }
+
+  public static void createCacheClientForMultiUserMode(Integer numOfUsers,
+      String authInitModule, Properties[] authProps, Properties javaProps,
+      Integer[] ports, Integer numConnections,
+      Boolean setupDynamicRegionFactory, String durableClientId,
+      Integer expectedResult) {
+
+    if (numOfUsers == null || numOfUsers < 1) {
+      fail("Number of users cannot be less than one");
+    }
+    multiUserAuthMode = true;
+    // Assert that number of users == length of arrays of the provided params
+//    if (numOfUsers != authInitModules.length) {
+//      fail("Number of authInitModules provided does not match with numOfUsers specified, "
+//          + authInitModules.length);
+//    }
+    if (numOfUsers != authProps.length) {
+      fail("Number of authProps provided does not match with numOfUsers specified, "
+          + authProps.length);
+    }
+//    if (numOfUsers != javaProps.length) {
+//      fail("Number of javaProps provided does not match with numOfUsers specified, "
+//          + javaProps.length);
+//    }
+//    if (numOfUsers != expectedResult.length) {
+//      fail("Number of expectedResult provided does not match with numOfUsers specified, "
+//          + expectedResult.length);
+//    }
+    if (authProps[0] == null) {
+      authProps[0] = new Properties();
+    }
+    authProps[0].setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    authProps[0].setProperty(DistributionConfig.LOCATORS_NAME, "");
+    authProps[0].setProperty(DistributionConfig.SECURITY_LOG_LEVEL_NAME,
+        "finest");
+    Properties props = new Properties();
+    if (authInitModule != null) {
+      authProps[0].setProperty(
+          DistributionConfig.SECURITY_CLIENT_AUTH_INIT_NAME, authInitModule);
+      props.setProperty(DistributionConfig.SECURITY_CLIENT_AUTH_INIT_NAME,
+          authInitModule);
+    }
+    if (durableClientId != null) {
+      props.setProperty(DistributionConfig.DURABLE_CLIENT_ID_NAME,
+          durableClientId);
+      props.setProperty(DistributionConfig.DURABLE_CLIENT_TIMEOUT_NAME, String
+          .valueOf(DistributionConfig.DEFAULT_DURABLE_CLIENT_TIMEOUT));
+    }
+
+    SecurityTestUtil tmpInstance = new SecurityTestUtil("temp");
+    tmpInstance.createSystem(props, javaProps);
+    AttributesFactory factory = new AttributesFactory();
+    int[] portsI = new int[ports.length];
+    for(int z=0;z<ports.length;z++) {
+      portsI[z] = ports[z].intValue();
+    }
+   
+    try {
+      tmpInstance.openCache();
+      PoolFactory poolFactory = PoolManager.createFactory();
+      poolFactory.setRetryAttempts(200);
+      poolFactory.setMultiuserAuthentication(multiUserAuthMode);
+      poolFactory.setSubscriptionEnabled(true);
+      pool = ClientServerTestCase.configureConnectionPoolWithNameAndFactory(factory,
+          DistributedTestCase.getIPLiteral(), portsI, true, 1,
+          numConnections == null ? -1 : numConnections.intValue(), null, null,
+          poolFactory);
+
+      if (setupDynamicRegionFactory.booleanValue()) {
+        initClientDynamicRegionFactory(pool.getName());
+      }
+      proxyCaches = new ProxyCache[numOfUsers];
+      for (int i=0; i<numOfUsers; i++) {
+        proxyCaches[i] = (ProxyCache)((PoolImpl) pool).createAuthenticatedCacheView(authProps[i]);
+      }
+
+      factory.setScope(Scope.LOCAL);
+      factory.setDataPolicy(DataPolicy.EMPTY);
+      RegionAttributes attrs = factory.create();
+      cache.createRegion(regionName, attrs);
+
+      if (expectedResult.intValue() != NO_EXCEPTION
+          && expectedResult.intValue() != NOFORCE_AUTHREQ_EXCEPTION) {
+        if (!multiUserAuthMode) {
+          fail("Expected an exception when starting client");
+        }
+      }
+    }
+    catch (AuthenticationRequiredException ex) {
+      if (expectedResult.intValue() == AUTHREQ_EXCEPTION
+          || expectedResult.intValue() == NOFORCE_AUTHREQ_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when starting client: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when starting client", ex);
+      }
+    }
+    catch (AuthenticationFailedException ex) {
+      if (expectedResult.intValue() == AUTHFAIL_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when starting client: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when starting client", ex);
+      }
+    }
+    catch (ServerRefusedConnectionException ex) {
+      if (expectedResult.intValue() == CONNREFUSED_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when starting client: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when starting client", ex);
+      }
+    }
+    catch (Exception ex) {
+      fail("Got unexpected exception when starting client", ex);
+    }
+  }
+
+  public static void createProxyCache(Integer[] userIndices, Properties[] props) {
+    int j = 0;
+    for (int i : userIndices) {
+      SecurityTestUtil.proxyCaches[i] = (ProxyCache)((PoolImpl) SecurityTestUtil.pool)
+          .createAuthenticatedCacheView(props[j]);
+      j++;
+    }
+  }
+
+  public static void stopCacheServers() {
+    Iterator iter = getCache().getCacheServers().iterator();
+    if (iter.hasNext()) {
+      CacheServer server = (CacheServer)iter.next();
+      server.stop();
+      assertFalse(server.isRunning());
+    }
+  }
+
+  public static void restartCacheServers() {
+    Iterator iter = getCache().getCacheServers().iterator();
+    if (iter.hasNext()) {
+      CacheServer server = (CacheServer)iter.next();
+      try {
+        server.start();
+      }
+      catch (Exception ex) {
+        fail("Unexpected exception when restarting cache servers", ex);
+      }
+      assertTrue(server.isRunning());
+    }
+  }
+
+  public static void startLocator(String name, Integer port, Object extraProps,
+      Object javaProps, String[] expectedExceptions) {
+
+    File logFile = new File(name + "-locator" + port.intValue() + ".log");
+    try {
+      Properties authProps = new Properties();
+      if (extraProps != null) {
+        authProps.putAll((Properties)extraProps);
+      }
+      authProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+      authProps.setProperty(DistributionConfig.LOCATORS_NAME, 
+                            DistributedTestCase.getIPLiteral() + "[" + port + "]");
+      authProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
+      clearStaticSSLContext();
+      setJavaProps((Properties)javaProps);
+      FileOutputStream logOut = new FileOutputStream(logFile);
+      PrintStream logStream = new PrintStream(logOut);
+      LogWriter logger = new PureLogWriter(InternalLogWriter.CONFIG_LEVEL,
+          logStream);
+      addExpectedExceptions(expectedExceptions, logger);
+      logStream.flush();
+      locator = Locator.startLocatorAndDS(port.intValue(), logFile, null,
+          authProps);
+    }
+    catch (IOException ex) {
+      fail("While starting locator on port " + port.intValue(), ex);
+    }
+  }
+
+  public static void stopLocator(Integer port, String[] expectedExceptions) {
+
+    try {
+      locator.stop();
+      removeExpectedExceptions(expectedExceptions, locator
+          .getDistributedSystem().getLogWriter());
+    }
+    catch (Exception ex) {
+      fail("While stopping locator on port " + port.intValue(), ex);
+    }
+  }
+
+  public static Cache getCache() {
+    return cache;
+  }
+
+  // Some useful region methods used by security tests
+
+  public static void waitForCondition(Callable cond) {
+    waitForCondition(cond, 100, 120);
+  }
+
+  public static void waitForCondition(final Callable cond, int sleepMillis,
+      int numTries) {
+    WaitCriterion ev = new WaitCriterion() {
+      public boolean done() {
+        try {
+          return ((Boolean)cond.call()).booleanValue();
+        }
+        catch (Exception e) {
+          fail("Unexpected exception", e);
+        }
+        return false; // NOTREACHED
+      }
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, sleepMillis * numTries, 200, true);
+  }
+
+  public static Object getLocalValue(Region region, Object key) {
+
+    Region.Entry entry = region.getEntry(key);
+    return (entry != null ? entry.getValue() : null);
+  }
+
+  public static void doProxyCacheClose() {
+    for (int i=0; i<proxyCaches.length; i++) {
+      proxyCaches[i].close();
+    }
+  }
+
+  private static void doPutsP(Integer num, Integer expectedResult,
+      boolean newVals) {
+    doPutsP(num, Integer.valueOf(0), expectedResult, newVals);
+  }
+
+  private static void doPutsP(Integer num, Integer multiUserIndex,
+      Integer expectedResult, boolean newVals) {
+
+    assertTrue(num.intValue() <= keys.length);
+    Region region = null;
+    try {
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiUserIndex].getRegion(regionName);
+        regionRef = region;
+      }
+      else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNotNull(region);
+    }
+    catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing puts: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when doing puts", ex);
+      }
+    }
+    for (int index = 0; index < num.intValue(); ++index) {
+      try {
+        if (newVals) {
+          region.put(keys[index], nvalues[index]);
+        }
+        else {
+          region.put(keys[index], values[index]);
+        }
+        if (expectedResult.intValue() != NO_EXCEPTION) {
+          fail("Expected a NotAuthorizedException while doing puts");
+        }
+      }
+      catch(NoAvailableServersException ex) {
+        if(expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+          getLogWriter().info(
+              "Got expected NoAvailableServers when doing puts: "
+              + ex.getCause());
+          continue;
+        }
+        else {
+          fail("Got unexpected exception when doing puts", ex);
+        }
+      }
+      catch (ServerConnectivityException ex) {
+        if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+            && (ex.getCause() instanceof NotAuthorizedException)) {
+          getLogWriter().info(
+              "Got expected NotAuthorizedException when doing puts: "
+                  + ex.getCause());
+          continue;
+        }
+        if ((expectedResult.intValue() == AUTHREQ_EXCEPTION)
+            && (ex.getCause() instanceof AuthenticationRequiredException)) {
+          getLogWriter().info(
+              "Got expected AuthenticationRequiredException when doing puts: "
+                  + ex.getCause());
+          continue;
+        }
+        if ((expectedResult.intValue() == AUTHFAIL_EXCEPTION)
+            && (ex.getCause() instanceof AuthenticationFailedException)) {
+          getLogWriter().info(
+              "Got expected AuthenticationFailedException when doing puts: "
+                  + ex.getCause());
+          continue;
+        }
+        else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing puts: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing puts", ex);
+        }
+      }
+      catch (Exception ex) {
+        if (expectedResult.intValue() == OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing puts: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing puts", ex);
+        }
+      }
+    }
+  }
+
+  private static void doGetAllP(Integer multiUserIndex,
+      Integer expectedResult, boolean useTX) {
+    Region region = null;
+    try {
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiUserIndex].getRegion(regionName);
+      }
+      else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNotNull(region);
+    }
+    catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing getAll: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when doing getAll", ex);
+      }
+    }
+    try {
+      List keys = new ArrayList();
+      keys.add("key1");
+      keys.add("key2");
+      if (useTX) {
+        getCache().getCacheTransactionManager().begin();
+      }
+      Map entries = region.getAll(keys);
+      // Also check getEntry()
+      region.getEntry("key1");
+      if (useTX) {
+        getCache().getCacheTransactionManager().commit();
+      }
+      assertNotNull(entries);
+      if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)) {
+        assertEquals(0, entries.size());
+      } else if ((expectedResult.intValue() == NO_EXCEPTION)) {
+        assertEquals(2, entries.size());
+        assertEquals("value1", entries.get("key1"));
+        assertEquals("value2", entries.get("key2"));
+      }
+    } catch (NoAvailableServersException ex) {
+      if (expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+        getLogWriter().info(
+            "Got expected NoAvailableServers when doing getAll: "
+                + ex.getCause());
+      } else {
+        fail("Got unexpected exception when doing getAll", ex);
+      }
+    } catch (ServerConnectivityException ex) {
+      if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+          && (ex.getCause() instanceof NotAuthorizedException)) {
+        getLogWriter().info(
+            "Got expected NotAuthorizedException when doing getAll: "
+                + ex.getCause());
+      } else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing getAll: " + ex);
+      } else {
+        fail("Got unexpected exception when doing getAll", ex);
+      }
+    } catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing getAll: " + ex);
+      } else {
+        fail("Got unexpected exception when doing getAll", ex);
+      }
+    }
+  }
+
+  private static void doGetsP(Integer num, Integer expectedResult,
+      boolean newVals) {
+    doGetsP(num, Integer.valueOf(0), expectedResult, newVals);
+  }
+
+  private static void doGetsP(Integer num, Integer multiUserIndex,
+      Integer expectedResult, boolean newVals) {
+
+    assertTrue(num.intValue() <= keys.length);
+    Region region = null;
+    try {
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiUserIndex].getRegion(regionName);
+      }
+      else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNotNull(region);
+    }
+    catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing gets: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when doing gets", ex);
+      }
+    }
+    for (int index = 0; index < num.intValue(); ++index) {
+      Object value = null;
+      try {
+        try {
+          region.localInvalidate(keys[index]);
+        }
+        catch (Exception ex) {
+        }
+        value = region.get(keys[index]);
+        if (expectedResult.intValue() != NO_EXCEPTION) {
+          fail("Expected a NotAuthorizedException while doing gets");
+        }
+      }
+      catch(NoAvailableServersException ex) {
+        if(expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+          getLogWriter().info(
+              "Got expected NoAvailableServers when doing gets: "
+              + ex.getCause());
+          continue;
+        }
+        else {
+          fail("Got unexpected exception when doing gets", ex);
+        }
+      }
+      catch (ServerConnectivityException ex) {
+        if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+            && (ex.getCause() instanceof NotAuthorizedException)) {
+          getLogWriter().info(
+              "Got expected NotAuthorizedException when doing gets: "
+                  + ex.getCause());
+          continue;
+        }
+        else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing gets: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing gets", ex);
+        }
+      }
+      catch (Exception ex) {
+        if (expectedResult.intValue() == OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing gets: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing gets", ex);
+        }
+      }
+      assertNotNull(value);
+      if (newVals) {
+        assertEquals(nvalues[index], value);
+      }
+      else {
+        assertEquals(values[index], value);
+      }
+    }
+  }
+
+  private static void doLocalGetsP(int num, boolean checkNVals) {
+
+    assertTrue(num <= keys.length);
+    String[] vals = values;
+    if (checkNVals) {
+      vals = nvalues;
+    }
+    final Region region = getCache().getRegion(regionName);
+    assertNotNull(region);
+    for (int index = 0; index < num; ++index) {
+      final String key = keys[index];
+      final String expectedVal = vals[index];
+      waitForCondition(new Callable() {
+        public Object call() throws Exception {
+          Object value = getLocalValue(region, key);
+          return Boolean.valueOf(expectedVal.equals(value));
+        }
+      }, 1000, 30 / num);
+    }
+    for (int index = 0; index < num; ++index) {
+      Region.Entry entry = region.getEntry(keys[index]);
+      assertNotNull(entry);
+      assertEquals(vals[index], entry.getValue());
+    }
+  }
+
+  private static void doRegionDestroysP(Integer multiuserIndex,
+      Integer expectedResult) {
+    Region region = null;
+    try {
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiuserIndex].getRegion(regionName);
+      } else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNotNull(region);
+    } catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when doing region destroy: " + ex);
+      } else {
+        fail("Got unexpected exception when doing region destroy", ex);
+      }
+    }
+
+    try {
+      region.destroyRegion();
+      if (expectedResult.intValue() != NO_EXCEPTION) {
+        fail("Expected a NotAuthorizedException while doing region destroy");
+      }
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiuserIndex].getRegion(regionName);
+      } else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNull(region);
+    } catch (NoAvailableServersException ex) {
+      if (expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+        getLogWriter().info(
+            "Got expected NoAvailableServers when doing region destroy: "
+                + ex.getCause());
+      } else {
+        fail("Got unexpected exception when doing region destroy", ex);
+      }
+    } catch (ServerConnectivityException ex) {
+      if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+          && (ex.getCause() instanceof NotAuthorizedException)) {
+        getLogWriter().info(
+            "Got expected NotAuthorizedException when doing region destroy: "
+                + ex.getCause());
+      } else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when doing region destroy: " + ex);
+      } else {
+        fail("Got unexpected exception when doing region destroy", ex);
+      }
+    } catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when doing region destroy: " + ex);
+      } else {
+        fail("Got unexpected exception when doing region destroy", ex);
+      }
+    }
+  }
+  
+  private static void doDestroysP(Integer num, Integer multiUserIndex,
+      Integer expectedResult, boolean newVals) {
+
+    assertTrue(num.intValue() <= keys.length);
+    Region region = null;
+    try {
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiUserIndex].getRegion(regionName);
+      }
+      else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNotNull(region);
+    }
+    catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing destroys: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when doing destroys", ex);
+      }
+    }
+    for (int index = 0; index < num.intValue(); ++index) {
+      try {
+        region.destroy(keys[index]);
+        if (expectedResult.intValue() != NO_EXCEPTION) {
+          fail("Expected a NotAuthorizedException while doing destroys");
+        }
+      }
+      catch(NoAvailableServersException ex) {
+        if(expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+          getLogWriter().info(
+              "Got expected NoAvailableServers when doing destroys: "
+              + ex.getCause());
+          continue;
+        }
+        else {
+          fail("Got unexpected exception when doing destroys", ex);
+        }
+      }
+      catch (ServerConnectivityException ex) {
+        if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+            && (ex.getCause() instanceof NotAuthorizedException)) {
+          getLogWriter().info(
+              "Got expected NotAuthorizedException when doing destroys: "
+                  + ex.getCause());
+          continue;
+        }
+        else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing destroys: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing destroys", ex);
+        }
+      }
+      catch (Exception ex) {
+        if (expectedResult.intValue() == OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing destroys: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing destroys", ex);
+        }
+      }
+    }
+  }
+
+  private static void doInvalidatesP(Integer num, Integer multiUserIndex,
+      Integer expectedResult, boolean newVals) {
+
+    assertTrue(num.intValue() <= keys.length);
+    Region region = null;
+    try {
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiUserIndex].getRegion(regionName);
+      }
+      else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNotNull(region);
+    }
+    catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing invalidates: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when doing invalidates", ex);
+      }
+    }
+    for (int index = 0; index < num.intValue(); ++index) {
+      try {
+        region.invalidate(keys[index]);
+        if (expectedResult.intValue() != NO_EXCEPTION) {
+          fail("Expected a NotAuthorizedException while doing invalidates");
+        }
+      }
+      catch(NoAvailableServersException ex) {
+        if(expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+          getLogWriter().info(
+              "Got expected NoAvailableServers when doing invalidates: "
+              + ex.getCause());
+          continue;
+        }
+        else {
+          fail("Got unexpected exception when doing invalidates", ex);
+        }
+      }
+      catch (ServerConnectivityException ex) {
+        if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+            && (ex.getCause() instanceof NotAuthorizedException)) {
+          getLogWriter().info(
+              "Got expected NotAuthorizedException when doing invalidates: "
+                  + ex.getCause());
+          continue;
+        }
+        else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing invalidates: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing invalidates", ex);
+        }
+      }
+      catch (Exception ex) {
+        if (expectedResult.intValue() == OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing invalidates: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing invalidates", ex);
+        }
+      }
+    }
+  }
+
+  private static void doContainsKeysP(Integer num, Integer multiUserIndex,
+      Integer expectedResult, boolean newVals, boolean expectedValue) {
+
+    assertTrue(num.intValue() <= keys.length);
+    Region region = null;
+    try {
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiUserIndex].getRegion(regionName);
+      }
+      else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNotNull(region);
+    }
+    catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing containsKey: " + ex);
+      }
+      else {
+        fail("Got unexpected exception when doing containsKey", ex);
+      }
+    }
+    for (int index = 0; index < num.intValue(); ++index) {
+      boolean result = false;
+      try {
+        result = region.containsKeyOnServer(keys[index]);
+        if (expectedResult.intValue() != NO_EXCEPTION) {
+          fail("Expected a NotAuthorizedException while doing containsKey");
+        }
+      }
+      catch(NoAvailableServersException ex) {
+        if(expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+          getLogWriter().info(
+              "Got expected NoAvailableServers when doing containsKey: "
+              + ex.getCause());
+          continue;
+        }
+        else {
+          fail("Got unexpected exception when doing containsKey", ex);
+        }
+      }
+      catch (ServerConnectivityException ex) {
+        if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+            && (ex.getCause() instanceof NotAuthorizedException)) {
+          getLogWriter().info(
+              "Got expected NotAuthorizedException when doing containsKey: "
+                  + ex.getCause());
+          continue;
+        }
+        else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing containsKey: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing containsKey", ex);
+        }
+      }
+      catch (Exception ex) {
+        if (expectedResult.intValue() == OTHER_EXCEPTION) {
+          getLogWriter().info("Got expected exception when doing containsKey: " + ex);
+        }
+        else {
+          fail("Got unexpected exception when doing containsKey", ex);
+        }
+      }
+      assertEquals(expectedValue, result);
+    }
+  }
+
+  private static void doQueriesP(Integer multiUserIndex,
+      Integer expectedResult, Integer expectedValue) {
+    Region region = null;
+    try {
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiUserIndex].getRegion(regionName);
+      } else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNotNull(region);
+    } catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing queries: " + ex);
+      } else {
+        fail("Got unexpected exception when doing queries", ex);
+      }
+    }
+    String queryStr = "SELECT DISTINCT * FROM " + region.getFullPath();
+    try {
+      SelectResults queryResults = region.query(queryStr);
+      Set resultSet = queryResults.asSet();
+      assertEquals(expectedValue.intValue(), resultSet.size());
+      if (expectedResult.intValue() != NO_EXCEPTION) {
+        fail("Expected a NotAuthorizedException while doing queries");
+      }
+    } catch (NoAvailableServersException ex) {
+      if (expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+        getLogWriter().info(
+            "Got expected NoAvailableServers when doing queries: "
+                + ex.getCause());
+      } else {
+        fail("Got unexpected exception when doing queries", ex);
+      }
+    } catch (ServerConnectivityException ex) {
+      if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+          && (ex.getCause() instanceof NotAuthorizedException)) {
+        getLogWriter().info(
+            "Got expected NotAuthorizedException when doing queries: "
+                + ex.getCause());
+      } else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing queries: " + ex);
+      } else {
+        fail("Got unexpected exception when doing queries", ex);
+      }
+    } catch (QueryInvocationTargetException qite) {
+      if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+          && (qite.getCause() instanceof NotAuthorizedException)) {
+        getLogWriter().info(
+            "Got expected NotAuthorizedException when doing queries: "
+                + qite.getCause());
+      } else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing queries: " + qite);
+      } else {
+        fail("Got unexpected exception when doing queries", qite);
+      }
+    } catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info("Got expected exception when doing queries: " + ex);
+      } else {
+        fail("Got unexpected exception when doing queries", ex);
+      }
+    }
+  }
+
+  private static void doFunctionExecuteP(Integer multiUserIndex,
+      Function function, Integer expectedResult, Object expectedValue,
+      String method) {
+    Region region = null;
+    try {
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiUserIndex].getRegion(regionName);
+      } else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNotNull(region);
+    } catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when executing function: " + ex);
+      } else {
+        fail("Got unexpected exception when executing function", ex);
+      }
+    }
+    try {
+      FunctionService.registerFunction(function);
+      Execution execution = null;
+      if ("region".equals(method)) {
+        execution = FunctionService.onRegion(region);
+      } else if ("server".equals(method)) {
+        if (multiUserAuthMode) {
+          execution = FunctionService.onServer(proxyCaches[multiUserIndex]);
+        } else {
+          execution = FunctionService.onServer(pool);
+        }
+      } else { // if ("servers".equals(method)) {
+        if (multiUserAuthMode) {
+          execution = FunctionService.onServers(proxyCaches[multiUserIndex]);
+        } else {
+          execution = FunctionService.onServers(pool);
+        }
+      }
+      execution.execute(function.getId());
+      if (expectedResult.intValue() != NO_EXCEPTION) {
+        fail("Expected a NotAuthorizedException while executing function");
+      }
+    } catch (NoAvailableServersException ex) {
+      if (expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+        getLogWriter().info(
+            "Got expected NoAvailableServers when executing function: "
+                + ex.getCause());
+      } else {
+        fail("Got unexpected exception when executing function", ex);
+      }
+    } catch (ServerConnectivityException ex) {
+      if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+          && (ex.getCause() instanceof NotAuthorizedException)) {
+        getLogWriter().info(
+            "Got expected NotAuthorizedException when executing function: "
+                + ex.getCause());
+      } else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when executing function: " + ex);
+      } else {
+        fail("Got unexpected exception when executing function", ex);
+      }
+    } catch (FunctionException ex) {
+      if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+          && ((ex.getCause() instanceof NotAuthorizedException) || ((ex
+              .getCause() instanceof ServerOperationException) && (((ServerOperationException)ex
+              .getCause()).getCause() instanceof NotAuthorizedException)))) {
+        getLogWriter().info(
+            "Got expected NotAuthorizedException when executing function: "
+                + ex.getCause());
+      } else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when executing function: " + ex);
+      } else {
+        fail("Got unexpected exception when executing function", ex);
+      }
+    } catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when executing function: " + ex);
+      } else {
+        fail("Got unexpected exception when executing function", ex);
+      }
+    }
+  }
+
+  private static void doQueryExecuteP(Integer multiUserIndex,
+      Integer expectedResult, Integer expectedValue) {
+    Region region = null;
+    try {
+      if (multiUserAuthMode) {
+        region = proxyCaches[multiUserIndex].getRegion(regionName);
+      } else {
+        region = getCache().getRegion(regionName);
+      }
+      assertNotNull(region);
+    } catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when executing query: " + ex);
+      } else {
+        fail("Got unexpected exception when executing query", ex);
+      }
+    }
+    try {
+      String queryString = "SELECT DISTINCT * FROM " + region.getFullPath();
+      Query query = null;
+      if (multiUserAuthMode) {
+        query = proxyCaches[multiUserIndex].getQueryService().newQuery(queryString);
+      }
+      else {
+        region.getCache().getQueryService().newQuery(queryString);
+      }
+      SelectResults result = (SelectResults)query.execute();
+      if (expectedResult.intValue() != NO_EXCEPTION) {
+        fail("Expected a NotAuthorizedException while executing function");
+      }
+      assertEquals(expectedValue.intValue(), result.asList().size());
+    } catch (NoAvailableServersException ex) {
+      if (expectedResult.intValue() == SecurityTestUtil.NO_AVAILABLE_SERVERS) {
+        getLogWriter().info(
+            "Got expected NoAvailableServers when executing query: "
+                + ex.getCause());
+      } else {
+        fail("Got unexpected exception when executing query", ex);
+      }
+    } catch (ServerConnectivityException ex) {
+      if ((expectedResult.intValue() == NOTAUTHZ_EXCEPTION)
+          && (ex.getCause() instanceof NotAuthorizedException)) {
+        getLogWriter().info(
+            "Got expected NotAuthorizedException when executing query: "
+                + ex.getCause());
+      } else if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when executing query: " + ex);
+      } else {
+        fail("Got unexpected exception when executing query", ex);
+      }
+    } catch (Exception ex) {
+      if (expectedResult.intValue() == OTHER_EXCEPTION) {
+        getLogWriter().info(
+            "Got expected exception when executing query: " + ex);
+      } else {
+        fail("Got unexpected exception when executing query", ex);
+      }
+    }
+  }
+
+  public static void doPuts(Integer num) {
+
+    doPutsP(num, new Integer(NO_EXCEPTION), false);
+  }
+
+  public static void doPuts(Integer num, Integer expectedResult) {
+
+    doPutsP(num, expectedResult, false);
+  }
+
+  public static void doMultiUserPuts(Integer num, Integer numOfUsers,
+      Integer[] expectedResults) {
+    if (numOfUsers != expectedResults.length) {
+      fail("SecurityTestUtil.doMultiUserPuts(): numOfUsers = " + numOfUsers
+          + ", but expected results " + expectedResults.length);
+    }
+    for (int i = 0; i < numOfUsers; i++) {
+      getLogWriter().info("PUT: MultiUser# " + i);
+      doPutsP(num, Integer.valueOf(i), expectedResults[i], false);
+    }
+  }
+
+  public static void doGets(Integer num) {
+
+    doGetsP(num, new Integer(NO_EXCEPTION), false);
+  }
+
+  public static void doGets(Integer num, Integer expectedResult) {
+
+    doGetsP(num, expectedResult, false);
+  }
+
+  public static void doMultiUserGetAll(Integer numOfUsers, Integer[] expectedResults) {
+    doMultiUserGetAll(numOfUsers, expectedResults, false);
+  }
+
+  public static void doMultiUserGetAll(Integer numOfUsers,
+      Integer[] expectedResults, boolean useTX) {
+    if (numOfUsers != expectedResults.length) {
+      fail("SecurityTestUtil.doMultiUserGetAll(): numOfUsers = " + numOfUsers
+          + ", but expected results " + expectedResults.length);
+    }
+    for (int i = 0; i < numOfUsers; i++) {
+      getLogWriter().info(
+          "GET_ALL" + (useTX ? " in TX" : "") + ": MultiUser# " + i);
+      doGetAllP(Integer.valueOf(i), expectedResults[i], useTX);
+    }
+  }
+  
+  public static void doMultiUserGets(Integer num, Integer numOfUsers,
+      Integer[] expectedResults) {
+    if (numOfUsers != expectedResults.length) {
+      fail("SecurityTestUtil.doMultiUserGets(): numOfUsers = " + numOfUsers
+          + ", but expected results " + expectedResults.length);
+    }
+    for (int i = 0; i < numOfUsers; i++) {
+      getLogWriter().info("GET: MultiUser# " + i);
+      doGetsP(num, Integer.valueOf(i), expectedResults[i], false);
+    }
+  }
+
+  public static void doMultiUserRegionDestroys(Integer numOfUsers,
+      Integer[] expectedResults) {
+    if (numOfUsers != expectedResults.length) {
+      fail("SecurityTestUtil.doMultiUserRegionDestroys(): numOfUsers = " + numOfUsers
+          + ", but expected results " + expectedResults.length);
+    }
+    for (int i = numOfUsers-1; i >= 0; i--) {
+      getLogWriter().info("DESTROY: MultiUser# " + i);
+      doRegionDestroysP(Integer.valueOf(i), expectedResults[i]);
+    }
+  }
+
+  public static void doMultiUserDestroys(Integer num, Integer numOfUsers,
+      Integer[] expectedResults) {
+    if (numOfUsers != expectedResults.length) {
+      fail("SecurityTestUtil.doMultiUserDestroys(): numOfUsers = " + numOfUsers
+          + ", but expected results " + expectedResults.length);
+    }
+    for (int i = 0; i < numOfUsers; i++) {
+      getLogWriter().info("DESTROY: MultiUser# " + i);
+      doDestroysP(num, Integer.valueOf(i), expectedResults[i], false);
+    }
+  }
+
+  public static void doMultiUserInvalidates(Integer num, Integer numOfUsers,
+      Integer[] expectedResults) {
+    if (numOfUsers != expectedResults.length) {
+      fail("SecurityTestUtil.doMultiUserInvalidates(): numOfUsers = " + numOfUsers
+          + ", but expected results " + expectedResults.length);
+    }
+    for (int i = 0; i < numOfUsers; i++) {
+      getLogWriter().info("INVALIDATE: MultiUser# " + i);
+      doInvalidatesP(num, Integer.valueOf(i), expectedResults[i], false);
+    }
+  }
+
+  public static void doMultiUserContainsKeys(Integer num, Integer numOfUsers,
+      Integer[] expectedResults, Boolean[] results) {
+    if (numOfUsers != expectedResults.length) {
+      fail("SecurityTestUtil.doMultiUserContainsKeys(): numOfUsers = " + numOfUsers
+          + ", but #expected results " + expectedResults.length);
+    }
+    if (numOfUsers != results.length) {
+      fail("SecurityTestUtil.doMultiUserContainsKeys(): numOfUsers = " + numOfUsers
+          + ", but #expected output " + results.length);
+    }
+    for (int i = 0; i < numOfUsers; i++) {
+      getLogWriter().info("CONTAINS_KEY: MultiUser# " + i);
+      doContainsKeysP(num, Integer.valueOf(i), expectedResults[i], false, results[i]);
+    }
+  }
+
+  public static void doMultiUserQueries(Integer numOfUsers,
+      Integer[] expectedResults, Integer valueSize) {
+    if (numOfUsers != expectedResults.length) {
+      fail("SecurityTestUtil.doMultiUserQueries(): numOfUsers = " + numOfUsers
+          + ", but #expected results " + expectedResults.length);
+    }
+    for (int i = 0; i < numOfUsers; i++) {
+      getLogWriter().info("QUERY: MultiUser# " + i);
+      doQueriesP(Integer.valueOf(i), expectedResults[i], valueSize);
+    }
+  }
+
+  public static void doMultiUserFE(Integer numOfUsers, Function function,
+      Integer[] expectedResults, Object[] results, Boolean isFailoverCase) {
+    if (numOfUsers != expectedResults.length) {
+      fail("SecurityTestUtil.doMultiUserFE(): numOfUsers = " + numOfUsers
+          + ", but #expected results " + expectedResults.length);
+    }
+    if (numOfUsers != results.length) {
+      fail("SecurityTestUtil.doMultiUserFE(): numOfUsers = " + numOfUsers
+          + ", but #expected output " + results.length);
+    }
+    for (int i = 0; i < numOfUsers; i++) {
+      getLogWriter().info("FunctionExecute:onRegion MultiUser# " + i);
+      doFunctionExecuteP(Integer.valueOf(i), function, expectedResults[i], results[i], "region");
+    }
+    for (int i = 0; i < numOfUsers; i++) {
+      getLogWriter().info("FunctionExecute:onServer MultiUser# " + i);
+      doFunctionExecuteP(Integer.valueOf(i), function, expectedResults[i], results[i], "server");
+    }
+    if (!isFailoverCase) {
+      for (int i = 0; i < numOfUsers; i++) {
+        getLogWriter().info("FunctionExecute:onServers MultiUser# " + i);
+        doFunctionExecuteP(Integer.valueOf(i), function, expectedResults[i],
+            results[i], "servers");
+      }
+    }
+  }
+
+  public static void doMultiUserQueryExecute(Integer numOfUsers,
+      Integer[] expectedResults, Integer result) {
+    if (numOfUsers != expectedResults.length) {
+      fail("SecurityTestUtil.doMultiUserFE(): numOfUsers = " + numOfUsers
+          + ", but #expected results " + expectedResults.length);
+    }
+    for (int i = 0; i < numOfUsers; i++) {
+      getLogWriter().info("QueryExecute: MultiUser# " + i);
+      doQueryExecuteP(Integer.valueOf(i), expectedResults[i], result);
+    }
+  }
+
+  public static void doLocalGets(Integer num) {
+
+    doLocalGetsP(num.intValue(), false);
+  }
+
+  public static void doNPuts(Integer num) {
+
+    doPutsP(num, new Integer(NO_EXCEPTION), true);
+  }
+
+  public static void doNPuts(Integer num, Integer expectedResult) {
+
+    doPutsP(num, expectedResult, true);
+  }
+
+  public static void doNGets(Integer num) {
+
+    doGetsP(num, new Integer(NO_EXCEPTION), true);
+  }
+
+  public static void doNGets(Integer num, Integer expectedResult) {
+
+    doGetsP(num, expectedResult, true);
+  }
+
+  public static void doNLocalGets(Integer num) {
+
+    doLocalGetsP(num.intValue(), true);
+  }
+
+  public static void doSimpleGet(String expectedResult) {
+    if (regionRef != null) {
+      try {
+        regionRef.get("KEY");
+        if (expectedResult != null && expectedResult.endsWith("Exception")) {
+          fail("Expected " + expectedResult + " but found none in doSimpleGet()");
+        }
+      } catch (Exception e) {
+        if (!e.getClass().getSimpleName().endsWith(expectedResult)) {
+          fail("Expected " + expectedResult + " but found "
+              + e.getClass().getSimpleName() + " in doSimpleGet()");
+        } else {
+          getLogWriter().fine(
+              "Got expected " + e.getClass().getSimpleName()
+                  + " in doSimpleGet()");
+        }
+      }
+    }
+  }
+
+  public static void doSimplePut(String expectedResult) {
+    if (regionRef != null) {
+      try {
+        regionRef.put("KEY", "VALUE");
+        if (expectedResult != null && expectedResult.endsWith("Exception")) {
+          fail("Expected " + expectedResult + " but found none in doSimplePut()");
+        }
+      } catch (Exception e) {
+        if (!e.getClass().getSimpleName().endsWith(expectedResult)) {
+          fail("Expected " + expectedResult + " but found "
+              + e.getClass().getSimpleName() + " in doSimplePut()", e);
+        } else {
+          getLogWriter().fine(
+              "Got expected " + e.getClass().getSimpleName()
+                  + " in doSimplePut()");
+        }
+      }
+    }
+  }
+  // Deal with javax SSL properties
+
+  private static void makeNullStaticField(Class cls) {
+
+    Field[] fields = cls.getDeclaredFields();
+    for (int index = 0; index < fields.length; ++index) {
+      Field field = fields[index];
+      try {
+        if (Modifier.isStatic(field.getModifiers())) {
+          field.setAccessible(true);
+          if (field.getClass().equals(boolean.class)) {
+            field.setBoolean(null, false);
+            assertFalse(field.getBoolean(null));
+          }
+          else if (cls.isInstance(field.get(null))) {
+            field.set(null, null);
+            assertNull(field.get(null));
+          }
+        }
+      }
+      catch (IllegalAccessException ex) {
+        getLogWriter()
+            .warning("Exception while clearing static SSL field.", ex);
+      }
+      catch (ClassCastException ex) {
+        getLogWriter()
+            .warning("Exception while clearing static SSL field.", ex);
+      }
+    }
+  }
+
+  private static void makeNullSSLFields(Object obj, Map fieldMap) {
+
+    Iterator fieldIter = fieldMap.entrySet().iterator();
+    while (fieldIter.hasNext()) {
+      Map.Entry entry = (Map.Entry)fieldIter.next();
+      Field field = (Field)entry.getKey();
+      Object fieldObj = entry.getValue();
+      try {
+        field.setAccessible(true);
+        makeNullStaticField(fieldObj.getClass());
+        field.set(obj, null);
+        assertNull(field.get(obj));
+      }
+      catch (IllegalAccessException ex) {
+        getLogWriter().warning("Exception while clearing SSL fields.", ex);
+      }
+    }
+  }
+
+  private static HashMap getSSLFields(Object obj, Class[] classes) {
+
+    HashMap resultFields = new HashMap();
+    Field[] fields = obj.getClass().getDeclaredFields();
+    for (int index = 0; index < fields.length; ++index) {
+      Field field = fields[index];
+      try {
+        field.setAccessible(true);
+        Object fieldObj = field.get(obj);
+        boolean isInstance = false;
+        for (int classIndex = 0; classIndex < classes.length; ++classIndex) {
+          if ((isInstance = classes[classIndex].isInstance(fieldObj)) == true) {
+            break;
+          }
+        }
+        if (isInstance) {
+          resultFields.put(field, fieldObj);
+        }
+      }
+      catch (IllegalAccessException ex) {
+        getLogWriter().warning("Exception while getting SSL fields.", ex);
+      }
+    }
+    return resultFields;
+  }
+
+  // This is a hack using reflection to clear the static objects in JSSE since
+  // otherwise changing the javax.* store related properties has no effect
+  // during the course of running dunit suite unless the VMs are restarted.
+  public static void clearStaticSSLContext() {
+
+    ServerSocketFactory defaultServerFact = SSLServerSocketFactory.getDefault();
+    // Get the class of this and use reflection to blank out any static
+    // SSLContext objects inside
+    Map contextMap = getSSLFields(defaultServerFact, new Class[] {
+        SSLContext.class, SSLContextSpi.class });
+    makeNullSSLFields(defaultServerFact, contextMap);
+    Iterator contextObjsIter = contextMap.values().iterator();
+    while (contextObjsIter.hasNext()) {
+      Object contextObj = contextObjsIter.next();
+      Map contextObjsMap = getSSLFields(contextObj, new Class[] {
+          TrustManager.class, KeyManager.class, TrustManager[].class,
+          KeyManager[].class });
+      makeNullSSLFields(contextObj, contextObjsMap);
+    }
+    makeNullStaticField(SSLServerSocketFactory.class);
+
+    // Do the same for normal SSL socket factory
+    SocketFactory defaultFact = SSLSocketFactory.getDefault();
+    contextMap = getSSLFields(defaultFact, new Class[] { SSLContext.class,
+        SSLContextSpi.class });
+    makeNullSSLFields(defaultFact, contextMap);
+    contextObjsIter = contextMap.values().iterator();
+    while (contextObjsIter.hasNext()) {
+      Object contextObj = contextObjsIter.next();
+      Map contextObjsMap = getSSLFields(contextObj, new Class[] {
+          TrustManager.class, KeyManager.class, TrustManager[].class,
+          KeyManager[].class });
+      makeNullSSLFields(contextObj, contextObjsMap);
+    }
+    makeNullStaticField(SSLSocketFactory.class);
+    makeNullStaticField(SSLContext.class);
+  }
+
+  private static LogWriter getLogger() {
+
+    LogWriter logger = null;
+    DistributedSystem dsys = system;
+    if (dsys == null || !dsys.isConnected()) {
+      while ((dsys = InternalDistributedSystem.getAnyInstance()) != null
+          && !dsys.isConnected()) {
+      }
+    }
+    if (dsys != null && dsys.isConnected()) {
+      logger = dsys.getLogWriter();
+    }
+    return logger;
+  }
+
+  public static void closeCache() {
+
+    LogWriter logger = getLogger();
+    if (logger != null) {
+      removeExpectedExceptions(SecurityTestUtil.expectedExceptions, logger);
+    }
+    if (cache != null && !cache.isClosed()) {
+      DistributedSystem sys = cache.getDistributedSystem();
+      cache.close();
+      sys.disconnect();
+      cache = null;
+    }
+    disconnectFromDS();
+  }
+
+  public static void closeCache(Boolean keepAlive) {
+    LogWriter logger = getLogger();
+    if (logger != null) {
+      removeExpectedExceptions(SecurityTestUtil.expectedExceptions, logger);
+    }
+    if (cache != null && !cache.isClosed()) {
+      DistributedSystem sys = cache.getDistributedSystem();
+      cache.close(keepAlive);
+      sys.disconnect();
+      cache = null;
+    }
+    disconnectFromDS();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a622d6ec/gemfire-core/src/test/java/templates/security/AuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/templates/security/AuthzCredentialGenerator.java b/gemfire-core/src/test/java/templates/security/AuthzCredentialGenerator.java
new file mode 100644
index 0000000..0e1ccf6
--- /dev/null
+++ b/gemfire-core/src/test/java/templates/security/AuthzCredentialGenerator.java
@@ -0,0 +1,465 @@
+
+package templates.security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.security.AccessControl;
+
+/**
+ * Encapsulates obtaining authorized and unauthorized credentials for a given
+ * operation in a region. Implementations will be for different kinds of
+ * authorization scheme and authentication scheme combos.
+ * 
+ * @author sumedh
+ * @since 5.5
+ */
+public abstract class AuthzCredentialGenerator {
+
+  /**
+   * Enumeration for various {@link AuthzCredentialGenerator} implementations.
+   * 
+   * The following schemes are supported as of now:
+   * <ul>
+   * <li><code>DummyAuthorization</code> with <code>DummyAuthenticator</code></li>
+   * <li><code>XMLAuthorization</code> with <code>DummyAuthenticator</code></li>
+   * <li><code>XMLAuthorization</code> with <code>LDAPAuthenticator</code></li>
+   * <li><code>XMLAuthorization</code> with <code>PKCSAuthenticator</code></li>
+   * <li><code>XMLAuthorization</code> when using SSL sockets</li>
+   * </ul>
+   * 
+   * To add a new authorization scheme the following needs to be done:
+   * <ul>
+   * <li>Add implementation for {@link AccessControl}.</li>
+   * <li>Choose the authentication schemes that it shall work with from
+   * {@link CredentialGenerator.ClassCode}</li>
+   * <li>Add a new enumeration value for the scheme in this class. Notice the
+   * size of <code>VALUES</code> array and increase that if it is getting
+   * overflowed. Note the methods and fields for existing schemes and add for
+   * the new one in a similar manner.</li>
+   * <li>Add an implementation for {@link AuthzCredentialGenerator}. Note the
+   * {@link AuthzCredentialGenerator#init} method where different authentication
+   * schemes can be passed and initialize differently for the authentication
+   * schemes that shall be handled.</li>
+   * <li>Modify the {@link AuthzCredentialGenerator#create} method to add
+   * creation of an instance of the new implementation for the
+   * <code>ClassCode</code> enumeration value.</li>
+   * </ul>
+   * All dunit tests will automagically start testing the new implementation
+   * after this.
+   * 
+   * @author sumedh
+   * @since 5.5
+   */
+  public static final class ClassCode {
+
+    private static final byte ID_DUMMY = 1;
+
+    private static final byte ID_XML = 2;
+
+    private static byte nextOrdinal = 0;
+
+    private static final ClassCode[] VALUES = new ClassCode[10];
+
+    private static final Map CodeNameMap = new HashMap();
+
+    public static final ClassCode DUMMY = new ClassCode(
+        "templates.security.DummyAuthorization.create", ID_DUMMY);
+
+    public static final ClassCode XML = new ClassCode(
+        "templates.security.XmlAuthorization.create", ID_XML);
+
+    /** The name of this class. */
+    private final String name;
+
+    /** byte used as ordinal to represent this class */
+    private final byte ordinal;
+
+    /**
+     * One of the following: ID_DUMMY, ID_LDAP, ID_PKI
+     */
+    private final byte classType;
+
+    /** Creates a new instance of class code. */
+    private ClassCode(String name, byte classType) {
+      this.name = name;
+      this.classType = classType;
+      this.ordinal = nextOrdinal++;
+      VALUES[this.ordinal] = this;
+      CodeNameMap.put(name, this);
+    }
+
+    public boolean isDummy() {
+      return (this.classType == ID_DUMMY);
+    }
+
+    public boolean isXml() {
+      return (this.classType == ID_XML);
+    }
+
+    /**
+     * Returns the <code>ClassCode</code> represented by specified ordinal.
+     */
+    public static ClassCode fromOrdinal(byte ordinal) {
+      return VALUES[ordinal];
+    }
+
+    /**
+     * Returns the <code>ClassCode</code> represented by specified string.
+     */
+    public static ClassCode parse(String operationName) {
+      return (ClassCode)CodeNameMap.get(operationName);
+    }
+
+    /**
+     * Returns all the possible values.
+     */
+    public static List getAll() {
+      List codes = new ArrayList();
+      Iterator iter = CodeNameMap.values().iterator();
+      while (iter.hasNext()) {
+        codes.add(iter.next());
+      }
+      return codes;
+    }
+
+    /**
+     * Returns the ordinal for this class code.
+     * 
+     * @return the ordinal of this class code.
+     */
+    public byte toOrdinal() {
+      return this.ordinal;
+    }
+
+    /**
+     * Returns a string representation for this class code.
+     * 
+     * @return the name of this class code.
+     */
+    final public String toString() {
+      return this.name;
+    }
+
+    /**
+     * Indicates whether other object is same as this one.
+     * 
+     * @return true if other object is same as this one.
+     */
+    @Override
+    final public boolean equals(final Object obj) {
+      if (obj == this) {
+        return true;
+      }
+      if (!(obj instanceof ClassCode)) {
+        return false;
+      }
+      final ClassCode other = (ClassCode)obj;
+      return (other.ordinal == this.ordinal);
+    }
+
+    /**
+     * Indicates whether other <code>ClassCode</code> is same as this one.
+     * 
+     * @return true if other <code>ClassCode</code> is same as this one.
+     */
+    final public boolean equals(final ClassCode opCode) {
+      return (opCode != null && opCode.ordinal == this.ordinal);
+    }
+
+    /**
+     * Returns a hash code value for this <code>ClassCode</code> which is the
+     * same as its ordinal.
+     * 
+     * @return the ordinal of this <code>ClassCode</code>.
+     */
+    @Override
+    final public int hashCode() {
+      return this.ordinal;
+    }
+
+  }
+
+  /**
+   * The {@link CredentialGenerator} being used.
+   */
+  protected CredentialGenerator cGen;
+
+  /**
+   * A set of system properties that should be added to the gemfire system
+   * properties before using the authorization module.
+   */
+  private Properties sysProps;
+
+  /**
+   * A factory method to create a new instance of an
+   * {@link AuthzCredentialGenerator} for the given {@link ClassCode}. Caller
+   * is supposed to invoke {@link AuthzCredentialGenerator#init} immediately
+   * after obtaining the instance.
+   * 
+   * @param classCode
+   *                the <code>ClassCode</code> of the
+   *                <code>AuthzCredentialGenerator</code> implementation
+   * 
+   * @return an instance of <code>AuthzCredentialGenerator</code> for the
+   *         given class code
+   */
+  public static AuthzCredentialGenerator create(ClassCode classCode) {
+    switch (classCode.classType) {
+      case ClassCode.ID_DUMMY:
+        return new DummyAuthzCredentialGenerator();
+      case ClassCode.ID_XML:
+        return new XmlAuthzCredentialGenerator();
+      default:
+        return null;
+    }
+  }
+
+  /**
+   * Initialize the authorized credential generator.
+   * 
+   * @param cGen
+   *                an instance of {@link CredentialGenerator} of the credential
+   *                implementation for which to obtain authorized/unauthorized
+   *                credentials.
+   * 
+   * @return false when the given {@link CredentialGenerator} is incompatible
+   *         with this authorization module.
+   */
+  public boolean init(CredentialGenerator cGen) {
+    this.cGen = cGen;
+    try {
+      this.sysProps = init();
+    }
+    catch (IllegalArgumentException ex) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * 
+   * @return A set of extra properties that should be added to Gemfire system
+   *         properties when not null.
+   */
+  public Properties getSystemProperties() {
+    return this.sysProps;
+  }
+
+  /**
+   * Get the {@link CredentialGenerator} being used by this instance.
+   */
+  public CredentialGenerator getCredentialGenerator() {
+    return this.cGen;
+  }
+
+  /**
+   * The {@link ClassCode} of the particular implementation.
+   * 
+   * @return the <code>ClassCode</code>
+   */
+  public abstract ClassCode classCode();
+
+  /**
+   * The name of the {@link AccessControl} factory function that should be used
+   * as the authorization module on the server side.
+   * 
+   * @return name of the <code>AccessControl</code> factory function
+   */
+  public abstract String getAuthorizationCallback();
+
+  /**
+   * Get a set of credentials generated using the given index allowed to perform
+   * the given {@link OperationCode}s for the given regions.
+   * 
+   * @param opCodes
+   *                the list of {@link OperationCode}s of the operations
+   *                requiring authorization; should not be null
+   * @param regionNames
+   *                list of the region names requiring authorization; a value of
+   *                null indicates all regions
+   * @param index
+   *                used to generate multiple such credentials by passing
+   *                different values for this
+   * 
+   * @return the set of credentials authorized to perform the given operation in
+   *         the given regions
+   */
+  public Properties getAllowedCredentials(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    int numTries = getNumPrincipalTries(opCodes, regionNames);
+    if (numTries <= 0) {
+      numTries = 1;
+    }
+    for (int tries = 0; tries < numTries; tries++) {
+      Principal principal = getAllowedPrincipal(opCodes, regionNames,
+          (index + tries) % numTries);
+      try {
+        return this.cGen.getValidCredentials(principal);
+      }
+      catch (IllegalArgumentException ex) {
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Get a set of credentials generated using the given index not allowed to
+   * perform the given {@link OperationCode}s for the given regions. The
+   * credentials are required to be valid for authentication.
+   * 
+   * @param opCode
+   *                the {@link OperationCode}s of the operations requiring
+   *                authorization failure; should not be null
+   * @param regionNames
+   *                list of the region names requiring authorization failure; a
+   *                value of null indicates all regions
+   * @param index
+   *                used to generate multiple such credentials by passing
+   *                different values for this
+   * 
+   * @return the set of credentials that are not authorized to perform the given
+   *         operation in the given region
+   */
+  public Properties getDisallowedCredentials(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    // This may not be very correct since we use the value of
+    // getNumPrincipalTries() but is used to avoid adding another method.
+    // Also something like getNumDisallowedPrincipals() will be normally always
+    // infinite, and the number here is just to perform some number of tries
+    // before giving up.
+    int numTries = getNumPrincipalTries(opCodes, regionNames);
+    if (numTries <= 0) {
+      numTries = 1;
+    }
+    for (int tries = 0; tries < numTries; tries++) {
+      Principal principal = getDisallowedPrincipal(opCodes, regionNames,
+          (index + tries) % numTries);
+      try {
+        return this.cGen.getValidCredentials(principal);
+      }
+      catch (IllegalArgumentException ex) {
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Initialize the authorized credential generator.
+   * 
+   * Required to be implemented by concrete classes that implement this abstract
+   * class.
+   * 
+   * @return A set of extra properties that should be added to Gemfire system
+   *         properties when not null.
+   * 
+   * @throws IllegalArgumentException
+   *                 when the {@link CredentialGenerator} is incompatible with
+   *                 this authorization module.
+   */
+  protected abstract Properties init() throws IllegalArgumentException;
+
+  /**
+   * Get the number of tries to be done for obtaining valid credentials for the
+   * given operations in the given region. It is required that
+   * {@link #getAllowedPrincipal} method returns valid principals for values of
+   * <code>index</code> from 0 through (n-1) where <code>n</code> is the
+   * value returned by this method. It is recommended that the principals so
+   * returned be unique for efficiency.
+   * 
+   * This will be used by {@link #getAllowedCredentials} to step through
+   * different principals and obtain a set of valid credentials.
+   * 
+   * Required to be implemented by concrete classes that implement this abstract
+   * class.
+   * 
+   * @param opCodes
+   *                the {@link OperationCode}s of the operations requiring
+   *                authorization
+   * @param regionNames
+   *                list of the region names requiring authorization; a value of
+   *                null indicates all regions
+   * @param index
+   *                used to generate multiple such credentials by passing
+   *                different values for this
+   * 
+   * @return the number of principals allowed to perform the given operation in
+   *         the given region
+   */
+  protected abstract int getNumPrincipalTries(OperationCode[] opCodes,
+      String[] regionNames);
+
+  /**
+   * Get a {@link Principal} generated using the given index allowed to perform
+   * the given {@link OperationCode}s for the given region.
+   * 
+   * Required to be implemented by concrete classes that implement this abstract
+   * class.
+   * 
+   * @param opCodes
+   *                the {@link OperationCode}s of the operations requiring
+   *                authorization
+   * @param regionNames
+   *                list of the region names requiring authorization; a value of
+   *                null indicates all regions
+   * @param index
+   *                used to generate multiple such principals by passing
+   *                different values for this
+   * 
+   * @return the {@link Principal} authorized to perform the given operation in
+   *         the given region
+   */
+  protected abstract Principal getAllowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index);
+
+  /**
+   * Get a {@link Principal} generated using the given index not allowed to
+   * perform the given {@link OperationCode}s for the given region.
+   * 
+   * Required to be implemented by concrete classes that implement this abstract
+   * class.
+   * 
+   * @param opCodes
+   *                the {@link OperationCode}s of the operations requiring
+   *                authorization failure
+   * @param regionNames
+   *                list of the region names requiring authorization failure; a
+   *                value of null indicates all regions
+   * @param index
+   *                used to generate multiple such principals by passing
+   *                different values for this
+   * 
+   * @return a {@link Principal} not authorized to perform the given operation
+   *         in the given region
+   */
+  protected abstract Principal getDisallowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index);
+}


[20/33] incubator-geode git commit: GEODE-709: CI Failure in AnalyzeSerializablesJUnitTest

Posted by je...@apache.org.
GEODE-709: CI Failure in AnalyzeSerializablesJUnitTest

JDK 1.7 added three new types of entries to the "constant pool" of
a compiled class: InvokeDynamic, MethodHandle and MethodType.  The test
was failing when it ran into an InvokeDynamic entry (type code 18) in some
compiled class.

Evidently we had no classes that required an InvokeDynamic entry until
recently.

I also changed the test so it will log the name of the class that causes
a failure in constant-pool parsing in the future.


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

Branch: refs/heads/feature/GEODE-14
Commit: 1a025728ac45d24f48d17f261a44fd8277f77b7e
Parents: 05f0413
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Dec 22 09:59:34 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 22 09:59:34 2015 -0800

----------------------------------------------------------------------
 .../gemfire/codeAnalysis/decode/cp/Cp.java      | 14 +++++++--
 .../codeAnalysis/decode/cp/CpInvokeDynamic.java | 33 ++++++++++++++++++++
 .../codeAnalysis/decode/cp/CpMethodHandle.java  | 33 ++++++++++++++++++++
 .../codeAnalysis/decode/cp/CpMethodType.java    | 31 ++++++++++++++++++
 4 files changed, 109 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a025728/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/Cp.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/Cp.java b/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/Cp.java
index 54634a1..d68d2e6 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/Cp.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/Cp.java
@@ -18,7 +18,8 @@ package com.gemstone.gemfire.codeAnalysis.decode.cp;
 import java.io.*;
 
 /**
- * Cp represents an entry in the constant pool of a class
+ * Cp represents an entry in the constant pool of a class.
+ * See https://docs.oracle.com/javase/specs/jvms/se8/html/jvms-4.html#jvms-4.4
  */
 public class Cp {
 
@@ -33,6 +34,9 @@ public class Cp {
     protected static final int TAG_Double = 6;
     protected static final int TAG_NameAndType = 12;
     protected static final int TAG_Utf8 = 1;
+    protected static final int TAG_MethodHandle = 15;
+    protected static final int TAG_MethodType = 16;
+    protected static final int TAG_InvokeDynamic = 18;
 
     public static Cp readCp( DataInputStream source ) throws IOException {
         byte tag;
@@ -61,8 +65,14 @@ public class Cp {
                 return new CpNameAndType(source);
             case TAG_Utf8:
                 return new CpUtf8(source);
+            case TAG_MethodHandle:
+                return new CpMethodHandle(source);
+            case TAG_MethodType:
+                return new CpMethodType(source);
+            case TAG_InvokeDynamic:
+                return new CpInvokeDynamic(source);
             default:
-                throw new Error("Unknown tag type in constant pool: " + tag);
+                throw new IOException("Unknown tag type in constant pool: " + tag);
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a025728/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpInvokeDynamic.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpInvokeDynamic.java b/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpInvokeDynamic.java
new file mode 100755
index 0000000..142700d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpInvokeDynamic.java
@@ -0,0 +1,33 @@
+/*
+ * 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.codeAnalysis.decode.cp;
+import java.io.*;
+
+import com.gemstone.gemfire.codeAnalysis.decode.CompiledClass;
+
+
+public class CpInvokeDynamic extends Cp {
+    int bootstrap_method_attr_index;
+    int name_and_type_index;
+    CpInvokeDynamic( DataInputStream source ) throws IOException {
+        bootstrap_method_attr_index = source.readUnsignedShort();
+        name_and_type_index = source.readUnsignedShort();
+    }
+    public String returnType(CompiledClass info) {
+        return "not yet implemented";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a025728/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpMethodHandle.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpMethodHandle.java b/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpMethodHandle.java
new file mode 100755
index 0000000..841fd8d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpMethodHandle.java
@@ -0,0 +1,33 @@
+/*
+ * 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.codeAnalysis.decode.cp;
+import java.io.*;
+
+import com.gemstone.gemfire.codeAnalysis.decode.CompiledClass;
+
+
+public class CpMethodHandle extends Cp {
+    byte reference_kind;
+    int reference_index;
+    CpMethodHandle( DataInputStream source ) throws IOException {
+        reference_kind = source.readByte();
+        reference_index = source.readUnsignedShort();
+    }
+    public String returnType(CompiledClass info) {
+        return "not yet implemented";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a025728/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpMethodType.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpMethodType.java b/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpMethodType.java
new file mode 100755
index 0000000..0436bca
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/codeAnalysis/decode/cp/CpMethodType.java
@@ -0,0 +1,31 @@
+/*
+ * 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.codeAnalysis.decode.cp;
+import java.io.*;
+
+import com.gemstone.gemfire.codeAnalysis.decode.CompiledClass;
+
+
+public class CpMethodType extends Cp {
+    int descriptor_index;
+    CpMethodType( DataInputStream source ) throws IOException {
+        descriptor_index = source.readUnsignedShort();
+    }
+    public String returnType(CompiledClass info) {
+        return "not yet implemented";
+    }
+}


[31/33] incubator-geode git commit: GEODE-707 cache loader not invoked on concurrent load if first load fails with an exception

Posted by je...@apache.org.
GEODE-707 cache loader not invoked on concurrent load if first load fails with an exception

avoid setting the future to {null, versionTag} when the loader throws an
exception so that a concurrent load attempt will be allowed


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

Branch: refs/heads/feature/GEODE-14
Commit: 81eafccce3cfce2500dcce786c65de1ae5b057dc
Parents: 935b76a
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Dec 29 07:54:56 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 29 07:54:56 2015 -0800

----------------------------------------------------------------------
 .../gemfire/internal/cache/LocalRegion.java     |  45 +++--
 .../internal/cache/PartitionedRegion.java       |  75 --------
 .../gemfire/cache30/SearchAndLoadDUnitTest.java | 177 ++++++++++++++++++-
 3 files changed, 199 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/81eafccc/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index caf07ce..2bc2f05 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -1261,7 +1261,7 @@ public class LocalRegion extends AbstractRegion
    * @param disableCopyOnRead if true then disable copy on read
    * @param preferCD true if the preferred result form is CachedDeserializable
    * @param clientEvent client's event, if any (for version tag retrieval)
-   * @param returnTombstones TODO
+   * @param returnTombstones whether destroyed entries should be returned
    * @param retainResult if true then the result may be a retained off-heap reference
    * @return the value for the given key
    */
@@ -1613,21 +1613,32 @@ public class LocalRegion extends AbstractRegion
         throw err;
       }
     }
-    // didn't find a future, do one more getDeserialized to catch race
-    // condition where the future was just removed by another get thread
+    // didn't find a future, do one more probe for the entry to catch a race
+    // condition where the future was just removed by another thread
     try {
-      localValue = getDeserializedValue(null, keyInfo, isCreate, disableCopyOnRead, preferCD, clientEvent, false, false/*allowReadFromHDFS*/, false);
-      // TODO verify that this method is not used for PR or BR and hence allowReadFromHDFS does not matter
-      // stats have now been updated
-      if (localValue != null && !Token.isInvalid(localValue)) {
-        result = localValue;
-        return result;
+      boolean partitioned = this.getDataPolicy().withPartitioning();
+      if (!partitioned) {
+        localValue = getDeserializedValue(null, keyInfo, isCreate, disableCopyOnRead, preferCD, clientEvent, false, false/*allowReadFromHDFS*/, false);
+
+        // stats have now been updated
+        if (localValue != null && !Token.isInvalid(localValue)) {
+          result = localValue;
+          return result;
+        }
+        isCreate = localValue == null;
+        result = findObjectInSystem(keyInfo, isCreate, null, generateCallbacks,
+            localValue, disableCopyOnRead, preferCD, null, clientEvent, returnTombstones, false/*allowReadFromHDFS*/);
+
+      } else {
+        
+        // This code was moved from PartitionedRegion.nonTxnFindObject().  That method has been removed.
+        // For PRs we don't want to deserialize the value and we can't use findObjectInSystem because
+        // it can invoke code that is transactional.
+        result = getSharedDataView().findObject(keyInfo, this, true/*isCreate*/, generateCallbacks,
+            localValue, disableCopyOnRead, preferCD, null, null, false, allowReadFromHDFS);
+        // TODO why are we not passing the client event or returnTombstones in the above invokation?
       }
-      isCreate = localValue == null;
 
-      result = findObjectInSystem(keyInfo, isCreate, null, generateCallbacks,
-          localValue, disableCopyOnRead, preferCD, null, clientEvent, returnTombstones, false/*allowReadFromHDFS*/);
-      
       if (result == null && localValue != null) {
         if (localValue != Token.TOMBSTONE || returnTombstones) {
           result = localValue;
@@ -1636,8 +1647,12 @@ public class LocalRegion extends AbstractRegion
       // findObjectInSystem does not call conditionalCopy
     }
     finally {
-      VersionTag tag = (clientEvent==null)? null : clientEvent.getVersionTag();
-      thisFuture.set(new Object[]{result, tag});
+      if (result != null) {
+        VersionTag tag = (clientEvent==null)? null : clientEvent.getVersionTag();
+        thisFuture.set(new Object[]{result, tag});
+      } else {
+        thisFuture.set(null);
+      }
       this.getFutures.remove(keyInfo.getKey());
     }
     if (!disableCopyOnRead) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/81eafccc/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
old mode 100644
new mode 100755
index a36d719..a14e99f
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
@@ -265,9 +265,6 @@ import com.gemstone.gemfire.i18n.StringId;
  * are copied up to a configurable level (for high availability) and placed on
  * multiple VMs for improved performance and increased storage capacity.
  * 
- * @since 5.0
- * @author Rohit Reja, Tushar Apshankar, Girish Thombare, Negi Tribhuwan, Greg
- *         Passmore, Mitch Thomas, Bruce Schuchardt
  */
 public class PartitionedRegion extends LocalRegion implements 
   CacheDistributionAdvisee, QueryExecutor {
@@ -3314,78 +3311,6 @@ public class PartitionedRegion extends LocalRegion implements
   }
 
   /**
-    * override the one in LocalRegion since we don't need to do getDeserialized.
-    */
-   @Override Object nonTxnFindObject(KeyInfo keyInfo, boolean isCreate,
-      boolean generateCallbacks, Object localValue, boolean disableCopyOnRead, boolean preferCD,
-      EntryEventImpl clientEvent, boolean returnTombstones, boolean allowReadFromHDFS) 
-      throws TimeoutException, CacheLoaderException
-  {
-    Object result = null;
-    FutureResult thisFuture = new FutureResult(getCancelCriterion());
-    Future otherFuture = (Future)this.getFutures.putIfAbsent(keyInfo.getKey(), thisFuture);
-    // only one thread can get their future into the map for this key at a time
-    if (otherFuture != null) {
-      try {
-        result = otherFuture.get();
-        if (result != null) {
-          if (!preferCD && result instanceof CachedDeserializable) {
-            CachedDeserializable cd = (CachedDeserializable)result;
-            // fix for bug 43023
-            if (!disableCopyOnRead && isCopyOnRead()) {
-              result = cd.getDeserializedWritableCopy(null, null);
-            } else {
-              result = cd.getDeserializedForReading();
-            }
-            
-          } else if (!disableCopyOnRead) {
-            result = conditionalCopy(result);
-          }
-          
-        //For sqlf since the deserialized value is nothing but chunk
-          // before returning the found value increase its use count
-         /* if(GemFireCacheImpl.sqlfSystem() && result instanceof Chunk) {
-            if(!((Chunk)result).use()) {
-              return null;
-            }
-          }*/
-           // what was a miss is now a hit
-          RegionEntry re = null;
-          if (isCreate) {
-            re = basicGetEntry(keyInfo.getKey());
-            updateStatsForGet(re, true);
-          }
-          return result;
-        }
-        // if value == null, try our own search/load
-      }
-      catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        // TODO check a CancelCriterion here?
-        return null;
-      }
-      catch (ExecutionException e) {
-        // unexpected since there is no background thread
-        AssertionError err = new AssertionError("unexpected exception");
-        err.initCause(err);
-        throw err;
-      }
-    }
-    try {
-      result = getSharedDataView().findObject(keyInfo, this, true/*isCreate*/, generateCallbacks,
-          localValue, disableCopyOnRead, preferCD, null, null, false, allowReadFromHDFS);
-    }
-    finally {
-      if (result instanceof Chunk) {
-        thisFuture.set(null);
-      } else {
-        thisFuture.set(result);
-      }
-      this.getFutures.remove(keyInfo.getKey());
-    }
-    return result;
-  }
-  /**
    * override the one in LocalRegion since we don't need to do getDeserialized.
    */
    @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/81eafccc/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/SearchAndLoadDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/SearchAndLoadDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/SearchAndLoadDUnitTest.java
index b33bda2..cf9ff9c 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/SearchAndLoadDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/SearchAndLoadDUnitTest.java
@@ -16,17 +16,17 @@
  */
 package com.gemstone.gemfire.cache30;
 
-//import com.gemstone.gemfire.*;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
 import com.gemstone.gemfire.cache.*;
 
 import dunit.*;
-//import hydra.ClientMgr;
 
 /**
  * This class tests various search load and write scenarios for distributed regions
- * @author Sudhir Menon
- *
  */
+@SuppressWarnings({"deprecation", "unchecked", "rawtypes", "serial"})
 public class SearchAndLoadDUnitTest extends CacheTestCase {
 
   static boolean loaderInvoked;
@@ -48,6 +48,10 @@ public class SearchAndLoadDUnitTest extends CacheTestCase {
   /** A <code>CacheWriter</code> used by a test */
   protected static TestCacheWriter writer;
 
+  static boolean exceptionThrown;
+  static final CountDownLatch readyForExceptionLatch = new CountDownLatch(1);
+  static final CountDownLatch loaderInvokedLatch = new CountDownLatch(1);
+
   public SearchAndLoadDUnitTest(String name) {
     super(name);
   }
@@ -171,8 +175,166 @@ public class SearchAndLoadDUnitTest extends CacheTestCase {
     });
   }
 
-  public void testNetLoadNoLoaders()
-  throws CacheException, InterruptedException {
+
+  /**
+   * This test is for a bug in which a cache loader threw an exception
+   * that caused the wrong value to be put in a Future in nonTxnFindObject.  This
+   * in turn caused a concurrent search for the object to not invoke the loader a
+   * second time.
+   * 
+   * VM0 is used to create a cache and a region having a loader that simulates the
+   * conditions that caused the bug.  One async thread then does a get() which invokes
+   * the loader.  Another async thread does a get() which reaches nonTxnFindObject
+   * and blocks waiting for the first thread's load to complete.  The loader then
+   * throws an exception that is sent back to the first thread.  The second thread
+   * should then cause the loader to be invoked again, and this time the loader will
+   * return a value.  Both threads then validate that they received the expected
+   * result.
+   */
+  public void testConcurrentLoad() throws Throwable {
+
+    Host host = Host.getHost(0);
+    VM vm0 = host.getVM(0);
+    
+    final String name = this.getUniqueName() + "Region";
+    final String objectName = "theKey";
+    final Integer value = new Integer(44);
+    final String exceptionString = "causing first cache-load to fail";
+
+    remoteLoaderInvoked = false;
+    loaderInvoked = false;
+    
+    vm0.invoke(new CacheSerializableRunnable("create region " + name + " in vm0") {
+      public void run2() {
+        remoteLoaderInvoked = false;
+        loaderInvoked = false;
+        AttributesFactory factory = new AttributesFactory();
+        factory.setScope(Scope.DISTRIBUTED_ACK);
+        factory.setConcurrencyChecksEnabled(true);
+        factory.setCacheLoader(new CacheLoader() {
+          boolean firstInvocation = true;
+          public synchronized Object load(LoaderHelper helper) {
+            System.out.println("invoked cache loader for " + helper.getKey());
+            loaderInvoked = true;
+            loaderInvokedLatch.countDown();
+            if (firstInvocation) {
+              firstInvocation = false;
+              try { 
+                // wait for both threads to be ready for the exception to be thrown
+                System.out.println("waiting for vm0t2 to be ready before throwing exception");
+                readyForExceptionLatch.await(30, TimeUnit.SECONDS);
+                // give the second thread time to get into loader code
+                Thread.sleep(5000);
+              } catch (InterruptedException e) {
+                fail("interrupted");
+              }
+              System.out.println("throwing exception");
+              exceptionThrown = true;
+              throw new RuntimeException(exceptionString);
+            }
+            System.out.println("returning value="+value);
+            return value;
+          }
+
+          public void close() {
+
+          }
+        });
+
+        Region region = createRegion(name,factory.create());
+        region.create(objectName, null);
+        addExpectedException(exceptionString);
+      }
+    });
+
+    AsyncInvocation async1 = null;
+    try {
+      async1 = vm0.invokeAsync(new CacheSerializableRunnable("Concurrently invoke the remote loader on the same key - t1") {
+        public void run2() {
+          Region region = getCache().getRegion("root/"+name);
+  
+          getLogWriter().info("t1 is invoking get("+objectName+")");
+          try {
+            getLogWriter().info("t1 retrieved value " + region.get(objectName));
+            fail("first load should have triggered an exception");
+          } catch (RuntimeException e) {
+            if (!e.getMessage().contains(exceptionString)) {
+              throw e;
+            }
+          }
+        }
+      });
+      vm0.invoke(new CacheSerializableRunnable("Concurrently invoke the loader on the same key - t2") {
+        public void run2() {
+          final Region region = getCache().getRegion("root/"+name);
+          final Object[] valueHolder = new Object[1];
+  
+          // wait for vm1 to cause the loader to be invoked
+          getLogWriter().info("t2 is waiting for loader to be invoked by t1");
+          try {
+            loaderInvokedLatch.await(30, TimeUnit.SECONDS);
+          } catch (InterruptedException e) {
+            fail("interrupted");
+          }
+          assertTrue(loaderInvoked);
+          
+          Thread t = new Thread("invoke get()") {
+            public void run() {
+              try {
+                valueHolder[0] = region.get(objectName);
+              } catch (RuntimeException e) {
+                valueHolder[0] = e;
+              }
+            }
+          };
+          
+          t.setDaemon(true);
+          t.start();
+          try {
+            // let the thread get to the point of blocking on vm1's Future
+            // in LocalRegion.nonTxnFindObject()
+            Thread.sleep(5000);
+          } catch (InterruptedException e) {
+            fail("interrupted");
+          }
+          
+          readyForExceptionLatch.countDown();
+          try {
+            t.join(30000);
+          } catch (InterruptedException e) {
+            fail("interrupted");
+          }
+          if (t.isAlive()) {
+            t.interrupt();
+            fail("get() operation blocked for too long - test needs some work");
+          }
+          
+          getLogWriter().info("t2 is invoking get("+objectName+")");
+          Object value = valueHolder[0];
+          if (value instanceof RuntimeException) {
+            if ( ((Exception)value).getMessage().contains(exceptionString) ) {
+              fail("second load should not have thrown an exception");
+            } else {
+              throw (RuntimeException)value;
+            }
+          } else {
+            getLogWriter().info("t2 retrieved value " + value);
+            assertNotNull(value);
+          }
+        }
+      });
+    } finally {
+      if (async1 != null) {
+        async1.join();
+        if (async1.exceptionOccurred()) {
+          throw async1.getException();
+        }
+      }
+    }
+  }
+  
+  
+  public void testNetLoadNoLoaders() throws CacheException, InterruptedException {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
@@ -318,7 +480,6 @@ public class SearchAndLoadDUnitTest extends CacheTestCase {
     VM vm2 = host.getVM(2);
     final String name = this.getUniqueName() + "-ACK";
     final String objectName = "B";
-    final Integer value = new Integer(43);
     loaderInvoked = false;
     remoteLoaderInvoked = false;
     remoteLoaderInvokedCount = 0;
@@ -369,7 +530,7 @@ public class SearchAndLoadDUnitTest extends CacheTestCase {
 
                 }
               });
-            Region region = createRegion(name,factory.create());
+            createRegion(name,factory.create());
           }
           catch (CacheException ex) {
             fail("While creating ACK region", ex);


[06/33] incubator-geode git commit: GEODE-663: add aditional security tests to the open source

Posted by je...@apache.org.
GEODE-663: add aditional security tests to the open source


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

Branch: refs/heads/feature/GEODE-14
Commit: 3720666f0d1e7c6dedb3cb18a1a0952d7446ac31
Parents: a622d6e
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Dec 11 15:27:50 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Dec 21 08:51:59 2015 -0800

----------------------------------------------------------------------
 .../security/ClientAuthenticationDUnitTest.java |   6 +-
 .../security/ClientAuthorizationDUnitTest.java  |   8 +-
 .../security/ClientAuthorizationTestBase.java   |  10 +-
 .../security/ClientMultiUserAuthzDUnitTest.java |   4 +-
 .../DeltaClientAuthorizationDUnitTest.java      |   4 +-
 .../DeltaClientPostAuthorizationDUnitTest.java  |   4 +-
 .../security/P2PAuthenticationDUnitTest.java    |   8 +-
 .../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 +++++++++++
 .../security/AuthzCredentialGenerator.java      | 465 -------------------
 .../templates/security/CredentialGenerator.java | 343 --------------
 .../security/DummyAuthzCredentialGenerator.java | 142 ------
 .../security/DummyCredentialGenerator.java      |  90 ----
 .../security/LdapUserCredentialGenerator.java   | 158 -------
 .../security/PKCSCredentialGenerator.java       | 110 -----
 .../security/SSLCredentialGenerator.java        | 116 -----
 .../UserPasswordWithExtraPropsAuthInit.java     |  76 ---
 .../security/XmlAuthzCredentialGenerator.java   | 262 -----------
 25 files changed, 1799 insertions(+), 1784 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
index 2fdbc05..2bd04e0 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
@@ -26,8 +26,8 @@ import java.util.Properties;
 import javax.net.ssl.SSLException;
 import javax.net.ssl.SSLHandshakeException;
 
-import templates.security.CredentialGenerator;
-import templates.security.CredentialGenerator.ClassCode;
+import security.CredentialGenerator;
+import security.CredentialGenerator.ClassCode;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -36,7 +36,7 @@ import com.gemstone.gemfire.internal.AvailablePort;
 import dunit.DistributedTestCase;
 import dunit.Host;
 import dunit.VM;
-import templates.security.DummyCredentialGenerator;
+import security.DummyCredentialGenerator;
 
 /**
  * Test for authentication from client to server. This tests for both valid and

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
index e89e0d3..b44ac33 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
@@ -26,10 +26,10 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 
-import templates.security.AuthzCredentialGenerator;
-import templates.security.CredentialGenerator;
-import templates.security.DummyCredentialGenerator;
-import templates.security.XmlAuthzCredentialGenerator;
+import security.AuthzCredentialGenerator;
+import security.CredentialGenerator;
+import security.DummyCredentialGenerator;
+import security.XmlAuthzCredentialGenerator;
 
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
 import com.gemstone.gemfire.internal.AvailablePort;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestBase.java
index ad6526a..e49f9ff 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestBase.java
@@ -31,9 +31,9 @@ import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
 
-import templates.security.AuthzCredentialGenerator;
-import templates.security.CredentialGenerator;
-import templates.security.AuthzCredentialGenerator.ClassCode;
+import security.AuthzCredentialGenerator;
+import security.CredentialGenerator;
+import security.AuthzCredentialGenerator.ClassCode;
 
 import com.gemstone.gemfire.cache.DynamicRegionFactory;
 import com.gemstone.gemfire.cache.InterestResultPolicy;
@@ -62,8 +62,8 @@ import com.gemstone.gemfire.internal.util.Callable;
 
 import dunit.DistributedTestCase;
 import dunit.VM;
-import templates.security.DummyCredentialGenerator;
-import templates.security.XmlAuthzCredentialGenerator;
+import security.DummyCredentialGenerator;
+import security.XmlAuthzCredentialGenerator;
 
 /**
  * Base class for tests for authorization from client to server. It contains

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
index ceeb989..627c415 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
@@ -24,8 +24,8 @@ package com.gemstone.gemfire.security;
 import java.util.Iterator;
 import java.util.Properties;
 
-import templates.security.AuthzCredentialGenerator;
-import templates.security.CredentialGenerator;
+import security.AuthzCredentialGenerator;
+import security.CredentialGenerator;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.execute.Function;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
index 16573c2..37b46cc 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
@@ -23,8 +23,8 @@ package com.gemstone.gemfire.security;
 
 import java.util.Properties;
 
-import templates.security.AuthzCredentialGenerator;
-import templates.security.CredentialGenerator;
+import security.AuthzCredentialGenerator;
+import security.CredentialGenerator;
 
 import com.gemstone.gemfire.DeltaTestImpl;
 import com.gemstone.gemfire.cache.Region;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
index 7cfa23d..0a0a0b8 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
@@ -28,8 +28,8 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 
-import templates.security.AuthzCredentialGenerator;
-import templates.security.CredentialGenerator;
+import security.AuthzCredentialGenerator;
+import security.CredentialGenerator;
 
 import com.gemstone.gemfire.DeltaTestImpl;
 import com.gemstone.gemfire.cache.InterestResultPolicy;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
index 0f83050..e4ce211 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
@@ -26,9 +26,9 @@ import java.util.Properties;
 
 import javax.net.ssl.SSLHandshakeException;
 
-import templates.security.CredentialGenerator;
-import templates.security.DummyCredentialGenerator;
-import templates.security.LdapUserCredentialGenerator;
+import security.CredentialGenerator;
+import security.DummyCredentialGenerator;
+import security.LdapUserCredentialGenerator;
 
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.distributed.DistributedSystem;
@@ -507,7 +507,7 @@ public class P2PAuthenticationDUnitTest extends DistributedTestCase {
     gen.init();
     Properties extraProps = gen.getSystemProperties();
     String authenticator = gen.getAuthenticator();
-    String authInit = "templates.security.UserPasswordWithExtraPropsAuthInit.create";
+    String authInit = "security.UserPasswordWithExtraPropsAuthInit.create";
     if (extraProps == null) {
       extraProps = new Properties();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/security/AuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/security/AuthzCredentialGenerator.java b/gemfire-core/src/test/java/security/AuthzCredentialGenerator.java
new file mode 100644
index 0000000..d5e4915
--- /dev/null
+++ b/gemfire-core/src/test/java/security/AuthzCredentialGenerator.java
@@ -0,0 +1,465 @@
+
+package security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.security.AccessControl;
+
+/**
+ * Encapsulates obtaining authorized and unauthorized credentials for a given
+ * operation in a region. Implementations will be for different kinds of
+ * authorization scheme and authentication scheme combos.
+ * 
+ * @author sumedh
+ * @since 5.5
+ */
+public abstract class AuthzCredentialGenerator {
+
+  /**
+   * Enumeration for various {@link AuthzCredentialGenerator} implementations.
+   * 
+   * The following schemes are supported as of now:
+   * <ul>
+   * <li><code>DummyAuthorization</code> with <code>DummyAuthenticator</code></li>
+   * <li><code>XMLAuthorization</code> with <code>DummyAuthenticator</code></li>
+   * <li><code>XMLAuthorization</code> with <code>LDAPAuthenticator</code></li>
+   * <li><code>XMLAuthorization</code> with <code>PKCSAuthenticator</code></li>
+   * <li><code>XMLAuthorization</code> when using SSL sockets</li>
+   * </ul>
+   * 
+   * To add a new authorization scheme the following needs to be done:
+   * <ul>
+   * <li>Add implementation for {@link AccessControl}.</li>
+   * <li>Choose the authentication schemes that it shall work with from
+   * {@link CredentialGenerator.ClassCode}</li>
+   * <li>Add a new enumeration value for the scheme in this class. Notice the
+   * size of <code>VALUES</code> array and increase that if it is getting
+   * overflowed. Note the methods and fields for existing schemes and add for
+   * the new one in a similar manner.</li>
+   * <li>Add an implementation for {@link AuthzCredentialGenerator}. Note the
+   * {@link AuthzCredentialGenerator#init} method where different authentication
+   * schemes can be passed and initialize differently for the authentication
+   * schemes that shall be handled.</li>
+   * <li>Modify the {@link AuthzCredentialGenerator#create} method to add
+   * creation of an instance of the new implementation for the
+   * <code>ClassCode</code> enumeration value.</li>
+   * </ul>
+   * All dunit tests will automagically start testing the new implementation
+   * after this.
+   * 
+   * @author sumedh
+   * @since 5.5
+   */
+  public static final class ClassCode {
+
+    private static final byte ID_DUMMY = 1;
+
+    private static final byte ID_XML = 2;
+
+    private static byte nextOrdinal = 0;
+
+    private static final ClassCode[] VALUES = new ClassCode[10];
+
+    private static final Map CodeNameMap = new HashMap();
+
+    public static final ClassCode DUMMY = new ClassCode(
+        "templates.security.DummyAuthorization.create", ID_DUMMY);
+
+    public static final ClassCode XML = new ClassCode(
+        "templates.security.XmlAuthorization.create", ID_XML);
+
+    /** The name of this class. */
+    private final String name;
+
+    /** byte used as ordinal to represent this class */
+    private final byte ordinal;
+
+    /**
+     * One of the following: ID_DUMMY, ID_LDAP, ID_PKI
+     */
+    private final byte classType;
+
+    /** Creates a new instance of class code. */
+    private ClassCode(String name, byte classType) {
+      this.name = name;
+      this.classType = classType;
+      this.ordinal = nextOrdinal++;
+      VALUES[this.ordinal] = this;
+      CodeNameMap.put(name, this);
+    }
+
+    public boolean isDummy() {
+      return (this.classType == ID_DUMMY);
+    }
+
+    public boolean isXml() {
+      return (this.classType == ID_XML);
+    }
+
+    /**
+     * Returns the <code>ClassCode</code> represented by specified ordinal.
+     */
+    public static ClassCode fromOrdinal(byte ordinal) {
+      return VALUES[ordinal];
+    }
+
+    /**
+     * Returns the <code>ClassCode</code> represented by specified string.
+     */
+    public static ClassCode parse(String operationName) {
+      return (ClassCode)CodeNameMap.get(operationName);
+    }
+
+    /**
+     * Returns all the possible values.
+     */
+    public static List getAll() {
+      List codes = new ArrayList();
+      Iterator iter = CodeNameMap.values().iterator();
+      while (iter.hasNext()) {
+        codes.add(iter.next());
+      }
+      return codes;
+    }
+
+    /**
+     * Returns the ordinal for this class code.
+     * 
+     * @return the ordinal of this class code.
+     */
+    public byte toOrdinal() {
+      return this.ordinal;
+    }
+
+    /**
+     * Returns a string representation for this class code.
+     * 
+     * @return the name of this class code.
+     */
+    final public String toString() {
+      return this.name;
+    }
+
+    /**
+     * Indicates whether other object is same as this one.
+     * 
+     * @return true if other object is same as this one.
+     */
+    @Override
+    final public boolean equals(final Object obj) {
+      if (obj == this) {
+        return true;
+      }
+      if (!(obj instanceof ClassCode)) {
+        return false;
+      }
+      final ClassCode other = (ClassCode)obj;
+      return (other.ordinal == this.ordinal);
+    }
+
+    /**
+     * Indicates whether other <code>ClassCode</code> is same as this one.
+     * 
+     * @return true if other <code>ClassCode</code> is same as this one.
+     */
+    final public boolean equals(final ClassCode opCode) {
+      return (opCode != null && opCode.ordinal == this.ordinal);
+    }
+
+    /**
+     * Returns a hash code value for this <code>ClassCode</code> which is the
+     * same as its ordinal.
+     * 
+     * @return the ordinal of this <code>ClassCode</code>.
+     */
+    @Override
+    final public int hashCode() {
+      return this.ordinal;
+    }
+
+  }
+
+  /**
+   * The {@link CredentialGenerator} being used.
+   */
+  protected CredentialGenerator cGen;
+
+  /**
+   * A set of system properties that should be added to the gemfire system
+   * properties before using the authorization module.
+   */
+  private Properties sysProps;
+
+  /**
+   * A factory method to create a new instance of an
+   * {@link AuthzCredentialGenerator} for the given {@link ClassCode}. Caller
+   * is supposed to invoke {@link AuthzCredentialGenerator#init} immediately
+   * after obtaining the instance.
+   * 
+   * @param classCode
+   *                the <code>ClassCode</code> of the
+   *                <code>AuthzCredentialGenerator</code> implementation
+   * 
+   * @return an instance of <code>AuthzCredentialGenerator</code> for the
+   *         given class code
+   */
+  public static AuthzCredentialGenerator create(ClassCode classCode) {
+    switch (classCode.classType) {
+      case ClassCode.ID_DUMMY:
+        return new DummyAuthzCredentialGenerator();
+      case ClassCode.ID_XML:
+        return new XmlAuthzCredentialGenerator();
+      default:
+        return null;
+    }
+  }
+
+  /**
+   * Initialize the authorized credential generator.
+   * 
+   * @param cGen
+   *                an instance of {@link CredentialGenerator} of the credential
+   *                implementation for which to obtain authorized/unauthorized
+   *                credentials.
+   * 
+   * @return false when the given {@link CredentialGenerator} is incompatible
+   *         with this authorization module.
+   */
+  public boolean init(CredentialGenerator cGen) {
+    this.cGen = cGen;
+    try {
+      this.sysProps = init();
+    }
+    catch (IllegalArgumentException ex) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * 
+   * @return A set of extra properties that should be added to Gemfire system
+   *         properties when not null.
+   */
+  public Properties getSystemProperties() {
+    return this.sysProps;
+  }
+
+  /**
+   * Get the {@link CredentialGenerator} being used by this instance.
+   */
+  public CredentialGenerator getCredentialGenerator() {
+    return this.cGen;
+  }
+
+  /**
+   * The {@link ClassCode} of the particular implementation.
+   * 
+   * @return the <code>ClassCode</code>
+   */
+  public abstract ClassCode classCode();
+
+  /**
+   * The name of the {@link AccessControl} factory function that should be used
+   * as the authorization module on the server side.
+   * 
+   * @return name of the <code>AccessControl</code> factory function
+   */
+  public abstract String getAuthorizationCallback();
+
+  /**
+   * Get a set of credentials generated using the given index allowed to perform
+   * the given {@link OperationCode}s for the given regions.
+   * 
+   * @param opCodes
+   *                the list of {@link OperationCode}s of the operations
+   *                requiring authorization; should not be null
+   * @param regionNames
+   *                list of the region names requiring authorization; a value of
+   *                null indicates all regions
+   * @param index
+   *                used to generate multiple such credentials by passing
+   *                different values for this
+   * 
+   * @return the set of credentials authorized to perform the given operation in
+   *         the given regions
+   */
+  public Properties getAllowedCredentials(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    int numTries = getNumPrincipalTries(opCodes, regionNames);
+    if (numTries <= 0) {
+      numTries = 1;
+    }
+    for (int tries = 0; tries < numTries; tries++) {
+      Principal principal = getAllowedPrincipal(opCodes, regionNames,
+          (index + tries) % numTries);
+      try {
+        return this.cGen.getValidCredentials(principal);
+      }
+      catch (IllegalArgumentException ex) {
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Get a set of credentials generated using the given index not allowed to
+   * perform the given {@link OperationCode}s for the given regions. The
+   * credentials are required to be valid for authentication.
+   * 
+   * @param opCode
+   *                the {@link OperationCode}s of the operations requiring
+   *                authorization failure; should not be null
+   * @param regionNames
+   *                list of the region names requiring authorization failure; a
+   *                value of null indicates all regions
+   * @param index
+   *                used to generate multiple such credentials by passing
+   *                different values for this
+   * 
+   * @return the set of credentials that are not authorized to perform the given
+   *         operation in the given region
+   */
+  public Properties getDisallowedCredentials(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    // This may not be very correct since we use the value of
+    // getNumPrincipalTries() but is used to avoid adding another method.
+    // Also something like getNumDisallowedPrincipals() will be normally always
+    // infinite, and the number here is just to perform some number of tries
+    // before giving up.
+    int numTries = getNumPrincipalTries(opCodes, regionNames);
+    if (numTries <= 0) {
+      numTries = 1;
+    }
+    for (int tries = 0; tries < numTries; tries++) {
+      Principal principal = getDisallowedPrincipal(opCodes, regionNames,
+          (index + tries) % numTries);
+      try {
+        return this.cGen.getValidCredentials(principal);
+      }
+      catch (IllegalArgumentException ex) {
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Initialize the authorized credential generator.
+   * 
+   * Required to be implemented by concrete classes that implement this abstract
+   * class.
+   * 
+   * @return A set of extra properties that should be added to Gemfire system
+   *         properties when not null.
+   * 
+   * @throws IllegalArgumentException
+   *                 when the {@link CredentialGenerator} is incompatible with
+   *                 this authorization module.
+   */
+  protected abstract Properties init() throws IllegalArgumentException;
+
+  /**
+   * Get the number of tries to be done for obtaining valid credentials for the
+   * given operations in the given region. It is required that
+   * {@link #getAllowedPrincipal} method returns valid principals for values of
+   * <code>index</code> from 0 through (n-1) where <code>n</code> is the
+   * value returned by this method. It is recommended that the principals so
+   * returned be unique for efficiency.
+   * 
+   * This will be used by {@link #getAllowedCredentials} to step through
+   * different principals and obtain a set of valid credentials.
+   * 
+   * Required to be implemented by concrete classes that implement this abstract
+   * class.
+   * 
+   * @param opCodes
+   *                the {@link OperationCode}s of the operations requiring
+   *                authorization
+   * @param regionNames
+   *                list of the region names requiring authorization; a value of
+   *                null indicates all regions
+   * @param index
+   *                used to generate multiple such credentials by passing
+   *                different values for this
+   * 
+   * @return the number of principals allowed to perform the given operation in
+   *         the given region
+   */
+  protected abstract int getNumPrincipalTries(OperationCode[] opCodes,
+      String[] regionNames);
+
+  /**
+   * Get a {@link Principal} generated using the given index allowed to perform
+   * the given {@link OperationCode}s for the given region.
+   * 
+   * Required to be implemented by concrete classes that implement this abstract
+   * class.
+   * 
+   * @param opCodes
+   *                the {@link OperationCode}s of the operations requiring
+   *                authorization
+   * @param regionNames
+   *                list of the region names requiring authorization; a value of
+   *                null indicates all regions
+   * @param index
+   *                used to generate multiple such principals by passing
+   *                different values for this
+   * 
+   * @return the {@link Principal} authorized to perform the given operation in
+   *         the given region
+   */
+  protected abstract Principal getAllowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index);
+
+  /**
+   * Get a {@link Principal} generated using the given index not allowed to
+   * perform the given {@link OperationCode}s for the given region.
+   * 
+   * Required to be implemented by concrete classes that implement this abstract
+   * class.
+   * 
+   * @param opCodes
+   *                the {@link OperationCode}s of the operations requiring
+   *                authorization failure
+   * @param regionNames
+   *                list of the region names requiring authorization failure; a
+   *                value of null indicates all regions
+   * @param index
+   *                used to generate multiple such principals by passing
+   *                different values for this
+   * 
+   * @return a {@link Principal} not authorized to perform the given operation
+   *         in the given region
+   */
+  protected abstract Principal getDisallowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index);
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/security/CredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/security/CredentialGenerator.java b/gemfire-core/src/test/java/security/CredentialGenerator.java
new file mode 100644
index 0000000..c16a8a1
--- /dev/null
+++ b/gemfire-core/src/test/java/security/CredentialGenerator.java
@@ -0,0 +1,343 @@
+
+package security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import com.gemstone.gemfire.security.AuthInitialize;
+import com.gemstone.gemfire.security.Authenticator;
+
+/**
+ * Encapsulates obtaining valid and invalid credentials. Implementations will be
+ * for different kinds of authentication schemes.
+ * 
+ * @author sumedh
+ * @since 5.5
+ */
+public abstract class CredentialGenerator {
+
+  /**
+   * Enumeration for various {@link CredentialGenerator} implementations.
+   * 
+   * The following schemes are supported as of now:
+   * <code>DummyAuthenticator</code>, <code>LdapUserAuthenticator</code>,
+   * <code>PKCSAuthenticator</code>. In addition SSL socket mode with mutual
+   * authentication is also supported.
+   * 
+   * To add a new authentication scheme the following needs to be done:
+   * <ul>
+   * <li>Add implementations for {@link AuthInitialize} and
+   * {@link Authenticator} classes for clients/peers.</li>
+   * <li>Add a new enumeration value for the scheme in this class. Notice the
+   * size of <code>VALUES</code> array and increase that if it is getting
+   * overflowed. Note the methods and fields for existing schemes and add for
+   * the new one in a similar manner.</li>
+   * <li>Add an implementation for {@link CredentialGenerator}.</li>
+   * <li>Modify the {@link CredentialGenerator.Factory#create} method to add
+   * creation of an instance of the new implementation for the
+   * <code>ClassCode</code> enumeration value.</li>
+   * </ul>
+   * All security dunit tests will automagically start testing the new
+   * implementation after this.
+   * 
+   * @author sumedh
+   * @since 5.5
+   */
+  public static final class ClassCode {
+
+    private static final byte ID_DUMMY = 1;
+
+    private static final byte ID_LDAP = 2;
+
+    private static final byte ID_PKCS = 3;
+
+    private static final byte ID_SSL = 4;
+
+    private static byte nextOrdinal = 0;
+
+    private static final ClassCode[] VALUES = new ClassCode[10];
+
+    private static final Map CodeNameMap = new HashMap();
+
+    public static final ClassCode DUMMY = new ClassCode(
+        "templates.security.DummyAuthenticator.create", ID_DUMMY);
+
+    public static final ClassCode LDAP = new ClassCode(
+        "templates.security.LdapUserAuthenticator.create", ID_LDAP);
+
+    public static final ClassCode PKCS = new ClassCode(
+        "templates.security.PKCSAuthenticator.create", ID_PKCS);
+
+    public static final ClassCode SSL = new ClassCode("SSL", ID_SSL);
+
+    /** The name of this class. */
+    private final String name;
+
+    /** byte used as ordinal to represent this class */
+    private final byte ordinal;
+
+    /**
+     * One of the following: ID_DUMMY, ID_LDAP, ID_PKCS
+     */
+    private final byte classType;
+
+    /** Creates a new instance of class code. */
+    private ClassCode(String name, byte classType) {
+      this.name = name;
+      this.classType = classType;
+      this.ordinal = nextOrdinal++;
+      VALUES[this.ordinal] = this;
+      CodeNameMap.put(name, this);
+    }
+
+    public boolean isDummy() {
+      return (this.classType == ID_DUMMY);
+    }
+
+    public boolean isLDAP() {
+      return (this.classType == ID_LDAP);
+    }
+
+    public boolean isPKCS() {
+      return (this.classType == ID_PKCS);
+    }
+
+    public boolean isSSL() {
+      return (this.classType == ID_SSL);
+    }
+
+    /**
+     * Returns the <code>ClassCode</code> represented by specified ordinal.
+     */
+    public static ClassCode fromOrdinal(byte ordinal) {
+      return VALUES[ordinal];
+    }
+
+    /**
+     * Returns the <code>ClassCode</code> represented by specified string.
+     */
+    public static ClassCode parse(String operationName) {
+      return (ClassCode)CodeNameMap.get(operationName);
+    }
+
+    /**
+     * Returns all the possible values.
+     */
+    public static List getAll() {
+      List codes = new ArrayList();
+      Iterator iter = CodeNameMap.values().iterator();
+      while (iter.hasNext()) {
+        codes.add(iter.next());
+      }
+      return codes;
+    }
+
+    /**
+     * Returns the ordinal for this operation code.
+     * 
+     * @return the ordinal of this operation.
+     */
+    public byte toOrdinal() {
+      return this.ordinal;
+    }
+
+    /**
+     * Returns a string representation for this operation.
+     * 
+     * @return the name of this operation.
+     */
+    final public String toString() {
+      return this.name;
+    }
+
+    /**
+     * Indicates whether other object is same as this one.
+     * 
+     * @return true if other object is same as this one.
+     */
+    @Override
+    final public boolean equals(final Object obj) {
+      if (obj == this) {
+        return true;
+      }
+      if (!(obj instanceof ClassCode)) {
+        return false;
+      }
+      final ClassCode other = (ClassCode)obj;
+      return (other.ordinal == this.ordinal);
+    }
+
+    /**
+     * Indicates whether other <code>ClassCode</code> is same as this one.
+     * 
+     * @return true if other <code>ClassCode</code> is same as this one.
+     */
+    final public boolean equals(final ClassCode opCode) {
+      return (opCode != null && opCode.ordinal == this.ordinal);
+    }
+
+    /**
+     * Returns a hash code value for this <code>ClassCode</code> which is the
+     * same as its ordinal.
+     * 
+     * @return the ordinal of this operation.
+     */
+    @Override
+    final public int hashCode() {
+      return this.ordinal;
+    }
+
+  }
+
+  /**
+   * A set of properties that should be added to the Gemfire system properties
+   * before using the authentication module.
+   */
+  private Properties sysProps = null;
+
+  /**
+   * A set of properties that should be added to the java system properties
+   * before using the authentication module.
+   */
+  protected Properties javaProps = null;
+
+  /**
+   * A factory method to create a new instance of an {@link CredentialGenerator}
+   * for the given {@link ClassCode}. Caller is supposed to invoke
+   * {@link CredentialGenerator#init} immediately after obtaining the instance.
+   * 
+   * @param classCode
+   *                the <code>ClassCode</code> of the
+   *                <code>CredentialGenerator</code> implementation
+   * 
+   * @return an instance of <code>CredentialGenerator</code> for the given
+   *         class code
+   */
+  public static CredentialGenerator create(ClassCode classCode) {
+    switch (classCode.classType) {
+      // Removing dummy one to reduce test run times
+      // case ClassCode.ID_DUMMY:
+      // return new DummyCredentialGenerator();
+      case ClassCode.ID_LDAP:
+        return new LdapUserCredentialGenerator();
+        // case ClassCode.ID_SSL:ø
+        // return new SSLCredentialGenerator();
+      case ClassCode.ID_PKCS:
+        return new PKCSCredentialGenerator();
+      default:
+        return null;
+    }
+  }
+
+  /**
+   * Initialize the credential generator.
+   * 
+   * @throws IllegalArgumentException
+   *                 when there is a problem during initialization
+   */
+  public void init() throws IllegalArgumentException {
+    this.sysProps = initialize();
+  }
+
+  /**
+   * Initialize the credential generator. This is provided separately from the
+   * {@link #init} method for convenience of implementations so that they do not
+   * need to store in {@link #sysProps}. The latter is convenient for the users
+   * who do not need to store these properties rather can obtain it later by
+   * invoking {@link #getSystemProperties}
+   * 
+   * Required to be implemented by concrete classes that implement this abstract
+   * class.
+   * 
+   * @return A set of extra properties that should be added to Gemfire system
+   *         properties when not null.
+   * 
+   * @throws IllegalArgumentException
+   *                 when there is a problem during initialization
+   */
+  protected abstract Properties initialize() throws IllegalArgumentException;
+
+  /**
+   * 
+   * @return A set of extra properties that should be added to Gemfire system
+   *         properties when not null.
+   */
+  public Properties getSystemProperties() {
+    return this.sysProps;
+  }
+
+  /**
+   * 
+   * @return A set of extra properties that should be added to Gemfire system
+   *         properties when not null.
+   */
+  public Properties getJavaProperties() {
+    return this.javaProps;
+  }
+
+  /**
+   * The {@link ClassCode} of this particular implementation.
+   * 
+   * @return the <code>ClassCode</code>
+   */
+  public abstract ClassCode classCode();
+
+  /**
+   * The name of the {@link AuthInitialize} factory function that should be used
+   * in conjunction with the credentials generated by this generator.
+   * 
+   * @return name of the <code>AuthInitialize</code> factory function
+   */
+  public abstract String getAuthInit();
+
+  /**
+   * The name of the {@link Authenticator} factory function that should be used
+   * in conjunction with the credentials generated by this generator.
+   * 
+   * @return name of the <code>Authenticator</code> factory function
+   */
+  public abstract String getAuthenticator();
+
+  /**
+   * Get a set of valid credentials generated using the given index.
+   */
+  public abstract Properties getValidCredentials(int index);
+
+  /**
+   * Get a set of valid credentials for the given {@link Principal}.
+   * 
+   * @return credentials for the given <code>Principal</code> or null if none
+   *         possible.
+   */
+  public abstract Properties getValidCredentials(Principal principal);
+
+  /**
+   * Get a set of invalid credentials generated using the given index.
+   */
+  public abstract Properties getInvalidCredentials(int index);
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/security/DummyAuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/security/DummyAuthzCredentialGenerator.java b/gemfire-core/src/test/java/security/DummyAuthzCredentialGenerator.java
new file mode 100644
index 0000000..7e40d13
--- /dev/null
+++ b/gemfire-core/src/test/java/security/DummyAuthzCredentialGenerator.java
@@ -0,0 +1,145 @@
+
+package security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import security.AuthzCredentialGenerator;
+import templates.security.DummyAuthorization;
+import templates.security.UsernamePrincipal;
+
+public class DummyAuthzCredentialGenerator extends AuthzCredentialGenerator {
+
+  public static final byte READER_ROLE = 1;
+
+  public static final byte WRITER_ROLE = 2;
+
+  public static final byte ADMIN_ROLE = 3;
+
+  private static Set readerOpsSet;
+
+  private static Set writerOpsSet;
+
+  static {
+
+    readerOpsSet = new HashSet();
+    for (int index = 0; index < DummyAuthorization.READER_OPS.length; index++) {
+      readerOpsSet.add(DummyAuthorization.READER_OPS[index]);
+    }
+    writerOpsSet = new HashSet();
+    for (int index = 0; index < DummyAuthorization.WRITER_OPS.length; index++) {
+      writerOpsSet.add(DummyAuthorization.WRITER_OPS[index]);
+    }
+  }
+
+  public DummyAuthzCredentialGenerator() {
+  }
+
+  protected Properties init() throws IllegalArgumentException {
+
+    if (!this.cGen.classCode().isDummy()) {
+      throw new IllegalArgumentException(
+          "DummyAuthorization module only works with DummyAuthenticator");
+    }
+    return null;
+  }
+
+  public ClassCode classCode() {
+    return ClassCode.DUMMY;
+  }
+
+  public String getAuthorizationCallback() {
+    return "templates.security.DummyAuthorization.create";
+  }
+
+  public static byte getRequiredRole(OperationCode[] opCodes) {
+
+    byte roleType = ADMIN_ROLE;
+    boolean requiresReader = true;
+    boolean requiresWriter = true;
+
+    for (int opNum = 0; opNum < opCodes.length; opNum++) {
+      if (requiresReader && !readerOpsSet.contains(opCodes[opNum])) {
+        requiresReader = false;
+      }
+      if (requiresWriter && !writerOpsSet.contains(opCodes[opNum])) {
+        requiresWriter = false;
+      }
+    }
+    if (requiresReader) {
+      roleType = READER_ROLE;
+    }
+    else if (requiresWriter) {
+      roleType = WRITER_ROLE;
+    }
+    return roleType;
+  }
+
+  private Principal getPrincipal(byte roleType, int index) {
+
+    String[] admins = new String[] { "root", "admin", "administrator" };
+    switch (roleType) {
+      case READER_ROLE:
+        return new UsernamePrincipal("reader" + index);
+      case WRITER_ROLE:
+        return new UsernamePrincipal("writer" + index);
+      default:
+        return new UsernamePrincipal(admins[index % admins.length]);
+    }
+  }
+
+  protected Principal getAllowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    byte roleType = getRequiredRole(opCodes);
+    return getPrincipal(roleType, index);
+  }
+
+  protected Principal getDisallowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    byte roleType = getRequiredRole(opCodes);
+    byte disallowedRoleType;
+    switch (roleType) {
+      case READER_ROLE:
+        disallowedRoleType = WRITER_ROLE;
+        break;
+      case WRITER_ROLE:
+        disallowedRoleType = READER_ROLE;
+        break;
+      default:
+        disallowedRoleType = READER_ROLE;
+        break;
+    }
+    return getPrincipal(disallowedRoleType, index);
+  }
+
+  protected int getNumPrincipalTries(OperationCode[] opCodes,
+      String[] regionNames) {
+    return 5;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/security/DummyCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/security/DummyCredentialGenerator.java b/gemfire-core/src/test/java/security/DummyCredentialGenerator.java
new file mode 100644
index 0000000..86b26a7
--- /dev/null
+++ b/gemfire-core/src/test/java/security/DummyCredentialGenerator.java
@@ -0,0 +1,94 @@
+
+package security;
+
+/*
+ * 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.
+ */
+
+
+import security.CredentialGenerator;
+import templates.security.DummyAuthenticator;
+import templates.security.UserPasswordAuthInit;
+
+import java.security.Principal;
+import java.util.Properties;
+
+public class DummyCredentialGenerator extends CredentialGenerator {
+
+  public DummyCredentialGenerator() {
+  }
+
+  protected Properties initialize() throws IllegalArgumentException {
+    return null;
+  }
+
+  public ClassCode classCode() {
+    return ClassCode.DUMMY;
+  }
+
+  public String getAuthInit() {
+    return "templates.security.UserPasswordAuthInit.create";
+  }
+
+  public String getAuthenticator() {
+    return "templates.security.DummyAuthenticator.create";
+  }
+
+  public Properties getValidCredentials(int index) {
+
+    String[] validGroups = new String[] { "admin", "user", "reader", "writer" };
+    String[] admins = new String[] { "root", "admin", "administrator" };
+
+    Properties props = new Properties();
+    int groupNum = (index % validGroups.length);
+    String userName;
+    if (groupNum == 0) {
+      userName = admins[index % admins.length];
+    }
+    else {
+      userName = validGroups[groupNum] + (index / validGroups.length);
+    }
+    props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
+    props.setProperty(UserPasswordAuthInit.PASSWORD, userName);
+    return props;
+  }
+
+  public Properties getValidCredentials(Principal principal) {
+
+    String userName = principal.getName();
+    if (DummyAuthenticator.testValidName(userName)) {
+      Properties props = new Properties();
+      props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
+      props.setProperty(UserPasswordAuthInit.PASSWORD, userName);
+      return props;
+    }
+    else {
+      throw new IllegalArgumentException("Dummy: [" + userName
+          + "] is not a valid user");
+    }
+  }
+
+  public Properties getInvalidCredentials(int index) {
+
+    Properties props = new Properties();
+    props.setProperty(UserPasswordAuthInit.USER_NAME, "invalid" + index);
+    props.setProperty(UserPasswordAuthInit.PASSWORD, "none");
+    return props;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/security/LdapUserCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/security/LdapUserCredentialGenerator.java b/gemfire-core/src/test/java/security/LdapUserCredentialGenerator.java
new file mode 100644
index 0000000..12bcb62
--- /dev/null
+++ b/gemfire-core/src/test/java/security/LdapUserCredentialGenerator.java
@@ -0,0 +1,160 @@
+
+package security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.Properties;
+
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
+import com.gemstone.gemfire.util.test.TestUtil;
+import templates.security.LdapUserAuthenticator;
+import templates.security.UserPasswordAuthInit;
+
+import java.util.Random;
+
+public class LdapUserCredentialGenerator extends CredentialGenerator {
+
+  private static final String USER_PREFIX = "gemfire";
+
+  private static boolean enableServerAuthentication = false;
+
+  private boolean serverAuthEnabled = false;
+
+  private static final Random prng = new Random();
+
+  private static final String[] algos = new String[] { "", "DESede", "AES:128",
+      "Blowfish:128" };
+
+  public LdapUserCredentialGenerator() {
+    // Toggle server authentication enabled for each test
+    // This is done instead of running all the tests with both
+    // server auth enabled/disabled to reduce test run time.
+    enableServerAuthentication = !enableServerAuthentication;
+    serverAuthEnabled = enableServerAuthentication;
+  }
+
+  @Override
+  protected Properties initialize() throws IllegalArgumentException {
+
+    Properties extraProps = new Properties();
+    String ldapServer = System.getProperty("gf.ldap.server", "ldap");
+    String ldapBaseDN = System.getProperty("gf.ldap.basedn", "ou=ldapTesting,dc=pune,dc=gemstone,dc=com");
+    String ldapUseSSL = System.getProperty("gf.ldap.usessl");
+    extraProps.setProperty(LdapUserAuthenticator.LDAP_SERVER_NAME, ldapServer);
+    extraProps.setProperty(LdapUserAuthenticator.LDAP_BASEDN_NAME, ldapBaseDN);
+    if (ldapUseSSL != null && ldapUseSSL.length() > 0) {
+      extraProps.setProperty(LdapUserAuthenticator.LDAP_SSL_NAME, ldapUseSSL);
+    }
+    if (serverAuthEnabled) {
+      String keyStoreFile = TestUtil.getResourcePath(LdapUserCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/gemfire1.keystore");
+      extraProps.setProperty(HandShake.PRIVATE_KEY_FILE_PROP, keyStoreFile);
+      extraProps.setProperty(HandShake.PRIVATE_KEY_ALIAS_PROP, "gemfire1");
+      extraProps.setProperty(HandShake.PRIVATE_KEY_PASSWD_PROP, "gemfire");
+    }
+    return extraProps;
+  }
+
+  @Override
+  public ClassCode classCode() {
+    return ClassCode.LDAP;
+  }
+
+  @Override
+  public String getAuthInit() {
+    return "templates.security.UserPasswordAuthInit.create";
+  }
+
+  @Override
+  public String getAuthenticator() {
+    return "templates.security.LdapUserAuthenticator.create";
+  }
+
+  @Override
+  public Properties getValidCredentials(int index) {
+
+    Properties props = new Properties();
+    props.setProperty(UserPasswordAuthInit.USER_NAME, USER_PREFIX
+        + ((index % 10) + 1));
+    props.setProperty(UserPasswordAuthInit.PASSWORD, USER_PREFIX
+        + ((index % 10) + 1));
+    props.setProperty(DistributionConfig.SECURITY_CLIENT_DHALGO_NAME,
+        algos[prng.nextInt(algos.length)]);
+    if (serverAuthEnabled) {
+      String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/publickeyfile");
+      props.setProperty(HandShake.PUBLIC_KEY_FILE_PROP, keyStoreFile);
+      props.setProperty(HandShake.PUBLIC_KEY_PASSWD_PROP, "gemfire");
+    }
+    return props;
+  }
+
+  @Override
+  public Properties getValidCredentials(Principal principal) {
+
+    Properties props = null;
+    String userName = principal.getName();
+    if (userName != null && userName.startsWith(USER_PREFIX)) {
+      boolean isValid;
+      try {
+        int suffix = Integer.parseInt(userName.substring(USER_PREFIX.length()));
+        isValid = (suffix >= 1 && suffix <= 10);
+      }
+      catch (Exception ex) {
+        isValid = false;
+      }
+      if (isValid) {
+        props = new Properties();
+        props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
+        props.setProperty(UserPasswordAuthInit.PASSWORD, userName);
+      }
+    }
+    if (props == null) {
+      throw new IllegalArgumentException("LDAP: [" + userName
+          + "] not a valid user");
+    }
+    props.setProperty(DistributionConfig.SECURITY_CLIENT_DHALGO_NAME,
+        algos[prng.nextInt(algos.length)]);
+    if (serverAuthEnabled) {
+      String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/publickeyfile");
+      props.setProperty(HandShake.PUBLIC_KEY_FILE_PROP, keyStoreFile);
+      props.setProperty(HandShake.PUBLIC_KEY_PASSWD_PROP, "gemfire");
+    }
+    return props;
+  }
+
+  @Override
+  public Properties getInvalidCredentials(int index) {
+
+    Properties props = new Properties();
+    props.setProperty(UserPasswordAuthInit.USER_NAME, "invalid" + index);
+    props.setProperty(UserPasswordAuthInit.PASSWORD, "none");
+    props.setProperty(DistributionConfig.SECURITY_CLIENT_DHALGO_NAME,
+        algos[prng.nextInt(algos.length)]);
+    if (serverAuthEnabled) {
+      String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/publickeyfile");
+      props.setProperty(HandShake.PUBLIC_KEY_FILE_PROP, keyStoreFile);
+      props.setProperty(HandShake.PUBLIC_KEY_PASSWD_PROP, "gemfire");
+    }
+    return props;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/security/PKCSCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/security/PKCSCredentialGenerator.java b/gemfire-core/src/test/java/security/PKCSCredentialGenerator.java
new file mode 100644
index 0000000..24c0100
--- /dev/null
+++ b/gemfire-core/src/test/java/security/PKCSCredentialGenerator.java
@@ -0,0 +1,112 @@
+
+package security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.security.Provider;
+import java.security.Security;
+import java.util.Properties;
+
+import com.gemstone.gemfire.util.test.TestUtil;
+import templates.security.PKCSAuthInit;
+import templates.security.PKCSAuthenticator;
+
+/**
+ * @author kneeraj
+ * 
+ */
+public class PKCSCredentialGenerator extends CredentialGenerator {
+
+  public static String keyStoreDir = getKeyStoreDir();
+
+  public static boolean usesIBMJSSE;
+
+  // Checks if the current JVM uses only IBM JSSE providers.
+  private static boolean usesIBMProviders() {
+    Provider[] providers = Security.getProviders();
+    for (int index = 0; index < providers.length; ++index) {
+      if (!providers[index].getName().toLowerCase().startsWith("ibm")) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static String getKeyStoreDir() {
+    usesIBMJSSE = usesIBMProviders();
+    if (usesIBMJSSE) {
+      return "/lib/keys/ibm";
+    }
+    else {
+      return "/lib/keys";
+    }
+  }
+
+  public ClassCode classCode() {
+    return ClassCode.PKCS;
+  }
+
+  public String getAuthInit() {
+    return "templates.security.PKCSAuthInit.create";
+  }
+
+  public String getAuthenticator() {
+    return "templates.security.PKCSAuthenticator.create";
+  }
+
+  public Properties getInvalidCredentials(int index) {
+    Properties props = new Properties();
+    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + "/gemfire11.keystore");
+    props.setProperty(PKCSAuthInit.KEYSTORE_FILE_PATH, keyStoreFile);
+    props.setProperty(PKCSAuthInit.KEYSTORE_ALIAS, "gemfire11");
+    props.setProperty(PKCSAuthInit.KEYSTORE_PASSWORD, "gemfire");
+    return props;
+  }
+
+  public Properties getValidCredentials(int index) {
+    Properties props = new Properties();
+    int aliasnum = (index % 10) + 1;
+    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + "/gemfire" + aliasnum + ".keystore");
+    props.setProperty(PKCSAuthInit.KEYSTORE_FILE_PATH, keyStoreFile);
+    props.setProperty(PKCSAuthInit.KEYSTORE_ALIAS, "gemfire" + aliasnum);
+    props.setProperty(PKCSAuthInit.KEYSTORE_PASSWORD, "gemfire");
+    return props;
+  }
+
+  public Properties getValidCredentials(Principal principal) {
+    Properties props = new Properties();
+    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + principal.getName() + ".keystore");
+    props.setProperty(PKCSAuthInit.KEYSTORE_FILE_PATH, keyStoreFile);
+    props.setProperty(PKCSAuthInit.KEYSTORE_ALIAS, principal.getName());
+    props.setProperty(PKCSAuthInit.KEYSTORE_PASSWORD, "gemfire");
+    return props;
+  }
+
+  protected Properties initialize() throws IllegalArgumentException {
+    Properties props = new Properties();
+    String keyStoreFile = TestUtil.getResourcePath(PKCSCredentialGenerator.class, keyStoreDir + "/publickeyfile");
+    props.setProperty(PKCSAuthenticator.PUBLIC_KEY_FILE, keyStoreFile);
+    props.setProperty(PKCSAuthenticator.PUBLIC_KEYSTORE_PASSWORD, "gemfire");
+    return props;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/security/SSLCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/security/SSLCredentialGenerator.java b/gemfire-core/src/test/java/security/SSLCredentialGenerator.java
new file mode 100644
index 0000000..29a1a30
--- /dev/null
+++ b/gemfire-core/src/test/java/security/SSLCredentialGenerator.java
@@ -0,0 +1,117 @@
+
+package security;
+
+/*
+ * 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.
+ */
+
+
+import java.io.File;
+import java.io.IOException;
+import java.security.Principal;
+import java.util.Properties;
+
+import com.gemstone.gemfire.security.AuthenticationFailedException;
+import security.CredentialGenerator;
+
+public class SSLCredentialGenerator extends CredentialGenerator {
+
+  private File findTrustedJKS() {
+    File ssldir = new File(System.getProperty("JTESTS") + "/ssl");
+    return new File(ssldir, "trusted.keystore");
+  }
+
+  private File findUntrustedJKS() {
+    File ssldir = new File(System.getProperty("JTESTS") + "/ssl");
+    return new File(ssldir, "untrusted.keystore");
+  }
+
+  private Properties getValidJavaSSLProperties() {
+    File jks = findTrustedJKS();
+    try {
+      Properties props = new Properties();
+      props.setProperty("javax.net.ssl.trustStore", jks.getCanonicalPath());
+      props.setProperty("javax.net.ssl.trustStorePassword", "password");
+      props.setProperty("javax.net.ssl.keyStore", jks.getCanonicalPath());
+      props.setProperty("javax.net.ssl.keyStorePassword", "password");
+      return props;
+    }
+    catch (IOException ex) {
+      throw new AuthenticationFailedException(
+          "SSL: Exception while opening the key store: " + ex);
+    }
+  }
+
+  private Properties getInvalidJavaSSLProperties() {
+    File jks = findUntrustedJKS();
+    try {
+      Properties props = new Properties();
+      props.setProperty("javax.net.ssl.trustStore", jks.getCanonicalPath());
+      props.setProperty("javax.net.ssl.trustStorePassword", "password");
+      props.setProperty("javax.net.ssl.keyStore", jks.getCanonicalPath());
+      props.setProperty("javax.net.ssl.keyStorePassword", "password");
+      return props;
+    }
+    catch (IOException ex) {
+      throw new AuthenticationFailedException(
+          "SSL: Exception while opening the key store: " + ex);
+    }
+  }
+
+  private Properties getSSLProperties() {
+    Properties props = new Properties();
+    props.setProperty("ssl-enabled", "true");
+    props.setProperty("ssl-require-authentication", "true");
+    props.setProperty("ssl-ciphers", "SSL_RSA_WITH_RC4_128_MD5");
+    props.setProperty("ssl-protocols", "TLSv1");
+    return props;
+  }
+
+  protected Properties initialize() throws IllegalArgumentException {
+    this.javaProps = getValidJavaSSLProperties();
+    return getSSLProperties();
+  }
+
+  public ClassCode classCode() {
+    return ClassCode.SSL;
+  }
+
+  public String getAuthInit() {
+    return null;
+  }
+
+  public String getAuthenticator() {
+    return null;
+  }
+
+  public Properties getValidCredentials(int index) {
+    this.javaProps = getValidJavaSSLProperties();
+    return getSSLProperties();
+  }
+
+  public Properties getValidCredentials(Principal principal) {
+    this.javaProps = getValidJavaSSLProperties();
+    return getSSLProperties();
+  }
+
+  public Properties getInvalidCredentials(int index) {
+    this.javaProps = getInvalidJavaSSLProperties();
+    return getSSLProperties();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/security/UserPasswordWithExtraPropsAuthInit.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/security/UserPasswordWithExtraPropsAuthInit.java b/gemfire-core/src/test/java/security/UserPasswordWithExtraPropsAuthInit.java
new file mode 100644
index 0000000..a41f73a
--- /dev/null
+++ b/gemfire-core/src/test/java/security/UserPasswordWithExtraPropsAuthInit.java
@@ -0,0 +1,77 @@
+
+package security;
+
+/*
+ * 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.
+ */
+
+
+import java.util.Properties;
+import java.util.Iterator;
+
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.security.AuthInitialize;
+import com.gemstone.gemfire.security.AuthenticationFailedException;
+import templates.security.UserPasswordAuthInit;
+
+/**
+ * An {@link AuthInitialize} implementation that obtains the user name and
+ * password as the credentials from the given set of properties. If 
+ * keep-extra-props property exits, it will copy rest of the
+ * properties provided in getCredential props argument will also be 
+ * copied as new credentials.
+ * 
+ * @author Soubhik
+ * @since 5.5
+ */
+public class UserPasswordWithExtraPropsAuthInit extends UserPasswordAuthInit {
+
+  public static final String EXTRA_PROPS = "security-keep-extra-props";
+
+  public static final String SECURITY_PREFIX = "security-";
+  
+  public static AuthInitialize create() {
+    return new UserPasswordWithExtraPropsAuthInit();
+  }
+
+  public UserPasswordWithExtraPropsAuthInit() {
+    super();
+  }
+
+  public Properties getCredentials(Properties props, DistributedMember server,
+      boolean isPeer) throws AuthenticationFailedException {
+
+    Properties newProps = super.getCredentials(props, server, isPeer);
+    String extraProps = props.getProperty(EXTRA_PROPS);
+    if(extraProps != null) {
+    	for(Iterator it = props.keySet().iterator(); it.hasNext();) {
+    		String key = (String)it.next();
+    		if( key.startsWith(SECURITY_PREFIX) && 
+    		    key.equalsIgnoreCase(USER_NAME) == false &&
+    		    key.equalsIgnoreCase(PASSWORD) == false &&
+    		    key.equalsIgnoreCase(EXTRA_PROPS) == false) {
+    			newProps.setProperty(key, props.getProperty(key));
+    		}
+    	}
+    	this.securitylog.fine("got everything and now have: "
+          + newProps.keySet().toString());
+    }
+    return newProps;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3720666f/gemfire-core/src/test/java/security/XmlAuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/security/XmlAuthzCredentialGenerator.java b/gemfire-core/src/test/java/security/XmlAuthzCredentialGenerator.java
new file mode 100644
index 0000000..929eafb
--- /dev/null
+++ b/gemfire-core/src/test/java/security/XmlAuthzCredentialGenerator.java
@@ -0,0 +1,264 @@
+
+package security;
+
+/*
+ * 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.
+ */
+
+
+import java.security.Principal;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.util.test.TestUtil;
+import templates.security.UsernamePrincipal;
+import templates.security.XmlAuthorization;
+
+public class XmlAuthzCredentialGenerator extends AuthzCredentialGenerator {
+
+  private static final String dummyXml = "authz-dummy.xml";
+
+  private static final String ldapXml = "authz-ldap.xml";
+
+  private static final String pkcsXml = "authz-pkcs.xml";
+
+  private static final String sslXml = "authz-ssl.xml";
+
+  private static final String[] QUERY_REGIONS = { "/Portfolios", "/Positions",
+      "/AuthRegion" };
+
+  public static OperationCode[] READER_OPS = { OperationCode.GET,
+      OperationCode.REGISTER_INTEREST, OperationCode.UNREGISTER_INTEREST,
+      OperationCode.KEY_SET, OperationCode.CONTAINS_KEY, OperationCode.EXECUTE_FUNCTION };
+
+  public static OperationCode[] WRITER_OPS = { OperationCode.PUT,
+      OperationCode.DESTROY, OperationCode.INVALIDATE, OperationCode.REGION_CLEAR };
+
+  public static OperationCode[] QUERY_OPS = { OperationCode.QUERY,
+      OperationCode.EXECUTE_CQ, OperationCode.STOP_CQ, OperationCode.CLOSE_CQ };
+
+  private static final byte READER_ROLE = 1;
+
+  private static final byte WRITER_ROLE = 2;
+
+  private static final byte QUERY_ROLE = 3;
+
+  private static final byte ADMIN_ROLE = 4;
+
+  private static Set readerOpsSet;
+
+  private static Set writerOpsSet;
+
+  private static Set queryOpsSet;
+
+  private static Set queryRegionSet;
+
+  static {
+
+    readerOpsSet = new HashSet();
+    for (int index = 0; index < READER_OPS.length; index++) {
+      readerOpsSet.add(READER_OPS[index]);
+    }
+    writerOpsSet = new HashSet();
+    for (int index = 0; index < WRITER_OPS.length; index++) {
+      writerOpsSet.add(WRITER_OPS[index]);
+    }
+    queryOpsSet = new HashSet();
+    for (int index = 0; index < QUERY_OPS.length; index++) {
+      queryOpsSet.add(QUERY_OPS[index]);
+    }
+    queryRegionSet = new HashSet();
+    for (int index = 0; index < QUERY_REGIONS.length; index++) {
+      queryRegionSet.add(QUERY_REGIONS[index]);
+    }
+  }
+
+  public XmlAuthzCredentialGenerator() {
+  }
+
+  protected Properties init() throws IllegalArgumentException {
+
+    Properties sysProps = new Properties();
+    String dirName = "/lib/";
+    if (this.cGen.classCode().isDummy()) {
+      String xmlFilename = TestUtil.getResourcePath(XmlAuthzCredentialGenerator.class, dirName + dummyXml);
+      sysProps.setProperty(XmlAuthorization.DOC_URI_PROP_NAME, xmlFilename);
+    }
+    else if (this.cGen.classCode().isLDAP()) {
+      String xmlFilename = TestUtil.getResourcePath(XmlAuthzCredentialGenerator.class, dirName + ldapXml);
+      sysProps.setProperty(XmlAuthorization.DOC_URI_PROP_NAME, xmlFilename);
+    }
+    // else if (this.cGen.classCode().isPKCS()) {
+    // sysProps
+    // .setProperty(XmlAuthorization.DOC_URI_PROP_NAME, dirName + pkcsXml);
+    // }
+    // else if (this.cGen.classCode().isSSL()) {
+    // sysProps
+    // .setProperty(XmlAuthorization.DOC_URI_PROP_NAME, dirName + sslXml);
+    // }
+    else {
+      throw new IllegalArgumentException(
+          "No XML defined for XmlAuthorization module to work with "
+              + this.cGen.getAuthenticator());
+    }
+    return sysProps;
+  }
+
+  public ClassCode classCode() {
+    return ClassCode.XML;
+  }
+
+  public String getAuthorizationCallback() {
+    return "templates.security.XmlAuthorization.create";
+  }
+
+  private Principal getDummyPrincipal(byte roleType, int index) {
+
+    String[] admins = new String[] { "root", "admin", "administrator" };
+    int numReaders = 3;
+    int numWriters = 3;
+
+    switch (roleType) {
+      case READER_ROLE:
+        return new UsernamePrincipal("reader" + (index % numReaders));
+      case WRITER_ROLE:
+        return new UsernamePrincipal("writer" + (index % numWriters));
+      case QUERY_ROLE:
+        return new UsernamePrincipal("reader" + ((index % 2) + 3));
+      default:
+        return new UsernamePrincipal(admins[index % admins.length]);
+    }
+  }
+
+  private Principal getLdapPrincipal(byte roleType, int index) {
+
+    final String userPrefix = "gemfire";
+    final int[] readerIndices = { 3, 4, 5 };
+    final int[] writerIndices = { 6, 7, 8 };
+    final int[] queryIndices = { 9, 10 };
+    final int[] adminIndices = { 1, 2 };
+
+    switch (roleType) {
+      case READER_ROLE:
+        int readerIndex = readerIndices[index % readerIndices.length];
+        return new UsernamePrincipal(userPrefix + readerIndex);
+      case WRITER_ROLE:
+        int writerIndex = writerIndices[index % writerIndices.length];
+        return new UsernamePrincipal(userPrefix + writerIndex);
+      case QUERY_ROLE:
+        int queryIndex = queryIndices[index % queryIndices.length];
+        return new UsernamePrincipal(userPrefix + queryIndex);
+      default:
+        int adminIndex = adminIndices[index % adminIndices.length];
+        return new UsernamePrincipal(userPrefix + adminIndex);
+    }
+  }
+
+  private byte getRequiredRole(OperationCode[] opCodes, String[] regionNames) {
+
+    byte roleType = ADMIN_ROLE;
+    boolean requiresReader = true;
+    boolean requiresWriter = true;
+    boolean requiresQuery = true;
+
+    for (int opNum = 0; opNum < opCodes.length; opNum++) {
+      OperationCode opCode = opCodes[opNum];
+      if (requiresReader && !readerOpsSet.contains(opCode)) {
+        requiresReader = false;
+      }
+      if (requiresWriter && !writerOpsSet.contains(opCode)) {
+        requiresWriter = false;
+      }
+      if (requiresQuery && !queryOpsSet.contains(opCode)) {
+        requiresQuery = false;
+      }
+    }
+    if (requiresReader) {
+      roleType = READER_ROLE;
+    }
+    else if (requiresWriter) {
+      roleType = WRITER_ROLE;
+    }
+    else if (requiresQuery) {
+      if (regionNames != null && regionNames.length > 0) {
+        for (int index = 0; index < regionNames.length; index++) {
+          String regionName = XmlAuthorization
+              .normalizeRegionName(regionNames[index]);
+          if (requiresQuery && !queryRegionSet.contains(regionName)) {
+            requiresQuery = false;
+            break;
+          }
+        }
+        if (requiresQuery) {
+          roleType = QUERY_ROLE;
+        }
+      }
+    }
+    return roleType;
+  }
+
+  protected Principal getAllowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    if (this.cGen.classCode().isDummy()) {
+      byte roleType = getRequiredRole(opCodes, regionNames);
+      return getDummyPrincipal(roleType, index);
+    }
+    else if (this.cGen.classCode().isLDAP()) {
+      byte roleType = getRequiredRole(opCodes, regionNames);
+      return getLdapPrincipal(roleType, index);
+    }
+    return null;
+  }
+
+  protected Principal getDisallowedPrincipal(OperationCode[] opCodes,
+      String[] regionNames, int index) {
+
+    byte roleType = getRequiredRole(opCodes, regionNames);
+    byte disallowedRoleType = READER_ROLE;
+    switch (roleType) {
+      case READER_ROLE:
+        disallowedRoleType = WRITER_ROLE;
+        break;
+      case WRITER_ROLE:
+        disallowedRoleType = READER_ROLE;
+        break;
+      case QUERY_ROLE:
+        disallowedRoleType = READER_ROLE;
+        break;
+      case ADMIN_ROLE:
+        disallowedRoleType = READER_ROLE;
+        break;
+    }
+    if (this.cGen.classCode().isDummy()) {
+      return getDummyPrincipal(disallowedRoleType, index);
+    }
+    else if (this.cGen.classCode().isLDAP()) {
+      return getLdapPrincipal(disallowedRoleType, index);
+    }
+    return null;
+  }
+
+  protected int getNumPrincipalTries(OperationCode[] opCodes,
+      String[] regionNames) {
+    return 5;
+  }
+
+}


[15/33] incubator-geode git commit: GEODE-543: Updating rat.gradle

Posted by je...@apache.org.
GEODE-543: Updating rat.gradle


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

Branch: refs/heads/feature/GEODE-14
Commit: 26ce9a416f59004480587707ec1984143c708edc
Parents: c19c3ca
Author: Jens Deppe <jd...@pivotal.io>
Authored: Mon Dec 21 10:51:03 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Dec 21 10:51:03 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/26ce9a41/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index cb6bdfb..3d45290 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -150,6 +150,7 @@ rat {
 
     // BSD License
     'gemfire-web-api/src/main/webapp/docs/lib/highlight.7.3.pack.js',
+    'gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/ANSIBuffer.java',
 
     // Apache License
     'gemfire-web-api/src/main/webapp/docs/o2c.html',


[13/33] incubator-geode git commit: GEODE-543: upgrade the Jline and Spring Shell libraries and fix the compilation erros

Posted by je...@apache.org.
GEODE-543: upgrade the Jline and Spring Shell libraries and fix the compilation erros

Closes #34
Closes #57

(cherry picked from commit 058aad3663cb00de3ac83f76b9c9b72a32952ca3)
Signed-off-by: Jinmei Liao <ji...@pivotal.io>

GEODE-543: update jline lib and fix compilation errors


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

Branch: refs/heads/feature/GEODE-14
Commit: 5787a48eafe6ac9817070fe50525bdab54cec842
Parents: 3ff5d78
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Wed Nov 11 12:06:22 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Dec 21 09:27:12 2015 -0800

----------------------------------------------------------------------
 gemfire-core/build.gradle                       |   2 +-
 .../management/internal/cli/CliUtil.java        |   2 +-
 .../management/internal/cli/Launcher.java       |   4 +-
 .../internal/cli/commands/ShellCommands.java    |   4 +-
 .../management/internal/cli/shell/Gfsh.java     |  15 +-
 .../internal/cli/shell/jline/ANSIBuffer.java    | 433 +++++++++++++++++++
 .../internal/cli/shell/jline/ANSIHandler.java   |   5 +-
 .../cli/shell/jline/CygwinMinttyTerminal.java   | 137 +-----
 .../internal/cli/shell/jline/GfshHistory.java   |  13 +-
 .../shell/jline/GfshUnsupportedTerminal.java    |   2 +-
 .../internal/cli/util/CLIConsoleBufferUtil.java |   8 +-
 .../PersistentPartitionedRegionTestBase.java    |   2 +-
 .../management/internal/cli/HeadlessGfsh.java   |   8 +-
 .../cli/commands/ShellCommandsDUnitTest.java    |   2 +-
 gradle/dependency-versions.properties           |   4 +-
 15 files changed, 472 insertions(+), 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-core/build.gradle b/gemfire-core/build.gradle
index 19aeee1..28fb3ba 100755
--- a/gemfire-core/build.gradle
+++ b/gemfire-core/build.gradle
@@ -52,7 +52,7 @@ dependencies {
   compile 'mx4j:mx4j-remote:' + project.'mx4j.version'
   compile 'mx4j:mx4j-tools:' + project.'mx4j.version'
   compile 'net.java.dev.jna:jna:' + project.'jna.version'
-  compile 'net.sourceforge.jline:jline:' + project.'jline.version'
+  compile 'jline:jline:' + project.'jline.version'
   provided 'org.apache.hadoop:hadoop-common:' + project.'hadoop.version'
   provided 'org.apache.hadoop:hadoop-annotations:' + project.'hadoop.version'
   provided 'org.apache.hadoop:hadoop-hdfs:' + project.'hadoop.version'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CliUtil.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CliUtil.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CliUtil.java
index bc1f7b7..09f8bd8 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CliUtil.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CliUtil.java
@@ -105,7 +105,7 @@ public class CliUtil {
 
     if (includeGfshDependencies) {
       // ConsoleReader from jline
-      jarProductName = checkLibraryByLoadingClass("jline.ConsoleReader", "JLine");
+      jarProductName = checkLibraryByLoadingClass("jline.console.ConsoleReader", "JLine");
       if (jarProductName != null) {
         return jarProductName;
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/Launcher.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/Launcher.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/Launcher.java
index 8bf1ce1..12e2c50 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/Launcher.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/Launcher.java
@@ -166,7 +166,7 @@ public final class Launcher {
           System.err.println(CliStrings.format(MSG_INVALID_COMMAND_OR_OPTION, CliUtil.arrayToString(args)));
           exitRequest = ExitShellRequest.FATAL_EXIT;
         } else {
-          if (!gfsh.executeCommand(commandLineCommand)) {
+          if (!gfsh.executeScriptLine(commandLineCommand)) {
               if (gfsh.getLastExecutionStatus() != 0) 
                 exitRequest = ExitShellRequest.FATAL_EXIT;
           } else if (gfsh.getLastExecutionStatus() != 0) {
@@ -224,7 +224,7 @@ public final class Launcher {
             String command = commandsToExecute.get(i);
             System.out.println(GfshParser.LINE_SEPARATOR + "(" + (i + 1) + ") Executing - " + command
                 + GfshParser.LINE_SEPARATOR);
-            if (!gfsh.executeCommand(command) || gfsh.getLastExecutionStatus() != 0) {
+            if (!gfsh.executeScriptLine(command) || gfsh.getLastExecutionStatus() != 0) {
               exitRequest = ExitShellRequest.FATAL_EXIT;
             }
           }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
index edab207..1bd7692 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
@@ -853,8 +853,8 @@ private void configureHttpsURLConnection(Map<String, String> sslConfigProps) thr
       int historySizeWordLength = historySizeString.length();
 
       GfshHistory gfshHistory = gfsh.getGfshHistory();
-      List<?> gfshHistoryList = gfshHistory.getHistoryList();
-      Iterator<?> it = gfshHistoryList.iterator();
+      //List<?> gfshHistoryList = gfshHistory.getHistoryList();
+      Iterator<?> it = gfshHistory.entries();
       boolean flagForLineNumbers = (saveHistoryTo != null && saveHistoryTo
           .length() > 0) ? false : true;
       long lineNumber = 0;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/Gfsh.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/Gfsh.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/Gfsh.java
index 1d14c5b..67a8ccb 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/Gfsh.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/Gfsh.java
@@ -36,9 +36,9 @@ import java.util.logging.Level;
 import java.util.logging.LogManager;
 import java.util.logging.Logger;
 
-import jline.ConsoleReader;
 import jline.Terminal;
 
+import jline.console.ConsoleReader;
 import org.springframework.shell.core.AbstractShell;
 import org.springframework.shell.core.CommandMarker;
 import org.springframework.shell.core.Converter;
@@ -380,7 +380,6 @@ public class Gfsh extends JLineShell {
   /**
    * See findResources in {@link AbstractShell}
    */
-  @Override
   protected Collection<URL> findResources(String resourceName) {
 //    return Collections.singleton(ClassPathLoader.getLatest().getResource(resourceName));
     return null;
@@ -420,7 +419,7 @@ public class Gfsh extends JLineShell {
    * @return true if execution is successful; false otherwise
    */
   @Override
-  public boolean executeCommand(final String line) {
+  public boolean executeScriptLine(final String line) {
     boolean success = false;
     String withPropsExpanded = line;
 
@@ -440,7 +439,7 @@ public class Gfsh extends JLineShell {
       if (gfshFileLogger.fineEnabled()) {
         gfshFileLogger.fine(logMessage + withPropsExpanded);
       }
-      success = super.executeCommand(withPropsExpanded);
+      success = super.executeScriptLine(withPropsExpanded);
     } catch (Exception e) {
       //TODO: should there be a way to differentiate error in shell & error on
       //server. May be by exception type.
@@ -633,12 +632,12 @@ public class Gfsh extends JLineShell {
   ///////////////////// JLineShell Class Methods End  //////////////////////////
 
   public int getTerminalHeight() {
-    return terminal != null ? terminal.getTerminalHeight() : DEFAULT_HEIGHT;
+    return terminal != null ? terminal.getHeight() : DEFAULT_HEIGHT;
   }
 
   public int getTerminalWidth() {
     if (terminal != null) {
-      return terminal.getTerminalWidth();
+      return terminal.getWidth();
     }
 
     Map<String, String> env = System.getenv();
@@ -805,7 +804,7 @@ public class Gfsh extends JLineShell {
                 ++commandSrNum;
                 Gfsh.println(commandSrNum+". Executing - " + cmdLet);                
                 Gfsh.println();
-                boolean executeSuccess = executeCommand(cmdLet);
+                boolean executeSuccess = executeScriptLine(cmdLet);
                 if (!executeSuccess) {
                   setLastExecutionStatus(-1);
                 }
@@ -922,7 +921,7 @@ public class Gfsh extends JLineShell {
       readLine = reader.readLine(prompt);
     } catch (IndexOutOfBoundsException e) {
       if (earlierLine.length() == 0) {
-        reader.printNewline();
+        reader.println();
         readLine = LINE_SEPARATOR;
         reader.getCursorBuffer().cursor = 0;
       } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/ANSIBuffer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/ANSIBuffer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/ANSIBuffer.java
new file mode 100644
index 0000000..572f899
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/ANSIBuffer.java
@@ -0,0 +1,433 @@
+/*
+ * Copyright (c) 2002-2007, Marc Prud'hommeaux. All rights reserved.
+ *
+ * This software is distributable under the BSD license. See the terms of the
+ * BSD license in the documentation provided with this software.
+ */
+package com.gemstone.gemfire.management.internal.cli.shell.jline;
+
+import org.springframework.shell.support.util.OsUtils;
+
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+
+/**
+ *  A buffer that can contain ANSI text.
+ *
+ *  @author  <a href="mailto:mwp1@cornell.edu">Marc Prud'hommeaux</a>
+ */
+public class ANSIBuffer {
+    private boolean ansiEnabled = true;
+    private final StringBuffer ansiBuffer = new StringBuffer();
+    private final StringBuffer plainBuffer = new StringBuffer();
+
+    public ANSIBuffer() {
+    }
+
+    public ANSIBuffer(final String str) {
+        append(str);
+    }
+
+    public void setAnsiEnabled(final boolean ansi) {
+        this.ansiEnabled = ansi;
+    }
+
+    public boolean getAnsiEnabled() {
+        return this.ansiEnabled;
+    }
+
+    public String getAnsiBuffer() {
+        return ansiBuffer.toString();
+    }
+
+    public String getPlainBuffer() {
+        return plainBuffer.toString();
+    }
+
+    public String toString(final boolean ansi) {
+        return ansi ? getAnsiBuffer() : getPlainBuffer();
+    }
+
+    public String toString() {
+        return toString(ansiEnabled);
+    }
+
+    public ANSIBuffer append(final String str) {
+        ansiBuffer.append(str);
+        plainBuffer.append(str);
+
+        return this;
+    }
+
+    public ANSIBuffer attrib(final String str, final int code) {
+        ansiBuffer.append(ANSICodes.attrib(code)).append(str)
+                  .append(ANSICodes.attrib(ANSICodes.OFF));
+        plainBuffer.append(str);
+
+        return this;
+    }
+
+    public ANSIBuffer red(final String str) {
+        return attrib(str, ANSICodes.FG_RED);
+    }
+
+    public ANSIBuffer blue(final String str) {
+        return attrib(str, ANSICodes.FG_BLUE);
+    }
+
+    public ANSIBuffer green(final String str) {
+        return attrib(str, ANSICodes.FG_GREEN);
+    }
+
+    public ANSIBuffer black(final String str) {
+        return attrib(str, ANSICodes.FG_BLACK);
+    }
+
+    public ANSIBuffer yellow(final String str) {
+        return attrib(str, ANSICodes.FG_YELLOW);
+    }
+
+    public ANSIBuffer magenta(final String str) {
+        return attrib(str, ANSICodes.FG_MAGENTA);
+    }
+
+    public ANSIBuffer cyan(final String str) {
+        return attrib(str, ANSICodes.FG_CYAN);
+    }
+
+    public ANSIBuffer bold(final String str) {
+        return attrib(str, ANSICodes.BOLD);
+    }
+
+    public ANSIBuffer underscore(final String str) {
+        return attrib(str, ANSICodes.UNDERSCORE);
+    }
+
+    public ANSIBuffer blink(final String str) {
+        return attrib(str, ANSICodes.BLINK);
+    }
+
+    public ANSIBuffer reverse(final String str) {
+        return attrib(str, ANSICodes.REVERSE);
+    }
+
+    public static class ANSICodes {
+        static final int OFF = 0;
+        static final int BOLD = 1;
+        static final int UNDERSCORE = 4;
+        static final int BLINK = 5;
+        static final int REVERSE = 7;
+        static final int CONCEALED = 8;
+        static final int FG_BLACK = 30;
+        static final int FG_RED = 31;
+        static final int FG_GREEN = 32;
+        static final int FG_YELLOW = 33;
+        static final int FG_BLUE = 34;
+        static final int FG_MAGENTA = 35;
+        static final int FG_CYAN = 36;
+        static final int FG_WHITE = 37;
+        static final char ESC = 27;
+
+        /**
+         *  Constructor is private since this is a utility class.
+         */
+        private ANSICodes() {
+        }
+
+        /**
+          * Sets the screen mode. The mode will be one of the following values:
+          * <pre>
+          * mode     description
+          * ----------------------------------------
+          *   0      40 x 148 x 25 monochrome (text)
+          *   1      40 x 148 x 25 color (text)
+          *   2      80 x 148 x 25 monochrome (text)
+          *   3      80 x 148 x 25 color (text)
+          *   4      320 x 148 x 200 4-color (graphics)
+          *   5      320 x 148 x 200 monochrome (graphics)
+          *   6      640 x 148 x 200 monochrome (graphics)
+          *   7      Enables line wrapping
+          *  13      320 x 148 x 200 color (graphics)
+          *  14      640 x 148 x 200 color (16-color graphics)
+          *  15      640 x 148 x 350 monochrome (2-color graphics)
+          *  16      640 x 148 x 350 color (16-color graphics)
+          *  17      640 x 148 x 480 monochrome (2-color graphics)
+          *  18      640 x 148 x 480 color (16-color graphics)
+          *  19      320 x 148 x 200 color (256-color graphics)
+          * </pre>
+          */
+        public static String setmode(final int mode) {
+            return ESC + "[=" + mode + "h";
+        }
+
+        /**
+          * Same as setmode () except for mode = 7, which disables line
+          * wrapping (useful for writing the right-most column without
+          * scrolling to the next line).
+          */
+        public static String resetmode(final int mode) {
+            return ESC + "[=" + mode + "l";
+        }
+
+        /**
+          * Clears the screen and moves the cursor to the home postition.
+          */
+        public static String clrscr() {
+            return ESC + "[2J";
+        }
+
+        /**
+          * Removes all characters from the current cursor position until
+          * the end of the line.
+          */
+        public static String clreol() {
+            return ESC + "[K";
+        }
+
+        /**
+          * Moves the cursor n positions to the left. If n is greater or
+          * equal to the current cursor column, the cursor is moved to the
+          * first column.
+          */
+        public static String left(final int n) {
+            return ESC + "[" + n + "D";
+        }
+
+        /**
+          * Moves the cursor n positions to the right. If n plus the current
+          * cursor column is greater than the rightmost column, the cursor
+          * is moved to the rightmost column.
+          */
+        public static String right(final int n) {
+            return ESC + "[" + n + "C";
+        }
+
+        /**
+          * Moves the cursor n rows up without changing the current column.
+          * If n is greater than or equal to the current row, the cursor is
+          * placed in the first row.
+          */
+        public static String up(final int n) {
+            return ESC + "[" + n + "A";
+        }
+
+        /**
+          * Moves the cursor n rows down. If n plus the current row is greater
+          * than the bottom row, the cursor is moved to the bottom row.
+          */
+        public static String down(final int n) {
+            return ESC + "[" + n + "B";
+        }
+
+        /*
+          * Moves the cursor to the given row and column. (1,1) represents
+          * the upper left corner. The lower right corner of a usual DOS
+          * screen is (25, 80).
+          */
+        public static String gotoxy(final int row, final int column) {
+            return ESC + "[" + row + ";" + column + "H";
+        }
+
+        /**
+          * Saves the current cursor position.
+          */
+        public static String save() {
+            return ESC + "[s";
+        }
+
+        /**
+          * Restores the saved cursor position.
+          */
+        public static String restore() {
+            return ESC + "[u";
+        }
+
+        /**
+          * Sets the character attribute. It will be
+         * one of the following character attributes:
+          *
+          * <pre>
+          * Text attributes
+          *    0    All attributes off
+          *    1    Bold on
+          *    4    Underscore (on monochrome display adapter only)
+          *    5    Blink on
+          *    7    Reverse video on
+          *    8    Concealed on
+          *
+          *   Foreground colors
+          *    30    Black
+          *    31    Red
+          *    32    Green
+          *    33    Yellow
+          *    34    Blue
+          *    35    Magenta
+          *    36    Cyan
+          *    37    White
+          *
+          *   Background colors
+          *    40    Black
+          *    41    Red
+          *    42    Green
+          *    43    Yellow
+          *    44    Blue
+          *    45    Magenta
+          *    46    Cyan
+          *    47    White
+          * </pre>
+          *
+          * The attributes remain in effect until the next attribute command
+          * is sent.
+          */
+        public static String attrib(final int attr) {
+            return ESC + "[" + attr + "m";
+        }
+
+        /**
+          * Sets the key with the given code to the given value. code must be
+          * derived from the following table, value must
+         * be any semicolon-separated
+          * combination of String (enclosed in double quotes) and numeric values.
+          * For example, to set F1 to the String "Hello F1", followed by a CRLF
+          * sequence, one can use: ANSI.setkey ("0;59", "\"Hello F1\";13;10").
+          * Heres's the table of key values:
+          * <pre>
+          * Key                       Code      SHIFT+code  CTRL+code  ALT+code
+          * ---------------------------------------------------------------
+          * F1                        0;59      0;84        0;94       0;104
+          * F2                        0;60      0;85        0;95       0;105
+          * F3                        0;61      0;86        0;96       0;106
+          * F4                        0;62      0;87        0;97       0;107
+          * F5                        0;63      0;88        0;98       0;108
+          * F6                        0;64      0;89        0;99       0;109
+          * F7                        0;65      0;90        0;100      0;110
+          * F8                        0;66      0;91        0;101      0;111
+          * F9                        0;67      0;92        0;102      0;112
+          * F10                       0;68      0;93        0;103      0;113
+          * F11                       0;133     0;135       0;137      0;139
+          * F12                       0;134     0;136       0;138      0;140
+          * HOME (num keypad)         0;71      55          0;119      --
+          * UP ARROW (num keypad)     0;72      56          (0;141)    --
+          * PAGE UP (num keypad)      0;73      57          0;132      --
+          * LEFT ARROW (num keypad)   0;75      52          0;115      --
+          * RIGHT ARROW (num keypad)  0;77      54          0;116      --
+          * END (num keypad)          0;79      49          0;117      --
+          * DOWN ARROW (num keypad)   0;80      50          (0;145)    --
+          * PAGE DOWN (num keypad)    0;81      51          0;118      --
+          * INSERT (num keypad)       0;82      48          (0;146)    --
+          * DELETE  (num keypad)      0;83      46          (0;147)    --
+          * HOME                      (224;71)  (224;71)    (224;119)  (224;151)
+          * UP ARROW                  (224;72)  (224;72)    (224;141)  (224;152)
+          * PAGE UP                   (224;73)  (224;73)    (224;132)  (224;153)
+          * LEFT ARROW                (224;75)  (224;75)    (224;115)  (224;155)
+          * RIGHT ARROW               (224;77)  (224;77)    (224;116)  (224;157)
+          * END                       (224;79)  (224;79)    (224;117)  (224;159)
+          * DOWN ARROW                (224;80)  (224;80)    (224;145)  (224;154)
+          * PAGE DOWN                 (224;81)  (224;81)    (224;118)  (224;161)
+          * INSERT                    (224;82)  (224;82)    (224;146)  (224;162)
+          * DELETE                    (224;83)  (224;83)    (224;147)  (224;163)
+          * PRINT SCREEN              --        --          0;114      --
+          * PAUSE/BREAK               --        --          0;0        --
+          * BACKSPACE                 8         8           127        (0)
+          * ENTER                     13        --          10         (0
+          * TAB                       9         0;15        (0;148)    (0;165)
+          * NULL                      0;3       --          --         --
+          * A                         97        65          1          0;30
+          * B                         98        66          2          0;48
+          * C                         99        66          3          0;46
+          * D                         100       68          4          0;32
+          * E                         101       69          5          0;18
+          * F                         102       70          6          0;33
+          * G                         103       71          7          0;34
+          * H                         104       72          8          0;35
+          * I                         105       73          9          0;23
+          * J                         106       74          10         0;36
+          * K                         107       75          11         0;37
+          * L                         108       76          12         0;38
+          * M                         109       77          13         0;50
+          * N                         110       78          14         0;49
+          * O                         111       79          15         0;24
+          * P                         112       80          16         0;25
+          * Q                         113       81          17         0;16
+          * R                         114       82          18         0;19
+          * S                         115       83          19         0;31
+          * T                         116       84          20         0;20
+          * U                         117       85          21         0;22
+          * V                         118       86          22         0;47
+          * W                         119       87          23         0;17
+          * X                         120       88          24         0;45
+          * Y                         121       89          25         0;21
+          * Z                         122       90          26         0;44
+          * 1                         49        33          --         0;120
+          * 2                         50        64          0          0;121
+          * 3                         51        35          --         0;122
+          * 4                         52        36          --         0;123
+          * 5                         53        37          --         0;124
+          * 6                         54        94          30         0;125
+          * 7                         55        38          --         0;126
+          * 8                         56        42          --         0;126
+          * 9                         57        40          --         0;127
+          * 0                         48        41          --         0;129
+          * -                         45        95          31         0;130
+          * =                         61        43          ---        0;131
+          * [                         91        123         27         0;26
+          * ]                         93        125         29         0;27
+          *                           92        124         28         0;43
+          * ;                         59        58          --         0;39
+          * '                         39        34          --         0;40
+          * ,                         44        60          --         0;51
+          * .                         46        62          --         0;52
+          * /                         47        63          --         0;53
+          * `                         96        126         --         (0;41)
+          * ENTER (keypad)            13        --          10         (0;166)
+          * / (keypad)                47        47          (0;142)    (0;74)
+          * * (keypad)                42        (0;144)     (0;78)     --
+          * - (keypad)                45        45          (0;149)    (0;164)
+          * + (keypad)                43        43          (0;150)    (0;55)
+          * 5 (keypad)                (0;76)    53          (0;143)    --
+          */
+        public static String setkey(final String code, final String value) {
+            return ESC + "[" + code + ";" + value + "p";
+        }
+    }
+
+    public static void main(final String[] args) throws Exception {
+        // sequence, one can use: ANSI.setkey ("0;59", "\"Hello F1\";13;10").
+        BufferedReader reader =
+            new BufferedReader(new InputStreamReader(System.in));
+        System.out.print(ANSICodes.setkey("97", "97;98;99;13")
+                         + ANSICodes.attrib(ANSICodes.OFF));
+        System.out.flush();
+
+        String line;
+
+        while ((line = reader.readLine()) != null) {
+            System.out.println("GOT: " + line);
+        }
+    }
+
+    private static final boolean ROO_BRIGHT_COLORS = Boolean.getBoolean("roo.bright");
+    private static final boolean SHELL_BRIGHT_COLORS = Boolean.getBoolean("spring.shell.bright");
+    private static final boolean BRIGHT_COLORS = ROO_BRIGHT_COLORS || SHELL_BRIGHT_COLORS;
+
+    public static ANSIBuffer getANSIBuffer() {
+        final char esc = (char) 27;
+        return new ANSIBuffer() {
+            @Override
+            public ANSIBuffer reverse(final String str) {
+                if (OsUtils.isWindows()) {
+                    return super.reverse(str).append(ANSICodes.attrib(esc));
+                }
+                return super.reverse(str);
+            };
+            @Override
+            public ANSIBuffer attrib(final String str, final int code) {
+                if (BRIGHT_COLORS && 30 <= code && code <= 37) {
+                    // This is a color code: add a 'bright' code
+                    return append(esc + "[" + code + ";1m").append(str).append(ANSICodes.attrib(0));
+                }
+                return super.attrib(str, code);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/ANSIHandler.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/ANSIHandler.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/ANSIHandler.java
index ccd9864..c8102ae 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/ANSIHandler.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/ANSIHandler.java
@@ -18,8 +18,6 @@ package com.gemstone.gemfire.management.internal.cli.shell.jline;
 
 import org.springframework.shell.core.JLineLogHandler;
 
-import jline.ANSIBuffer;
-
 /**
  * Overrides jline.History to add History without newline characters.
  * 
@@ -50,7 +48,8 @@ public class ANSIHandler {
     String decoratedInput = input;
     
     if (isAnsiEnabled()) {
-      ANSIBuffer ansiBuffer = JLineLogHandler.getANSIBuffer();
+      ANSIBuffer ansiBuffer = ANSIBuffer.getANSIBuffer();
+
 
       for (ANSIStyle ansiStyle : styles) {
         switch (ansiStyle) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/CygwinMinttyTerminal.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/CygwinMinttyTerminal.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/CygwinMinttyTerminal.java
index d84bbe7..f486774 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/CygwinMinttyTerminal.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/CygwinMinttyTerminal.java
@@ -39,147 +39,18 @@ public class CygwinMinttyTerminal extends UnixTerminal {
   
   
   String encoding = System.getProperty("input.encoding", "UTF-8");
-  ReplayPrefixOneCharInputStream replayStream = new ReplayPrefixOneCharInputStream(encoding);
   InputStreamReader replayReader;
 
-  public CygwinMinttyTerminal() {
-      try {
-          replayReader = new InputStreamReader(replayStream, encoding);
-      } catch (Exception e) {
-          throw new RuntimeException(e);
-      }
+  public CygwinMinttyTerminal() throws Exception{
   }
 
   @Override
-  public void initializeTerminal() throws IOException, InterruptedException {
+  public void init() throws Exception{
 
   }
 
   @Override
-  public void restoreTerminal() throws Exception {
-    resetTerminal();
+  public void restore() throws Exception {
+    reset();
   }
-
-  @Override
-  public int readVirtualKey(InputStream in) throws IOException {
-    int c = readCharacter(in);
-
-    //if (backspaceDeleteSwitched)
-        if (c == DELETE)
-            c = BACKSPACE;
-        else if (c == BACKSPACE)
-            c = DELETE;
-
-    // in Unix terminals, arrow keys are represented by
-    // a sequence of 3 characters. E.g., the up arrow
-    // key yields 27, 91, 68
-    if (c == ARROW_START && in.available() > 0) {
-        // Escape key is also 27, so we use InputStream.available()
-        // to distinguish those. If 27 represents an arrow, there
-        // should be two more chars immediately available.
-        while (c == ARROW_START) {
-            c = readCharacter(in);
-        }
-        if (c == ARROW_PREFIX || c == O_PREFIX) {
-            c = readCharacter(in);
-            if (c == ARROW_UP) {
-                return CTRL_P;
-            } else if (c == ARROW_DOWN) {
-                return CTRL_N;
-            } else if (c == ARROW_LEFT) {
-                return CTRL_B;
-            } else if (c == ARROW_RIGHT) {
-                return CTRL_F;
-            } else if (c == HOME_CODE) {
-                return CTRL_A;
-            } else if (c == END_CODE) {
-                return CTRL_E;
-            } else if (c == DEL_THIRD) {
-                c = readCharacter(in); // read 4th
-                return DELETE;
-            }
-        } 
-    } 
-    // handle unicode characters, thanks for a patch from amyi@inf.ed.ac.uk
-    if (c > 128) {      
-      // handle unicode characters longer than 2 bytes,
-      // thanks to Marc.Herbert@continuent.com        
-        replayStream.setInput(c, in);
-//      replayReader = new InputStreamReader(replayStream, encoding);
-        c = replayReader.read();
-        
-    }
-    return c;
-  }
-  
-  /**
-   * This is awkward and inefficient, but probably the minimal way to add
-   * UTF-8 support to JLine
-   *
-   * @author <a href="mailto:Marc.Herbert@continuent.com">Marc Herbert</a>
-   */
-  static class ReplayPrefixOneCharInputStream extends InputStream {
-      byte firstByte;
-      int byteLength;
-      InputStream wrappedStream;
-      int byteRead;
-
-      final String encoding;
-      
-      public ReplayPrefixOneCharInputStream(String encoding) {
-          this.encoding = encoding;
-      }
-      
-      public void setInput(int recorded, InputStream wrapped) throws IOException {
-          this.byteRead = 0;
-          this.firstByte = (byte) recorded;
-          this.wrappedStream = wrapped;
-
-          byteLength = 1;
-          if (encoding.equalsIgnoreCase("UTF-8"))
-              setInputUTF8(recorded, wrapped);
-          else if (encoding.equalsIgnoreCase("UTF-16"))
-              byteLength = 2;
-          else if (encoding.equalsIgnoreCase("UTF-32"))
-              byteLength = 4;
-      }
-          
-          
-      public void setInputUTF8(int recorded, InputStream wrapped) throws IOException {
-          // 110yyyyy 10zzzzzz
-          if ((firstByte & (byte) 0xE0) == (byte) 0xC0)
-              this.byteLength = 2;
-          // 1110xxxx 10yyyyyy 10zzzzzz
-          else if ((firstByte & (byte) 0xF0) == (byte) 0xE0)
-              this.byteLength = 3;
-          // 11110www 10xxxxxx 10yyyyyy 10zzzzzz
-          else if ((firstByte & (byte) 0xF8) == (byte) 0xF0)
-              this.byteLength = 4;
-          else
-              throw new IOException("invalid UTF-8 first byte: " + firstByte);
-      }
-
-      public int read() throws IOException {
-          if (available() == 0)
-              return -1;
-
-          byteRead++;
-
-          if (byteRead == 1)
-              return firstByte;
-
-          return wrappedStream.read();
-      }
-
-      /**
-      * InputStreamReader is greedy and will try to read bytes in advance. We
-      * do NOT want this to happen since we use a temporary/"losing bytes"
-      * InputStreamReader above, that's why we hide the real
-      * wrappedStream.available() here.
-      */
-      public int available() {
-          return byteLength - byteRead;
-      }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/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 9f18cae..dc3fbe1 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
@@ -18,7 +18,10 @@ package com.gemstone.gemfire.management.internal.cli.shell.jline;
 
 import com.gemstone.gemfire.management.internal.cli.parser.preprocessor.PreprocessorUtils;
 
-import jline.History;
+import jline.console.history.MemoryHistory;
+
+import java.io.File;
+import java.io.IOException;
 
 /**
  * Overrides jline.History to add History without newline characters.
@@ -26,14 +29,14 @@ import jline.History;
  * @author Abhishek Chaudhari
  * @since 7.0
  */
-public class GfshHistory extends History {
+public class GfshHistory extends MemoryHistory {
+
   // let the history from history file get added initially
   private boolean autoFlush = true;
-  
-  @Override
+
   public void addToHistory(String buffer) {
     if (isAutoFlush()) {
-      super.addToHistory(toHistoryLoggable(buffer));
+      super.add(toHistoryLoggable(buffer));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/GfshUnsupportedTerminal.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/GfshUnsupportedTerminal.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/GfshUnsupportedTerminal.java
index 59609ba..3bc839e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/GfshUnsupportedTerminal.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/jline/GfshUnsupportedTerminal.java
@@ -27,7 +27,7 @@ import jline.UnsupportedTerminal;
  */
 public class GfshUnsupportedTerminal extends UnsupportedTerminal {
   @Override
-  public boolean isANSISupported() {
+  public synchronized boolean isAnsiSupported() {
     return false;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/CLIConsoleBufferUtil.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/CLIConsoleBufferUtil.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/CLIConsoleBufferUtil.java
index 70a00a1..9d1bdbe 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/CLIConsoleBufferUtil.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/CLIConsoleBufferUtil.java
@@ -16,7 +16,7 @@
  */
 package com.gemstone.gemfire.management.internal.cli.util;
 
-import jline.ConsoleReader;
+import jline.console.ConsoleReader;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 
 public class CLIConsoleBufferUtil {
@@ -24,9 +24,9 @@ public class CLIConsoleBufferUtil {
     
     ConsoleReader reader = Gfsh.getConsoleReader();    
     if (reader != null) {
-      StringBuffer buffer = reader.getCursorBuffer().getBuffer();    
-      if(buffer.length() > messege.length()){
-        int appendSpaces = buffer.length() - messege.length();
+      int bufferLength = reader.getCursorBuffer().length();
+      if(bufferLength > messege.length()){
+        int appendSpaces = bufferLength - messege.length();
         for(int i = 0; i < appendSpaces; i++){
           messege = messege + " ";
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
index 6ce1a13..057be81 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
@@ -172,7 +172,7 @@ public abstract class PersistentPartitionedRegionTestBase extends CacheTestCase
           public void run() {
             Cache cache = getCache();
             Region region = cache.getRegion(regionName);
-            
+
             for(int i =startKey; i < endKey; i++) {
               assertEquals("For key " + i, value, region.get(i));
             }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfsh.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfsh.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfsh.java
index 9ca9809..f8b3fd4 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfsh.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfsh.java
@@ -19,8 +19,7 @@ package com.gemstone.gemfire.management.internal.cli;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.shell.GfshConfig;
 import com.gemstone.gemfire.management.internal.cli.shell.jline.GfshUnsupportedTerminal;
-import edu.umd.cs.findbugs.annotations.SuppressWarnings;
-import jline.ConsoleReader;
+import jline.console.ConsoleReader;
 import org.springframework.shell.core.ExitShellRequest;
 import org.springframework.shell.event.ShellStatus.Status;
 
@@ -106,7 +105,7 @@ public class HeadlessGfsh implements ResultHandler {
     boolean status = false;
     try {
       outputString = null;
-      status = shell.executeCommand(command);
+      status = shell.executeScriptLine(command);
     } catch (Exception e) {
       outputString = e.getMessage();
     }
@@ -291,9 +290,8 @@ public class HeadlessGfsh implements ResultHandler {
       try {
         output = new ByteArrayOutputStream(1024 * 10);
         PrintStream sysout = new PrintStream(output);
-        Writer wrappedOut = new BufferedWriter(new OutputStreamWriter(sysout));
         setGfshOutErr(sysout);
-        return new ConsoleReader(new FileInputStream(FileDescriptor.in), new PrintWriter(wrappedOut));
+        return new ConsoleReader(new FileInputStream(FileDescriptor.in), sysout);
       } catch (IOException e) {
         throw new RuntimeException(e);
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
index 0fc6be4..c8695ad 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
@@ -338,7 +338,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
       String resultString = commandResultToString(cmdResult);
       getLogWriter().info("testClearHistory resultString=" + resultString);
       assertTrue(resultString.contains(CliStrings.HISTORY__MSG__CLEARED_HISTORY));
-      assertTrue(gfshInstance.getGfshHistory().getHistoryList().size() <= 1);
+      assertTrue(gfshInstance.getGfshHistory().size()<= 1);
     } else {
       fail("testClearHistory failed");
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5787a48e/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 3833f54..71cfd43 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -50,7 +50,7 @@ javax.servlet-api.version = 3.1.0
 javax.transaction-api.version = 1.2
 jedis.version = 2.7.2
 jetty.version = 9.3.6.v20151106
-jline.version = 1.0.S2-B
+jline.version = 2.12
 jmock.version = 2.8.1
 jna.version = 4.0.0
 json4s.version = 3.2.4
@@ -73,7 +73,7 @@ snappy-java.version = 1.1.1.6
 spring-data-commons.version = 1.9.1.RELEASE
 spring-data-gemfire.version = 1.5.1.RELEASE
 spring-hateos.version = 0.16.0.RELEASE
-spring-shell.version = 1.0.0.RELEASE
+spring-shell.version = 1.1.0.RELEASE
 springframework.version = 3.2.12.RELEASE
 stephenc-findbugs.version = 1.3.9-1
 spymemcached.version = 2.9.0



[27/33] incubator-geode git commit: removing my setting from .gitignore that has no business being there

Posted by je...@apache.org.
removing my setting from .gitignore that has no business being there


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

Branch: refs/heads/feature/GEODE-14
Commit: 1c2cad1d36ef9c99db1e05b1620d117935829147
Parents: 121710c
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Wed Dec 23 09:46:08 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Wed Dec 23 09:46:08 2015 -0800

----------------------------------------------------------------------
 .gitignore | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c2cad1d/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 0034762..3f17c16 100644
--- a/.gitignore
+++ b/.gitignore
@@ -21,5 +21,4 @@ build-eclipse/
 *.patch
 *.diff
 *.dat
-# bruce: my synch settings for windows->linux transfers
-transferhost.txt
+


[23/33] incubator-geode git commit: Closes #64 *Committed in 94bce4d394b24cceb2f183db0b35aa7688679fb5*

Posted by je...@apache.org.
Closes #64 *Committed in 94bce4d394b24cceb2f183db0b35aa7688679fb5*


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

Branch: refs/heads/feature/GEODE-14
Commit: a03e8b701753455991c1ca4b4110967be968dac2
Parents: 94bce4d
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue Dec 22 10:56:08 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Dec 22 10:56:08 2015 -0800

----------------------------------------------------------------------

----------------------------------------------------------------------



[30/33] incubator-geode git commit: GEODE-614: Increase maximum duration

Posted by je...@apache.org.
GEODE-614: Increase maximum duration

Increase ExpectedTimeout maximum duration to prevent false failures
when build job loses CPU for more than 2 seconds.


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

Branch: refs/heads/feature/GEODE-14
Commit: 935b76a393fee86cc491afeba54e233eb0bf6a5e
Parents: 093ac12
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue Dec 22 14:28:07 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Mon Dec 28 10:14:53 2015 -0800

----------------------------------------------------------------------
 .../com/gemstone/gemfire/internal/process/PidFileJUnitTest.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/935b76a3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
index 89f786a..5f81c2b 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
@@ -139,7 +139,7 @@ public class PidFileJUnitTest {
     timeout.expect(TimeoutException.class);
     timeout.expectMessage("Invalid pid 'null' found");
     timeout.expectMinimumDuration(1000);
-    timeout.expectMaximumDuration(2000);
+    timeout.expectMaximumDuration(10000);
     timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
     
     new PidFile(file).readPid(1500, TimeUnit.MILLISECONDS);


[19/33] incubator-geode git commit: Excluding binary publickeyfile from rat

Posted by je...@apache.org.
Excluding binary publickeyfile from rat


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

Branch: refs/heads/feature/GEODE-14
Commit: 05f041391b5cab0f55fda80d9cd3669c65166a58
Parents: 625ea36
Author: Dan Smith <up...@apache.org>
Authored: Tue Dec 22 09:18:49 2015 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Tue Dec 22 09:19:10 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/05f04139/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 3d45290..fb4648e 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -65,6 +65,7 @@ rat {
     '**/*.ser',
     '**/*.truststore',
     '**/*.xls',
+    '**/publickeyfile',
 
     // other text files
     'gemfire-spark-connector/project/plugins.sbt',