You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2016/06/21 03:14:25 UTC

[4/9] incubator-geode git commit: GEODE-17: used new security interface for integrated security.

GEODE-17: used new security interface for integrated security.

* Create new interface for integrated security
* separate jmx and cli from client-server security
* client-server security using integrated security


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

Branch: refs/heads/feature/GEODE-1571
Commit: f447023e7518f27217e95f22b80cb76e2670c9e8
Parents: b53d28f
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue Jun 7 11:51:12 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Jun 20 10:00:28 2016 -0700

----------------------------------------------------------------------
 .../cache/operations/OperationContext.java      |  33 +-
 .../internal/GetOperationContextImpl.java       |   2 +-
 .../cache/tier/sockets/AcceptorImpl.java        |  70 ++-
 .../cache/tier/sockets/ClientUserAuths.java     |  24 +
 .../cache/tier/sockets/ServerConnection.java    | 501 +++++----------
 .../cache/tier/sockets/command/Get70.java       | 191 +++---
 .../tier/sockets/command/RemoveUserAuth.java    |  71 +--
 .../internal/security/GeodeSecurityUtil.java    | 192 ++++--
 .../security/shiro/CustomAuthRealm.java         | 146 +----
 .../management/AsyncEventQueueMXBean.java       |   7 +-
 .../gemfire/management/CacheServerMXBean.java   |  17 +-
 .../gemfire/management/DiskStoreMXBean.java     |  17 +-
 .../DistributedLockServiceMXBean.java           |  11 +-
 .../management/DistributedRegionMXBean.java     |   7 +-
 .../management/DistributedSystemMXBean.java     |  22 +-
 .../management/GatewayReceiverMXBean.java       |  11 +-
 .../gemfire/management/GatewaySenderMXBean.java |  17 +-
 .../gemfire/management/LocatorMXBean.java       |   7 +-
 .../gemfire/management/LockServiceMXBean.java   |  13 +-
 .../gemfire/management/ManagerMXBean.java       |  19 +-
 .../gemfire/management/MemberMXBean.java        |  12 +-
 .../gemfire/management/RegionMXBean.java        |   7 +-
 .../internal/cli/commands/ClientCommands.java   |  30 +-
 .../internal/cli/commands/ConfigCommands.java   |  41 +-
 .../CreateAlterDestroyRegionCommands.java       |   8 +-
 .../internal/cli/commands/DataCommands.java     |  13 +-
 .../internal/cli/commands/DeployCommands.java   |  31 +-
 .../cli/commands/DiskStoreCommands.java         |  65 +-
 .../cli/commands/DurableClientCommands.java     |  28 +-
 ...ExportImportSharedConfigurationCommands.java |  30 +-
 .../internal/cli/commands/FunctionCommands.java |  30 +-
 .../internal/cli/commands/IndexCommands.java    |  16 +-
 .../cli/commands/LauncherLifecycleCommands.java |  72 ++-
 .../internal/cli/commands/MemberCommands.java   |  22 +-
 .../cli/commands/MiscellaneousCommands.java     |  96 +--
 .../internal/cli/commands/PDXCommands.java      |  86 +--
 .../internal/cli/commands/QueueCommands.java    |  19 +-
 .../internal/cli/commands/RegionCommands.java   |   8 +-
 .../internal/cli/commands/StatusCommands.java   |  16 +-
 .../internal/cli/commands/WanCommands.java      |  60 +-
 .../internal/security/MBeanServerWrapper.java   |  36 +-
 .../internal/security/ResourceOperation.java    |   7 +-
 .../security/ResourceOperationContext.java      |  85 ---
 .../gemfire/security/AccessControl.java         |   5 +-
 .../gemfire/security/ExternalSecurity.java      |  34 +
 .../gemfire/security/GeodePermission.java       |  93 +++
 .../templates/SampleJsonAuthorization.java      |  53 +-
 .../extension/mock/MockExtensionCommands.java   |  24 +-
 .../internal/cli/CommandManagerJUnitTest.java   |  37 +-
 .../shell/GfshExecutionStrategyJUnitTest.java   |  27 +-
 ...CacheServerMBeanAuthenticationJUnitTest.java |   1 -
 .../security/GeodePermissionJUnitTest.java      |  94 +++
 .../GeodeSecurityUtilWithIniFileJUnitTest.java  |  15 +-
 .../ResourceOperationContextJUnitTest.java      |  94 ---
 .../internal/security/TestCommand.java          |  44 +-
 .../security/ClientAuthenticationDUnitTest.java |   8 +-
 .../security/ClientAuthenticationTestCase.java  |  32 +-
 .../security/ClientAuthenticationTestUtils.java |  29 +-
 .../gemfire/security/SecurityTestUtils.java     | 622 +++++++++++++------
 .../gemfire/codeAnalysis/excludedClasses.txt    |   4 +-
 .../gemfire/tools/pulse/tests/Server.java       |  40 +-
 61 files changed, 1741 insertions(+), 1711 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
index 20b528c..a8970bf 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
@@ -17,8 +17,6 @@
 
 package com.gemstone.gemfire.cache.operations;
 
-import org.apache.shiro.authz.permission.WildcardPermission;
-
 /**
  * Encapsulates a cache operation and the data associated with it for both the
  * pre-operation and post-operation cases. Implementations for specific
@@ -30,14 +28,7 @@ import org.apache.shiro.authz.permission.WildcardPermission;
  *
  * @since GemFire 5.5
  */
-public abstract class OperationContext extends WildcardPermission{
-  public static String ALL_REGIONS="*";
-
-  public enum Resource {
-    NULL,
-    CLUSTER,
-    DATA
-  };
+public abstract class OperationContext{
 
   public enum OperationCode {
     @Deprecated
@@ -77,11 +68,7 @@ public abstract class OperationContext extends WildcardPermission{
     @Deprecated
     EXECUTE_FUNCTION,
     @Deprecated
-    GET_DURABLE_CQS,
-    NULL,
-    MANAGE,
-    WRITE,
-    READ;
+    GET_DURABLE_CQS;
 
     /**
      * Check if this is an entry get operation.
@@ -300,14 +287,6 @@ public abstract class OperationContext extends WildcardPermission{
    */
   public abstract OperationCode getOperationCode();
 
-  public Resource getResource(){
-    return Resource.NULL;
-  }
-
-  public String getRegionName(){
-    return ALL_REGIONS;
-  }
-
   /**
    * True if the context is for post-operation.
    *
@@ -359,12 +338,4 @@ public abstract class OperationContext extends WildcardPermission{
         || opCode.isRegionDestroy() || opCode.isRegionClear());
   }
 
-  @Override
-  public String toString(){
-    if(ALL_REGIONS.equals(getRegionName()))
-      return getResource()+":"+getOperationCode();
-    else
-      return getResource()+":"+getOperationCode()+":"+getRegionName();
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
index f664061..2904776 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
@@ -32,7 +32,7 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 public class GetOperationContextImpl extends GetOperationContext implements Releasable {
 
   private boolean released;
-  
+
   public GetOperationContextImpl(Object key, boolean postOperation) {
     super(key, postOperation);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
index ffcb8c5..6d0c5cb 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
@@ -17,6 +17,47 @@
 
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
+import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.BindException;
+import java.net.Inet6Address;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.nio.channels.CancelledKeyException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.ClosedSelectorException;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.net.ssl.SSLException;
+
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.ToDataException;
@@ -25,7 +66,12 @@ import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.wan.GatewayTransportFilter;
-import com.gemstone.gemfire.distributed.internal.*;
+import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.LonerDistributionManager;
+import com.gemstone.gemfire.distributed.internal.PooledExecutorWithDMStats;
+import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.cache.BucketAdvisor;
@@ -41,22 +87,11 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.internal.tcp.ConnectionTable;
 import com.gemstone.gemfire.internal.util.ArrayUtils;
-import org.apache.logging.log4j.Logger;
-
-import javax.net.ssl.SSLException;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.net.*;
-import java.nio.ByteBuffer;
-import java.nio.channels.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicInteger;
 
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
+import org.apache.logging.log4j.Logger;
 
 /**
  * Implements the acceptor thread on the bridge server. Accepts connections from
@@ -232,6 +267,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
   private long acceptorId;
 
   private static boolean isAuthenticationRequired;
+  private static boolean isIntegratedSecurity;
 
   private static boolean isPostAuthzCallbackPresent;
 
@@ -607,6 +643,8 @@ public class AcceptorImpl extends Acceptor implements Runnable
     isAuthenticationRequired = (authenticator != null && authenticator.length() > 0) ? true
         : false;
 
+    isIntegratedSecurity = GeodeSecurityUtil.isIntegratedSecurity(authenticator);
+
     String postAuthzFactoryName = this.cache.getDistributedSystem()
         .getProperties().getProperty(SECURITY_CLIENT_ACCESSOR_PP);
 
@@ -1807,6 +1845,10 @@ public class AcceptorImpl extends Acceptor implements Runnable
     return isAuthenticationRequired;
   }
 
+  public static boolean isIntegratedSecurity(){
+    return isIntegratedSecurity;
+  }
+
   public static boolean isPostAuthzCallbackPresent() {
     return isPostAuthzCallbackPresent;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientUserAuths.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientUserAuths.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientUserAuths.java
index 383f0c6..840bbfc 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientUserAuths.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientUserAuths.java
@@ -25,6 +25,8 @@ import com.gemstone.gemfire.internal.security.AuthorizeRequest;
 import com.gemstone.gemfire.internal.security.AuthorizeRequestPP;
 import com.gemstone.gemfire.security.NotAuthorizedException;
 
+import org.apache.shiro.subject.Subject;
+
 public class ClientUserAuths
 {
  // private AtomicLong counter = new AtomicLong(1);
@@ -34,6 +36,7 @@ public class ClientUserAuths
 
   private ConcurrentHashMap<Long, UserAuthAttributes> uniqueIdVsUserAuth = new ConcurrentHashMap<Long, UserAuthAttributes>();
   private ConcurrentHashMap<String, UserAuthAttributes> cqNameVsUserAuth = new ConcurrentHashMap<String, UserAuthAttributes>();
+  private ConcurrentHashMap<Long, Subject> uniqueIdVsSubject = new ConcurrentHashMap<Long, Subject>();
 
   public long putUserAuth(UserAuthAttributes userAuthAttr)
   {
@@ -43,6 +46,12 @@ public class ClientUserAuths
     uniqueIdVsUserAuth.put(newId, userAuthAttr);
     return newId;
   }
+
+  public long putSubject(Subject subject){
+    long newId = getNextID();
+    uniqueIdVsSubject.put(newId, subject);
+    return newId;
+  }
   
   public ClientUserAuths(int clientProxyHashcode)
   {
@@ -70,6 +79,19 @@ public class ClientUserAuths
   {
     return uniqueIdVsUserAuth.get(userId);
   }
+
+  public Subject getSubject(long userId){
+    return uniqueIdVsSubject.get(userId);
+  }
+
+  public boolean removeSubject(long userId) {
+    Subject subject = uniqueIdVsSubject.remove(userId);
+    if(subject == null)
+      return false;
+
+    subject.logout();
+    return true;
+  }
   
   public UserAuthAttributes getUserAuthAttributes(String cqName)
   {
@@ -133,6 +155,8 @@ public class ClientUserAuths
     }
     return false;
   }
+
+
   
   public void cleanUserAuth(UserAuthAttributes userAuth)
   {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
index e7c102c..7bb35da 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
@@ -17,6 +17,24 @@
 
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
+import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectableChannel;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.security.Principal;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.SystemFailure;
@@ -30,7 +48,12 @@ import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.tier.*;
+import com.gemstone.gemfire.internal.cache.tier.Acceptor;
+import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
+import com.gemstone.gemfire.internal.cache.tier.ClientHandShake;
+import com.gemstone.gemfire.internal.cache.tier.Command;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
+import com.gemstone.gemfire.internal.cache.tier.MessageType;
 import com.gemstone.gemfire.internal.cache.tier.sockets.command.Default;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
@@ -38,29 +61,15 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
 import com.gemstone.gemfire.internal.security.AuthorizeRequestPP;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.internal.util.Breadcrumbs;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.AuthenticationRequiredException;
 import com.gemstone.gemfire.security.GemFireSecurityException;
-import org.apache.logging.log4j.Logger;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectableChannel;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.security.Principal;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.LinkedBlockingQueue;
 
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
+import org.apache.logging.log4j.Logger;
+import org.apache.shiro.subject.Subject;
+import org.apache.shiro.util.ThreadState;
 
 /**
  * Provides an implementation for the server socket end of the hierarchical
@@ -749,6 +758,7 @@ public class ServerConnection implements Runnable {
   private void doNormalMsg() {
     Message msg = null;
     msg = BaseCommand.readRequest(this);
+    ThreadState threadState = null;
     try {
       if (msg != null) {
         //this.logger.fine("donormalMsg() msgType " + msg.getMessageType());
@@ -794,6 +804,14 @@ public class ServerConnection implements Runnable {
         if (command == null) {
           command = Default.getCommand();
         }
+
+        // if a subject exists for this uniqueId, binds the subject to this thread so that we can do authorization later
+        if(AcceptorImpl.isIntegratedSecurity()) {
+          long uniqueId = getUniqueId();
+          Subject subject = this.clientUserAuths.getSubject(uniqueId);
+          threadState = GeodeSecurityUtil.bindSubject(subject);
+        }
+
         command.execute(msg, this);
       }
     }
@@ -802,6 +820,9 @@ public class ServerConnection implements Runnable {
       // processed.
       setNotProcessingMessage();
       clearRequestMsg();
+      if(threadState!=null){
+        threadState.clear();
+      }
     }
 
   }
@@ -970,7 +991,14 @@ public class ServerConnection implements Runnable {
       }
       
       try {
-        return this.clientUserAuths.removeUserId(aIds.getUniqueId(), keepalive);
+        // first try integrated security
+        boolean removed = this.clientUserAuths.removeSubject(aIds.getUniqueId());
+
+        // if not successfull, try the old way
+        if(!removed)
+          removed = this.clientUserAuths.removeUserId(aIds.getUniqueId(), keepalive);
+        return removed;
+
       } catch (NullPointerException npe) {
         // Bug #52023.
         logger.debug("Exception {}", npe);
@@ -1008,7 +1036,7 @@ public class ServerConnection implements Runnable {
         return new byte[0];
       }
       if (!msg.isSecureMode()) {
-        //throw exception not authorized 
+        throw new  AuthenticationFailedException("Authentication failed");
       }
       
       byte [] secureBytes = msg.getSecureBytes();
@@ -1032,18 +1060,28 @@ public class ServerConnection implements Runnable {
       ByteArrayInputStream bis = new ByteArrayInputStream(credBytes);
       DataInputStream dinp = new DataInputStream(bis);
       Properties credentials = DataSerializer.readProperties(dinp);
-      
-      
-      DistributedSystem system = this.getDistributedSystem();
-      String methodName = system.getProperties().getProperty(
+
+      String username = credentials.getProperty("security-username");
+      String password = credentials.getProperty("security-password");
+
+      // When here, security is enfored on server, if login returns a subject, then it's the newly integrated security, otherwise, do it the old way.
+      long uniqueId;
+      Subject subject = GeodeSecurityUtil.login(username, password);
+      if(subject!=null){
+        uniqueId = this.clientUserAuths.putSubject(subject);
+      }
+      else {
+        DistributedSystem system = this.getDistributedSystem();
+        String methodName = system.getProperties().getProperty(
           SECURITY_CLIENT_AUTHENTICATOR);
-      
-      Principal principal = HandShake.verifyCredentials(methodName, credentials,
-          system.getSecurityProperties(), (InternalLogWriter)system.getLogWriter(), (InternalLogWriter)system
-              .getSecurityLogWriter(), this.proxyId.getDistributedMember());
-  
-      //this sets principal in map as well....
-      long uniqueId = ServerHandShakeProcessor.getUniqueId(this, principal);
+
+        Principal principal = HandShake.verifyCredentials(methodName, credentials,
+          system.getSecurityProperties(), (InternalLogWriter) system.getLogWriter(), (InternalLogWriter) system
+            .getSecurityLogWriter(), this.proxyId.getDistributedMember());
+
+        //this sets principal in map as well....
+        uniqueId = ServerHandShakeProcessor.getUniqueId(this, principal);
+      }
       
       //create secure part which will be send in respones    
       
@@ -1356,185 +1394,11 @@ public class ServerConnection implements Runnable {
       return LocalizedStrings.ServerConnection_ERROR_IN_GETSOCKETSTRING_0.toLocalizedString(e.getLocalizedMessage());
     }
   }
-
-  
-
-  
-
-  
-
-  
-  
-
-  
-
-//  private void writePingReply(Message origMsg) throws IOException {
-//    replyMsg.setMessageType(MessageType.REPLY);
-//    replyMsg.setNumberOfParts(1);
-//    replyMsg.setTransactionId(origMsg.getTransactionId());
-//    replyMsg.addBytesPart(OK_BYTES);
-//    replyMsg.send(logger, origMsg.getTransactionId());
-//    if (logger.finerEnabled()) {
-//      logger.finer(getName() + ": rpl tx: " + origMsg.getTransactionId());
-//    }
-//  }
-
-  
-
-  
-
-  
-//  private void writeBatchException(Message origMsg, String message, int index) throws IOException {
-//    Exception be = new BatchException(message, index);
-//    errorMsg.setMessageType(MessageType.EXCEPTION);
-//    errorMsg.setNumberOfParts(2);
-//    errorMsg.setTransactionId(origMsg.getTransactionId());
-//    errorMsg.addObjPart(be);
-//    errorMsg.addStringPart(be.toString());
-//    errorMsg.send();
-//    if (logger.fineEnabled()) {
-//      logger.fine(this.name + ": Wrote batch exception: ", be);
-//    }
-//  }
-
  
   void clearRequestMsg() {
     requestMsg.clear();
   }
 
-  
-
-  
-
-//   /**
-//    * Examine an entry, and build an InterestEvent for it
-//    * @param region region we're fetching from
-//    * @param entryKey entry key that we may want the event for
-//    * @return the event or null if entry does not exist
-//    */
-//   private InterestEvent getInterestEvent(Region region, Object entryKey)
-//   {
-//     Region.Entry entry = null;
-//     try {
-//       entry = region.getEntry(entryKey);
-//     } catch (Exception likelyAPartitionedRegion) { // ignore, change when a partitioned region supports getEntry
-//     }
-//     if (entry == null) {
-//       return null;
-//     }
-//     if (entry instanceof LocalRegion.NonTXEntry) {
-//       final LocalRegion.NonTXEntry regionEntry = (LocalRegion.NonTXEntry)entry;
-//       boolean isDeserialized = true;
-//       // Get the value in the VM
-//       Object value = regionEntry.getRegionEntry().getValueInVM();
-//       // If the value in the VM is a CachedDeserializable,
-//       // get its value. If it is Token.REMOVED, Token.DESTROYED,
-//       // Token.INVALID, or Token.LOCAL_INVALID
-//       // set it to null. If it is NOT_AVAILABLE, get the value from
-//       // disk. If it is already a byte[], set isObject to false.
-//       if (value instanceof CachedDeserializable) {
-//         value = ((CachedDeserializable)value).getValue();
-//         isDeserialized = !(value instanceof byte[]);
-//       }
-//       else if (value == Token.REMOVED || value == Token.DESTROYED) {
-//         return null;
-//       }
-//       else if (value == Token.INVALID || value == Token.LOCAL_INVALID) {
-//         return null; // fix for bug 35884
-//       }
-//       else if (value instanceof byte[]) {
-//         // key, value, and isDeserialized already set
-//       }
-//       else if (value == EntryEvent.NOT_AVAILABLE) {
-//         // This will occur with a disk region entry where the value
-//         // is on disk. Currently the getValue call will deserialize
-//         // the value. This means that for disk regions, value classes
-//         // must exist on the server. If this code is changed, look at
-//         // the run method above for similar code to change.
-//         value = regionEntry.getRegionEntry().getValue((LocalRegion)region);
-//         if (value instanceof CachedDeserializable) {
-//           value = ((CachedDeserializable)value).getValue();
-//           isDeserialized = !(value instanceof byte[]);
-//         }
-//       }
-//       return new InterestEvent(entryKey, value, isDeserialized);
-//     } else {
-//       return null;
-//     }
-//   }
-
-//   /**
-//    * Process an interest request of type {@link InterestType#FILTER_CLASS}
-//    * @param region the region
-//    * @param className the key
-//    * @param policy the policy
-//    * @throws IOException
-//    */
-//   private void handleFilter(LocalRegion region, String className,
-//       InterestResultPolicy policy) throws IOException
-//   {
-//     ArrayList keyList = new ArrayList(this.maximumChunkSize);
-
-//     // Handle the filtering class pattern
-
-//     Class filterClass;
-//     InterestFilter filter;
-//     try {
-//       filterClass = ClassLoadUtil.classFromName((String) className);
-//       filter = (InterestFilter) filterClass.newInstance();
-//     } catch(ClassNotFoundException cnfe) {
-//       throw new RuntimeException("Class " + className + " not found in classpath.", cnfe);
-//     } catch(Exception e) {
-//       throw new RuntimeException("Class " + className + " could not be instantiated.", e);
-//     }
-
-// //    if(!(filter instanceof InterestFilter)) {
-// //      throw new RuntimeException("Class " + key + " does not implement InterestFilter.");
-// //    }
-
-//     for (Iterator it = region.keys().iterator(); it.hasNext();) {
-//       Object entryKey = it.next();
-//       InterestEvent ie = getInterestEvent(region, entryKey);
-//       if (ie == null) {
-//         // key no longer existed so skip it
-//         continue;
-//       }
-//       if(!filter.notifyOnRegister(ie)) {
-//         //the filter does not want to know about this entry, so skip it.
-//          continue;
-//       }
-//       appendInterestResponseKey(region, className, entryKey, keyList, "filter list");
-//     }
-//     // Send the last chunk (the only chunk for individual and list keys)
-//     // always send it back, even if the list is of zero size.
-//     sendRegisterInterestResponseChunk(region, className, keyList, true);
-//   }
-
- 
-
-//  /**
-//   * Process an interest request of type {@link InterestType#FILTER_CLASS}
-//   * @param region the region
-//   * @param className the key
-//   * @param policy the policy
-//   * @throws IOException
-//   */
-//  private void handleFilterPR(PartitionedRegion region, String className,
-//      InterestResultPolicy policy) throws IOException
-//  {
-//    // this interest type isn't exposed to the customer yet
-//    throw new UnsupportedOperationException("filter classes not yet supporeted");
-//  }
-
-  
-  
- 
-  
-
-  
-
-  
-
   public void incrementLatestBatchIdReplied(int justProcessed) {
     // not synchronized because it only has a single caller
   if(justProcessed-this.latestBatchIdReplied!=1) {
@@ -1630,17 +1494,6 @@ public class ServerConnection implements Runnable {
         /*|| this.communicationMode == Acceptor.CLIENT_TO_SERVER_FOR_QUEUE*/) {
       getAcceptor().decClientServerCnxCount();
     }
-//       if (logger.fineEnabled()) {
-//         logger.fine (this.name + ": about to close socket");
-//       }
-//       try {
-//         theSocket.shutdownInput();
-//       } catch (Exception e) {
-//       }
-//       try {
-//         theSocket.shutdownOutput();
-//       } catch (Exception e) {
-//       }
     try {
       theSocket.close();
     } catch (Exception e) {
@@ -1944,146 +1797,116 @@ public class ServerConnection implements Runnable {
       hdos.close();
     }
   }
-  
-  public AuthorizeRequest getAuthzRequest() 
-      throws AuthenticationRequiredException, IOException {
-    //look client version and return authzrequest
-    //for backward client it will be store in member variable userAuthId 
-    //for other look "requestMsg" here and get unique-id from this to get the authzrequest
-    
-    if (AcceptorImpl.isAuthenticationRequired()) {
-      long uniqueId = 0;
-      
-      if (this.handshake.getVersion().compareTo(Version.GFE_65) < 0
-          || this.communicationMode == Acceptor.GATEWAY_TO_GATEWAY) {
-        uniqueId = this.userAuthId;
-      } else {
-        try {
-          //this.logger.fine("getAuthzRequest() isSecureMode = " + this.requestMsg.isSecureMode());
-          if (this.requestMsg.isSecureMode()) {
-            //get uniqueID from message
-            byte [] secureBytes = this.requestMsg.getSecureBytes();
-
-            secureBytes =  ((HandShake)this.handshake).decryptBytes(secureBytes);
-            AuthIds aIds = new AuthIds(secureBytes);
-
-            if ( /*this.connectionId != Connection.DEFAULT_CONNECTION_ID &&*/ this.connectionId != aIds.getConnectionId()) {
-              throw new AuthenticationRequiredException(
-                  LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED
-                    .toLocalizedString());
-            } else {
-              uniqueId = aIds.getUniqueId();
-            }
-            
-          } else {
+
+  public long getUniqueId(){
+    long uniqueId = 0;
+
+    if (this.handshake.getVersion().compareTo(Version.GFE_65) < 0
+      || this.communicationMode == Acceptor.GATEWAY_TO_GATEWAY) {
+      uniqueId = this.userAuthId;
+    } else {
+      try {
+        //this.logger.fine("getAuthzRequest() isSecureMode = " + this.requestMsg.isSecureMode());
+        if (this.requestMsg.isSecureMode()) {
+          //get uniqueID from message
+          byte [] secureBytes = this.requestMsg.getSecureBytes();
+
+          secureBytes =  ((HandShake)this.handshake).decryptBytes(secureBytes);
+          AuthIds aIds = new AuthIds(secureBytes);
+
+          if (this.connectionId != aIds.getConnectionId()) {
             throw new AuthenticationRequiredException(
-                LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED
-                  .toLocalizedString());
-          }
-        } catch (AuthenticationRequiredException are) {
-          throw are;
-        }
-        catch(Exception ex ) {
-          throw new AuthenticationRequiredException(
               LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED
                 .toLocalizedString());
-        }
-      }
-      UserAuthAttributes uaa = null;
-      try {
-        uaa = this.clientUserAuths.getUserAuthAttributes(uniqueId);
-      } catch (NullPointerException npe) {
-        if (this.isTerminated()) {
-          // Bug #52023.
-          throw new IOException("Server connection is terminated.");
+          } else {
+            uniqueId = aIds.getUniqueId();
+          }
+
         } else {
-          logger.debug("Unexpected exception {}", npe);
+          throw new AuthenticationRequiredException(
+            LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED
+              .toLocalizedString());
         }
+      } catch (AuthenticationRequiredException are) {
+        throw are;
       }
-      if (uaa == null) {
+      catch(Exception ex ) {
         throw new AuthenticationRequiredException(
-            "User authorization attributes not found.");
+          LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED
+            .toLocalizedString());
       }
-      AuthorizeRequest authReq = uaa.getAuthzRequest();
-      if (logger.isDebugEnabled()) {
-        logger.debug("getAuthzRequest() authrequest: {}", ((authReq == null) ? "NULL (only authentication is required)" : "not null"));
-      }
-      
-      return authReq;
     }
-    else {
+    return uniqueId;
+  }
+
+  public AuthorizeRequest getAuthzRequest()
+      throws AuthenticationRequiredException, IOException {
+    //look client version and return authzrequest
+    //for backward client it will be store in member variable userAuthId
+    //for other look "requestMsg" here and get unique-id from this to get the authzrequest
+
+    if (!AcceptorImpl.isAuthenticationRequired())
       return null;
+
+    if(AcceptorImpl.isIntegratedSecurity())
+      return null;
+
+    long uniqueId = getUniqueId();
+
+    UserAuthAttributes uaa = null;
+    try {
+      uaa = this.clientUserAuths.getUserAuthAttributes(uniqueId);
+    } catch (NullPointerException npe) {
+      if (this.isTerminated()) {
+        // Bug #52023.
+        throw new IOException("Server connection is terminated.");
+      } else {
+        logger.debug("Unexpected exception {}", npe);
+      }
     }
+    if (uaa == null) {
+      throw new AuthenticationRequiredException(
+          "User authorization attributes not found.");
+    }
+    AuthorizeRequest authReq = uaa.getAuthzRequest();
+    if (logger.isDebugEnabled()) {
+      logger.debug("getAuthzRequest() authrequest: {}", ((authReq == null) ? "NULL (only authentication is required)" : "not null"));
+    }
+    return authReq;
   }
 
   public AuthorizeRequestPP getPostAuthzRequest() 
   throws AuthenticationRequiredException, IOException {
-  //look client version and return authzrequest
-  //for backward client it will be store in member variable userAuthId 
-  //for other look "requestMsg" here and get unique-id from this to get the authzrequest
-    if (AcceptorImpl.isAuthenticationRequired()) {
-      long uniqueId = 0;
-      
-      if (this.handshake.getVersion().compareTo(Version.GFE_65) < 0
-          || this.communicationMode == Acceptor.GATEWAY_TO_GATEWAY) {
-        uniqueId = this.userAuthId;
+    if (!AcceptorImpl.isAuthenticationRequired())
+      return null;
+
+    if(AcceptorImpl.isIntegratedSecurity())
+      return null;
+
+    //look client version and return authzrequest
+    //for backward client it will be store in member variable userAuthId
+    //for other look "requestMsg" here and get unique-id from this to get the authzrequest
+    long uniqueId = getUniqueId();
+
+    UserAuthAttributes uaa = null;
+    try {
+      uaa = this.clientUserAuths.getUserAuthAttributes(uniqueId);
+    } catch (NullPointerException npe) {
+      if (this.isTerminated()) {
+        // Bug #52023.
+        throw new IOException("Server connection is terminated.");
       } else {
-        try {
-          //this.logger.fine("getPostAuthzRequest() isSecureMode = " + this.requestMsg.isSecureMode());
-          if (this.requestMsg.isSecureMode()) {
-            byte [] secureBytes = this.requestMsg.getSecureBytes();
-            
-            secureBytes =  ((HandShake)this.handshake).decryptBytes(secureBytes);
-            
-            AuthIds aIds = new AuthIds(secureBytes);
-            if ( /*this.connectionId != Connection.DEFAULT_CONNECTION_ID && */this.connectionId != aIds.getConnectionId()) {
-              throw new AuthenticationRequiredException(
-                  LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED
-                    .toLocalizedString());
-            } else {
-              uniqueId = aIds.getUniqueId();
-            }
-            
-          } 
-          else{
-            throw new AuthenticationRequiredException(
-                LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED
-                  .toLocalizedString());
-          }
-        } catch (AuthenticationRequiredException are) {
-          throw are;
-        }
-        catch(Exception ex) {
-          throw new AuthenticationRequiredException(
-              LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED
-                .toLocalizedString());
-        }
+        logger.debug("Unexpected exception {}", npe);
       }
-      
-      UserAuthAttributes uaa = null;
-      try {
-        uaa = this.clientUserAuths.getUserAuthAttributes(uniqueId);
-      } catch (NullPointerException npe) {
-        if (this.isTerminated()) {
-          // Bug #52023.
-          throw new IOException("Server connection is terminated.");
-        } else {
-          logger.debug("Unexpected exception {}", npe);
-        }
-      }
-      if (uaa == null) {
-        throw new AuthenticationRequiredException(
-            "User authorization attributes not found.");
-      }
-      
-      AuthorizeRequestPP postAuthReq = uaa.getPostAuthzRequest();
-      
-      return postAuthReq;
     }
-    else
-      return null;
-    
-    //return this.postAuthzRequest;
+    if (uaa == null) {
+      throw new AuthenticationRequiredException(
+          "User authorization attributes not found.");
+    }
+
+    AuthorizeRequestPP postAuthReq = uaa.getPostAuthzRequest();
+
+    return postAuthReq;
   }
 
   /** returns the member ID byte array to be used for creating EventID objects */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Get70.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Get70.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Get70.java
index 55047c7..5154652 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Get70.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Get70.java
@@ -19,13 +19,15 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.client.internal.GetOp;
 import com.gemstone.gemfire.cache.operations.GetOperationContext;
 import com.gemstone.gemfire.cache.operations.internal.GetOperationContextImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionStats;
+import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.Token;
@@ -46,10 +48,8 @@ import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
 import com.gemstone.gemfire.internal.security.AuthorizeRequestPP;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.security.NotAuthorizedException;
-import com.gemstone.gemfire.i18n.StringId;
-
-import java.io.IOException;
 
 public class Get70 extends BaseCommand {
 
@@ -140,111 +140,114 @@ public class Get70 extends BaseCommand {
       String s = errMessage.toLocalizedString();
       logger.warn("{}: {}", servConn.getName(), s);
       writeErrorResponse(msg, MessageType.REQUESTDATAERROR, s, servConn);
-      // responded = true;
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      Region region = crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = LocalizedStrings.Request__0_WAS_NOT_FOUND_DURING_GET_REQUEST.toLocalizedString(regionName);
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
+
+    // for integrated security
+    GeodeSecurityUtil.authorizeRegionRead(regionName);
+
+    Region region = crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = LocalizedStrings.Request__0_WAS_NOT_FOUND_DURING_GET_REQUEST.toLocalizedString(regionName);
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    GetOperationContext getContext = null;
+    try {
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      if (authzRequest != null) {
+        getContext = authzRequest
+          .getAuthorize(regionName, key, callbackArg);
+        callbackArg = getContext.getCallbackArg();
       }
-      else {
-        
-        GetOperationContext getContext = null;
-        
+    }
+    catch (NotAuthorizedException ex) {
+      writeException(msg, ex, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    // Get the value and update the statistics. Do not deserialize
+    // the value if it is a byte[].
+    Entry entry;
+    try {
+      entry = getEntry(region, key, callbackArg, servConn);
+    }
+    catch (Exception e) {
+      writeException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    @Retained final Object originalData = entry.value;
+    Object data = originalData;
+    try {
+      boolean isObject = entry.isObject;
+      VersionTag versionTag = entry.versionTag;
+      boolean keyNotPresent = entry.keyNotPresent;
+
+
+      try {
+        AuthorizeRequestPP postAuthzRequest = servConn.getPostAuthzRequest();
+        if (postAuthzRequest != null) {
           try {
-            AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-              if (authzRequest != null) {
-              getContext = authzRequest
-                  .getAuthorize(regionName, key, callbackArg);
-              callbackArg = getContext.getCallbackArg();
+            getContext = postAuthzRequest.getAuthorize(regionName, key, data,
+              isObject, getContext);
+            GetOperationContextImpl gci = (GetOperationContextImpl) getContext;
+            Object newData = gci.getRawValue();
+            if (newData != data) {
+              // user changed the value
+              isObject = getContext.isObject();
+              data = newData;
             }
           }
-          catch (NotAuthorizedException ex) {
-            writeException(msg, ex, false, servConn);
-            servConn.setAsTrue(RESPONDED);
-            return;
-          }
-
-        // Get the value and update the statistics. Do not deserialize
-        // the value if it is a byte[].
-        Entry entry;
-        try {
-          entry = getEntry(region, key, callbackArg, servConn);
-        }
-        catch (Exception e) {
-          writeException(msg, e, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
-
-        @Retained final Object originalData = entry.value;
-        Object data = originalData;
-        try {
-        boolean isObject = entry.isObject;
-        VersionTag versionTag = entry.versionTag;
-        boolean keyNotPresent = entry.keyNotPresent;
-        
-        
-        try {
-          AuthorizeRequestPP postAuthzRequest = servConn.getPostAuthzRequest();
-          if (postAuthzRequest != null) {
-            try {
-              getContext = postAuthzRequest.getAuthorize(regionName, key, data,
-                  isObject, getContext);
-              GetOperationContextImpl gci = (GetOperationContextImpl) getContext;
-              Object newData = gci.getRawValue();
-              if (newData != data) {
-                // user changed the value
-                isObject = getContext.isObject();
-                data = newData;
-              }
-            } finally {
-              if (getContext != null) {
-                ((GetOperationContextImpl)getContext).release();
-              }
+          finally {
+            if (getContext != null) {
+              ((GetOperationContextImpl) getContext).release();
             }
           }
         }
-        catch (NotAuthorizedException ex) {
-          writeException(msg, ex, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
-        {
-          long oldStart = start;
-          start = DistributionStats.getStatTime();
-          stats.incProcessGetTime(start - oldStart);
-        }
-        
-        if (region instanceof PartitionedRegion) {
-          PartitionedRegion pr = (PartitionedRegion)region;
-          if (pr.isNetworkHop() != (byte)0) {
-            writeResponseWithRefreshMetadata(data, callbackArg, msg, isObject,
-                servConn, pr, pr.isNetworkHop(), versionTag, keyNotPresent);
-            pr.setIsNetworkHop((byte)0);
-            pr.setMetadataVersion(Byte.valueOf((byte)0));
-          }
-          else {
-            writeResponse(data, callbackArg, msg, isObject, versionTag, keyNotPresent, servConn);
-          }
+      }
+      catch (NotAuthorizedException ex) {
+        writeException(msg, ex, false, servConn);
+        servConn.setAsTrue(RESPONDED);
+        return;
+      }
+      {
+        long oldStart = start;
+        start = DistributionStats.getStatTime();
+        stats.incProcessGetTime(start - oldStart);
+      }
+
+      if (region instanceof PartitionedRegion) {
+        PartitionedRegion pr = (PartitionedRegion) region;
+        if (pr.isNetworkHop() != (byte) 0) {
+          writeResponseWithRefreshMetadata(data, callbackArg, msg, isObject,
+            servConn, pr, pr.isNetworkHop(), versionTag, keyNotPresent);
+          pr.setIsNetworkHop((byte) 0);
+          pr.setMetadataVersion(Byte.valueOf((byte) 0));
         }
         else {
           writeResponse(data, callbackArg, msg, isObject, versionTag, keyNotPresent, servConn);
         }
-        } finally {
-          OffHeapHelper.release(originalData);
-        }
-        
-        servConn.setAsTrue(RESPONDED);
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Wrote get response back to {} for region {} {}", servConn.getName(), servConn.getSocketString(), regionName, entry);
-        }
-        stats.incWriteGetResponseTime(DistributionStats.getStatTime() - start);
+      }
+      else {
+        writeResponse(data, callbackArg, msg, isObject, versionTag, keyNotPresent, servConn);
       }
     }
+    finally {
+      OffHeapHelper.release(originalData);
+    }
+
+    servConn.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Wrote get response back to {} for region {} {}", servConn.getName(), servConn.getSocketString(), regionName, entry);
+    }
+    stats.incWriteGetResponseTime(DistributionStats.getStatTime() - start);
+
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveUserAuth.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveUserAuth.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveUserAuth.java
index ac6023a..bba1944 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveUserAuth.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveUserAuth.java
@@ -39,45 +39,42 @@ public class RemoveUserAuth extends BaseCommand {
       throws IOException, ClassNotFoundException, InterruptedException {
     boolean isSecureMode = msg.isSecureMode();
     
-   // if (!isSecureMode)
-     //client has not send secuirty header, need to send exception and log this in security (file)
-
-    if (isSecureMode) {
+    if (!isSecureMode){
+     // need to throw exception
+     return;
+    }
 
-      try {
-        servConn.setAsTrue(REQUIRES_RESPONSE);
-        Part keepalivePart = msg.getPart(0);
-        byte[] keepaliveByte = keepalivePart.getSerializedForm();
-        boolean keepalive = (keepaliveByte == null || keepaliveByte[0] == 0) ? false
-            : true;
-        servConn.getSecurityLogWriter().fine(
-            "remove user auth keep alive " + keepalive);
-        servConn.removeUserAuth(msg, keepalive);
-        writeReply(msg, servConn);
-      } catch (GemFireSecurityException gfse) {
-        if (servConn.getSecurityLogWriter().warningEnabled()) {
-          servConn.getSecurityLogWriter()
-              .warning(
-                  LocalizedStrings.ONE_ARG,
-                  servConn.getName() + ": Security exception: "
-                      + gfse.getMessage());
-        }
-        writeException(msg, gfse, false, servConn);
-      } catch (Exception ex) {
-        // TODO Auto-generated catch block
-        if (servConn.getLogWriter().warningEnabled()) {
-          servConn
-              .getLogWriter()
-              .warning(
-                  LocalizedStrings.CacheClientNotifier_AN_EXCEPTION_WAS_THROWN_FOR_CLIENT_0_1,
-                  new Object[] {servConn.getProxyID(), ""}, ex);
-        }
-        writeException(msg, ex, false, servConn);
-      } finally {
-        servConn.setAsTrue(RESPONDED);
+    try {
+      servConn.setAsTrue(REQUIRES_RESPONSE);
+      Part keepalivePart = msg.getPart(0);
+      byte[] keepaliveByte = keepalivePart.getSerializedForm();
+      boolean keepalive = (keepaliveByte == null || keepaliveByte[0] == 0) ? false
+          : true;
+      servConn.getSecurityLogWriter().fine(
+          "remove user auth keep alive " + keepalive);
+      servConn.removeUserAuth(msg, keepalive);
+      writeReply(msg, servConn);
+    } catch (GemFireSecurityException gfse) {
+      if (servConn.getSecurityLogWriter().warningEnabled()) {
+        servConn.getSecurityLogWriter()
+            .warning(
+                LocalizedStrings.ONE_ARG,
+                servConn.getName() + ": Security exception: "
+                    + gfse.getMessage());
+      }
+      writeException(msg, gfse, false, servConn);
+    } catch (Exception ex) {
+      // TODO Auto-generated catch block
+      if (servConn.getLogWriter().warningEnabled()) {
+        servConn
+            .getLogWriter()
+            .warning(
+                LocalizedStrings.CacheClientNotifier_AN_EXCEPTION_WAS_THROWN_FOR_CLIENT_0_1,
+                new Object[] {servConn.getProxyID(), ""}, ex);
       }
-    } else {
-      // need to throw exception
+      writeException(msg, ex, false, servConn);
+    } finally {
+      servConn.setAsTrue(RESPONDED);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
index 1a4f91b..fa499d9 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
@@ -17,17 +17,14 @@
 
 package com.gemstone.gemfire.internal.security;
 
-import com.gemstone.gemfire.cache.operations.OperationContext;
-import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import com.gemstone.gemfire.cache.operations.OperationContext.Resource;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.security.shiro.CustomAuthRealm;
-import com.gemstone.gemfire.internal.security.shiro.ShiroPrincipal;
-import com.gemstone.gemfire.management.internal.security.ResourceOperation;
-import com.gemstone.gemfire.management.internal.security.ResourceOperationContext;
-import com.gemstone.gemfire.security.AuthenticationFailedException;
-import com.gemstone.gemfire.security.GemFireSecurityException;
-import com.gemstone.gemfire.security.NotAuthorizedException;
+import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
+
+import java.lang.reflect.Method;
+import java.security.AccessController;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Callable;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.Logger;
 import org.apache.shiro.SecurityUtils;
@@ -40,14 +37,23 @@ import org.apache.shiro.mgt.DefaultSecurityManager;
 import org.apache.shiro.mgt.SecurityManager;
 import org.apache.shiro.realm.Realm;
 import org.apache.shiro.subject.Subject;
+import org.apache.shiro.subject.support.SubjectThreadState;
 import org.apache.shiro.util.ThreadContext;
+import org.apache.shiro.util.ThreadState;
 
-import java.security.AccessController;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.Callable;
-
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
+import com.gemstone.gemfire.internal.ClassLoadUtil;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.security.shiro.CustomAuthRealm;
+import com.gemstone.gemfire.internal.security.shiro.ShiroPrincipal;
+import com.gemstone.gemfire.management.internal.security.ResourceOperation;
+import com.gemstone.gemfire.security.AuthenticationFailedException;
+import com.gemstone.gemfire.security.AuthenticationRequiredException;
+import com.gemstone.gemfire.security.ExternalSecurity;
+import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.security.GeodePermission;
+import com.gemstone.gemfire.security.GeodePermission.Operation;
+import com.gemstone.gemfire.security.GeodePermission.Resource;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 
 public class GeodeSecurityUtil {
 
@@ -56,12 +62,12 @@ public class GeodeSecurityUtil {
   /**
    * It first looks the shiro subject in AccessControlContext since JMX will use multiple threads to process operations from the same client.
    * then it looks into Shiro's thead context.
-   *
    * @return the shiro subject, null if security is not enabled
    */
-  public static Subject getSubject(){
-    if(!isSecured())
+  public static Subject getSubject() {
+    if (!isSecured()) {
       return null;
+    }
 
     Subject currentUser = null;
 
@@ -70,9 +76,9 @@ public class GeodeSecurityUtil {
     javax.security.auth.Subject jmxSubject =
       javax.security.auth.Subject.getSubject(AccessController.getContext());
 
-    if(jmxSubject!=null){
+    if (jmxSubject != null) {
       Set<ShiroPrincipal> principals = jmxSubject.getPrincipals(ShiroPrincipal.class);
-      if(principals.size()>0){
+      if (principals.size() > 0) {
         ShiroPrincipal principal = principals.iterator().next();
         currentUser = principal.getSubject();
         ThreadContext.bind(currentUser);
@@ -83,7 +89,7 @@ public class GeodeSecurityUtil {
     // in other cases like admin rest call or pulse authorization
     currentUser = SecurityUtils.getSubject();
 
-    if(currentUser==null || currentUser.getPrincipal()==null){
+    if (currentUser == null || currentUser.getPrincipal() == null) {
       throw new GemFireSecurityException("Error: Anonymous User");
     }
 
@@ -91,23 +97,24 @@ public class GeodeSecurityUtil {
   }
 
   /**
-   *
    * @param username
    * @param password
    * @return null if security is not enabled, otherwise return a shiro subject
    */
-  public static Subject login(String username, String password){
-    if(!isSecured())
+  public static Subject login(String username, String password) {
+    if (!isSecured()) {
       return null;
+    }
 
     Subject currentUser = SecurityUtils.getSubject();
 
     UsernamePasswordToken token =
       new UsernamePasswordToken(username, password);
     try {
-      logger.info("Logging in "+username);
+      logger.info("Logging in " + username);
       currentUser.login(token);
-    } catch (ShiroException e) {
+    }
+    catch (ShiroException e) {
       logger.info(e.getMessage(), e);
       throw new AuthenticationFailedException("Authentication error. Please check your username/password.", e);
     }
@@ -115,16 +122,17 @@ public class GeodeSecurityUtil {
     return currentUser;
   }
 
-  public static void logout(){
+  public static void logout() {
     Subject currentUser = getSubject();
-    if(currentUser==null)
+    if (currentUser == null) {
       return;
+    }
 
     try {
-      logger.info("Logging out "+currentUser.getPrincipal());
+      logger.info("Logging out " + currentUser.getPrincipal());
       currentUser.logout();
     }
-    catch(ShiroException e){
+    catch (ShiroException e) {
       logger.info(e.getMessage(), e);
       throw new GemFireSecurityException(e.getMessage(), e);
     }
@@ -132,93 +140,123 @@ public class GeodeSecurityUtil {
     ThreadContext.remove();
   }
 
-  public static Callable associateWith(Callable callable){
+  public static Callable associateWith(Callable callable) {
     Subject currentUser = getSubject();
-    if(currentUser==null)
+    if (currentUser == null) {
       return callable;
+    }
 
     return currentUser.associateWith(callable);
   }
 
+  /**
+   * this binds the passed-in subject to the executing thread, normally, you would do this:
+   * ThreadState state = null;
+   * try{
+   *   state = GeodeSecurityUtil.bindSubject(subject);
+   *   //do the rest of the work as this subject
+   * }
+   * finally{
+   *   if(state!=null)
+   *      state.clear();
+   * }
+   *
+   * @param subject
+   * @return
+   */
+  public static ThreadState bindSubject(Subject subject){
+    if (subject == null) {
+      return null;
+    }
+
+    ThreadState threadState = new SubjectThreadState(subject);
+    threadState.bind();
+    return threadState;
+  }
+
   public static void authorize(ResourceOperation resourceOperation) {
-    if(resourceOperation==null)
+    if (resourceOperation == null) {
       return;
+    }
 
     authorize(resourceOperation.resource().name(),
       resourceOperation.operation().name(),
       null);
   }
 
-  public static void authorizeClusterManage(){
+  public static void authorizeClusterManage() {
     authorize("CLUSTER", "MANAGE");
   }
 
-  public static void authorizeClusterWrite(){
+  public static void authorizeClusterWrite() {
     authorize("CLUSTER", "WRITE");
   }
 
-  public static void authorizeClusterRead(){
+  public static void authorizeClusterRead() {
     authorize("CLUSTER", "READ");
   }
 
-  public static void authorizeDataManage(){
+  public static void authorizeDataManage() {
     authorize("DATA", "MANAGE");
   }
 
-  public static void authorizeDataWrite(){
+  public static void authorizeDataWrite() {
     authorize("DATA", "WRITE");
   }
 
-  public static void authorizeDataRead(){
+  public static void authorizeDataRead() {
     authorize("DATA", "READ");
   }
 
-  public static void authorizeRegionManage(String regionName){
+  public static void authorizeRegionManage(String regionName) {
     authorize("DATA", "MANAGE", regionName);
   }
 
-  public static void authorizeRegionWrite(String regionName){
+  public static void authorizeRegionWrite(String regionName) {
     authorize("DATA", "WRITE", regionName);
   }
 
-  public static void authorizeRegionRead(String regionName){
+  public static void authorizeRegionRead(String regionName) {
     authorize("DATA", "READ", regionName);
   }
 
-  public static void authorize(String resource, String operation){
+  public static void authorize(String resource, String operation) {
     authorize(resource, operation, null);
   }
 
-  private static void authorize(String resource, String operation, String regionName){
+  private static void authorize(String resource, String operation, String regionName) {
     regionName = StringUtils.stripStart(regionName, "/");
-    authorize(new ResourceOperationContext(resource, operation, regionName));
+    authorize(new GeodePermission(resource, operation, regionName));
   }
 
-  public static void authorize(OperationContext context) {
-    if(context==null)
+  public static void authorize(GeodePermission context) {
+    Subject currentUser = getSubject();
+    if (currentUser == null) {
       return;
+    }
 
-    if(context.getResource()== Resource.NULL && context.getOperationCode()== OperationCode.NULL)
+    if (context == null) {
       return;
+    }
 
-    Subject currentUser = getSubject();
-    if(currentUser==null)
+    if (context.getResource() == Resource.NULL && context.getOperation() == Operation.NULL) {
       return;
+    }
 
     try {
       currentUser.checkPermission(context);
     }
-    catch(ShiroException e){
+    catch (ShiroException e) {
       logger.info(currentUser.getPrincipal() + " not authorized for " + context);
       throw new NotAuthorizedException(e.getMessage(), e);
     }
   }
 
-  private static boolean isSecured(){
-    try{
+  private static boolean isSecured() {
+    try {
       SecurityUtils.getSecurityManager();
     }
-    catch(UnavailableSecurityManagerException e){
+    catch (UnavailableSecurityManagerException e) {
       return false;
     }
     return true;
@@ -228,33 +266,59 @@ public class GeodeSecurityUtil {
    * initialize Shiro's Security Manager and Security Utilities
    * @param securityProps
    */
-  public static void initSecurity(Properties securityProps){
-    if(securityProps==null)
+  public static void initSecurity(Properties securityProps) {
+    if (securityProps == null) {
       return;
+    }
 
     String shiroConfig = securityProps.getProperty(SECURITY_SHIRO_INIT);
-    String customAuthenticator =securityProps.getProperty(SECURITY_CLIENT_AUTHENTICATOR);
+    String customAuthenticator = securityProps.getProperty(SECURITY_CLIENT_AUTHENTICATOR);
+
+    Object auth = getAuthenticatorObject(customAuthenticator);
+
     if (!com.gemstone.gemfire.internal.lang.StringUtils.isBlank(shiroConfig)) {
-      IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:"+shiroConfig);
+      IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:" + shiroConfig);
 
       // we will need to make sure that shiro uses a case sensitive permission resolver
       Section main = factory.getIni().addSection("main");
       main.put("geodePermissionResolver", "com.gemstone.gemfire.internal.security.shiro.GeodePermissionResolver");
-      if(!main.containsKey("iniRealm.permissionResolver")) {
+      if (!main.containsKey("iniRealm.permissionResolver")) {
         main.put("iniRealm.permissionResolver", "$geodePermissionResolver");
       }
 
       SecurityManager securityManager = factory.getInstance();
       SecurityUtils.setSecurityManager(securityManager);
     }
-    else if (!com.gemstone.gemfire.internal.lang.StringUtils.isBlank(customAuthenticator)) {
-      Realm realm = new CustomAuthRealm(securityProps);
+    // only set up shiro realm if user has implemented ExternalSecurity
+    else if (auth != null && auth instanceof ExternalSecurity) {
+      ExternalSecurity externalSecurity = (ExternalSecurity) auth;
+      externalSecurity.init(securityProps);
+      Realm realm = new CustomAuthRealm(externalSecurity);
       SecurityManager securityManager = new DefaultSecurityManager(realm);
       SecurityUtils.setSecurityManager(securityManager);
     }
-    else{
+    else {
       SecurityUtils.setSecurityManager(null);
     }
   }
 
+  public static Object getAuthenticatorObject(String authenticatorFactoryName) {
+    if (StringUtils.isBlank(authenticatorFactoryName)) {
+      return null;
+    }
+
+    try {
+      Method instanceGetter = ClassLoadUtil.methodFromName(authenticatorFactoryName);
+      return instanceGetter.invoke(null, (Object[]) null);
+    }
+    catch (Exception ex) {
+      throw new AuthenticationRequiredException(ex.toString(), ex);
+    }
+  }
+
+  public static boolean isIntegratedSecurity(String authenticatorFactoryName) {
+    Object auth = getAuthenticatorObject(authenticatorFactoryName);
+    return (auth instanceof ExternalSecurity);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/CustomAuthRealm.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/CustomAuthRealm.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/CustomAuthRealm.java
index 6da9656..060878d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/CustomAuthRealm.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/CustomAuthRealm.java
@@ -16,56 +16,48 @@
  */
 package com.gemstone.gemfire.internal.security.shiro;
 
-import com.gemstone.gemfire.cache.operations.OperationContext;
-import com.gemstone.gemfire.internal.ClassLoadUtil;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.lang.StringUtils;
+import java.security.Principal;
+import java.util.Properties;
+
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.internal.security.ResourceConstants;
-import com.gemstone.gemfire.security.AccessControl;
-import com.gemstone.gemfire.security.Authenticator;
+import com.gemstone.gemfire.security.ExternalSecurity;
+import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.security.GeodePermission;
+
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
-import org.apache.shiro.authc.*;
+import org.apache.shiro.authc.AuthenticationException;
+import org.apache.shiro.authc.AuthenticationInfo;
+import org.apache.shiro.authc.AuthenticationToken;
+import org.apache.shiro.authc.SimpleAuthenticationInfo;
+import org.apache.shiro.authc.UsernamePasswordToken;
 import org.apache.shiro.authz.AuthorizationInfo;
 import org.apache.shiro.authz.Permission;
 import org.apache.shiro.realm.AuthorizingRealm;
 import org.apache.shiro.subject.PrincipalCollection;
 
-import javax.management.remote.JMXPrincipal;
-import javax.security.auth.Subject;
-import java.lang.reflect.Method;
-import java.security.AccessControlContext;
-import java.security.AccessController;
-import java.security.Principal;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import static com.gemstone.gemfire.management.internal.security.ResourceConstants.ACCESS_DENIED_MESSAGE;
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
-
 public class CustomAuthRealm extends AuthorizingRealm{
   public static final String REALM_NAME = "CUSTOMAUTHREALM";
 
   private static final Logger logger = LogManager.getLogger(CustomAuthRealm.class);
-  private String authzFactoryName;
-  private String postAuthzFactoryName;
-  private String authenticatorFactoryName;
-  private Properties securityProps = null;
-  private ConcurrentMap<Principal, AccessControl> cachedAuthZCallback;
-  private ConcurrentMap<Principal, AccessControl> cachedPostAuthZCallback;
-
-  public CustomAuthRealm(Properties securityProps) {
-    this.securityProps = securityProps;
-    this.authzFactoryName = securityProps.getProperty(SECURITY_CLIENT_ACCESSOR);
-    this.postAuthzFactoryName = securityProps.getProperty(SECURITY_CLIENT_ACCESSOR_PP);
-    this.authenticatorFactoryName = securityProps.getProperty(SECURITY_CLIENT_AUTHENTICATOR);
-    this.cachedAuthZCallback = new ConcurrentHashMap<>();
-    this.cachedPostAuthZCallback = new ConcurrentHashMap<>();
+  private ExternalSecurity externalSecurity = null;
+
+  public CustomAuthRealm(ExternalSecurity auth) {
+    externalSecurity = auth;
   }
 
-  @Override
+
+  public CustomAuthRealm (String authenticatorFactory) {
+    Object auth = GeodeSecurityUtil.getAuthenticatorObject(authenticatorFactory);
+
+    if(!(auth instanceof ExternalSecurity)){
+      throw new GemFireSecurityException("Integrated Security requires ExternalSecurity interface.");
+    }
+    externalSecurity = (ExternalSecurity) auth;
+  }
+
+    @Override
   protected AuthenticationInfo doGetAuthenticationInfo(AuthenticationToken token) throws AuthenticationException {
     UsernamePasswordToken authToken = (UsernamePasswordToken) token;
     String username = authToken.getUsername();
@@ -75,7 +67,7 @@ public class CustomAuthRealm extends AuthorizingRealm{
     credentialProps.put(ResourceConstants.USER_NAME, username);
     credentialProps.put(ResourceConstants.PASSWORD, password);
 
-    Principal principal  = getAuthenticator(securityProps).authenticate(credentialProps);
+    Principal principal  = externalSecurity.authenticate(credentialProps);
 
     return new SimpleAuthenticationInfo(principal, authToken.getPassword(), REALM_NAME);
   }
@@ -89,83 +81,9 @@ public class CustomAuthRealm extends AuthorizingRealm{
 
   @Override
   public boolean isPermitted(PrincipalCollection principals, Permission permission) {
-    OperationContext context =(OperationContext)permission;
-    Principal principal = (Principal)principals.getPrimaryPrincipal();
-    // if no access control is specified, then we allow all
-    if(StringUtils.isBlank(authzFactoryName))
-      return true;
-    AccessControl accessControl = getAccessControl(principal, false);
-    return accessControl.authorizeOperation(context.getRegionName(), context);
-  }
-
-  public AccessControl getAccessControl(Principal principal, boolean isPost) {
-    if (!isPost) {
-      if (cachedAuthZCallback.containsKey(principal)) {
-        return cachedAuthZCallback.get(principal);
-      } else if (!StringUtils.isBlank(authzFactoryName)) {
-        try {
-          Method authzMethod = ClassLoadUtil.methodFromName(authzFactoryName);
-          AccessControl authzCallback = (AccessControl) authzMethod.invoke(null, (Object[]) null);
-          authzCallback.init(principal, null);
-          cachedAuthZCallback.put(principal, authzCallback);
-          return authzCallback;
-        } catch (Exception ex) {
-          throw new AuthenticationException(
-              ex.toString(), ex);
-        }
-      }
-    } else {
-      if (cachedPostAuthZCallback.containsKey(principal)) {
-        return cachedPostAuthZCallback.get(principal);
-      } else if (!StringUtils.isBlank(postAuthzFactoryName)) {
-        try {
-          Method authzMethod = ClassLoadUtil.methodFromName(postAuthzFactoryName);
-          AccessControl postAuthzCallback = (AccessControl) authzMethod.invoke(null, (Object[]) null);
-          postAuthzCallback.init(principal, null);
-          cachedPostAuthZCallback.put(principal, postAuthzCallback);
-          return postAuthzCallback;
-        } catch (Exception ex) {
-          throw new AuthenticationException(
-              ex.toString(), ex);
-        }
-      }
-    }
-    return null;
-  }
-
-  private Authenticator getAuthenticator(Properties gfSecurityProperties) throws AuthenticationException {
-    Authenticator auth;
-    try {
-      Method instanceGetter = ClassLoadUtil.methodFromName(this.authenticatorFactoryName);
-      auth = (Authenticator) instanceGetter.invoke(null, (Object[]) null);
-    } catch (Exception ex) {
-      throw new AuthenticationException(
-          ex.toString(), ex);
-    }
-    if (auth == null) {
-      throw new AuthenticationException(
-          LocalizedStrings.HandShake_AUTHENTICATOR_INSTANCE_COULD_NOT_BE_OBTAINED.toLocalizedString());
-    }
-    auth.init(gfSecurityProperties);
-    return auth;
-  }
-
-  public void postAuthorize(OperationContext context) {
-    if (StringUtils.isBlank(postAuthzFactoryName)){
-      return ;
-    }
-
-    AccessControlContext acc = AccessController.getContext();
-    Subject subject = Subject.getSubject(acc);
-    Set<JMXPrincipal> principals = subject.getPrincipals(JMXPrincipal.class);
-    if (principals == null || principals.isEmpty()) {
-      throw new SecurityException(ACCESS_DENIED_MESSAGE);
-    }
-    Principal principal = principals.iterator().next();
-    AccessControl accessControl = getAccessControl(principal, true);
-    if (!accessControl.authorizeOperation(null, context)) {
-      throw new SecurityException(ACCESS_DENIED_MESSAGE);
-    }
+    GeodePermission context = (GeodePermission) permission;
+    Principal principal = (Principal) principals.getPrimaryPrincipal();
+    return externalSecurity.authorize(principal, context);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/management/AsyncEventQueueMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/AsyncEventQueueMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/AsyncEventQueueMXBean.java
index b56012f..16df5f2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/AsyncEventQueueMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/AsyncEventQueueMXBean.java
@@ -17,10 +17,9 @@
 package com.gemstone.gemfire.management;
 
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
-import com.gemstone.gemfire.cache.operations.OperationContext;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
-
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
+import com.gemstone.gemfire.security.GeodePermission.Operation;
+import com.gemstone.gemfire.security.GeodePermission.Resource;
 
 /**
  * MBean that provides access to an {@link AsyncEventQueue}.
@@ -28,7 +27,7 @@ import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
  * @since GemFire 7.0
  * 
  */
-@ResourceOperation(resource = Resource.CLUSTER, operation = OperationContext.OperationCode.READ)
+@ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
 public interface AsyncEventQueueMXBean {
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/management/CacheServerMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/CacheServerMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/CacheServerMXBean.java
index e0e4803..deffc5d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/CacheServerMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/CacheServerMXBean.java
@@ -18,9 +18,8 @@ package com.gemstone.gemfire.management;
 
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
-
-import static com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
+import com.gemstone.gemfire.security.GeodePermission.Operation;
+import com.gemstone.gemfire.security.GeodePermission.Resource;
 
 /**
  * MBean that provides access to information and management functionality for a
@@ -56,7 +55,7 @@ import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
  * @since GemFire 7.0
  * 
  */
-@ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
+@ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
 public interface CacheServerMXBean {
 
   /**
@@ -283,7 +282,7 @@ public interface CacheServerMXBean {
    * @param indexName
    *          Name of the index to be removed.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public void removeIndex(String indexName) throws Exception;
 
   /**
@@ -301,7 +300,7 @@ public interface CacheServerMXBean {
    * not affected. Using the client side CQ methods to modify a CQ.
    */
   @Deprecated
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.READ)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.READ)
   public void executeContinuousQuery(String queryName) throws Exception;
 
   /**
@@ -315,7 +314,7 @@ public interface CacheServerMXBean {
    * not affected. Using the client side CQ methods to modify a CQ.         
    */
   @Deprecated
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public void stopContinuousQuery(String queryName) throws Exception;
 
   /**
@@ -328,7 +327,7 @@ public interface CacheServerMXBean {
    * not affected. Using the client side CQ methods to modify a CQ.         
    */
   @Deprecated
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public void closeAllContinuousQuery(String regionName) throws Exception;
   
   
@@ -342,7 +341,7 @@ public interface CacheServerMXBean {
    * not affected. Using the client side CQ methods to modify a CQ.         
    */
   @Deprecated
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public void closeContinuousQuery(String queryName) throws Exception;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/management/DiskStoreMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/DiskStoreMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/DiskStoreMXBean.java
index a7878ec..7729505 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/DiskStoreMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/DiskStoreMXBean.java
@@ -18,9 +18,8 @@ package com.gemstone.gemfire.management;
 
 import com.gemstone.gemfire.cache.DiskStore;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
-
-import static com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
+import com.gemstone.gemfire.security.GeodePermission.Operation;
+import com.gemstone.gemfire.security.GeodePermission.Resource;
 
 /**
  * MBean that provides access to information and management functionality for a
@@ -29,7 +28,7 @@ import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
  * @since GemFire 7.0
  * 
  */
-@ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
+@ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
 public interface DiskStoreMXBean {
   
   /**
@@ -160,7 +159,7 @@ public interface DiskStoreMXBean {
    * compaction is true then the application will wait for the other op-logs to
    * be compacted and additional space is available.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public void forceRoll();
 
   /**
@@ -174,14 +173,14 @@ public interface DiskStoreMXBean {
    *         that no op-logs were ready to be compacted or that a compaction was
    *         already in progress.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public boolean forceCompaction();
   
   /**
    * Causes any data that is currently in the asynchronous queue to be written
    * to disk. Does not return until the flush is complete.
    */
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public void flush();
 
   /**
@@ -207,7 +206,7 @@ public interface DiskStoreMXBean {
    * 
    * @param warningPercent the warning percent
    */
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public void setDiskUsageWarningPercentage(float warningPercent);
   
   /**
@@ -215,6 +214,6 @@ public interface DiskStoreMXBean {
    * 
    * @param criticalPercent the critical percent
    */
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public void setDiskUsageCriticalPercentage(float criticalPercent);
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedLockServiceMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedLockServiceMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedLockServiceMXBean.java
index 73c8aae..b18723a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedLockServiceMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedLockServiceMXBean.java
@@ -16,13 +16,12 @@
  */
 package com.gemstone.gemfire.management;
 
-import com.gemstone.gemfire.cache.operations.OperationContext;
-import com.gemstone.gemfire.distributed.DistributedLockService;
-import com.gemstone.gemfire.management.internal.security.ResourceOperation;
-
 import java.util.Map;
 
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
+import com.gemstone.gemfire.distributed.DistributedLockService;
+import com.gemstone.gemfire.management.internal.security.ResourceOperation;
+import com.gemstone.gemfire.security.GeodePermission.Operation;
+import com.gemstone.gemfire.security.GeodePermission.Resource;
 
 /**
  * MBean that provides access to information for a named instance of {@link DistributedLockService}.
@@ -32,7 +31,7 @@ import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
  * @since GemFire 7.0
  * 
  */
-@ResourceOperation(resource = Resource.CLUSTER, operation = OperationContext.OperationCode.READ)
+@ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
 public interface DistributedLockServiceMXBean {
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f447023e/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedRegionMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedRegionMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedRegionMXBean.java
index 2f71cea..6bf462e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedRegionMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedRegionMXBean.java
@@ -20,11 +20,10 @@ import com.gemstone.gemfire.cache.CacheListener;
 import com.gemstone.gemfire.cache.CacheWriter;
 import com.gemstone.gemfire.cache.EvictionAlgorithm;
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.operations.OperationContext;
 import com.gemstone.gemfire.cache.wan.GatewaySender;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
-
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
+import com.gemstone.gemfire.security.GeodePermission.Operation;
+import com.gemstone.gemfire.security.GeodePermission.Resource;
 
 /**
  * MBean that provides access to information and management functionality for a
@@ -33,7 +32,7 @@ import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
  * @since GemFire 7.0
  * 
  */
-@ResourceOperation(resource = Resource.CLUSTER, operation = OperationContext.OperationCode.READ)
+@ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
 public interface DistributedRegionMXBean {
 
   /**