You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2015/12/22 20:26:03 UTC

[01/35] incubator-geode git commit: GEODE-568: wait at most 5 seconds before checking the size of the internal region to allow eviction to complete

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-217 9283f26cb -> ca1550700


GEODE-568: wait at most 5 seconds before checking the size of the internal region to allow eviction to complete


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

Branch: refs/heads/feature/GEODE-217
Commit: d58588a44a536c12adf0c5dcb64beda06b18e1f3
Parents: ec9d16a
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon Dec 14 15:14:15 2015 -0800
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Dec 14 15:14:15 2015 -0800

----------------------------------------------------------------------
 .../gemfire/management/CacheManagementDUnitTest.java     | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d58588a4/gemfire-core/src/test/java/com/gemstone/gemfire/management/CacheManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/CacheManagementDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/CacheManagementDUnitTest.java
index 19fa5d6..b9762a4 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/CacheManagementDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/CacheManagementDUnitTest.java
@@ -23,6 +23,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
 
 import javax.management.InstanceNotFoundException;
 import javax.management.JMException;
@@ -44,6 +46,8 @@ import com.gemstone.gemfire.management.internal.ManagementConstants;
 import com.gemstone.gemfire.management.internal.SystemManagementService;
 import com.gemstone.gemfire.management.internal.NotificationHub.NotificationHubListener;
 
+import static com.jayway.awaitility.Awaitility.*;
+import static org.hamcrest.Matchers.*;
 import dunit.AsyncInvocation;
 import dunit.DistributedTestCase;
 import dunit.SerializableRunnable;
@@ -938,10 +942,9 @@ public class CacheManagementDUnitTest extends ManagementTestBase {
         // Even though we got 15 notification only 10 should be there due to
         // eviction attributes set in notification region
 
-        assertEquals(10, member1NotifRegion.size());
-        assertEquals(10, member2NotifRegion.size());
-        assertEquals(10, member3NotifRegion.size());
-
+        waitAtMost(5, TimeUnit.SECONDS).untilCall(to(member1NotifRegion).size(), equalTo(10));
+        waitAtMost(5, TimeUnit.SECONDS).untilCall(to(member2NotifRegion).size(), equalTo(10));
+        waitAtMost(5, TimeUnit.SECONDS).untilCall(to(member3NotifRegion).size(), equalTo(10));
       }
     });
 


[14/35] incubator-geode git commit: implementation of AvailablePort.isPortAvailable for multicast

Posted by kl...@apache.org.
implementation of AvailablePort.isPortAvailable for multicast

The AvailablePort utility has not had support for finding a free multicast
port since feature/GEODE-77 was merged to the develop branch.  This change-
set enables the old multicast code in AvailablePort and implements the
check & response for its datagrams in JGroupsMessenger.  The JGroups
transport method for receiving packets is overridden to look for these
datagrams and respond to them.

I also renamed AvailablePort.JGroups to AvailablePort.MULTICAST, which
describes the kind of port that's being tested a little better.


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

Branch: refs/heads/feature/GEODE-217
Commit: b826b4b0077dc159cbcee68775cc197b025d5586
Parents: 5d1f38e
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Dec 18 08:31:52 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri Dec 18 08:31:52 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/messenger/Transport.java     |  26 +++++
 .../gemfire/internal/AvailablePort.java         | 103 +++++++++----------
 .../distributed/DistributedSystemDUnitTest.java |   6 +-
 .../gemfire/distributed/LocatorDUnitTest.java   |   2 +-
 .../messenger/JGroupsMessengerJUnitTest.java    |   4 +-
 .../gemfire/internal/AvailablePortHelper.java   |   2 +-
 .../internal/AvailablePortJUnitTest.java        |   2 +
 7 files changed, 86 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
index 1687261..fbb4b28 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
@@ -138,5 +138,31 @@ public class Transport extends UDP {
     }
   }
 
+  // overridden to implement AvailablePort response
+  @Override
+  public void receive(Address sender, byte[] data, int offset, int length, boolean copy_buffer) {
+    if(data == null) return;
+
+    // drop message from self; it has already been looped back up (https://issues.jboss.org/browse/JGRP-1765)
+    if(local_physical_addr != null && local_physical_addr.equals(sender))
+        return;
+
+    if (length-offset == 4
+        && data[offset] == 'p'
+        && data[offset+1] == 'i'
+        && data[offset+2] == 'n'
+        && data[offset+3] == 'g') {
+      // AvailablePort check
+      data[offset+1] = 'o';
+      try {
+        sendToSingleMember(sender, data, offset, length);
+      } catch (Exception e) {
+        log.fatal("Unable to respond to available-port check", e);
+      }
+      return;
+    }
+
+    super.receive(sender,  data,  offset,  length, copy_buffer);
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
index c4907ad..93c6bc2 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
@@ -35,8 +35,8 @@ public class AvailablePort {
   /** Is the port available for a Socket (TCP) connection? */
   public static final int SOCKET = 0;
 
-  /** Is the port available for a JGroups (UDP) connection */
-  public static final int JGROUPS = 1;
+  /** Is the port available for a JGroups (UDP) multicast connection */
+  public static final int MULTICAST = 1;
 
   ///////////////////////  Static Methods  ///////////////////////
   
@@ -50,7 +50,7 @@ public class AvailablePort {
       if (protocol == SOCKET) {
         name = System.getProperty("gemfire.bind-address");
       }
-      else if (protocol == JGROUPS) {
+      else if (protocol == MULTICAST) {
         name = System.getProperty("gemfire.mcast-address");
       }
       if (name != null) {
@@ -72,7 +72,7 @@ public class AvailablePort {
    *        The port to check
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    *
    * @throws IllegalArgumentException
    *         <code>protocol</code> is unknown
@@ -90,7 +90,7 @@ public class AvailablePort {
    *        The port to check
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    * @param addr the bind address (or mcast address) to use
    *
    * @throws IllegalArgumentException
@@ -106,47 +106,44 @@ public class AvailablePort {
       }
     }
     
-    else if (protocol == JGROUPS) {
+    else if (protocol == MULTICAST) {
       DatagramSocket socket = null;
       try {
-        // TODO - need to find out if anyone is listening on this port
-        return true;
-
-//        socket = new MulticastSocket();
-//        socket.setSoTimeout(Integer.getInteger("AvailablePort.timeout", 2000).intValue());
-//        byte[] buffer = new byte[4];
-//        buffer[0] = (byte)'p';
-//        buffer[1] = (byte)'i';
-//        buffer[2] = (byte)'n';
-//        buffer[3] = (byte)'g';
-//        SocketAddress mcaddr = new InetSocketAddress(
-//          addr==null? DistributionConfig.DEFAULT_MCAST_ADDRESS : addr, port);
-//        DatagramPacket packet = new DatagramPacket(buffer, 0, buffer.length, mcaddr);
-//        socket.send(packet);
-//        try {
-//          socket.receive(packet);
-//          packet.getData();  // make sure there's data, but no need to process it
-//          return false;
-//        }
-//        catch (SocketTimeoutException ste) {
-//          //System.out.println("socket read timed out");
-//          return true;
-//        }
-//        catch (Exception e) {
-//          e.printStackTrace();
-//          return false;
-//        }
-//      }
-//      catch (java.io.IOException ioe) {
-//        if (ioe.getMessage().equals("Network is unreachable")) {
-//          throw new RuntimeException(LocalizedStrings.AvailablePort_NETWORK_IS_UNREACHABLE.toLocalizedString(), ioe);
-//        }
-//        ioe.printStackTrace();
-//        return false;
-//      }
-//      catch (Exception e) {
-//        e.printStackTrace();
-//        return false;
+        socket = new MulticastSocket();
+        socket.setSoTimeout(Integer.getInteger("AvailablePort.timeout", 2000).intValue());
+        byte[] buffer = new byte[4];
+        buffer[0] = (byte)'p';
+        buffer[1] = (byte)'i';
+        buffer[2] = (byte)'n';
+        buffer[3] = (byte)'g';
+        SocketAddress mcaddr = new InetSocketAddress(
+          addr==null? DistributionConfig.DEFAULT_MCAST_ADDRESS : addr, port);
+        DatagramPacket packet = new DatagramPacket(buffer, 0, buffer.length, mcaddr);
+        socket.send(packet);
+        try {
+          socket.receive(packet);
+          packet.getData();  // make sure there's data, but no need to process it
+          return false;
+        }
+        catch (SocketTimeoutException ste) {
+          //System.out.println("socket read timed out");
+          return true;
+        }
+        catch (Exception e) {
+          e.printStackTrace();
+          return false;
+        }
+      }
+      catch (java.io.IOException ioe) {
+        if (ioe.getMessage().equals("Network is unreachable")) {
+          throw new RuntimeException(LocalizedStrings.AvailablePort_NETWORK_IS_UNREACHABLE.toLocalizedString(), ioe);
+        }
+        ioe.printStackTrace();
+        return false;
+      }
+      catch (Exception e) {
+        e.printStackTrace();
+        return false;
       }
       finally {
         if (socket != null) {
@@ -173,7 +170,7 @@ public class AvailablePort {
       } else {
         return keepOneInterface(addr, port);
       }
-    } else if (protocol == JGROUPS) {
+    } else if (protocol == MULTICAST) {
       throw new IllegalArgumentException("You can not keep the JGROUPS protocol");
     } else {
       throw new IllegalArgumentException(LocalizedStrings.AvailablePort_UNKNOWN_PROTOCOL_0.toLocalizedString(Integer.valueOf(protocol)));
@@ -297,7 +294,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    *
    * @throws IllegalArgumentException
    *         <code>protocol</code> is unknown
@@ -314,7 +311,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    *
    * @throws IllegalArgumentException
    *         <code>protocol</code> is unknown
@@ -329,7 +326,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    *
    * @throws IllegalArgumentException
    *         <code>protocol</code> is unknown
@@ -345,7 +342,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    * @param addr the bind-address or mcast address to use
    *
    * @throws IllegalArgumentException
@@ -356,7 +353,7 @@ public class AvailablePort {
       int port = getRandomWildcardBindPortNumber();
       if (isPortAvailable(port, protocol, addr)) {
         // don't return the products default multicast port
-        if ( !(protocol == JGROUPS && port == DistributionConfig.DEFAULT_MCAST_PORT) ){
+        if ( !(protocol == MULTICAST && port == DistributionConfig.DEFAULT_MCAST_PORT) ){
           return port;
         }
       }
@@ -377,7 +374,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    * @param addr the bind-address or mcast address to use
    *
    * @throws IllegalArgumentException
@@ -399,7 +396,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    * @param addr the bind-address or mcast address to use
    *
    * @throws IllegalArgumentException
@@ -543,7 +540,7 @@ public class AvailablePort {
 
     } else if (protocolString.equalsIgnoreCase("javagroups") ||
       protocolString.equalsIgnoreCase("jgroups")) {
-      protocol = JGROUPS;
+      protocol = MULTICAST;
 
     } else {
       usage("Unknown protocol: " + protocolString);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
index a3ff8df..9f1e90b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
@@ -317,7 +317,7 @@ public class DistributedSystemDUnitTest extends DistributedTestCase {
   // TODO this needs to use a locator
   public void _testMembershipPortRange() throws Exception {
     Properties config = new Properties();
-    int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+    int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
     int unicastPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     config.setProperty("mcast-port", String.valueOf(mcastPort));
     config.setProperty("locators", "");
@@ -337,7 +337,7 @@ public class DistributedSystemDUnitTest extends DistributedTestCase {
   // TODO this needs to use a locator
   public void _testMembershipPortRangeWithExactThreeValues() throws Exception {
     Properties config = new Properties();
-    int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+    int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
     config.setProperty("mcast-port", String.valueOf(mcastPort));
     config.setProperty("locators", "");
     config.setProperty(DistributionConfig.MEMBERSHIP_PORT_RANGE_NAME, ""
@@ -358,7 +358,7 @@ public class DistributedSystemDUnitTest extends DistributedTestCase {
   // TODO this needs to use a locator
   public void _testConflictingUDPPort() throws Exception {
     final Properties config = new Properties();
-    final int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+    final int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
     final int unicastPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     config.setProperty("mcast-port", String.valueOf(mcastPort));
     config.setProperty("locators", "");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/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 7d34ab1..80f527b 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
@@ -1454,7 +1454,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
     final int port2 = freeTCPPorts[1];
     this.port2 = port2;
     deleteLocatorStateFile(port1, port2);
-    final int mcastport = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+    final int mcastport = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
     
     final String host0 = getServerHostName(host); 
     final String locators = host0 + "[" + port1 + "]," +

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
index c844583..9d3dbac 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
@@ -68,6 +68,7 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinReq
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinResponseMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.LeaveRequestMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messenger.JGroupsMessenger.JGroupsReceiver;
+import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
@@ -812,8 +813,9 @@ public class JGroupsMessengerJUnitTest {
   public void testMulticastTest() throws Exception {
     initMocks(true);
     boolean result = messenger.testMulticast(50);
-    // this shouldln't succeed
+    // this shouldln't succeed because there's no-one to respond
     assertFalse(result);
+    assertFalse(AvailablePort.isPortAvailable(services.getConfig().getDistributionConfig().getMcastPort(), AvailablePort.MULTICAST));
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortHelper.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortHelper.java
index 5230ff9..246e6c7 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortHelper.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortHelper.java
@@ -110,7 +110,7 @@ public class AvailablePortHelper {
   public static int[] getRandomAvailableUDPPorts(int count) {
     Set set = new HashSet();
     while (set.size() < count) {
-      int port = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+      int port = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
       set.add(new Integer(port));
     }
     int[] ports = new int[set.size()];

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
index 7244822..6ba91b8 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
@@ -69,4 +69,6 @@ public class AvailablePortJUnitTest {
     socket.bind(new InetSocketAddress((InetAddress)null, port));
     assertFalse(AvailablePort.isPortAvailable(port, AvailablePort.SOCKET));
   }
+  
+  // multicast availability is tested in JGroupsMessengerJUnitTest
 }


[25/35] 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 kl...@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();
+    }
+  }
+
+}


[30/35] incubator-geode git commit: GEODE-543: Updating rat.gradle

Posted by kl...@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-217
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',


[23/35] 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 kl...@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);
+}


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

Posted by kl...@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-217
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


[15/35] incubator-geode git commit: GEODE-608: Update rat excludes to ignore binary files

Posted by kl...@apache.org.
GEODE-608: Update rat excludes to ignore binary files

Add png, jpg, gif, dia, pdf, keystore, truststore, and ico
to the excludes list.


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

Branch: refs/heads/feature/GEODE-217
Commit: 0851f186074be4265d8088e3a5e1969c85eed635
Parents: b826b4b
Author: Anthony Baker <ab...@pivotal.io>
Authored: Fri Dec 18 11:59:54 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Fri Dec 18 12:07:14 2015 -0800

----------------------------------------------------------------------
 gradle/rat.gradle | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0851f186/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 63eff0b..2068b88 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -54,9 +54,16 @@ rat {
 
     // binary files
     '**/*.cer',
+    '**/*.dia',
     '**/*.gfs',
-    '**/keystore',
+    '**/*.gif',
+    '**/*.ico',
+    '**/*.jpg',
+    '**/*.keystore',
+    '**/*.pdf',
+    '**/*.png',
     '**/*.ser',
+    '**/*.truststore',
     '**/*.xls',
 
     // other text files


[22/35] 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 kl...@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',
 


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

Posted by kl...@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-217
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;


[24/35] 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 kl...@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());
+  }
+
+}


[06/35] incubator-geode git commit: GEODE-338 FDDUnitTest. testFDSocketFixOnlyServers The issue this test was trying to reproduce no longer appears to be affecting the latest jvm Removing this test from the code base Created GEODE-679 to track removal of

Posted by kl...@apache.org.
GEODE-338 FDDUnitTest. testFDSocketFixOnlyServers
The issue this test was trying to reproduce no longer appears to be affecting the latest jvm
Removing this test from the code base
Created GEODE-679 to track removal of private selector pool code


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

Branch: refs/heads/feature/GEODE-217
Commit: d40d8a70d084d963530cc132c9004cf39b1b3a76
Parents: e63ad64
Author: Jason Huynh <hu...@gmail.com>
Authored: Tue Dec 15 14:58:58 2015 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Dec 15 15:11:24 2015 -0800

----------------------------------------------------------------------
 .../gemstone/gemfire/internal/FDDUnitTest.java  | 282 -------------------
 1 file changed, 282 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d40d8a70/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
deleted file mode 100644
index f1ce420..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal;
-
-import java.util.Properties;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.PartitionAttributesFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionFactory;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.client.ClientCache;
-import com.gemstone.gemfire.cache.client.ClientCacheFactory;
-import com.gemstone.gemfire.cache.client.ClientRegionFactory;
-import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
-import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
-import com.gemstone.gemfire.cache30.CacheTestCase;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
-import com.gemstone.gemfire.internal.stats50.VMStats50;
-
-import dunit.Host;
-import dunit.SerializableCallable;
-import dunit.SerializableRunnable;
-import dunit.VM;
-
-/**
- * 
- * @author jhuynh
- *
- */
-public class FDDUnitTest extends CacheTestCase {
-
-  VM vm0;
-  VM vm1;
-  VM vm2;
-
-  
-  public FDDUnitTest(String name) {
-    super(name);
-  }
-
-  public void setUp() throws Exception {
-    super.setUp();
-    //getSystem();
-   
-    Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-    vm1 = host.getVM(1);
-    vm2 = host.getVM(2);
-  }
-  
-  public void tearDown2() throws Exception {
-    vm0.invoke(cleanup());
-    vm1.invoke(cleanup());
-    vm2.invoke(cleanup());
-    vm0.invoke(CacheServerTestUtil.class, "closeCache");
-    vm1.invoke(CacheServerTestUtil.class, "closeCache");
-    vm2.invoke(CacheServerTestUtil.class, "closeCache");
-  }
-
-  public void testEmpty() {
-    //Ticket #GEODE-338.  Disable the test for now and rewrite as a junit test.
-  }
-  
-  public void disable_testFDSocketFixOnlyServers() throws Exception {
-    String os = System.getProperty("os.name");
-    if (os != null) {
-      if (os.indexOf("Windows") != -1) {
-        System.out.println("This test is disabled on Windows");
-        //we are running this test on windows.  fd stats are not available in windows so let's
-        //just not run this test
-        return;
-      }
-    }
-    try {
-      StringBuffer incaseOfFailure = new StringBuffer();
-      final int[] port = AvailablePortHelper.getRandomAvailableTCPPorts(3);
-      int numThreads = 30;
-
-      startCacheServer(vm0, port[0]);
-      startCacheServer(vm1, port[1]);
-      startCacheServer(vm2, port[2]);
-
-      createRegion(vm0, "portfolios", RegionShortcut.PARTITION_REDUNDANT);
-      createRegion(vm1, "portfolios", RegionShortcut.PARTITION_REDUNDANT);
-      createRegion(vm2, "portfolios", RegionShortcut.PARTITION_REDUNDANT);
-
-      // run test without selector pooling
-      setUseSelectorPooling(vm0, false);
-      long startingFDs = checkFD(vm0);
-      doPuts(vm0, numThreads, "portfolios");
-      long endFDs = checkFD(vm0);
-      long numFDs = endFDs - startingFDs;
-      incaseOfFailure.append("NoSelectorPooling startFDs: " + startingFDs + " endFDs: " + endFDs + " diff:" + numFDs + " ");
-
-      // run test with selector pooling
-      setUseSelectorPooling(vm0, true);
-      long startingFDsWithPooling = checkFD(vm0);
-      doPuts(vm0, numThreads, "portfolios");
-      long endFDsWithPooling = checkFD(vm0);
-      long numFDsWithPooling = endFDsWithPooling - startingFDsWithPooling;
-      incaseOfFailure.append("SelectorPooling#1 startFDs: " + startingFDsWithPooling + " endFDs: " + endFDsWithPooling + " diff:" + numFDsWithPooling + " ");
-      assertTrue(incaseOfFailure.toString(), numFDsWithPooling < numFDs);
-
-      // run it again and see if the number still is below
-      startingFDsWithPooling = checkFD(vm0);
-      doPuts(vm0, numThreads, "portfolios");
-      endFDsWithPooling = checkFD(vm0);
-      numFDsWithPooling = endFDsWithPooling - startingFDsWithPooling;
-      // if you see these asserts failing, it could be that we are not using the
-      // selector pool
-      incaseOfFailure.append("SelectorPooling#2 startFDs: " + startingFDsWithPooling + " endFDs: " + endFDsWithPooling + " diff:" + numFDsWithPooling + " ");
-      assertTrue(incaseOfFailure.toString(), numFDsWithPooling < numFDs);
-
-    } finally {
-      setUseSelectorPooling(vm0, true);
-    }
-
-  }
-  
-  private void setUseSelectorPooling(VM vm, final boolean useSelectorPooling) {
-    vm.invoke(new SerializableRunnable("setting use selectorPooling to " + useSelectorPooling) {
-      public void run() {
-        SocketUtils.USE_SELECTOR_POOLING = useSelectorPooling;
-      }
-    });
-  }
-  private Long checkFD(VM vm) {
-    return (Long)vm.invoke(new SerializableCallable() {
-      public Object call() throws Exception {
-        VMStatsContract stats = ((InternalDistributedSystem)system).getStatSampler().getVMStats();
-        VMStats50 vmstats = (VMStats50) stats;
-        return vmstats.getVMStats().get("fdsOpen").longValue();
-      }
-    });
-  }
-  
-  private void doPuts(VM vm, final int numThreads, final String regionName) {
-    vm.invoke(new SerializableCallable() {
-      public Object call() throws Exception {
-        final Region region = getCache().getRegion(regionName);
-        if (region == null) {
-          throw new Exception("No Region found");
-        }
-        ExecutorService executor = Executors.newFixedThreadPool(numThreads);
-          for (int i = 0; i < numThreads; i++) {
-            executor.execute(new Runnable() {
-              public void run() {
-                  for (int i = 0; i < 10; i++) {
-                    String myValue = "string" + i;
-                    region.put("k" + i, myValue);
-                    try {
-                      Thread.sleep(75); 
-                    }
-                    catch (InterruptedException e) {
-                      Thread.currentThread().interrupt();
-                    }
-                    
-                  }
-              }
-            });
-          }
-        
-          executor.shutdown();
-       return executor.awaitTermination(90, TimeUnit.SECONDS);
-      }
-    });
-  }
-  
-  private String createGarbage(int valueIndex) {
-    StringBuffer[] randomStringArray = new StringBuffer[100];
-    for (int i = 0; i < randomStringArray.length; i++) {
-      randomStringArray[i] = new StringBuffer();
-      randomStringArray[i].append("value" + valueIndex + "," + Math.random());
-    }
-    return randomStringArray[(int)(Math.random() * randomStringArray.length)].toString();
-  }
- 
-  private void createRegion(VM vm, final String regionName, final RegionShortcut shortcut) {
-    vm.invoke(new SerializableCallable() {
-      public Object call() throws Exception {
-        PartitionAttributesFactory paf = new PartitionAttributesFactory();
-        RegionFactory factory = getCache().createRegionFactory(shortcut)
-            .setPartitionAttributes(paf.create());
-        factory.create(regionName);
-        return null;
-      }
-    });
-  }
-  
-  private void createRegionOnClient(VM vm, final String regionName, final ClientRegionShortcut shortcut) {
-    vm.invoke(new SerializableCallable() {
-      public Object call() throws Exception {
-        PartitionAttributesFactory paf = new PartitionAttributesFactory();
-        ClientRegionFactory factory = ((ClientCache)getCache()).createClientRegionFactory(shortcut);
-        factory.create(regionName);
-        return null;
-      }
-    });
-  }
-  
-  private void startCacheServer(VM server, final int port) throws Exception {
-    server.invoke(new SerializableCallable() {
-      public Object call() throws Exception {
-        //System.setProperty("IDLE_THREAD_TIMEOUT", "50");
-        disconnectFromDS();
-        
-        getSystem(getServerProperties());
-        
-        GemFireCacheImpl cache = (GemFireCacheImpl)getCache();
-        
-        CacheServer cacheServer = cache.addCacheServer();
-        cacheServer.setPort(port);
-        cacheServer.start();  
-        return null;
-      }
-    });
-  }
-  
-  private void startClient(VM client, final VM server, final int port) {
-    client.invoke(new CacheSerializableRunnable("Start client") {
-      public void run2() throws CacheException {
-        //System.setProperty("IDLE_THREAD_TIMEOUT", "50");
-        Properties props = getClientProps();
-        getSystem(props);
-        
-        final ClientCacheFactory ccf = new ClientCacheFactory(props);
-        ccf.addPoolServer(getServerHostName(server.getHost()), port);
-        ccf.setPoolSubscriptionEnabled(true);
-        
-        ClientCache cache = (ClientCache)getClientCache(ccf);
-      }
-    });
-  }
-  
-  private Runnable cleanup() {
-    return new SerializableRunnable() {
-      public void run() {
-        //System.setProperty("IDLE_THREAD_TIMEOUT", "30000*60");
-      }
-    };
-  }
- 
-  protected Properties getClientProps() {
-    Properties p = new Properties();
-    p.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    p.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    p.setProperty(DistributionConfig.CONSERVE_SOCKETS_NAME, "false");
-    return p;
-  }
-
-  protected Properties getServerProperties() {
-    Properties p = new Properties();
-    p.setProperty(DistributionConfig.LOCATORS_NAME, "localhost["+getDUnitLocatorPort()+"]");
-    p.setProperty(DistributionConfig.CONSERVE_SOCKETS_NAME, "false");
-    //p.setProperty(DistributionConfig.SOCKET_LEASE_TIME_NAME, "120000");
-    return p;
-  }
- 
-
-}


[11/35] incubator-geode git commit: GEODE-678: dlock reenter will no longer timeout

Posted by kl...@apache.org.
GEODE-678: dlock reenter will no longer timeout


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

Branch: refs/heads/feature/GEODE-217
Commit: d31a85ff7898732857d5fd1abee8030c8beeca66
Parents: 0e366b0
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Dec 15 15:36:24 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 17 14:18:23 2015 -0800

----------------------------------------------------------------------
 .../internal/locks/DLockGrantor.java            |  5 +-
 .../locks/DLockReentrantLockJUnitTest.java      | 84 ++++++++++++++++++++
 2 files changed, 85 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d31a85ff/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
index b4e646c..a5b8e0f 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
@@ -984,10 +984,7 @@ public class DLockGrantor {
     }
     try {
       checkDestroyed();
-      if (request.checkForTimeout()) {
-        // no cleanup here because we bypassed lock permissions
-        return 0;
-      }
+      // to fix GEODE-678 no longer call request.checkForTimeout
       DLockGrantToken grant = getGrantToken(request.getObjectName());
       if (grant == null) {
         if (logger.isTraceEnabled(LogMarker.DLS)) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d31a85ff/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java
new file mode 100644
index 0000000..2dbd231
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed.internal.locks;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+
+@Category(IntegrationTest.class)
+public class DLockReentrantLockJUnitTest {
+  final long id = 1213L;
+
+  private Cache cache;
+  private Region<Long, String> region;
+
+  @Before
+  public void setup() {
+    cache = new CacheFactory().set("mcast-port", "0").set("locators", "").create();
+    final RegionFactory<Long, String> regionFactory = cache.createRegionFactory("REPLICATE");
+    regionFactory.setScope(Scope.GLOBAL);
+    region = regionFactory.create("ReentrantLockRegion");
+    region.put(id, new String("TestValue1"));
+  }
+
+  @After
+  public void tearDown() {
+    cache.close();
+  }
+
+  /**
+   * Tests GEM-96/GEODE-678
+   */
+  @Test
+  public void testReentrantLock() throws Exception {
+
+    Assert.assertEquals(Scope.GLOBAL, region.getAttributes().getScope());
+
+    final Lock lock1 = region.getDistributedLock(id);
+    final Lock lock2 = region.getDistributedLock(id);
+
+    for (int i = 0; i < 50; i++) {
+      lock1.lock();
+      boolean reenteredLock = false;
+      try {
+        reenteredLock = lock2.tryLock(1, TimeUnit.NANOSECONDS);
+        if (!reenteredLock) {
+          System.out.println("ERROR: could not reenter lock");
+        }
+        Assert.assertTrue("Failed getting lock at 2:" + i, reenteredLock);
+      } finally {
+        if (reenteredLock) {
+          lock2.unlock();
+        }
+        lock1.unlock();
+      }
+    }
+  }
+}


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

Posted by kl...@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;
-  }
-
-}


[10/35] incubator-geode git commit: Merge branch 'feature/GEODE-568' into develop

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

Closes #59


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

Branch: refs/heads/feature/GEODE-217
Commit: 0e366b0263eefc685d0e4f04122ae165b5088afe
Parents: f2a852d d58588a
Author: Jens Deppe <jd...@pivotal.io>
Authored: Thu Dec 17 10:34:57 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Thu Dec 17 10:34:57 2015 -0800

----------------------------------------------------------------------
 .../gemfire/management/CacheManagementDUnitTest.java     | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[26/35] 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 kl...@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-217
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);
+  }
+
+}


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

Posted by kl...@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-217
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;
+  }
+
+}


[31/35] incubator-geode git commit: GEODE-299

Posted by kl...@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-217
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");
             }
           }
         });


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

Posted by kl...@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-217
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/35] incubator-geode git commit: GEODE-667: Update system-rules dependency from 1.12.1 to 1.15.0

Posted by kl...@apache.org.
GEODE-667: Update system-rules dependency from 1.12.1 to 1.15.0


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

Branch: refs/heads/feature/GEODE-217
Commit: 7c368e9553bac0ee7d5d31e252912480346f2f39
Parents: c30e533
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Dec 14 09:32:30 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Dec 15 14:07:06 2015 -0800

----------------------------------------------------------------------
 gradle/dependency-versions.properties | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7c368e95/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 684e60e..3833f54 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -79,4 +79,4 @@ stephenc-findbugs.version = 1.3.9-1
 spymemcached.version = 2.9.0
 swagger.version = 1.3.2
 swagger-springmvc.version = 0.8.2
-system-rules.version = 1.12.1
+system-rules.version = 1.15.0


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

Posted by kl...@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-217
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);


[05/35] incubator-geode git commit: GEODE-668 Improve GMSHealthMonitor logs to help debugging

Posted by kl...@apache.org.
GEODE-668 Improve GMSHealthMonitor logs to help debugging


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

Branch: refs/heads/feature/GEODE-217
Commit: e63ad64b693c10f3de2809697a1b60eef8514e38
Parents: df32ad1
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Tue Dec 15 14:51:31 2015 -0800
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Tue Dec 15 14:51:31 2015 -0800

----------------------------------------------------------------------
 .../internal/membership/gms/fd/GMSHealthMonitor.java      | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e63ad64b/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 8c38ba6..2aac935 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
@@ -514,7 +514,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
         GMSMember gmbr = (GMSMember) suspectMember.getNetMember();
         writeMemberToStream(gmbr, out);
         clientSocket.shutdownOutput();
-        logger.debug("Connected - reading response", suspectMember);
+        logger.debug("Connected - reading response from suspect member {}", suspectMember);
         int b = in.read();
         logger.debug("Received {}", (b == OK ? "OK" : (b == ERROR ? "ERROR" : b)), suspectMember);
         if (b == OK) {
@@ -531,7 +531,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
         return false;
       }
     } catch (SocketTimeoutException e) {
-      logger.debug("tcp/ip connection timed out");
+      logger.debug("Final check TCP/IP connection timed out for suspect member {}", suspectMember);
       return false;
     } catch (IOException e) {
       logger.trace("Unexpected exception", e);
@@ -1201,15 +1201,15 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
               if (!pinged && !isStopping) {
                 TimeStamp ts = memberTimeStamps.get(mbr);
                 if (ts == null || ts.getTime() <= startTime) {
-                  logger.info("Final check failed - requesting removal");
+                  logger.info("Final check failed - requesting removal of suspect member " + mbr);
                   services.getJoinLeave().remove(mbr, reason);
                   failed = true;
                 } else {
-                  logger.info("check failed but detected recent message traffic");
+                  logger.info("Final check failed but detected recent message traffic for suspect member " + mbr);
                 }
               }
               if (!failed) {
-                logger.info("Final check passed");
+                logger.info("Final check passed for suspect member " + mbr);
               }
               // whether it's alive or not, at this point we allow it to
               // be watched again


[12/35] incubator-geode git commit: GEODE-680: refactor Chunk to cleanup unit tests 1) Promoted constant fields to default access level. 2) Modified getAddressForReading to delegate to getUnsafeAddress.

Posted by kl...@apache.org.
GEODE-680: refactor Chunk to cleanup unit tests
1) Promoted constant fields to default access level.
2) Modified getAddressForReading to delegate to getUnsafeAddress.


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

Branch: refs/heads/feature/GEODE-217
Commit: bb794ee205400ac4e5655cc1e6410889d29f28b1
Parents: d31a85f
Author: Sai Boorlagadda <sb...@pivotal.io>
Authored: Wed Dec 16 13:03:19 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 17 14:25:44 2015 -0800

----------------------------------------------------------------------
 .../gemfire/internal/offheap/Chunk.java         | 26 ++++++++++----------
 .../internal/offheap/GemFireChunkJUnitTest.java | 23 ++++++++---------
 2 files changed, 23 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb794ee2/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
index ed4bc43..4d1252d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
@@ -74,21 +74,21 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
      * a delta whose max value would be HUGE_MULTIPLE-1.
      * The lower two bytes contains the use count.
      */
-    private final static int REF_COUNT_OFFSET = 4;
+    final static int REF_COUNT_OFFSET = 4;
     /**
      * The upper two bits are used for the isSerialized
      * and isCompressed flags.
      */
-    private final static int IS_SERIALIZED_BIT =    0x80000000;
-    private final static int IS_COMPRESSED_BIT =    0x40000000;
-    private final static int SRC_TYPE_MASK = 0x38000000;
-    private final static int SRC_TYPE_SHIFT = 16/*refCount*/+8/*dataSize*/+3/*magicSize*/;
-    private final static int MAGIC_MASK = 0x07000000;
-    private final static int MAGIC_NUMBER = 0x05000000;
-    private final static int DATA_SIZE_DELTA_MASK = 0x00ff0000;
-    private final static int DATA_SIZE_SHIFT = 16;
-    private final static int REF_COUNT_MASK =       0x0000ffff;
-    private final static int MAX_REF_COUNT = 0xFFFF;
+    final static int IS_SERIALIZED_BIT =    0x80000000;
+    final static int IS_COMPRESSED_BIT =    0x40000000;
+    final static int SRC_TYPE_MASK = 0x38000000;
+    final static int SRC_TYPE_SHIFT = 16/*refCount*/+8/*dataSize*/+3/*magicSize*/;
+    final static int MAGIC_MASK = 0x07000000;
+    final static int MAGIC_NUMBER = 0x05000000;
+    final static int DATA_SIZE_DELTA_MASK = 0x00ff0000;
+    final static int DATA_SIZE_SHIFT = 16;
+    final static int REF_COUNT_MASK =       0x0000ffff;
+    final static int MAX_REF_COUNT = 0xFFFF;
     final static long FILL_PATTERN = 0x3c3c3c3c3c3c3c3cL;
     final static byte FILL_BYTE = 0x3c;
     
@@ -376,8 +376,8 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
     }
 
     public long getAddressForReading(int offset, int size) {
-      assert offset+size <= getDataSize();
-      return getBaseDataAddress() + offset;
+      //delegate to getUnsafeAddress - as both the methods does return the memory address from given offset
+      return getUnsafeAddress(offset, size);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb794ee2/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
index 20cb8dc..81a59e3 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
@@ -255,7 +255,7 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
 
-    int headerBeforeSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+    int headerBeforeSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + Chunk.REF_COUNT_OFFSET);
 
     assertThat(chunk.isSerialized()).isFalse();
 
@@ -263,9 +263,9 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     assertThat(chunk.isSerialized()).isTrue();
 
-    int headerAfterSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+    int headerAfterSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + Chunk.REF_COUNT_OFFSET);
 
-    assertThat(headerAfterSerializedBitSet).isEqualTo(headerBeforeSerializedBitSet | 0x80000000/* IS_SERIALIZED_BIT */);
+    assertThat(headerAfterSerializedBitSet).isEqualTo(headerBeforeSerializedBitSet | Chunk.IS_SERIALIZED_BIT);
 
     chunk.release();
   }
@@ -289,7 +289,7 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
 
-    int headerBeforeCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+    int headerBeforeCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + Chunk.REF_COUNT_OFFSET);
 
     assertThat(chunk.isCompressed()).isFalse();
 
@@ -297,9 +297,9 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     assertThat(chunk.isCompressed()).isTrue();
 
-    int headerAfterCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+    int headerAfterCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + Chunk.REF_COUNT_OFFSET);
 
-    assertThat(headerAfterCompressedBitSet).isEqualTo(headerBeforeCompressedBitSet | 0x40000000/* IS_SERIALIZED_BIT */);
+    assertThat(headerAfterCompressedBitSet).isEqualTo(headerBeforeCompressedBitSet | Chunk.IS_COMPRESSED_BIT);
 
     chunk.release();
   }
@@ -730,11 +730,8 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
   public void retainShouldThrowExceptionAfterMaxNumberOfTimesRetained() {
     GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
 
-    // max retain Chunk.MAX_REF_COUNT
-    int MAX_REF_COUNT = 0xFFFF;
-
     // loop though and invoke retain for MAX_REF_COUNT-1 times, as create chunk above counted as one reference
-    for (int i = 0; i < MAX_REF_COUNT - 1; i++)
+    for (int i = 0; i < Chunk.MAX_REF_COUNT - 1; i++)
       chunk.retain();
 
     // invoke for the one more time should throw exception
@@ -891,7 +888,7 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
   public void getSrcTypeOrdinalFromAddressShouldReturnOrdinal() {
     GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
 
-    assertThat(Chunk.getSrcTypeOrdinal(chunk.getMemoryAddress())).isEqualTo(4);
+    assertThat(Chunk.getSrcTypeOrdinal(chunk.getMemoryAddress())).isEqualTo(Chunk.SRC_TYPE_GFE >> Chunk.SRC_TYPE_SHIFT);
 
     chunk.release();
   }
@@ -900,8 +897,8 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
   public void getSrcTypeOrdinalFromRawBitsShouldReturnOrdinal() {
     GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
 
-    int rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4 /* REF_COUNT_OFFSET */);
-    assertThat(Chunk.getSrcTypeOrdinalFromRawBits(rawBits)).isEqualTo(4);
+    int rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + Chunk.REF_COUNT_OFFSET);
+    assertThat(Chunk.getSrcTypeOrdinalFromRawBits(rawBits)).isEqualTo(Chunk.SRC_TYPE_GFE >> Chunk.SRC_TYPE_SHIFT);
 
     chunk.release();
   }


[19/35] 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 kl...@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-217
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/35] incubator-geode git commit: Modifying the fix for handling old client IDs

Posted by kl...@apache.org.
Modifying the fix for handling old client IDs

The previous fix for handling old client identifiers left the IDs always
thinking that the client was running version GFE_82.  The new fix makes
the client defer to its encapsulated InternalDistributedMember identifier
to determine the client's version and handles deserialization of the
encapsulated pre-GFE_90 identifier in InternalDistributedMember.fromData().

This fix has the added benefit of reducing the size of client identifiers.


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

Branch: refs/heads/feature/GEODE-217
Commit: f2a852d880d166d8b4a4a11218a38f49e6016864
Parents: fdec70b
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Thu Dec 17 08:47:10 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Thu Dec 17 08:51:31 2015 -0800

----------------------------------------------------------------------
 .../internal/direct/DirectChannel.java          |  2 +-
 .../membership/InternalDistributedMember.java   | 13 ++++-
 .../internal/membership/NetMember.java          |  4 ++
 .../internal/membership/gms/GMSMember.java      |  7 +--
 .../tier/sockets/ClientProxyMembershipID.java   | 15 ++----
 .../internal/cache/tier/sockets/HandShake.java  | 16 +++----
 .../client/ClientCacheFactoryJUnitTest.java     | 50 ++++++++++++++++++++
 .../sanctionedDataSerializables.txt             | 12 ++---
 8 files changed, 89 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
index d4df3bf..a4245a0 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
@@ -596,7 +596,7 @@ public class DirectChannel {
           logger.trace(LogMarker.DM, "Not a member: {}", destination);
         }
         if (ce == null) ce = new ConnectExceptions();
-        ce.addFailure(destination, new ShunnedMemberException(LocalizedStrings.DirectChannel_SHUNNING_0.toLocalizedString()));
+        ce.addFailure(destination, new ShunnedMemberException(LocalizedStrings.DirectChannel_SHUNNING_0.toLocalizedString(destination)));
       }
       else {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
index b112b92..a77c211 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
@@ -18,6 +18,7 @@ package com.gemstone.gemfire.distributed.internal.membership;
 
 import java.io.DataInput;
 import java.io.DataOutput;
+import java.io.EOFException;
 import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
@@ -1023,7 +1024,11 @@ public final class InternalDistributedMember
     // just in case this is just a non-versioned read
     // from a file we ought to check the version
     if (this.version >= Version.GFE_90.ordinal()) {
-      netMbr.readAdditionalData(in);
+      try {
+        netMbr.readAdditionalData(in);
+      } catch (EOFException e) {
+        // nope - it's from a pre-GEODE client or WAN site
+      }
     }
   }
   
@@ -1254,6 +1259,12 @@ public final class InternalDistributedMember
       sb.append(hostname);
     }
   }*/
+  
+  public final void setVersionObjectForTest(Version v) {
+    this.version = v.ordinal();
+    this.versionObj = v;
+    netMbr.setVersion(v);
+  }
 
   public final Version getVersionObject() {
     return this.versionObj;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetMember.java
index 6811073..128f96c 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetMember.java
@@ -21,6 +21,8 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.net.InetAddress;
 
+import com.gemstone.gemfire.internal.Version;
+
 /**
  * This is the SPI for the basic element of membership provided in the
  * GemFire system.
@@ -68,6 +70,8 @@ public interface NetMember extends Comparable<NetMember>
   public void setPreferredForCoordinator(boolean preferred);
   
   public byte getMemberWeight();
+  
+  public void setVersion(Version v);
 
   /** write identity information not known by DistributedMember instances */
   public void writeAdditionalData(DataOutput out) throws IOException;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
index b1a4883..84ace6c 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
@@ -26,11 +26,9 @@ import org.jgroups.util.UUID;
 
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.distributed.DurableClientAttributes;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
 import com.gemstone.gemfire.distributed.internal.membership.NetMember;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
@@ -349,6 +347,10 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
   public void setVmKind(int vmKind) {
     this.vmKind = vmKind;
   }
+  
+  public void setVersion(Version v) {
+    this.versionOrdinal = v.ordinal();
+  }
 
   public void setBirthViewId(int birthViewId) {
     this.vmViewId = birthViewId;
@@ -441,7 +443,6 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
 
   @Override
   public void writeAdditionalData(DataOutput out) throws IOException {
-    // note: member weight and other GMS-only info is not included here
     out.writeLong(uuidMSBs);
     out.writeLong(uuidLSBs);
     out.write(memberWeight);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
index daa07f4..44c6152 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
@@ -39,6 +39,7 @@ import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedM
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.VersionedDataInputStream;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -101,10 +102,6 @@ public final class ClientProxyMembershipID
 
   protected int uniqueId;
   
-  //Version information is not available during the handshake
-  //see comments in HandShake.write()
-  private transient static final Version clientVersion = Version.GFE_82;
-
   // private final String proxyIDStr;
   // private final String clientIdStr ;
 
@@ -353,14 +350,13 @@ public final class ClientProxyMembershipID
 
   public void fromData(DataInput in) throws IOException, ClassNotFoundException
   {
-    
     this.identity = DataSerializer.readByteArray(in);
     this.uniqueId = in.readInt();
 //    {toString(); this.transientPort = ((InternalDistributedMember)this.memberId).getPort();}
   }
   
   public Version getClientVersion() {
-    return this.clientVersion;
+    return ((InternalDistributedMember)getDistributedMember()).getVersionObject();
   }
 
   public String getDSMembership()
@@ -405,15 +401,12 @@ public final class ClientProxyMembershipID
   public DistributedMember getDistributedMember()  {
     if (memberId == null) {      
       ByteArrayInputStream bais = new ByteArrayInputStream(identity);
-      DataInputStream dis = new VersionedDataInputStream(bais, clientVersion);
+      DataInputStream dis = new VersionedDataInputStream(bais, Version.CURRENT);
       try {
         memberId = (DistributedMember)DataSerializer.readObject(dis);
       }
       catch (Exception e) {
-        DistributedSystem ds = InternalDistributedSystem.getAnyInstance();
-        if(ds != null){
-          logger.error(LocalizedMessage.create(LocalizedStrings.ClientProxyMembershipID_UNABLE_TO_DESERIALIZE_MEMBERSHIP_ID), e);
-        }
+        logger.error(LocalizedMessage.create(LocalizedStrings.ClientProxyMembershipID_UNABLE_TO_DESERIALIZE_MEMBERSHIP_ID), e);
       }
     }
     return memberId;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
index be3a7bf..100549a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
@@ -1314,14 +1314,14 @@ public class HandShake implements ClientHandShake
       //Successful handshake for GATEWAY_TO_GATEWAY mode sets the peer version in connection
       if(communicationMode == Acceptor.GATEWAY_TO_GATEWAY  && !
           (acceptanceCode == REPLY_EXCEPTION_AUTHENTICATION_REQUIRED ||
-              acceptanceCode ==  REPLY_EXCEPTION_AUTHENTICATION_FAILED)) {
-         short wanSiteVersion = Version.readOrdinal(dis);
-         conn.setWanSiteVersion(wanSiteVersion);
-         // establish a versioned stream for the other site, if necessary         
-         if (wanSiteVersion < Version.CURRENT_ORDINAL) {
-           dis = new VersionedDataInputStream(dis, Version.fromOrdinalOrCurrent(wanSiteVersion));
-         }
-        } 
+          acceptanceCode ==  REPLY_EXCEPTION_AUTHENTICATION_FAILED)) {
+        short wanSiteVersion = Version.readOrdinal(dis);
+        conn.setWanSiteVersion(wanSiteVersion);
+        // establish a versioned stream for the other site, if necessary         
+        if (wanSiteVersion < Version.CURRENT_ORDINAL) {
+          dis = new VersionedDataInputStream(dis, Version.fromOrdinalOrCurrent(wanSiteVersion));
+        }
+      } 
 
       // No need to check for return value since DataInputStream already throws
       // EOFException in case of EOF

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest.java
index d15863b..6c19018 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest.java
@@ -20,6 +20,8 @@ package com.gemstone.gemfire.cache.client;
 import static org.junit.Assert.*;
 import static org.junit.runners.MethodSorters.*;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
 import java.io.File;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -28,20 +30,28 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Properties;
 
+import org.jgroups.util.UUID;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.FixMethodOrder;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.RegionService;
 import com.gemstone.gemfire.cache.client.internal.ProxyCache;
 import com.gemstone.gemfire.cache.client.internal.UserAttributes;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.distributed.internal.membership.gms.GMSMember;
 import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.VersionedDataInputStream;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.pdx.ReflectionBasedAutoSerializer;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
@@ -290,4 +300,44 @@ public class ClientCacheFactoryJUnitTest {
     .setPdxSerializer(new ReflectionBasedAutoSerializer())
     .create();
   }
+  
+  @Test
+  public void testOldClientIDDeserialization() throws Exception {
+    // during a HandShake a clientID is read w/o knowing the client's
+    // version
+    cc = new ClientCacheFactory().create();
+    GemFireCacheImpl gfc = (GemFireCacheImpl)cc;
+    InternalDistributedMember memberID = (InternalDistributedMember)cc.getDistributedSystem().getDistributedMember();
+    GMSMember gmsID = (GMSMember)memberID.getNetMember();
+    memberID.setVersionObjectForTest(Version.GFE_82);
+    assertEquals(Version.GFE_82, memberID.getVersionObject());
+    ClientProxyMembershipID clientID = ClientProxyMembershipID.getClientId(memberID);
+    HeapDataOutputStream out = new HeapDataOutputStream(Version.GFE_82);
+    DataSerializer.writeObject(clientID, out);
+
+    DataInputStream in = new VersionedDataInputStream(new ByteArrayInputStream(out.toByteArray()), Version.CURRENT); 
+    ClientProxyMembershipID newID = DataSerializer.readObject(in);
+    InternalDistributedMember newMemberID = (InternalDistributedMember)newID.getDistributedMember();
+    assertEquals(Version.GFE_82, newMemberID.getVersionObject());
+    assertEquals(Version.GFE_82, newID.getClientVersion());
+    GMSMember newGmsID = (GMSMember)newMemberID.getNetMember();
+    assertEquals(0, newGmsID.getUuidLSBs());
+    assertEquals(0, newGmsID.getUuidMSBs());
+    
+    gmsID.setUUID(new UUID(1234l, 5678l));
+    memberID.setVersionObjectForTest(Version.CURRENT);
+    clientID = ClientProxyMembershipID.getClientId(memberID);
+    out = new HeapDataOutputStream(Version.CURRENT);
+    DataSerializer.writeObject(clientID, out);
+
+    in = new VersionedDataInputStream(new ByteArrayInputStream(out.toByteArray()), Version.CURRENT);
+    newID = DataSerializer.readObject(in);
+    newMemberID = (InternalDistributedMember)newID.getDistributedMember();
+    assertEquals(Version.CURRENT, newMemberID.getVersionObject());
+    assertEquals(Version.CURRENT, newID.getClientVersion());
+    newGmsID = (GMSMember)newMemberID.getNetMember();
+    assertEquals(gmsID.getUuidLSBs(), newGmsID.getUuidLSBs());
+    assertEquals(gmsID.getUuidMSBs(), newGmsID.getUuidMSBs());
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
index d553ad9..d3a141a 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
@@ -329,12 +329,12 @@ fromData,16,2a2bb700172a2bb900180100b50008b1
 toData,16,2a2bb700192b2ab40008b9001a0200b1
 
 com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember,6
-fromData,29,2a2bb600992ab4001ab2008fb6006fa1000d2ab400282bb900900200b1
-fromDataPre_GFE_7_1_0_0,292,2bb8009a4d2bb9009b01003e2a2bb80087b500062ab2000399000e2c2ab40006b8009ca700072cb60007b500062bb9009d010036041504047e99000704a700040336051504057e99000704a700040336062a1504077e99000704a7000403b500382a2bb9009b0100b500102a2bb9009b0100b500022a2bb9009d0100b500122a2bb80089b5001f2a2bb80087b500172ab40012100da0000e2a2bb80087b50018a700172bb800873a071907c6000c2a1907b8009eb500132bb800873a072bb8008ab6008b36082abb008c5919071508b7008db5001dbb0032592ab400102ab400022ab400122ab400132ab400172ab4001f2ab4001db700333a092a2c1d150515062bb8009fb6006f1909b80027b500282ab7000b2ab400129e000704a7000403b80041b1
-fromDataPre_GFE_9_0_0_0,296,2bb8009a4d2bb9009b01003e2a2bb80087b500062ab2000399000e2c2ab40006b8009ca700072cb60007b500062bb9009d010036041504047e99000704a700040336051504057e99000704a700040336062a1504077e99000704a7000403b500382a2bb9009b0100b500102a2bb9009b0100b500022a2bb9009d0100b500122a2bb80089b5001f2a2bb80087b500172ab40012100da0000e2a2bb80087b50018a700172bb800873a071907c6000c2a1907b8009eb500132bb800873a072bb8008ab6008b36082abb008c5919071508b7008db5001d2a15042bb7008ebb0032592ab400102ab400022ab400122ab400132ab400172ab4001f2ab4001db700333a092a2c1d150515062ab4001a1909b80027b500282ab7000b2ab400129e000704a7000403b80041b1
-toData,29,2a2bb600932ab4001ab2008fb6006fa1000d2ab600942bb900830200b1
-toDataPre_GFE_7_1_0_0,226,2ab400129e000704a7000403b800412ab6004d2bb800952b2ab6004cb9009602002ab400062bb8007a033d2ab40028b9007b01009900071c04803d2ab40028b9006a01009900071c05803d2ab400389900071c07803d2b1c1100ff7e91b9009702002b2ab40010b9009602002b2ab40002b9009602002b2ab40012b9009702002ab4001f2bb8007d2ab400172bb8007a2ab40012100da0000e2ab400182bb8007aa7000e2ab40013b800982bb8007a2ab4001dc700081245a7000a2ab4001db6007e2bb8007a2ab4001dc7000911012ca7000a2ab4001db6007fb800802bb80081b1
-toDataPre_GFE_9_0_0_0,240,2ab400129e000704a7000403b800412ab6004d2bb800952b2ab6004cb9009602002ab400062bb8007a033d2ab40028b9007b01009900071c04803d2ab40028b9006a01009900071c05803d2ab400389900071c07803d1c1008803d2b1c1100ff7e91b9009702002b2ab40010b9009602002b2ab40002b9009602002b2ab40012b9009702002ab4001f2bb8007d2ab400172bb8007a2ab40012100da0000e2ab400182bb8007aa7000e2ab40013b800982bb8007a2ab4001dc700081245a7000a2ab4001db6007e2bb8007a2ab4001dc7000911012ca7000a2ab4001db6007fb800802bb800812b2ab4001a04b80082b1
+fromData,33,2a2bb6009a2ab4001ab20090b60070a100112ab400282bb900910200a700044db1
+fromDataPre_GFE_7_1_0_0,292,2bb8009b4d2bb9009c01003e2a2bb80088b500062ab2000399000e2c2ab40006b8009da700072cb60007b500062bb9009e010036041504047e99000704a700040336051504057e99000704a700040336062a1504077e99000704a7000403b500392a2bb9009c0100b500102a2bb9009c0100b500022a2bb9009e0100b500122a2bb8008ab5001f2a2bb80088b500172ab40012100da0000e2a2bb80088b50018a700172bb800883a071907c6000c2a1907b8009fb500132bb800883a072bb8008bb6008c36082abb008d5919071508b7008eb5001dbb0033592ab400102ab400022ab400122ab400132ab400172ab4001f2ab4001db700343a092a2c1d150515062bb800a0b600701909b80027b500282ab7000b2ab400129e000704a7000403b80042b1
+fromDataPre_GFE_9_0_0_0,296,2bb8009b4d2bb9009c01003e2a2bb80088b500062ab2000399000e2c2ab40006b8009da700072cb60007b500062bb9009e010036041504047e99000704a700040336051504057e99000704a700040336062a1504077e99000704a7000403b500392a2bb9009c0100b500102a2bb9009c0100b500022a2bb9009e0100b500122a2bb8008ab5001f2a2bb80088b500172ab40012100da0000e2a2bb80088b50018a700172bb800883a071907c6000c2a1907b8009fb500132bb800883a072bb8008bb6008c36082abb008d5919071508b7008eb5001d2a15042bb7008fbb0033592ab400102ab400022ab400122ab400132ab400172ab4001f2ab4001db700343a092a2c1d150515062ab4001a1909b80027b500282ab7000b2ab400129e000704a7000403b80042b1
+toData,29,2a2bb600942ab4001ab20090b60070a1000d2ab600952bb900840200b1
+toDataPre_GFE_7_1_0_0,226,2ab400129e000704a7000403b800422ab6004e2bb800962b2ab6004db9009702002ab400062bb8007b033d2ab40028b9007c01009900071c04803d2ab40028b9006b01009900071c05803d2ab400399900071c07803d2b1c1100ff7e91b9009802002b2ab40010b9009702002b2ab40002b9009702002b2ab40012b9009802002ab4001f2bb8007e2ab400172bb8007b2ab40012100da0000e2ab400182bb8007ba7000e2ab40013b800992bb8007b2ab4001dc700081246a7000a2ab4001db6007f2bb8007b2ab4001dc7000911012ca7000a2ab4001db60080b800812bb80082b1
+toDataPre_GFE_9_0_0_0,240,2ab400129e000704a7000403b800422ab6004e2bb800962b2ab6004db9009702002ab400062bb8007b033d2ab40028b9007c01009900071c04803d2ab40028b9006b01009900071c05803d2ab400399900071c07803d1c1008803d2b1c1100ff7e91b9009802002b2ab40010b9009702002b2ab40002b9009702002b2ab40012b9009802002ab4001f2bb8007e2ab400172bb8007b2ab40012100da0000e2ab400182bb8007ba7000e2ab40013b800992bb8007b2ab4001dc700081246a7000a2ab4001db6007f2bb8007b2ab4001dc7000911012ca7000a2ab4001db60080b800812bb800822b2ab4001a04b80083b1
 
 com/gemstone/gemfire/distributed/internal/membership/MemberAttributes,2
 fromData,73,2a2bb900270100b500022a2bb900270100b500032a2bb900270100b500042a2bb80028b500092a2bb80029b500072bb800284d2bb8002ab6002b3e2abb002c592c1db7002db5000ab1


[35/35] incubator-geode git commit: Change AvailablePort.JGROUPS to AvailablePort.MULTICAST

Posted by kl...@apache.org.
Change AvailablePort.JGROUPS to AvailablePort.MULTICAST


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

Branch: refs/heads/feature/GEODE-217
Commit: ca1550700e5e00a4a2804beaa75268212c1285ee
Parents: 76bca9e
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue Dec 22 11:24:31 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Dec 22 11:24:31 2015 -0800

----------------------------------------------------------------------
 .../test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java   | 2 +-
 .../java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca155070/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java
index a66fad5..6fd135f 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DUnitTestRule.java
@@ -450,7 +450,7 @@ public class DUnitTestRule implements SerializableTestRule {
    */
   public static InternalDistributedSystem getMcastSystem() {
     Properties props = getDistributedSystemProperties();
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
     props.put(DistributionConfig.MCAST_PORT_NAME, ""+port);
     props.put(DistributionConfig.MCAST_TTL_NAME, "0");
     props.put(DistributionConfig.LOCATORS_NAME, "");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ca155070/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
index 2a63ee3..12725a7 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
@@ -402,7 +402,7 @@ public abstract class DistributedTestCase implements java.io.Serializable {
    */
   public final InternalDistributedSystem getMcastSystem() {
     Properties props = this.getDistributedSystemProperties();
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
     props.put(DistributionConfig.MCAST_PORT_NAME, ""+port);
     props.put(DistributionConfig.MCAST_TTL_NAME, "0");
     props.put(DistributionConfig.LOCATORS_NAME, "");


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

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


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

Branch: refs/heads/feature/GEODE-217
Commit: 76bca9ee4d0509fe6d407e7bfcb83d09321412e5
Parents: 9283f26 625ea36
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue Dec 22 09:19:19 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Dec 22 09:19:19 2015 -0800

----------------------------------------------------------------------
 gemfire-core/build.gradle                       |   10 +-
 .../cache/query/internal/QueryMonitor.java      |    8 +-
 .../distributed/internal/InternalLocator.java   |    2 +-
 .../internal/SharedConfiguration.java           |    1 +
 .../internal/direct/DirectChannel.java          |    2 +-
 .../internal/locks/DLockGrantor.java            |    5 +-
 .../membership/InternalDistributedMember.java   |   13 +-
 .../internal/membership/NetMember.java          |    4 +
 .../internal/membership/gms/GMSMember.java      |    7 +-
 .../membership/gms/fd/GMSHealthMonitor.java     |  126 +-
 .../membership/gms/messenger/Transport.java     |   26 +
 .../internal/tcpserver/TcpClient.java           |    2 +-
 .../internal/tcpserver/TcpServer.java           |    2 +-
 .../gemfire/internal/AvailablePort.java         |  103 +-
 .../tier/sockets/ClientProxyMembershipID.java   |   15 +-
 .../internal/cache/tier/sockets/HandShake.java  |   16 +-
 .../gemfire/internal/offheap/Chunk.java         |   26 +-
 .../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 +-
 .../client/ClientCacheFactoryJUnitTest.java     |   50 +
 .../distributed/DistributedSystemDUnitTest.java |    6 +-
 .../gemfire/distributed/LocatorDUnitTest.java   |    5 +-
 .../locks/DLockReentrantLockJUnitTest.java      |   84 +
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |   42 +-
 .../messenger/JGroupsMessengerJUnitTest.java    |    4 +-
 .../gemfire/internal/AvailablePortHelper.java   |    2 +-
 .../internal/AvailablePortJUnitTest.java        |    2 +
 .../gemstone/gemfire/internal/FDDUnitTest.java  |  282 ---
 .../PersistentPartitionedRegionDUnitTest.java   |    6 +
 .../PersistentPartitionedRegionTestBase.java    |    2 +-
 .../offheap/AbstractStoredObjectTestBase.java   |    2 +-
 .../offheap/ChunkWithHeapFormJUnitTest.java     |   64 +
 .../offheap/GemFireChunkFactoryJUnitTest.java   |  129 ++
 .../internal/offheap/GemFireChunkJUnitTest.java |  921 +++++++++
 .../offheap/GemFireChunkSliceJUnitTest.java     |   72 +
 .../offheap/SimpleMemoryAllocatorJUnitTest.java |    6 +
 .../internal/offheap/StoredObjectTestSuite.java |   33 +
 .../management/CacheManagementDUnitTest.java    |   11 +-
 .../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 +++
 .../sanctionedDataSerializables.txt             |   12 +-
 .../codeAnalysis/sanctionedSerializables.txt    |    4 +-
 .../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                               |   12 +-
 103 files changed, 11619 insertions(+), 799 deletions(-)
----------------------------------------------------------------------



[08/35] incubator-geode git commit: GEODE-651: add Chunk unit tests This closes #60

Posted by kl...@apache.org.
GEODE-651: add Chunk unit tests
This closes #60


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

Branch: refs/heads/feature/GEODE-217
Commit: fdec70bcd46c568637f0dad976f3778625858608
Parents: 8c9af2a
Author: Sai Boorlagadda <sb...@pivotal.io>
Authored: Thu Dec 10 17:33:57 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Dec 15 16:50:39 2015 -0800

----------------------------------------------------------------------
 .../offheap/AbstractStoredObjectTestBase.java   |   2 +-
 .../offheap/ChunkWithHeapFormJUnitTest.java     |  64 ++
 .../offheap/GemFireChunkFactoryJUnitTest.java   | 129 +++
 .../internal/offheap/GemFireChunkJUnitTest.java | 924 +++++++++++++++++++
 .../offheap/GemFireChunkSliceJUnitTest.java     |  72 ++
 .../internal/offheap/StoredObjectTestSuite.java |  33 +
 6 files changed, 1223 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java
index a5fac48..24cb2fd 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java
@@ -51,7 +51,7 @@ public abstract class AbstractStoredObjectTestBase {
         Object regionEntryValue = getValue();
         StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
 
-        Integer actualRegionEntryValue = (Integer) storedObject.getValueAsDeserializedHeapObject();
+        Object actualRegionEntryValue = storedObject.getValueAsDeserializedHeapObject();
         assertEquals(regionEntryValue, actualRegionEntryValue);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapFormJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapFormJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapFormJUnitTest.java
new file mode 100644
index 0000000..bc32367
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapFormJUnitTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class ChunkWithHeapFormJUnitTest extends GemFireChunkJUnitTest {
+
+  @Test
+  public void getRawBytesShouldReturnCachedHeapForm() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    byte[] valueInBytes = getValueAsByteArray();
+    ChunkWithHeapForm heapForm = new ChunkWithHeapForm(chunk, valueInBytes);
+
+    assertNotNull(heapForm);
+
+    assertSame(valueInBytes, heapForm.getRawBytes());
+  }
+
+  @Test
+  public void getChunkWithoutHeapFormShouldReturnGemFireChunk() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    byte[] valueInBytes = getValueAsByteArray();
+    ChunkWithHeapForm heapForm = new ChunkWithHeapForm(chunk, valueInBytes);
+
+    Chunk chunkWithOutHeapForm = heapForm.getChunkWithoutHeapForm();
+
+    assertNotNull(chunkWithOutHeapForm);
+    assertEquals(GemFireChunk.class, chunkWithOutHeapForm.getClass());
+
+    assertEquals(chunk, heapForm.getChunkWithoutHeapForm());
+
+    assertEquals(chunk.getMemoryAddress(), chunkWithOutHeapForm.getMemoryAddress());
+    assertArrayEquals(chunk.getRawBytes(), chunkWithOutHeapForm.getRawBytes());
+    assertNotSame(valueInBytes, chunkWithOutHeapForm.getRawBytes());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactoryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactoryJUnitTest.java
new file mode 100644
index 0000000..d12b823
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactoryJUnitTest.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.mock;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class GemFireChunkFactoryJUnitTest {
+
+  private MemoryAllocator ma;
+
+  @Before
+  public void setUp() {
+    OutOfOffHeapMemoryListener ooohml = mock(OutOfOffHeapMemoryListener.class);
+    OffHeapMemoryStats stats = mock(OffHeapMemoryStats.class);
+    LogWriter lw = mock(LogWriter.class);
+
+    ma = SimpleMemoryAllocatorImpl.create(ooohml, stats, lw, 1, OffHeapStorage.MIN_SLAB_SIZE * 1, OffHeapStorage.MIN_SLAB_SIZE);
+  }
+
+  @After
+  public void tearDown() {
+    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+  }
+
+  private GemFireChunk createChunk(Object value) {
+    byte[] v = EntryEventImpl.serialize(value);
+
+    boolean isSerialized = true;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    return chunk;
+  }
+
+  @Test
+  public void factoryShouldCreateNewChunkWithGivenAddress() {
+    GemFireChunk chunk = createChunk(Long.MAX_VALUE);
+
+    ChunkFactory factory = new GemFireChunkFactory();
+    Chunk newChunk = factory.newChunk(chunk.getMemoryAddress());
+
+    assertNotNull(newChunk);
+    assertEquals(GemFireChunk.class, newChunk.getClass());
+
+    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void factoryShouldCreateNewChunkWithGivenAddressAndType() {
+    GemFireChunk chunk = createChunk(Long.MAX_VALUE);
+
+    ChunkFactory factory = new GemFireChunkFactory();
+    Chunk newChunk = factory.newChunk(chunk.getMemoryAddress(), GemFireChunk.TYPE);
+
+    assertNotNull(newChunk);
+    assertEquals(GemFireChunk.class, newChunk.getClass());
+
+    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
+    assertThat(newChunk.getChunkType()).isEqualTo(GemFireChunk.TYPE);
+
+    chunk.release();
+  }
+
+  @Test
+  public void shouldGetChunkTypeFromAddress() {
+    byte[] v = EntryEventImpl.serialize(Long.MAX_VALUE);
+
+    boolean isSerialized = true;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    ChunkFactory factory = new GemFireChunkFactory();
+    ChunkType actualType = factory.getChunkTypeForAddress(chunk.getMemoryAddress());
+
+    assertEquals(GemFireChunk.TYPE, actualType);
+
+    chunk.release();
+  }
+
+  @Test
+  public void shouldGetChunkTypeFromRawBits() {
+    byte[] v = EntryEventImpl.serialize(Long.MAX_VALUE);
+
+    boolean isSerialized = true;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    int rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4 /* REF_COUNT_OFFSET */);
+
+    ChunkFactory factory = new GemFireChunkFactory();
+    ChunkType actualType = factory.getChunkTypeForRawBits(rawBits);
+    assertEquals(GemFireChunk.TYPE, actualType);
+
+    chunk.release();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
new file mode 100644
index 0000000..20cb8dc
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
@@ -0,0 +1,924 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.compression.Compressor;
+import com.gemstone.gemfire.internal.DSCODE;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
+import com.gemstone.gemfire.internal.cache.CachePerfStats;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.internal.cache.RegionEntryContext;
+import com.gemstone.gemfire.internal.offheap.MemoryBlock.State;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
+
+  private MemoryAllocator ma;
+
+  static {
+    ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
+  }
+
+  @Before
+  public void setUp() {
+    OutOfOffHeapMemoryListener ooohml = mock(OutOfOffHeapMemoryListener.class);
+    OffHeapMemoryStats stats = mock(OffHeapMemoryStats.class);
+    LogWriter lw = mock(LogWriter.class);
+
+    ma = SimpleMemoryAllocatorImpl.create(ooohml, stats, lw, 3, OffHeapStorage.MIN_SLAB_SIZE * 3, OffHeapStorage.MIN_SLAB_SIZE);
+  }
+
+  @After
+  public void tearDown() {
+    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+  }
+
+  @Override
+  public Object getValue() {
+    return Long.valueOf(Long.MAX_VALUE);
+  }
+
+  @Override
+  public byte[] getValueAsByteArray() {
+    return convertValueToByteArray(getValue());
+  }
+
+  private byte[] convertValueToByteArray(Object value) {
+    return ByteBuffer.allocate(Long.SIZE / Byte.SIZE).putLong((Long) value).array();
+  }
+
+  @Override
+  public Object convertByteArrayToObject(byte[] valueInByteArray) {
+    return ByteBuffer.wrap(valueInByteArray).getLong();
+  }
+
+  @Override
+  public Object convertSerializedByteArrayToObject(byte[] valueInSerializedByteArray) {
+    return EntryEventImpl.deserialize(valueInSerializedByteArray);
+  }
+
+  @Override
+  public GemFireChunk createValueAsUnserializedStoredObject(Object value) {
+    byte[] valueInByteArray;
+    if (value instanceof Long) {
+      valueInByteArray = convertValueToByteArray(value);
+    } else {
+      valueInByteArray = (byte[]) value;
+    }
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    return createChunk(valueInByteArray, isSerialized, isCompressed);
+  }
+
+  @Override
+  public GemFireChunk createValueAsSerializedStoredObject(Object value) {
+    byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
+
+    boolean isSerialized = true;
+    boolean isCompressed = false;
+
+    return createChunk(valueInSerializedByteArray, isSerialized, isCompressed);
+  }
+
+  private GemFireChunk createChunk(byte[] v, boolean isSerialized, boolean isCompressed) {
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed, GemFireChunk.TYPE);
+    return chunk;
+  }
+
+  @Test
+  public void chunkCanBeCreatedFromAnotherChunk() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    GemFireChunk newChunk = new GemFireChunk(chunk);
+
+    assertNotNull(newChunk);
+    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void chunkCanBeCreatedWithOnlyMemoryAddress() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    GemFireChunk newChunk = new GemFireChunk(chunk.getMemoryAddress());
+
+    assertNotNull(newChunk);
+    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void chunkSliceCanBeCreatedFromAnotherChunk() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    int position = 1;
+    int end = 2;
+
+    GemFireChunk newChunk = (GemFireChunk) chunk.slice(position, end);
+
+    assertNotNull(newChunk);
+    assertThat(newChunk.getClass()).isEqualTo(GemFireChunkSlice.class);
+    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void fillSerializedValueShouldFillWrapperWithSerializedValueIfValueIsSerialized() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    // mock the things
+    BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
+
+    byte userBits = 0;
+    byte serializedUserBits = 1;
+    chunk.fillSerializedValue(wrapper, userBits);
+
+    verify(wrapper, times(1)).setChunkData(chunk, serializedUserBits);
+
+    chunk.release();
+  }
+
+  @Test
+  public void fillSerializedValueShouldFillWrapperWithDeserializedValueIfValueIsNotSerialized() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    // mock the things
+    BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
+
+    byte userBits = 1;
+    chunk.fillSerializedValue(wrapper, userBits);
+
+    verify(wrapper, times(1)).setChunkData(chunk, userBits);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getShortClassNameShouldReturnShortClassName() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getShortClassName()).isEqualTo("GemFireChunk");
+
+    chunk.release();
+  }
+
+  @Test
+  public void chunksAreEqualsOnlyByAddress() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    GemFireChunk newChunk = new GemFireChunk(chunk.getMemoryAddress());
+    assertThat(chunk.equals(newChunk)).isTrue();
+
+    GemFireChunk chunkWithSameValue = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.equals(chunkWithSameValue)).isFalse();
+
+    Object someObject = getValue();
+    assertThat(chunk.equals(someObject)).isFalse();
+
+    chunk.release();
+    chunkWithSameValue.release();
+  }
+
+  @Test
+  public void chunksShouldBeComparedBySize() {
+    GemFireChunk chunk1 = createValueAsSerializedStoredObject(getValue());
+
+    GemFireChunk chunk2 = chunk1;
+    assertThat(chunk1.compareTo(chunk2)).isEqualTo(0);
+
+    GemFireChunk chunkWithSameValue = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk1.compareTo(chunkWithSameValue)).isEqualTo(Long.signum(chunk1.getMemoryAddress() - chunkWithSameValue.getMemoryAddress()));
+
+    GemFireChunk chunk3 = createValueAsSerializedStoredObject(Long.MAX_VALUE);
+    GemFireChunk chunk4 = createValueAsSerializedStoredObject(Long.MAX_VALUE);
+
+    int newSizeForChunk3 = 2;
+    int newSizeForChunk4 = 3;
+
+    assertThat(chunk3.compareTo(chunk4)).isEqualTo(Integer.signum(newSizeForChunk3 - newSizeForChunk4));
+
+    chunk1.release();
+    chunk4.release();
+  }
+
+  @Test
+  public void setSerializedShouldSetTheSerializedBit() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    int headerBeforeSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+
+    assertThat(chunk.isSerialized()).isFalse();
+
+    chunk.setSerialized(true); // set to true
+
+    assertThat(chunk.isSerialized()).isTrue();
+
+    int headerAfterSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+
+    assertThat(headerAfterSerializedBitSet).isEqualTo(headerBeforeSerializedBitSet | 0x80000000/* IS_SERIALIZED_BIT */);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void setSerialziedShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.setSerialized(true);
+
+    chunk.release();
+  }
+
+  @Test
+  public void setCompressedShouldSetTheCompressedBit() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    int headerBeforeCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+
+    assertThat(chunk.isCompressed()).isFalse();
+
+    chunk.setCompressed(true); // set to true
+
+    assertThat(chunk.isCompressed()).isTrue();
+
+    int headerAfterCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+
+    assertThat(headerAfterCompressedBitSet).isEqualTo(headerBeforeCompressedBitSet | 0x40000000/* IS_SERIALIZED_BIT */);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void setCompressedShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.setCompressed(true);
+
+    chunk.release();
+  }
+
+  @Test
+  public void setDataSizeShouldSetTheDataSizeBits() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    int beforeSize = chunk.getDataSize();
+
+    chunk.setDataSize(2);
+
+    int afterSize = chunk.getDataSize();
+
+    assertThat(afterSize).isEqualTo(2);
+    assertThat(afterSize).isNotEqualTo(beforeSize);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void setDataSizeShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.setDataSize(1);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void initializeUseCountShouldThrowIllegalStateExceptionIfChunkIsAlreadyRetained() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.retain();
+    chunk.initializeUseCount();
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void initializeUseCountShouldThrowIllegalStateExceptionIfChunkIsAlreadyReleased() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.initializeUseCount();
+
+    chunk.release();
+  }
+
+  @Test
+  public void isSerializedPdxInstanceShouldReturnTrueIfItsPDXInstance() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    byte[] serailizedValue = chunk.getSerializedValue();
+    serailizedValue[0] = DSCODE.PDX;
+    chunk.setSerializedValue(serailizedValue);
+
+    assertThat(chunk.isSerializedPdxInstance()).isTrue();
+
+    serailizedValue = chunk.getSerializedValue();
+    serailizedValue[0] = DSCODE.PDX_ENUM;
+    chunk.setSerializedValue(serailizedValue);
+
+    assertThat(chunk.isSerializedPdxInstance()).isTrue();
+
+    serailizedValue = chunk.getSerializedValue();
+    serailizedValue[0] = DSCODE.PDX_INLINE_ENUM;
+    chunk.setSerializedValue(serailizedValue);
+
+    assertThat(chunk.isSerializedPdxInstance()).isTrue();
+
+    chunk.release();
+  }
+
+  @Test
+  public void isSerializedPdxInstanceShouldReturnFalseIfItsNotPDXInstance() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.isSerializedPdxInstance()).isFalse();
+
+    chunk.release();
+  }
+
+  @Test
+  public void checkDataEqualsByChunk() {
+    GemFireChunk chunk1 = createValueAsSerializedStoredObject(getValue());
+    GemFireChunk sameAsChunk1 = chunk1;
+
+    assertThat(chunk1.checkDataEquals(sameAsChunk1)).isTrue();
+
+    GemFireChunk unserializedChunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk1.checkDataEquals(unserializedChunk)).isFalse();
+
+    GemFireChunk chunkDifferBySize = createValueAsSerializedStoredObject(getValue());
+    chunkDifferBySize.setSize(0);
+    assertThat(chunk1.checkDataEquals(chunkDifferBySize)).isFalse();
+
+    GemFireChunk chunkDifferByValue = createValueAsSerializedStoredObject(Long.MAX_VALUE - 1);
+    assertThat(chunk1.checkDataEquals(chunkDifferByValue)).isFalse();
+
+    GemFireChunk newChunk1 = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk1.checkDataEquals(newChunk1)).isTrue();
+
+    chunk1.release();
+    unserializedChunk.release();
+    chunkDifferBySize.release();
+    chunkDifferByValue.release();
+    newChunk1.release();
+  }
+
+  @Test
+  public void checkDataEqualsBySerializedValue() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.checkDataEquals(new byte[1])).isFalse();
+
+    GemFireChunk chunkDifferByValue = createValueAsSerializedStoredObject(Long.MAX_VALUE - 1);
+    assertThat(chunk.checkDataEquals(chunkDifferByValue.getSerializedValue())).isFalse();
+
+    GemFireChunk newChunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.checkDataEquals(newChunk.getSerializedValue())).isTrue();
+
+    chunk.release();
+    chunkDifferByValue.release();
+    newChunk.release();
+  }
+
+  @Test
+  public void getDecompressedBytesShouldReturnDecompressedBytesIfCompressed() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = true;
+    boolean isCompressed = true;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    RegionEntryContext regionContext = mock(RegionEntryContext.class);
+    CachePerfStats cacheStats = mock(CachePerfStats.class);
+    Compressor compressor = mock(Compressor.class);
+
+    long startTime = 10000L;
+
+    // mock required things
+    when(regionContext.getCompressor()).thenReturn(compressor);
+    when(compressor.decompress(regionEntryValueAsBytes)).thenReturn(regionEntryValueAsBytes);
+    when(regionContext.getCachePerfStats()).thenReturn(cacheStats);
+    when(cacheStats.startDecompression()).thenReturn(startTime);
+
+    // invoke the thing
+    byte[] bytes = chunk.getDecompressedBytes(regionContext);
+
+    // verify the thing happened
+    verify(cacheStats, atLeastOnce()).startDecompression();
+    verify(compressor, times(1)).decompress(regionEntryValueAsBytes);
+    verify(cacheStats, atLeastOnce()).endDecompression(startTime);
+
+    assertArrayEquals(regionEntryValueAsBytes, bytes);
+
+    chunk.release();
+  }
+
+  @Test
+  public void incSizeShouldIncrementSize() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    int beforeSize = chunk.getSize();
+
+    chunk.incSize(1);
+    assertThat(chunk.getSize()).isEqualTo(beforeSize + 1);
+
+    chunk.incSize(2);
+    assertThat(chunk.getSize()).isEqualTo(beforeSize + 1 + 2);
+
+    chunk.release();
+  }
+
+  @Test
+  public void readyForFreeShouldResetTheRefCount() {
+    Chunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    int refCountBeforeFreeing = chunk.getRefCount();
+    assertThat(refCountBeforeFreeing).isEqualTo(1);
+
+    chunk.readyForFree();
+
+    int refCountAfterFreeing = chunk.getRefCount();
+    assertThat(refCountAfterFreeing).isEqualTo(0);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void readyForAllocationShouldThrowExceptionIfAlreadyAllocated() {
+    Chunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    // chunk is already allocated when we created it, so calling readyForAllocation should throw exception.
+    chunk.readyForAllocation(GemFireChunk.TYPE);
+
+    chunk.release();
+  }
+
+  @Test
+  public void checkIsAllocatedShouldReturnIfAllocated() {
+    Chunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.checkIsAllocated();
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void checkIsAllocatedShouldThrowExceptionIfNotAllocated() {
+    Chunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.release();
+    chunk.checkIsAllocated();
+
+    chunk.release();
+  }
+
+  @Test
+  public void sendToShouldWriteSerializedValueToDataOutputIfValueIsSerialized() throws IOException {
+    Chunk chunk = createValueAsSerializedStoredObject(getValue());
+    Chunk spyChunk = spy(chunk);
+
+    HeapDataOutputStream dataOutput = mock(HeapDataOutputStream.class);
+    ByteBuffer directByteBuffer = ByteBuffer.allocate(1024);
+
+    doReturn(directByteBuffer).when(spyChunk).createDirectByteBuffer();
+    doNothing().when(dataOutput).write(directByteBuffer);
+
+    spyChunk.sendTo(dataOutput);
+
+    verify(dataOutput, times(1)).write(directByteBuffer);
+
+    chunk.release();
+  }
+
+  @Test
+  public void sendToShouldWriteUnserializedValueToDataOutputIfValueIsUnserialized() throws IOException {
+    byte[] regionEntryValue = getValueAsByteArray();
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    // writeByte is a final method and cannot be mocked, so creating a real one
+    HeapDataOutputStream dataOutput = new HeapDataOutputStream(Version.CURRENT);
+
+    chunk.sendTo(dataOutput);
+
+    byte[] actual = dataOutput.toByteArray();
+
+    byte[] expected = new byte[regionEntryValue.length + 2];
+    expected[0] = DSCODE.BYTE_ARRAY;
+    expected[1] = (byte) regionEntryValue.length;
+    System.arraycopy(regionEntryValue, 0, expected, 2, regionEntryValue.length);
+
+    assertNotNull(dataOutput);
+    assertThat(actual).isEqualTo(expected);
+
+    chunk.release();
+  }
+
+  @Test
+  public void sendAsByteArrayShouldWriteValueToDataOutput() throws IOException {
+    byte[] regionEntryValue = getValueAsByteArray();
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    // writeByte is a final method and cannot be mocked, so creating a real one
+    HeapDataOutputStream dataOutput = new HeapDataOutputStream(Version.CURRENT);
+
+    chunk.sendAsByteArray(dataOutput);
+
+    byte[] actual = dataOutput.toByteArray();
+
+    byte[] expected = new byte[regionEntryValue.length + 1];
+    expected[0] = (byte) regionEntryValue.length;
+    System.arraycopy(regionEntryValue, 0, expected, 1, regionEntryValue.length);
+
+    assertNotNull(dataOutput);
+    assertThat(actual).isEqualTo(expected);
+
+    chunk.release();
+  }
+
+  @Test
+  public void createDirectByteBufferShouldCreateAByteBuffer() {
+    byte[] regionEntryValue = getValueAsByteArray();
+
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    ByteBuffer buffer = chunk.createDirectByteBuffer();
+
+    byte[] actual = new byte[regionEntryValue.length];
+    buffer.get(actual);
+
+    assertArrayEquals(regionEntryValue, actual);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getDirectByteBufferShouldCreateAByteBuffer() {
+    byte[] regionEntryValue = getValueAsByteArray();
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    ByteBuffer buffer = chunk.createDirectByteBuffer();
+    long bufferAddress = Chunk.getDirectByteBufferAddress(buffer);
+
+    // returned address should be starting of the value (after skipping HEADER_SIZE bytes)
+    assertEquals(chunk.getMemoryAddress() + Chunk.OFF_HEAP_HEADER_SIZE, bufferAddress);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getAddressForReadingShouldFailIfItsOutsideOfChunk() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getAddressForReading(0, chunk.getDataSize() + 1);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getAddressForReadingShouldReturnDataAddressFromGivenOffset() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    int offset = 1;
+    long requestedAddress = chunk.getAddressForReading(offset, 1);
+
+    assertThat(requestedAddress).isEqualTo(chunk.getBaseDataAddress() + offset);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getSizeInBytesShouldReturnSize() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.getSizeInBytes()).isEqualTo(chunk.getSize());
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getUnsafeAddressShouldFailIfOffsetIsNegative() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getUnsafeAddress(-1, 1);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getUnsafeAddressShouldFailIfSizeIsNegative() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getUnsafeAddress(1, -1);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getUnsafeAddressShouldFailIfItsOutsideOfChunk() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getUnsafeAddress(0, chunk.getDataSize() + 1);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getUnsafeAddressShouldReturnUnsafeAddress() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    int offset = 1;
+    long unsafeAddress = chunk.getUnsafeAddress(offset, 1);
+
+    assertThat(unsafeAddress).isEqualTo(chunk.getBaseDataAddress() + offset);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void readByteAndWriteByteShouldFailIfOffsetIsOutside() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    chunk.readByte(chunk.getDataSize() + 1);
+
+    chunk.writeByte(chunk.getDataSize() + 1, Byte.MAX_VALUE);
+
+    chunk.release();
+  }
+
+  @Test
+  public void writeByteShouldWriteAtCorrectLocation() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    byte valueBeforeWrite = chunk.readByte(2);
+
+    Byte expected = Byte.MAX_VALUE;
+    chunk.writeByte(2, expected);
+
+    Byte actual = chunk.readByte(2);
+
+    assertThat(actual).isNotEqualTo(valueBeforeWrite);
+    assertThat(actual).isEqualTo(expected);
+
+    chunk.release();
+  }
+
+  @Test
+  public void retainShouldIncrementRefCount() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.retain();
+    assertThat(chunk.getRefCount()).isEqualTo(2);
+
+    chunk.retain();
+    assertThat(chunk.getRefCount()).isEqualTo(3);
+
+    chunk.release();
+    chunk.release();
+    chunk.release();
+    boolean retainAfterRelease = chunk.retain();
+
+    assertThat(retainAfterRelease).isFalse();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void retainShouldThrowExceptionAfterMaxNumberOfTimesRetained() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    // max retain Chunk.MAX_REF_COUNT
+    int MAX_REF_COUNT = 0xFFFF;
+
+    // loop though and invoke retain for MAX_REF_COUNT-1 times, as create chunk above counted as one reference
+    for (int i = 0; i < MAX_REF_COUNT - 1; i++)
+      chunk.retain();
+
+    // invoke for the one more time should throw exception
+    chunk.retain();
+  }
+
+  @Test
+  public void releaseShouldDecrementRefCount() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.retain();
+    chunk.retain();
+    assertThat(chunk.getRefCount()).isEqualTo(3);
+
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(2);
+
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.retain();
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(0);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void releaseShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.release();
+  }
+
+  @Test
+  public void testToStringForOffHeapByteSource() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    String expected = ":<dataSize=" + chunk.getDataSize() + " refCount=" + chunk.getRefCount() + " addr=" + Long.toHexString(chunk.getMemoryAddress()) + ">";
+    assertThat(chunk.toStringForOffHeapByteSource()).endsWith(expected);
+
+    // test toString
+    Chunk spy = spy(chunk);
+    spy.toString();
+    verify(spy, times(1)).toStringForOffHeapByteSource();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getStateShouldReturnAllocatedIfRefCountIsGreaterThanZero() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertEquals(State.ALLOCATED, chunk.getState());
+
+    chunk.release();
+  }
+
+  @Test
+  public void getStateShouldReturnDeallocatedIfRefCountIsZero() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    assertEquals(State.DEALLOCATED, chunk.getState());
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void getNextBlockShouldThrowUnSupportedOperationException() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.getNextBlock();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getBlockSizeShouldBeSameSameGetSize() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertEquals(chunk.getSize(), chunk.getBlockSize());
+
+    chunk.release();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void copyBytesShouldThrowUnSupportedOperationException() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.copyBytes(1, 2, 1);
+
+    chunk.release();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void getSlabIdShouldThrowUnSupportedOperationException() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.getSlabId();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getFreeListIdShouldReturnMinusOne() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getFreeListId()).isEqualTo(-1);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getDataTypeShouldReturnNull() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getDataType()).isNull();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getDataDataShouldReturnNull() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getDataValue()).isNull();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void getRawBytesShouldThrowExceptionIfValueIsCompressed() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = true;
+    boolean isCompressed = true;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    chunk.getRawBytes();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getSerializedValueShouldSerializeTheValue() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    byte[] serializedValue = chunk.getSerializedValue();
+
+    assertThat(serializedValue).isEqualTo(EntryEventImpl.serialize(regionEntryValueAsBytes));
+
+    chunk.release();
+  }
+
+  @Test
+  public void getSrcTypeOrdinalFromAddressShouldReturnOrdinal() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    assertThat(Chunk.getSrcTypeOrdinal(chunk.getMemoryAddress())).isEqualTo(4);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getSrcTypeOrdinalFromRawBitsShouldReturnOrdinal() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    int rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4 /* REF_COUNT_OFFSET */);
+    assertThat(Chunk.getSrcTypeOrdinalFromRawBits(rawBits)).isEqualTo(4);
+
+    chunk.release();
+  }
+
+  @Test
+  public void fillShouldFillTheChunk() {
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(new byte[100], isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    // first fill the unused part with FILL_PATTERN
+    Chunk.fill(chunk.getMemoryAddress());
+
+    // Validate that it is filled
+    chunk.validateFill();
+
+    chunk.release();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSliceJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSliceJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSliceJUnitTest.java
new file mode 100644
index 0000000..39ee620
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSliceJUnitTest.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class GemFireChunkSliceJUnitTest extends GemFireChunkJUnitTest {
+
+  @Test
+  public void sliceShouldHaveAValidDataSize() {
+    int position = 1;
+    int end = 2;
+
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    GemFireChunkSlice slice = (GemFireChunkSlice) chunk.slice(position, end);
+
+    assertNotNull(slice);
+    assertEquals(GemFireChunkSlice.class, slice.getClass());
+
+    assertEquals(end - position, slice.getDataSize());
+  }
+
+  @Test
+  public void sliceShouldHaveAValidBaseDataAddress() {
+    int position = 1;
+    int end = 2;
+
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    GemFireChunkSlice slice = (GemFireChunkSlice) chunk.slice(position, end);
+
+    assertNotNull(slice);
+    assertEquals(GemFireChunkSlice.class, slice.getClass());
+
+    assertEquals(chunk.getBaseDataAddress() + position, slice.getBaseDataAddress());
+  }
+
+  @Test
+  public void sliceShouldHaveAValidBaseOffset() {
+    int position = 1;
+    int end = 2;
+
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    GemFireChunkSlice slice = (GemFireChunkSlice) chunk.slice(position, end);
+
+    assertNotNull(slice);
+    assertEquals(GemFireChunkSlice.class, slice.getClass());
+
+    assertEquals(chunk.getBaseDataOffset() + position, slice.getBaseDataOffset());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java
new file mode 100644
index 0000000..d4f9e97
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.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.internal.offheap;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@Suite.SuiteClasses({
+	DataAsAddressJUnitTest.class,
+	GemFireChunkJUnitTest.class,
+	ChunkWithHeapFormJUnitTest.class,
+	GemFireChunkSliceJUnitTest.class,
+	GemFireChunkFactoryJUnitTest.class
+})
+@RunWith(Suite.class)
+public class StoredObjectTestSuite {
+
+}



[16/35] incubator-geode git commit: GEODE-588: CI failure in LocatorDUnitTest

Posted by kl...@apache.org.
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-217
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);


[02/35] incubator-geode git commit: GEODE-666: Update awaitility dependency from 1.6.5 to 1.7.0

Posted by kl...@apache.org.
GEODE-666: Update awaitility dependency from 1.6.5 to 1.7.0


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

Branch: refs/heads/feature/GEODE-217
Commit: c30e533853c25abf23ccee8b5a63255fce4c1885
Parents: ae2d529
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Dec 14 09:31:58 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Dec 15 14:07:04 2015 -0800

----------------------------------------------------------------------
 gradle/dependency-versions.properties | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c30e5338/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 8ea38c3..684e60e 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -22,7 +22,7 @@ annotations.version = 3.0.0
 antlr.version = 2.7.7
 asm.version = 5.0.3
 assertj-core.version = 3.2.0
-awaitility.version = 1.6.5
+awaitility.version = 1.7.0
 bcel.version = 5.2
 catch-exception.version = 1.4.4
 catch-throwable.version = 1.4.4


[13/35] incubator-geode git commit: Revision bb794ee205400ac4e5655cc1e6410889d29f28b1 closed #61

Posted by kl...@apache.org.
Revision bb794ee205400ac4e5655cc1e6410889d29f28b1 closed #61


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

Branch: refs/heads/feature/GEODE-217
Commit: 5d1f38ed3c2bcff5871630abbe92df032988b881
Parents: bb794ee
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Dec 17 14:27:48 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 17 14:27:48 2015 -0800

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

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



[07/35] incubator-geode git commit: GEODE-653: Add unit test for GMSHealthMonitor checkIfAvailable Removed unused code Minor javadoc corrections

Posted by kl...@apache.org.
GEODE-653: Add unit test for GMSHealthMonitor checkIfAvailable
Removed unused code
Minor javadoc corrections


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

Branch: refs/heads/feature/GEODE-217
Commit: 8c9af2ab4f4a03eda607fe3d6c95511e7d181117
Parents: d40d8a7
Author: Jason Huynh <hu...@gmail.com>
Authored: Fri Dec 11 10:17:39 2015 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Dec 15 16:07:33 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/fd/GMSHealthMonitor.java     | 114 +------------------
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |  42 ++++---
 2 files changed, 31 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c9af2ab/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 2aac935..33c7e76 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
@@ -86,12 +86,12 @@ import com.gemstone.gemfire.internal.Version;
  * to remove that member from view.
  * 
  * It has {@link #suspect(InternalDistributedMember, String)} api, which can be used
- * to initiate suspect processing for any member. First is checks whether member is
- * responding or not. Then it informs to probable coordinators to remove that member from
+ * to initiate suspect processing for any member. First is checks whether the member is
+ * responding or not. Then it informs probable coordinators to remove that member from
  * view.
  * 
  * It has {@link #checkIfAvailable(DistributedMember, String, boolean)} api to see
- * if that member is alive. Then based on removal flag it initiate the suspect processing
+ * if that member is alive. Then based on removal flag it initiates the suspect processing
  * for that member.
  * 
  * */
@@ -158,9 +158,6 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
 
   private ExecutorService checkExecutor;
 
-//  List<SuspectRequest> suspectRequests = new ArrayList<SuspectRequest>();
-//  private RequestCollector<SuspectRequest> suspectRequestCollectorThread;
-
   /**
    * to stop check scheduler
    */
@@ -464,17 +461,6 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
   }
 
   /**
-   * Check for recent messaging activity from the given member
-   * @param suspectMember
-   * @return whether there has been activity within memberTimeout ms
-   */
-  private boolean checkRecentActivity(InternalDistributedMember suspectMember) {
-    TimeStamp ts = memberTimeStamps.get(suspectMember);
-    return (ts != null && (System.currentTimeMillis() - ts.getTime()) <= memberTimeout);
-  }
-  
-
-  /**
    * During final check, establish TCP connection between current member and suspect member.
    * And exchange PING/PONG message to see if the suspect member is still alive.
    * 
@@ -1238,100 +1224,6 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
     return this.socketPort;
   }
 
-  interface Callback<T> {
-    public void process(List<T> requests);
-  }
-
-  /***
-   * this thread will collect suspect message for some time interval
-   * then it send message to current coordinator first if its not in
-   * suspected list. if its in then it will send message to next probable
-   * coordinator. NOTE: this thread will not check-server for verification
-   * assuming many servers are going down and lets coordinator deals with it.
-   * 
-   * Should we wait for ack from coordinator/probable coordinator that I got
-   * request to suspect these members.
-   * 
-   */
-  class RequestCollector<T> extends Thread {
-    volatile boolean shutdown = false;
-    final List<T> listToTrack;
-    final Callback<T> callback;
-    final long timeout;
-
-    public RequestCollector(String name, ThreadGroup tg, List<T> l, Callback<T> c, long t) {
-      super(tg, name);
-      listToTrack = l;
-      callback = c;
-      timeout = t;
-    }
-
-    void shutdown() {
-      shutdown = true;
-      synchronized (listToTrack) {
-        listToTrack.notify();
-        interrupt();
-      }
-    }
-
-    boolean isShutdown() {
-      return shutdown;
-    }
-
-    @Override
-    public void run() {
-      List<T> requests = null;
-      logger.debug("Suspect thread is starting");
-      long okayToSendSuspectRequest = System.currentTimeMillis() + timeout;
-      try {
-        for (;;) {
-          synchronized (listToTrack) {
-            if (shutdown || services.getCancelCriterion().isCancelInProgress()) {
-              return;
-            }
-            if (listToTrack.isEmpty()) {
-              try {
-                logger.trace("Result collector is waiting");
-                listToTrack.wait();
-              } catch (InterruptedException e) {
-                return;
-              }
-            } else {
-              long now = System.currentTimeMillis();
-              if (now < okayToSendSuspectRequest) {
-                // sleep to let more suspect requests arrive
-                try {
-                  sleep(okayToSendSuspectRequest - now);
-                  continue;
-                } catch (InterruptedException e) {
-                  return;
-                }
-              } else {
-                if (requests == null) {
-                  requests = new ArrayList<T>(listToTrack);
-                } else {
-                  requests.addAll(listToTrack);
-                }
-                listToTrack.clear();
-                okayToSendSuspectRequest = System.currentTimeMillis() + timeout;
-              }
-            }
-          } // synchronized
-          if (requests != null && !requests.isEmpty()) {
-            if (logger != null && logger.isDebugEnabled()) {
-              logger.info("Health Monitor is sending {} member suspect requests to coordinator", requests.size());
-            }
-            callback.process(requests);
-            requests = null;
-          }
-        }
-      } finally {
-        shutdown = true;
-        logger.debug("Suspect thread is stopped");
-      }
-    }
-  }
-
   private void sendSuspectRequest(final List<SuspectRequest> requests) {
     // the background suspect-collector thread is currently disabled
 //    synchronized (suspectRequests) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c9af2ab/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 eb17ca8..d539374 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
@@ -20,7 +20,11 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -42,6 +46,8 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -407,26 +413,34 @@ public class GMSHealthMonitorJUnitTest {
    * validates HealthMonitor.CheckIfAvailable api
    */
   @Test
-  public void testCheckIfAvailable() {
-
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-
-    gmsHealthMonitor.installView(v);
-
+  public void testCheckIfAvailableNoHeartBeatDontRemoveMember() {
     long startTime = System.currentTimeMillis();
-
     boolean retVal = gmsHealthMonitor.checkIfAvailable(mockMembers.get(1), "Not responding", false);
-
     long timeTaken = System.currentTimeMillis() - startTime;
 
-    assertTrue("This should have taken member ping timeout 100ms ", timeTaken > 90);
-    assertTrue("CheckIfAvailable should have return false", !retVal);
+    assertTrue("This should have taken member ping timeout 100ms ", timeTaken >= gmsHealthMonitor.memberTimeout);
+    assertFalse("CheckIfAvailable should have return false", retVal);
   }
 
   @Test
+  public void testCheckIfAvailableWithSimulatedHeartBeat() {
+    InternalDistributedMember memberToCheck = mockMembers.get(1);
+    HeartbeatMessage fakeHeartbeat = new HeartbeatMessage();
+    fakeHeartbeat.setSender(memberToCheck);
+    when(messenger.send(any(HeartbeatRequestMessage.class))).then(new Answer() {
+      @Override
+      public Object answer(InvocationOnMock invocation) throws Throwable {
+        gmsHealthMonitor.processMessage(fakeHeartbeat);
+        return null;
+      }
+    });
+    
+    boolean retVal = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
+    assertTrue("CheckIfAvailable should have return true", retVal);
+  }
+  
+  
+  @Test
   public void testShutdown() {
 
     NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());


[04/35] incubator-geode git commit: GEODE_468 - fixed AnalyzeSerializablesJUnitTest failure

Posted by kl...@apache.org.
GEODE_468 - fixed AnalyzeSerializablesJUnitTest failure

the recent failure was due to the removal of the Stub class and the
renaming of MissingStubException to ShunnedMemberException.


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

Branch: refs/heads/feature/GEODE-217
Commit: df32ad1c657a7b42efc5d00a407045d282788217
Parents: 7c368e9
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Dec 15 14:39:36 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 15 14:40:55 2015 -0800

----------------------------------------------------------------------
 .../gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/df32ad1c/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
index 03e82d0..8e5c7fd 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
@@ -249,7 +249,7 @@ com/gemstone/gemfire/distributed/internal/deadlock/GemFireDeadlockDetector$Colle
 com/gemstone/gemfire/distributed/internal/deadlock/LocalLockInfo,true,1,info:com/gemstone/gemfire/internal/concurrent/LI,locatility:java/io/Serializable
 com/gemstone/gemfire/distributed/internal/deadlock/LocalThread,true,1,locality:java/io/Serializable,threadId:long,threadName:java/lang/String,threadStack:java/lang/String
 com/gemstone/gemfire/distributed/internal/deadlock/MessageDependencyMonitor$MessageKey,true,414781046295505260,myId:com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember,processorId:int
-com/gemstone/gemfire/distributed/internal/direct/MissingStubException,true,-6455664684151074915
+com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException,true,-6455664684151074915
 com/gemstone/gemfire/distributed/internal/locks/DistributedMemberLock$LockReentryPolicy,false
 com/gemstone/gemfire/distributed/internal/locks/LockGrantorDestroyedException,true,-3540124531032570817
 com/gemstone/gemfire/distributed/internal/membership/gms/messages/InstallViewMessage$messageType,false
@@ -627,7 +627,7 @@ com/gemstone/gemfire/internal/tcp/ByteBufferInputStream,false,buffer:com/gemston
 com/gemstone/gemfire/internal/tcp/ConnectExceptions,true,-4173688946448867706,causes:java/util/List,members:java/util/List
 com/gemstone/gemfire/internal/tcp/ConnectionException,true,-1977443644277412122
 com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream,false
-com/gemstone/gemfire/internal/tcp/MemberShunnedException,true,-8453126202477831557,member:com/gemstone/gemfire/internal/tcp/Stub
+com/gemstone/gemfire/internal/tcp/MemberShunnedException,true,-8453126202477831557,member:com/gemstone/gemfire/distributed/DistributedMember
 com/gemstone/gemfire/internal/tcp/ReenteredConnectException,false
 com/gemstone/gemfire/internal/tcp/VersionedByteBufferInputStream,false,version:com/gemstone/gemfire/internal/Version
 com/gemstone/gemfire/internal/util/Breadcrumbs$CrumbType,false


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

Posted by kl...@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-217
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";


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

Posted by kl...@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-217
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;


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

Posted by kl...@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-217
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