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 2016/05/03 23:51:54 UTC

[05/50] [abbrv] incubator-geode git commit: GEODE-17: Shiro Integration

GEODE-17: Shiro Integration

GEODE-17: associate async request with correct shiro user

GEODE-17

GEODE-17: fix shiro dependency


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

Branch: refs/heads/feature/GEODE-1255
Commit: c733f0c2da856103652a383b8611330b57ed8fc4
Parents: def78f1
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon Feb 29 07:47:47 2016 -0800
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu Apr 21 10:17:18 2016 -0700

----------------------------------------------------------------------
 geode-assembly/build.gradle                     |   1 +
 geode-core/build.gradle                         |   4 +-
 .../cache/operations/OperationContext.java      |   8 +-
 .../internal/DistributionConfig.java            |   7 +
 .../internal/DistributionConfigImpl.java        |  13 +
 .../management/DistributedSystemMXBean.java     |  15 +-
 .../management/internal/AuthManager.java        | 305 -------------------
 .../management/internal/ManagementAgent.java    |  73 ++---
 .../internal/SystemManagementService.java       |  44 ++-
 .../internal/beans/MemberMBeanBridge.java       |  17 +-
 .../management/internal/cli/CommandManager.java |   3 -
 .../CreateAlterDestroyRegionCommands.java       |  43 +--
 .../internal/cli/commands/DataCommands.java     |  42 +--
 .../internal/cli/commands/RegionCommands.java   |  25 +-
 .../internal/cli/remote/CommandProcessor.java   |  25 +-
 .../internal/cli/result/ErrorResultData.java    |  10 +-
 .../internal/cli/result/ResultBuilder.java      |   7 +-
 .../internal/cli/shell/JmxOperationInvoker.java |   5 +-
 .../internal/security/AccessControlMBean.java   |  58 ----
 .../internal/security/CLIOperationContext.java  | 102 -------
 .../internal/security/MBeanServerWrapper.java   |  24 +-
 .../security/ManagementInterceptor.java         | 243 ---------------
 .../security/ResourceOperationContext.java      |  28 +-
 .../controllers/AbstractCommandsController.java |  37 +--
 .../controllers/ConfigCommandsController.java   |  28 +-
 .../web/controllers/DataCommandsController.java |  36 +--
 .../DiskStoreCommandsController.java            |  20 +-
 .../controllers/FunctionCommandsController.java |  14 +-
 .../MiscellaneousCommandsController.java        |  12 +-
 .../EnvironmentVariablesHandlerInterceptor.java |  74 +----
 .../gemfire/security/CustomAuthRealm.java       | 173 +++++++++++
 .../gemfire/security/JMXShiroAuthenticator.java |  66 ++++
 .../gemstone/gemfire/security/ShiroUtil.java    |  94 ++++++
 .../security/AccessControlMBeanJUnitTest.java   |   4 +-
 .../CacheServerMBeanAuthorizationJUnitTest.java |  30 +-
 .../CacheServerMBeanShiroJUnitTest.java         |  73 +++++
 .../security/CliCommandsSecurityTest.java       |   7 +-
 .../security/DataCommandsSecurityTest.java      |  25 +-
 .../DiskStoreMXBeanSecurityJUnitTest.java       |   6 +-
 .../GatewaySenderMBeanSecurityTest.java         |  34 +--
 .../security/GfshCommandsSecurityTest.java      | 121 ++++++--
 .../security/GfshShellConnectionRule.java       |  25 +-
 .../internal/security/JSONAuthorization.java    |  55 ++--
 .../JsonAuthorizationCacheStartRule.java        |   4 +-
 .../LockServiceMBeanAuthorizationJUnitTest.java |  16 +-
 .../security/MBeanServerConnectionRule.java     |  15 +-
 .../ManagerMBeanAuthorizationJUnitTest.java     |  16 +-
 .../security/MemberMBeanSecurityJUnitTest.java  |   8 +-
 .../internal/security/ShiroCacheStartRule.java  |  63 ++++
 .../internal/security/TestCommand.java          |  30 +-
 .../internal/security/cacheServer.json          |   4 +-
 geode-core/src/test/resources/shiro.ini         |  16 +
 .../gemfire/tools/pulse/tests/Server.java       |  36 +--
 gradle/dependency-versions.properties           |   1 +
 54 files changed, 1005 insertions(+), 1240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/geode-assembly/build.gradle b/geode-assembly/build.gradle
index b7d05e2..7faac14 100755
--- a/geode-assembly/build.gradle
+++ b/geode-assembly/build.gradle
@@ -152,6 +152,7 @@ def cp = {
       it.contains('log4j-jcl') ||
       it.contains('log4j-jul') ||
       it.contains('log4j-slf4j-impl') ||
+      it.contains('shiro') ||
       it.contains('slf4j-api') ||
       it.contains('spring-core') ||
       it.contains('spring-shell') ||

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/build.gradle
----------------------------------------------------------------------
diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index 2206018..2c0e871 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -58,7 +58,7 @@ dependencies {
   compile ('mx4j:mx4j-remote:' + project.'mx4j.version') {
     ext.optional = true;
   }
-  compile ('mx4j:mx4j-tools:' + project.'mx4j.version') { 
+  compile ('mx4j:mx4j-tools:' + project.'mx4j.version') {
     ext.optional = true;
   }
   compile ('net.java.dev.jna:jna:' + project.'jna.version')
@@ -118,6 +118,8 @@ dependencies {
   compile ('org.xerial.snappy:snappy-java:' + project.'snappy-java.version') {
     ext.optional = true
   }
+
+  compile 'org.apache.shiro:shiro-core:' + project.'shiro.version'
  
   compile project(':geode-common')
   compile project(':geode-joptsimple')

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/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 1a41e83..dd290c5 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,6 +17,8 @@
 
 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
@@ -28,7 +30,7 @@ package com.gemstone.gemfire.cache.operations;
  *
  * @since 5.5
  */
-public abstract class OperationContext {
+public abstract class OperationContext extends WildcardPermission{
 
   public enum Resource {
     CLUSTER,
@@ -299,6 +301,10 @@ public abstract class OperationContext {
     return Resource.DATA;
   }
 
+  public String getRegionName(){
+    return null;
+  }
+
   /**
    * True if the context is for post-operation.
    *

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
index c1a96ea..c0e560c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
@@ -3738,6 +3738,13 @@ public interface DistributionConfig extends Config, LogConfig {
   @ConfigAttributeSetter(name=LOCK_MEMORY_NAME)
   public void setLockMemory(boolean value);
 
+  @ConfigAttribute(type=String.class)
+  public String SHIRO_INIT_NAME="shiro-init";
+
+  @ConfigAttributeSetter(name=SHIRO_INIT_NAME)
+  public void setShiroInit(String value);
+  @ConfigAttributeGetter(name=SHIRO_INIT_NAME)
+  public String getShiroInit();
 
 
   //*************** Initializers to gather all the annotations in this class ************************

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
index 2f0e1d9..bac66c6 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
@@ -387,6 +387,8 @@ public class DistributionConfigImpl
   
   /** Whether pages should be locked into memory or allowed to swap to disk */
   private boolean lockMemory = DEFAULT_LOCK_MEMORY;
+
+  private String shiroInit = "";
   
   //////////////////////  Constructors  //////////////////////
 
@@ -566,6 +568,7 @@ public class DistributionConfigImpl
     
     this.lockMemory = other.getLockMemory();
     this.distributedTransactions = other.getDistributedTransactions();
+    this.shiroInit = other.getShiroInit();
   }
 
   /**
@@ -2268,6 +2271,16 @@ public class DistributionConfigImpl
     this.lockMemory = value;
   }
 
+  @Override
+  public void setShiroInit(String value) {
+    this.shiroInit = value;
+  }
+
+  @Override
+  public String getShiroInit() {
+    return this.shiroInit;
+  }
+
   ///////////////////////  Utility Methods  ///////////////////////
   /**
    * Two instances of <code>DistributedConfigImpl</code> are equal if all of 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedSystemMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedSystemMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedSystemMXBean.java
index 5232aed..de60e33 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedSystemMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedSystemMXBean.java
@@ -16,18 +16,17 @@
  */
 package com.gemstone.gemfire.management;
 
+import static com.gemstone.gemfire.cache.operations.OperationContext.*;
+
+import java.util.Map;
+import javax.management.ObjectName;
+
 import com.gemstone.gemfire.cache.DiskStore;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
 
-import javax.management.ObjectName;
-import java.util.Map;
-
-import static com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
-
 /**
  * MBean that provides access to information and management operations for a
  * {@link DistributedSystem}.
@@ -73,7 +72,7 @@ import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
  * @since 7.0
  *
  */
-@ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
+//@ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
 public interface DistributedSystemMXBean {
 
   /**
@@ -101,7 +100,7 @@ public interface DistributedSystemMXBean {
    *          members.
    * @return a list of names for locator members.
    */
-  
+
   public String[] listLocatorMembers(boolean onlyStandAloneLocators);
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/AuthManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/AuthManager.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/AuthManager.java
deleted file mode 100644
index 31a5ab3..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/AuthManager.java
+++ /dev/null
@@ -1,305 +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.management.internal;
-
-import static com.gemstone.gemfire.management.internal.security.ResourceConstants.ACCESS_DENIED_MESSAGE;
-
-import java.lang.reflect.Method;
-import java.security.Principal;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.ConcurrentHashMap;
-
-import com.gemstone.gemfire.GemFireConfigException;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.operations.OperationContext;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.i18n.LogWriterI18n;
-import com.gemstone.gemfire.internal.ClassLoadUtil;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.logging.InternalLogWriter;
-import com.gemstone.gemfire.security.AccessControl;
-import com.gemstone.gemfire.security.AuthenticationFailedException;
-import com.gemstone.gemfire.security.AuthenticationRequiredException;
-import com.gemstone.gemfire.security.Authenticator;
-import com.gemstone.gemfire.internal.lang.StringUtils;
-
-
-/**
- * This class acts as a single gateway to authorize and authenticate REST ADMIN
- * APIS. This stores credentials against CommandAuthZRequest object which is
- * used to identify a particular client. As REST requests are state less we need
- * to store this map to avoid re-authenticating same client on subsequent
- * requests. However this map needs to be purged and cleaned up on some expiry
- * policy.
- *
- *
- * @author rishim
- *
- */
-public class AuthManager {
-
-  private Map<Properties, CommandAuthZRequest> authMap = new ConcurrentHashMap<Properties, CommandAuthZRequest>();
-
-  private Cache cache;
-
-  private final LogWriterI18n logger;
-
-  private long DEFAULT_EXPIRY_TIME = 30; // in minutes
-
-  private long EXPIRY_TIME ;
-
-  String authzFactoryName;
-
-  String postAuthzFactoryName;
-
-  public static String EXPIRY_TIME_FOR_REST_ADMIN_AUTH = "gemfire.expriyTimeForRESTAdminAuth";
-
-  public AuthManager(Cache cache) {
-    this.cache = cache;
-    this.logger = cache.getSecurityLoggerI18n();
-    this.EXPIRY_TIME = Long.getLong(EXPIRY_TIME_FOR_REST_ADMIN_AUTH, DEFAULT_EXPIRY_TIME);
-    DistributedSystem system = cache.getDistributedSystem();
-    Properties sysProps = system.getProperties();
-    this.authzFactoryName = sysProps.getProperty(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME);
-    this.postAuthzFactoryName = sysProps.getProperty(DistributionConfig.SECURITY_CLIENT_ACCESSOR_PP_NAME);
-  }
-
-  private Authenticator getAuthenticator(String authenticatorMethod, Properties securityProperties,
-      InternalLogWriter logWriter, InternalLogWriter securityLogWriter) throws AuthenticationFailedException {
-    Authenticator auth;
-    try {
-
-      Method instanceGetter = ClassLoadUtil.methodFromName(authenticatorMethod);
-      auth = (Authenticator) instanceGetter.invoke(null, (Object[]) null);
-    } catch (Exception ex) {
-      throw new AuthenticationFailedException(
-          LocalizedStrings.HandShake_FAILED_TO_ACQUIRE_AUTHENTICATOR_OBJECT.toLocalizedString(), ex);
-    }
-    if (auth == null) {
-      throw new AuthenticationFailedException(
-          LocalizedStrings.HandShake_AUTHENTICATOR_INSTANCE_COULD_NOT_BE_OBTAINED.toLocalizedString());
-    }
-    auth.init(securityProperties, logWriter, securityLogWriter);
-    return auth;
-
-  }
-
-  public void verifyCredentials(Properties credentials) {
-
-    DistributedSystem system = this.cache.getDistributedSystem();
-    Properties sysProps = system.getProperties();
-    String authenticator = sysProps.getProperty(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME);
-
-    if (authenticator != null && authenticator.length() > 0) {
-
-      CommandAuthZRequest authZRequest = authMap.get(credentials);
-
-      if (authZRequest != null && !authZRequest.hasExpired()) {
-        return; //Already existing credentials . Return from here
-
-      } else {
-        Principal principal = verifyCredentials(authenticator, credentials, system.getSecurityProperties(),
-            (InternalLogWriter) this.cache.getLogger(), (InternalLogWriter) this.cache.getSecurityLogger(), cache
-                .getDistributedSystem().getDistributedMember());
-
-        if(authZRequest != null){ //i.e its an expired credential
-          CommandAuthZRequest expiredAuth = authMap.remove(credentials);
-          try{
-            expiredAuth.close();
-          }catch(Exception e){
-            logger.error(e);//Don't throw an exception , just logs it
-          }
-        }
-
-        authZRequest = new CommandAuthZRequest(principal).init();
-        authMap.put(credentials, authZRequest);
-      }
-    }
-
-  }
-
-  public void expireAllAuthZ() {
-    for (CommandAuthZRequest auth : authMap.values()) {
-      try {
-        auth.close();
-
-      } catch (Exception e) {
-        logger.error(e);// Don't throw an exception , just log it, as it depends on the user code.
-      }finally{
-        authMap.clear();
-      }
-    }
-  }
-
-  public void authorize(Properties credentials, OperationContext context) {
-
-    if (!StringUtils.isBlank(authzFactoryName)) {
-      CommandAuthZRequest authZRequest = authMap.get(credentials);
-      boolean authorized = authZRequest.authorize(context);
-      if (!authorized)
-        throw new SecurityException(ACCESS_DENIED_MESSAGE);
-    }
-  }
-
-  public void postAuthorize(Properties credentials, OperationContext context) {
-    if (!StringUtils.isBlank(postAuthzFactoryName)) {
-      CommandAuthZRequest authZRequest = authMap.get(credentials);
-      boolean authorized = authZRequest.postAuthorize(context);
-      if (!authorized)
-        throw new SecurityException(ACCESS_DENIED_MESSAGE);
-    }
-
-  }
-
-  private Principal verifyCredentials(String authenticatorMethod, Properties credentials,
-      Properties securityProperties, InternalLogWriter logWriter, InternalLogWriter securityLogWriter,
-      DistributedMember member) throws AuthenticationRequiredException, AuthenticationFailedException {
-
-    Authenticator authenticator = getAuthenticator(authenticatorMethod, securityProperties, logWriter,
-        securityLogWriter);
-    Principal principal;
-
-    try {
-      principal = authenticator.authenticate(credentials, member);
-    } finally {
-      authenticator.close();
-    }
-
-    return principal;
-
-  }
-
-  public class CommandAuthZRequest {
-
-    private Principal principal;
-
-    private AccessControl authzCallback;
-
-    private AccessControl postAuthzCallback;
-
-    private long initTime = System.currentTimeMillis();
-
-    public CommandAuthZRequest(Principal principal) {
-      this.principal = principal;
-    }
-
-    public boolean authorize(OperationContext context) {
-      if (authzCallback != null) {
-        return authzCallback.authorizeOperation(null, context);
-      }
-      return true; // If no AccessControl is set then always return true
-    }
-
-    public boolean postAuthorize(OperationContext context) {
-      if (postAuthzCallback != null) {
-        return postAuthzCallback.authorizeOperation(null, context);
-      }
-      return true; // If no AccessControl is set then always return true
-    }
-
-    public boolean hasExpired(){
-      if(System.currentTimeMillis() - initTime >= EXPIRY_TIME * 60 * 1000){
-        return true;
-      }
-      return false;
-    }
-
-    public void close() {
-      if (authzCallback != null) {
-        authzCallback.close();
-      }
-      if (postAuthzCallback != null) {
-        postAuthzCallback.close();
-      }
-    }
-
-    private CommandAuthZRequest init() {
-      try {
-        if (!StringUtils.isBlank(authzFactoryName)) {
-          Method authzMethod = ClassLoadUtil.methodFromName(authzFactoryName);
-          this.authzCallback = (AccessControl) authzMethod.invoke(null, (Object[]) null);
-          this.authzCallback.init(principal, null, cache);
-        }
-        if (!StringUtils.isBlank(postAuthzFactoryName)) {
-          Method postAuthzMethod = ClassLoadUtil.methodFromName(postAuthzFactoryName);
-          this.postAuthzCallback = (AccessControl) postAuthzMethod.invoke(null, (Object[]) null);
-          this.postAuthzCallback.init(principal, null, cache);
-        }
-      } catch (IllegalAccessException e) {
-        logger.error(e);
-        throw new GemFireConfigException("Error while configuring accesscontrol for rest resource", e);
-      } catch (Exception e) {
-        logger.error(e);
-        throw new GemFireConfigException("Error while configuring accesscontrol for rest resource", e);
-      }
-      return this;
-    }
-
-    public AccessControl getAuthzCallback() {
-      return authzCallback;
-    }
-
-    public AccessControl getPostAuthzCallback() {
-      return postAuthzCallback;
-    }
-
-    @Override
-    public int hashCode() {
-      final int prime = 31;
-      int result = 1;
-      result = prime * result + getOuterType().hashCode();
-      result = prime * result + (int) (initTime ^ (initTime >>> 32));
-      result = prime * result + ((principal == null) ? 0 : principal.hashCode());
-      return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj)
-        return true;
-      if (obj == null)
-        return false;
-      if (getClass() != obj.getClass())
-        return false;
-      CommandAuthZRequest other = (CommandAuthZRequest) obj;
-      if (!getOuterType().equals(other.getOuterType()))
-        return false;
-      if (initTime != other.initTime)
-        return false;
-      if (principal == null) {
-        if (other.principal != null)
-          return false;
-      } else if (!principal.equals(other.principal))
-        return false;
-      return true;
-    }
-
-    private AuthManager getOuterType() {
-      return AuthManager.this;
-    }
-
-  }
-
-  public Map<Properties, CommandAuthZRequest> getAuthMap() {
-    return this.authMap;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
index 5dd2f80..0ba62e7 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
@@ -38,7 +38,6 @@ import javax.management.remote.rmi.RMIJRMPServerImpl;
 import javax.management.remote.rmi.RMIServerImpl;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
-import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -52,8 +51,8 @@ import com.gemstone.gemfire.management.ManagementException;
 import com.gemstone.gemfire.management.ManagementService;
 import com.gemstone.gemfire.management.ManagerMXBean;
 import com.gemstone.gemfire.management.internal.security.MBeanServerWrapper;
-import com.gemstone.gemfire.management.internal.security.ManagementInterceptor;
 import com.gemstone.gemfire.management.internal.unsafe.ReadOpFileAccessController;
+import com.gemstone.gemfire.security.JMXShiroAuthenticator;
 import org.apache.logging.log4j.Logger;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
@@ -80,10 +79,9 @@ public class ManagementAgent {
   private boolean running = false;
   private Registry registry;
   private JMXConnectorServer cs;
+  private JMXShiroAuthenticator shiroAuthenticator;
   private final DistributionConfig config;
   private boolean isHttpServiceRunning = false;
-  private ManagementInterceptor managementInterceptor = null;
-  private MBeanServerWrapper mBeanServerWrapper = null;
 
   /**
    * This system property is set to true when the embedded HTTP server is
@@ -100,14 +98,6 @@ public class ManagementAgent {
     return this.running;
   }
 
-  public ManagementInterceptor getManagementInterceptor() {
-    return managementInterceptor;
-  }
-
-  public MBeanServerWrapper getMBeanServerWrapper() {
-    return mBeanServerWrapper;
-  }
-
   public synchronized boolean isHttpServiceRunning() {
     return isHttpServiceRunning;
   }
@@ -307,10 +297,6 @@ public class ManagementAgent {
     }
   }
 
-  private boolean isRunningInTomcat() {
-    return (System.getProperty("catalina.base") != null || System.getProperty("catalina.home") != null);
-  }
-
   private void setStatusMessage(ManagerMXBean mBean, String message) {
     mBean.setPulseURL("");
     mBean.setStatusMessage(message);
@@ -398,29 +384,6 @@ public class ManagementAgent {
     // Environment map. KIRK: why is this declared as HashMap?
     final HashMap<String, Object> env = new HashMap<String, Object>();
 
-    Cache cache = CacheFactory.getAnyInstance();
-    if (isCustomAuthenticator()) {
-      managementInterceptor = new ManagementInterceptor(cache.getDistributedSystem().getProperties());
-      env.put(JMXConnectorServer.AUTHENTICATOR, managementInterceptor);
-    }
-    else {
-      /* Disable the old authenticator mechanism */
-      String pwFile = this.config.getJmxManagerPasswordFile();
-      if (pwFile != null && pwFile.length() > 0) {
-        env.put("jmx.remote.x.password.file", pwFile);
-      }
-
-      String accessFile = this.config.getJmxManagerAccessFile();
-      if (accessFile != null && accessFile.length() > 0) {
-        // Lets not use default connector based authorization
-        // env.put("jmx.remote.x.access.file", accessFile);
-        // Rewire the mbs hierarchy to set accessController
-        ReadOpFileAccessController controller = new ReadOpFileAccessController(accessFile);
-        controller.setMBeanServer(mbs);
-        mbs = controller;
-      }
-    }
-
     // Manually creates and binds a JMX RMI Connector Server stub with the
     // registry created above: the port we pass here is the port that can
     // be specified in "service:jmx:rmi://"+hostname+":"+port - where the
@@ -474,13 +437,33 @@ public class ManagementAgent {
       }
     };
 
-    if (isCustomAuthorizer()) {
-      if(managementInterceptor==null){
-        managementInterceptor = new ManagementInterceptor(cache.getDistributedSystem().getProperties());
+    String shiroConfig = this.config.getShiroInit();
+    if (! StringUtils.isBlank(shiroConfig) || isCustomAuthenticator()) {
+      shiroAuthenticator = new JMXShiroAuthenticator();
+      env.put(JMXConnectorServer.AUTHENTICATOR, shiroAuthenticator);
+      cs.addNotificationListener(shiroAuthenticator, null, cs.getAttributes());
+      if(! StringUtils.isBlank(shiroConfig) || isCustomAuthorizer()) {
+        MBeanServerWrapper mBeanServerWrapper = new MBeanServerWrapper();
+        cs.setMBeanServerForwarder(mBeanServerWrapper);
+      }
+    }
+
+    else {
+      /* Disable the old authenticator mechanism */
+      String pwFile = this.config.getJmxManagerPasswordFile();
+      if (pwFile != null && pwFile.length() > 0) {
+        env.put("jmx.remote.x.password.file", pwFile);
+      }
+
+      String accessFile = this.config.getJmxManagerAccessFile();
+      if (accessFile != null && accessFile.length() > 0) {
+        // Lets not use default connector based authorization
+        // env.put("jmx.remote.x.access.file", accessFile);
+        // Rewire the mbs hierarchy to set accessController
+        ReadOpFileAccessController controller = new ReadOpFileAccessController(accessFile);
+        controller.setMBeanServer(mbs);
+        mbs = controller;
       }
-      mBeanServerWrapper = new MBeanServerWrapper(managementInterceptor);
-      cs.setMBeanServerForwarder(mBeanServerWrapper);
-      logger.info("Starting RMI Connector with Security Interceptor");
     }
 
     cs.start();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
index 4a48430..45ec358 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
@@ -20,22 +20,21 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
-
 import javax.management.Notification;
 import javax.management.ObjectName;
 
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.ResourceEvent;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.management.AlreadyRunningException;
 import com.gemstone.gemfire.management.AsyncEventQueueMXBean;
@@ -55,7 +54,14 @@ import com.gemstone.gemfire.management.RegionMXBean;
 import com.gemstone.gemfire.management.internal.beans.ManagementAdapter;
 import com.gemstone.gemfire.management.membership.MembershipEvent;
 import com.gemstone.gemfire.management.membership.MembershipListener;
-
+import com.gemstone.gemfire.security.CustomAuthRealm;
+import org.apache.logging.log4j.Logger;
+import org.apache.shiro.SecurityUtils;
+import org.apache.shiro.config.IniSecurityManagerFactory;
+import org.apache.shiro.mgt.DefaultSecurityManager;
+import org.apache.shiro.mgt.SecurityManager;
+import org.apache.shiro.realm.Realm;
+import org.apache.shiro.util.ThreadContext;
 
 /**
  * This is the concrete implementation of ManagementService
@@ -130,9 +136,6 @@ public final class SystemManagementService extends BaseManagementService {
    */
   private List<ProxyListener> proxyListeners;
 
-  private AuthManager authManager;
-
-
   private UniversalListenerContainer universalListenerContainer = new UniversalListenerContainer();
   
   public static BaseManagementService newSystemManagementService(Cache cache) {
@@ -154,6 +157,21 @@ public final class SystemManagementService extends BaseManagementService {
     this.jmxAdapter = new MBeanJMXAdapter();      
     this.repo = new ManagementResourceRepo();
 
+    DistributionConfig config = system.getConfig();
+
+    // setup shiro for authentication and authorization if it's desired
+    String shiroConfig = config.getShiroInit();
+    String customAuthenticator = config.getSecurityClientAuthenticator();
+    if (!StringUtils.isBlank(shiroConfig)) {
+      IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:"+shiroConfig);
+      SecurityManager securityManager = factory.getInstance();
+      SecurityUtils.setSecurityManager(securityManager);
+    }
+    else if (!StringUtils.isBlank(customAuthenticator)) {
+      Realm realm = new CustomAuthRealm(config.toProperties());
+      SecurityManager securityManager = new DefaultSecurityManager(realm);
+      SecurityUtils.setSecurityManager(securityManager);
+    }
 
     this.notificationHub = new NotificationHub(repo);
     if (system.getConfig().getJmxManager()) {
@@ -181,7 +199,6 @@ public final class SystemManagementService extends BaseManagementService {
       this.listener = new ManagementMembershipListener(this);
       system.getDistributionManager().addMembershipListener(listener);
       isStarted = true;
-      this.authManager = new AuthManager(cache);
       return this;
     } catch (CancelException e) {
       // Rethrow all CancelExceptions (fix for defect 46339)
@@ -273,8 +290,11 @@ public final class SystemManagementService extends BaseManagementService {
       }
       if (this.agent != null && this.agent.isRunning()) {
         this.agent.stopAgent();
-      }     
-      this.authManager.expireAllAuthZ();
+      }
+
+      // clean out Shiro's thread local content
+      ThreadContext.remove();
+
       getGemFireCacheImpl().getJmxManagerAdvisor().broadcastChange();
       instances.remove(cache);
       localManager  = null;
@@ -837,8 +857,4 @@ public final class SystemManagementService extends BaseManagementService {
   public void removeMembershipListener(MembershipListener listener) {
     universalListenerContainer.removeMembershipListener(listener);    
   }
-
-  public AuthManager getAuthManager(){
-    return this.authManager;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
index 1425572..cb69a80 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
@@ -32,17 +32,13 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-
 import javax.management.JMRuntimeException;
 import javax.management.MBeanServer;
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.Statistics;
 import com.gemstone.gemfire.StatisticsType;
 import com.gemstone.gemfire.cache.CacheClosedException;
@@ -92,7 +88,6 @@ import com.gemstone.gemfire.internal.cache.lru.LRUStatistics;
 import com.gemstone.gemfire.internal.cache.persistence.BackupManager;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.logging.ManagerLogWriter;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.logging.log4j.LogWriterAppender;
@@ -129,8 +124,11 @@ import com.gemstone.gemfire.management.internal.cli.CommandResponseBuilder;
 import com.gemstone.gemfire.management.internal.cli.remote.CommandExecutionContext;
 import com.gemstone.gemfire.management.internal.cli.remote.MemberCommandService;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.result.ErrorResultData;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
+import com.gemstone.gemfire.security.GemFireSecurityException;
+import org.apache.logging.log4j.Logger;
 
 /**
  * This class acts as an Bridge between MemberMBean and GemFire Cache and
@@ -1758,7 +1756,6 @@ public class MemberMBeanBridge {
     if (isGfshRequest) {
       CommandExecutionContext.setShellRequest();
     }
-//    System.out.println("isGfshRequest :: "+isGfshRequest);
     
     Result result = ((MemberCommandService)commandService).processCommand(commandString, env);
     if (!(result instanceof CommandResult)) {// TODO - Abhishek - Shouldn't be needed
@@ -1766,11 +1763,19 @@ public class MemberMBeanBridge {
         result = ResultBuilder.createInfoResult(result.nextLine());
       }
     }
+
     if (isGfshRequest) {
       String responseJson = CommandResponseBuilder.createCommandResponseJson(getMember(), (CommandResult) result);
   //    System.out.println("responseJson :: "+responseJson);
       return responseJson;
     } else {
+      // throw GemFireSecurityException is the returned error code is 415
+      if(((CommandResult) result).getResultData() instanceof ErrorResultData){
+        ErrorResultData resultData = (ErrorResultData) ((CommandResult)result).getResultData();
+        if(resultData.getErrorCode()==ResultBuilder.ERRORCODE_UNAUTHORIZED){
+          throw new GemFireSecurityException(resultData.getGfJsonObject().toString());
+        }
+      }
       return ResultBuilder.resultAsString(result);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandManager.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandManager.java
index 42539db..46a8226 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandManager.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandManager.java
@@ -54,7 +54,6 @@ import com.gemstone.gemfire.management.internal.cli.parser.GfshOptionParser;
 import com.gemstone.gemfire.management.internal.cli.parser.Option;
 import com.gemstone.gemfire.management.internal.cli.parser.jopt.JoptOptionParser;
 import com.gemstone.gemfire.management.internal.cli.util.ClasspathScanLoadHelper;
-import com.gemstone.gemfire.management.internal.security.CLIOperationContext;
 
 /**
  * 
@@ -457,8 +456,6 @@ public class CommandManager {
           }
         }
         
-        CLIOperationContext.registerCommand(this, method, commandTarget);
-        
       } else if (method.getAnnotation(CliAvailabilityIndicator.class) != null) {
         // Now add this availability Indicator to the list of
         // availability Indicators

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
index 5977c10..cdbd3db 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
@@ -16,6 +16,24 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.cache.operations.OperationContext.*;
+
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.regex.Pattern;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -61,29 +79,11 @@ import com.gemstone.gemfire.management.internal.cli.util.RegionPath;
 import com.gemstone.gemfire.management.internal.configuration.SharedConfigurationWriter;
 import com.gemstone.gemfire.management.internal.configuration.domain.XmlEntity;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
+import com.gemstone.gemfire.security.ShiroUtil;
 import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.regex.Pattern;
-
-import static com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
-
 /**
  *
  * @since 7.0
@@ -528,6 +528,9 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
                   specifiedDefaultValue = "0",
                   help = CliStrings.ALTER_REGION__EVICTIONMAX__HELP)
       Integer evictionMax) {
+
+    ShiroUtil.authorize("DATA", "MANAGE", regionPath);
+
     Result result = null;
     XmlEntity xmlEntity = null;
 
@@ -1004,6 +1007,8 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
           help = CliStrings.DESTROY_REGION__REGION__HELP)
       String regionPath) {
 
+    ShiroUtil.authorize("DATA", "MANAGE", regionPath);
+
     if (regionPath == null) {
       return ResultBuilder.createInfoResult(CliStrings.DESTROY_REGION__MSG__SPECIFY_REGIONPATH_TO_DESTROY);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
index 18fcc16..c1c04a3 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
@@ -16,6 +16,22 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.cache.operations.OperationContext.*;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringTokenizer;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheClosedException;
@@ -57,28 +73,12 @@ import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
+import com.gemstone.gemfire.security.ShiroUtil;
 import org.springframework.shell.core.CommandMarker;
 import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.StringTokenizer;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
-
 /**
  * 
  * @since 7.0
@@ -102,6 +102,7 @@ public class DataCommands implements CommandMarker {
       @CliOption(key = CliStrings.REBALANCE__EXCLUDEREGION, unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, help = CliStrings.REBALANCE__EXCLUDEREGION__HELP) String[] excludeRegions,
       @CliOption(key = CliStrings.REBALANCE__TIMEOUT, unspecifiedDefaultValue = "-1", help = CliStrings.REBALANCE__TIMEOUT__HELP) long timeout,
       @CliOption(key = CliStrings.REBALANCE__SIMULATE, specifiedDefaultValue = "true", unspecifiedDefaultValue = "false", help = CliStrings.REBALANCE__SIMULATE__HELP) boolean simulate) {
+
     ExecutorService commandExecutors = Executors.newSingleThreadExecutor();
     List<Future<Result>> commandResult = new ArrayList<Future<Result>>();
     Result result = null;
@@ -839,6 +840,7 @@ public class DataCommands implements CommandMarker {
       @CliOption(key = CliStrings.EXPORT_DATA__FILE, unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, mandatory = true, help = CliStrings.EXPORT_DATA__FILE__HELP) String filePath,
       @CliOption(key = CliStrings.EXPORT_DATA__MEMBER, unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, optionContext = ConverterHint.MEMBERIDNAME, mandatory = true, help = CliStrings.EXPORT_DATA__MEMBER__HELP) String memberNameOrId) {
 
+    ShiroUtil.authorize("DATA", "READ", regionName);
     final Cache cache = CacheFactory.getAnyInstance();
     final DistributedMember targetMember = CliUtil
         .getDistributedMemberByNameOrId(memberNameOrId);
@@ -895,6 +897,8 @@ public class DataCommands implements CommandMarker {
       @CliOption(key = CliStrings.IMPORT_DATA__FILE, mandatory = true, unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, help = CliStrings.IMPORT_DATA__FILE__HELP) String filePath,
       @CliOption(key = CliStrings.IMPORT_DATA__MEMBER, mandatory = true, unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, optionContext = ConverterHint.MEMBERIDNAME, help = CliStrings.IMPORT_DATA__MEMBER__HELP) String memberNameOrId) {
 
+    ShiroUtil.authorize("DATA", "WRITE", regionName);
+
     Result result = null;
 
     try {
@@ -954,6 +958,7 @@ public class DataCommands implements CommandMarker {
       @CliOption(key = { CliStrings.PUT__VALUEKLASS }, help = CliStrings.PUT__VALUEKLASS__HELP) String valueClass,
       @CliOption(key = { CliStrings.PUT__PUTIFABSENT }, help = CliStrings.PUT__PUTIFABSENT__HELP, unspecifiedDefaultValue = "false") boolean putIfAbsent) {
 
+    ShiroUtil.authorize("DATA", "WRITE", regionPath);
     Cache cache = CacheFactory.getAnyInstance();
     DataCommandResult dataResult = null;
     if (regionPath == null || regionPath.isEmpty()) {
@@ -1021,6 +1026,7 @@ public class DataCommands implements CommandMarker {
       @CliOption(key = { CliStrings.GET__VALUEKLASS }, help = CliStrings.GET__VALUEKLASS__HELP) String valueClass,
       @CliOption(key = CliStrings.GET__LOAD, unspecifiedDefaultValue = "true", specifiedDefaultValue = "true", help = CliStrings.GET__LOAD__HELP) Boolean loadOnCacheMiss)
   {
+    ShiroUtil.authorize("DATA", "READ", regionPath);
 
     Cache cache = CacheFactory.getAnyInstance();
     DataCommandResult dataResult = null;
@@ -1075,7 +1081,7 @@ public class DataCommands implements CommandMarker {
       @CliOption(key = { CliStrings.LOCATE_ENTRY__KEYCLASS }, help = CliStrings.LOCATE_ENTRY__KEYCLASS__HELP) String keyClass,
       @CliOption(key = { CliStrings.LOCATE_ENTRY__VALUEKLASS }, help = CliStrings.LOCATE_ENTRY__VALUEKLASS__HELP) String valueClass,
       @CliOption(key = { CliStrings.LOCATE_ENTRY__RECURSIVE }, help = CliStrings.LOCATE_ENTRY__RECURSIVE__HELP, unspecifiedDefaultValue = "false") boolean recursive) {
-
+    ShiroUtil.authorize("DATA", "READ", regionPath);
     // Cache cache = CacheFactory.getAnyInstance();
     DataCommandResult dataResult = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/RegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/RegionCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/RegionCommands.java
index fed5dbd..ac69d32 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/RegionCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/RegionCommands.java
@@ -16,6 +16,16 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.cache.operations.OperationContext.*;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.Region;
@@ -42,22 +52,12 @@ import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.util.RegionAttributesNames;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
+import com.gemstone.gemfire.security.ShiroUtil;
 import org.springframework.shell.core.CommandMarker;
 import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
-import static com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
-
 /***
  * Class containing implementation of commands based on region:
  * <ul>
@@ -87,8 +87,6 @@ public class RegionCommands implements CommandMarker {
       optionContext = ConverterHint.MEMBERIDNAME,
       help = CliStrings.LIST_REGION__MEMBER__HELP)
       String memberNameOrId) {
-    
-    
     Result result = null;
     try {
       Set<RegionInformation> regionInfoSet = new LinkedHashSet<RegionInformation>();
@@ -165,6 +163,7 @@ public class RegionCommands implements CommandMarker {
       mandatory = true)
       String regionName) {
 
+    ShiroUtil.authorize("CLUSTER", "READ", regionName);
     Result result = null;
     try {
       

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
index a8d1839..87053cc 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
@@ -17,12 +17,10 @@
 package com.gemstone.gemfire.management.internal.cli.remote;
 
 import java.io.IOException;
+import java.lang.reflect.Method;
 import java.util.Map;
 import java.util.Properties;
 
-import org.springframework.shell.core.Parser;
-import org.springframework.shell.event.ParseResult;
-
 import com.gemstone.gemfire.management.cli.CommandProcessingException;
 import com.gemstone.gemfire.management.cli.CommandStatement;
 import com.gemstone.gemfire.management.cli.Result;
@@ -31,6 +29,11 @@ import com.gemstone.gemfire.management.internal.cli.GfshParser;
 import com.gemstone.gemfire.management.internal.cli.LogWrapper;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.util.CommentSkipHelper;
+import com.gemstone.gemfire.management.internal.security.ResourceOperation;
+import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.security.ShiroUtil;
+import org.springframework.shell.core.Parser;
+import org.springframework.shell.event.ParseResult;
 
 /**
  * 
@@ -101,6 +104,14 @@ public class CommandProcessor {
           }
           ((CommandStatementImpl)cmdStmt).setParseResult(parseResult);
         }
+
+        //do general authorization check here
+        Method method = parseResult.getMethod();
+        ResourceOperation resourceOperation = method.getAnnotation(ResourceOperation.class);
+        if(resourceOperation!=null){
+          ShiroUtil.authorize(resourceOperation);
+        }
+
         result = executionStrategy.execute(parseResult);
         if (result instanceof Result) {
           commandResult = (Result) result;
@@ -116,7 +127,13 @@ public class CommandProcessor {
           logWrapper.info("Could not parse \""+cmdStmt.getCommandString()+"\".", e);
         }
         return ResultBuilder.createParsingErrorResult(e.getMessage());
-      } catch (RuntimeException e) {
+      } catch (GemFireSecurityException e) {
+        setLastExecutionStatus(1);
+        if (logWrapper.infoEnabled()) {
+          logWrapper.info("Could not execute \""+cmdStmt.getCommandString()+"\".", e);
+        }
+        return ResultBuilder.createGemFireUnAuthorizedErrorResult("Unauthorized while processing command <" +cmdStmt.getCommandString()+"> Reason : " + e.getMessage());
+      }catch (RuntimeException e) {
         setLastExecutionStatus(1);
         if (logWrapper.infoEnabled()) {
           logWrapper.info("Could not execute \""+cmdStmt.getCommandString()+"\".", e);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ErrorResultData.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ErrorResultData.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ErrorResultData.java
index f8db1d3..b324c8e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ErrorResultData.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ErrorResultData.java
@@ -37,8 +37,12 @@ public class ErrorResultData extends InfoResultData {
     super(gfJsonObject);
   }
   
-  public int getErrorCode(int errorCode) {
-    return (Integer) contentObject.get(ERROR_CODE);
+  public int getErrorCode() {
+    Integer code = (Integer) contentObject.get(ERROR_CODE);
+    if(code==null){
+      return ResultBuilder.ERRORCODE_DEFAULT;
+    }
+    return code;
   }
   
   /**
@@ -57,7 +61,7 @@ public class ErrorResultData extends InfoResultData {
     
     return this;
   }
-  
+
   /**
    * 
    * @param headerText

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
index 16d3f70..6b435d3 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
@@ -38,7 +38,7 @@ public class ResultBuilder {
   public static final int ERRORCODE_DEFAULT           = 400;
   public static final int ERRORCODE_CONNECTION_ERROR  = 405;
   public static final int ERRORCODE_SHELLCLIENT_ERROR = 410;
-  //Error code for auth-error??
+  public static final int ERRORCODE_UNAUTHORIZED      = 415;
   
   // errors on member
   public static final int ERRORCODE_PARSING_ERROR     = 501;
@@ -115,6 +115,11 @@ public class ResultBuilder {
                   "Could not process command due to GemFire error. " + message);
   }
 
+  public static Result createGemFireUnAuthorizedErrorResult(String message) {
+    return createErrorResult(ERRORCODE_UNAUTHORIZED,
+        "Could not process command due to GemFire error. " + message);
+  }
+
   public static Result createUserErrorResult(String message) {
     return createErrorResult(ERRORCODE_USER_ERROR, message);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
index 20f4ce8..3804111 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
@@ -151,19 +151,16 @@ public class JmxOperationInvoker implements OperationInvoker {
       this.connector = JMXConnectorFactory.connect(url, env);
       this.mbsc = connector.getMBeanServerConnection();
       this.connector.addConnectionNotificationListener(new JMXConnectionListener(this), null, null);
-      this.connector.connect(); // TODO this call to connect is not needed
       this.distributedSystemMXBeanProxy = JMX.newMXBeanProxy(mbsc, MBeanJMXAdapter.getDistributedSystemName(), DistributedSystemMXBean.class);
 
-      if (this.distributedSystemMXBeanProxy == null || !JMX.isMXBeanInterface(DistributedSystemMXBean.class)) {
+      if (this.distributedSystemMXBeanProxy == null ) {
         LogWrapper.getInstance().info("DistributedSystemMXBean is not present on member with endpoints : "+this.endpoints);
-        connector.close();
         throw new JMXConnectionException(JMXConnectionException.MANAGER_NOT_FOUND_EXCEPTION);
       }
       else {
         this.managerMemberObjectName = this.distributedSystemMXBeanProxy.getMemberObjectName();
         if (this.managerMemberObjectName == null || !JMX.isMXBeanInterface(MemberMXBean.class)) {
           LogWrapper.getInstance().info("MemberMXBean with ObjectName "+this.managerMemberObjectName+" is not present on member with endpoints : "+endpoints);
-          this.connector.close();
           throw new JMXConnectionException(JMXConnectionException.MANAGER_NOT_FOUND_EXCEPTION);
         }
         else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlMBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlMBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlMBean.java
deleted file mode 100644
index ea83771..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlMBean.java
+++ /dev/null
@@ -1,58 +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.management.internal.security;
-
-import com.gemstone.gemfire.security.AccessControl;
-
-import javax.management.remote.JMXPrincipal;
-import javax.security.auth.Subject;
-import java.security.AccessControlContext;
-import java.security.AccessController;
-import java.security.Principal;
-import java.util.Set;
-
-/**
- * AccessControlMBean Implementation. This retrieves JMXPrincipal from AccessController
- * and performs authorization for given role using gemfire AccessControl Plugin
- *
- * @author tushark
- * @since 9.0
- */
-public class AccessControlMBean implements AccessControlMXBean {
-
-  private ManagementInterceptor interceptor;
-
-  public AccessControlMBean(ManagementInterceptor interceptor) {
-    this.interceptor = interceptor;
-  }
-
-  @Override
-  public boolean authorize(String resource, String permission) {
-    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");
-    }
-    Principal principal = principals.iterator().next();
-    AccessControl gemAccControl = interceptor.getAccessControl(principal, false);
-    boolean authorized = gemAccControl.authorizeOperation(null,
-        new ResourceOperationContext(resource, permission));
-    return authorized;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/CLIOperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/CLIOperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/CLIOperationContext.java
deleted file mode 100644
index ad6be59..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/CLIOperationContext.java
+++ /dev/null
@@ -1,102 +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.management.internal.security;
-
-import java.lang.annotation.Annotation;
-import java.lang.reflect.Method;
-import java.util.HashMap;
-import java.util.Map;
-
-import com.gemstone.gemfire.management.cli.CommandProcessingException;
-import com.gemstone.gemfire.management.internal.cli.CommandManager;
-import com.gemstone.gemfire.management.internal.cli.GfshParseResult;
-import com.gemstone.gemfire.management.internal.cli.GfshParser;
-import com.gemstone.gemfire.management.internal.cli.parser.CommandTarget;
-import org.springframework.shell.event.ParseResult;
-
-/**
- * It represents command being executed and all passed options and option-values.
- * ResourceOpCode returned by CLIOperationContext is retrieved from ResourceOperation
- * annotation on the target command
- *
- * @author tushark
- * @since 9.0
- */
-public class CLIOperationContext extends ResourceOperationContext {
-
-	private Map<String,String> commandOptions = null;
-	private String command = null;
-	
-	private static Map<String,ResourceOperation> commandToCodeMapping = new HashMap<String,ResourceOperation>();
-	private static CommandManager commandManager = null;
-	private static GfshParser parser = null;	
-	
-	private CLIOperationContext(String commandName, Map<String,String> commandOptions, ResourceOperation op) throws CommandProcessingException, IllegalStateException{
-		this.command = commandName;
-		this.commandOptions = commandOptions;
-		setResourceOperation(op);
-  }
-
-	public static CLIOperationContext getOperationContext(String commandString){
-		GfshParseResult parseResult = (GfshParseResult) parseCommand(commandString);
-		ResourceOperation op = findResourceCode(parseResult.getCommandName());
-		if(op==null)
-			return null;
-
-		return new CLIOperationContext(parseResult.getCommandName(), parseResult.getParamValueStrings(), op);
-	}
-	
-	private static ParseResult parseCommand(String commentLessLine) throws CommandProcessingException, IllegalStateException {
-    if (commentLessLine != null) {
-      return parser.parse(commentLessLine);
-    }
-    throw new IllegalStateException("Command String should not be null.");
-  }
-	
-	public static void registerCommand(CommandManager cmdManager, Method method, CommandTarget commandTarget){
-	  if(commandManager==null){
-	    commandManager = cmdManager;
-	    parser = new GfshParser(cmdManager);
-	  }
-		Annotation ans[] = method.getDeclaredAnnotations();
-		for(Annotation an : ans){
-			if(an instanceof ResourceOperation) {
-				commandToCodeMapping.put(commandTarget.getCommandName(), (ResourceOperation)an);
-			}
-		}
-	}
-
-	private static void cache(String commandName, ResourceOperation op) {
-		commandToCodeMapping.put(commandName, op);
-	}
-
-	public Map<String, String> getCommandOptions() {
-		return commandOptions;
-	}
-
-	public String getCommand(){
-		return command;
-	}
-
-	private static ResourceOperation findResourceCode(String commandName) {
-		return commandToCodeMapping.get(commandName);
-	}
-	
-	public String toString(){
-	  return super.toString() + " commmand=" + command + " options=" + commandOptions;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/MBeanServerWrapper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/MBeanServerWrapper.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/MBeanServerWrapper.java
index e1c84a5..c7cb058 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/MBeanServerWrapper.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/MBeanServerWrapper.java
@@ -46,6 +46,7 @@ import javax.management.remote.MBeanServerForwarder;
 
 import com.gemstone.gemfire.management.internal.ManagementConstants;
 import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.security.ShiroUtil;
 
 /**
  * This class intercepts all MBean requests for GemFire MBeans and passed it to
@@ -55,11 +56,8 @@ import com.gemstone.gemfire.security.GemFireSecurityException;
  */
 public class MBeanServerWrapper implements MBeanServerForwarder {
   private MBeanServer mbs;
-  private ManagementInterceptor interceptor;
-
   
-  public MBeanServerWrapper(ManagementInterceptor interceptor){
-    this.interceptor = interceptor;
+  public MBeanServerWrapper(){
   }
 
   private void doAuthorization(ResourceOperationContext context){
@@ -67,14 +65,14 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
     if(context == null)
       return;
 
-    interceptor.authorize(context);
+    ShiroUtil.authorize(context);
   }
 
   private void doAuthorizationPost(ResourceOperationContext context){
     if(context == null)
       return;
 
-    interceptor.postAuthorize(context);
+    //interceptor.postAuthorize(context);
   }
 
   private void checkDomain(ObjectName name){
@@ -218,14 +216,12 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
   @Override
   public Object invoke(ObjectName name, String operationName, Object[] params, String[] signature)
       throws InstanceNotFoundException, MBeanException, ReflectionException {
+    // skip authorization check if operation is "processCommand" since we will check authorization in the command itself
     ResourceOperationContext ctx = null;
-    if("processCommand".equals(operationName) && params.length>=1){
-      ctx = CLIOperationContext.getOperationContext((String)params[0]);
-    }
-    else {
+    if(!"processCommand".equals(operationName)) {
       ctx = getOperationContext(name, operationName, true);
+      doAuthorization(ctx);
     }
-    doAuthorization(ctx);
 
     Object result = mbs.invoke(name, operationName, params, signature);
     if(ctx!=null)
@@ -271,7 +267,7 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
     String resource = (String)descriptor.getFieldValue("resource");
     String operationCode = (String)descriptor.getFieldValue("operation");
     if(resource!=null && operationCode!=null){
-      return new ResourceOperationContext(resource, operationCode);
+      return new ResourceOperationContext(resource, operationCode, null);
     }
     return defaultValue;
   }
@@ -397,10 +393,6 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
     return mbs;
   }
 
-  public ManagementInterceptor getInterceptor() {
-    return interceptor;
-  }
-
   @Override
   public void setMBeanServer(MBeanServer mbs) {
     this.mbs = mbs;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ManagementInterceptor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ManagementInterceptor.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ManagementInterceptor.java
deleted file mode 100644
index 6ee9da5..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ManagementInterceptor.java
+++ /dev/null
@@ -1,243 +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.management.internal.security;
-
-import com.gemstone.gemfire.GemFireConfigException;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.ClassLoadUtil;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.lang.StringUtils;
-import com.gemstone.gemfire.security.AccessControl;
-import com.gemstone.gemfire.security.AuthenticationFailedException;
-import com.gemstone.gemfire.security.Authenticator;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-import javax.management.InstanceAlreadyExistsException;
-import javax.management.MBeanRegistrationException;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.NotCompliantMBeanException;
-import javax.management.ObjectName;
-import javax.management.remote.JMXAuthenticator;
-import javax.management.remote.JMXPrincipal;
-import javax.security.auth.Subject;
-import java.lang.management.ManagementFactory;
-import java.lang.reflect.Method;
-import java.security.AccessControlContext;
-import java.security.AccessController;
-import java.security.Principal;
-import java.util.Collections;
-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.management.internal.security.ResourceConstants.WRONGE_CREDENTIALS_MESSAGE;
-
-/**
- * ManagementInterceptor is central go-to place for all M&M Clients Authentication and Authorization
- * requests
- *
- * @since 9.0
- */
-public class ManagementInterceptor implements JMXAuthenticator {
-
-  // FIXME: Merged from GEODE-17. Are they necessary?
-  public static final String USER_NAME = "security-username";
-  public static final String PASSWORD = "security-password";
-  public static final String OBJECT_NAME_ACCESSCONTROL = "GemFire:service=AccessControl,type=Distributed";
-
-  private static final Logger logger = LogManager.getLogger(ManagementInterceptor.class);
-  //  private Cache cache;
-  private String authzFactoryName;
-  private String postAuthzFactoryName;
-  private String authenticatorFactoryName;
-  private ConcurrentMap<Principal, AccessControl> cachedAuthZCallback;
-  private ConcurrentMap<Principal, AccessControl> cachedPostAuthZCallback;
-  private Properties sysProps = null;
-
-  public ManagementInterceptor(Properties sysProps) {
-    this.sysProps = sysProps;
-    this.authzFactoryName = sysProps.getProperty(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME);
-    this.postAuthzFactoryName = sysProps.getProperty(DistributionConfig.SECURITY_CLIENT_ACCESSOR_PP_NAME);
-    this.authenticatorFactoryName = sysProps.getProperty(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME);
-    this.cachedAuthZCallback = new ConcurrentHashMap<>();
-    this.cachedPostAuthZCallback = new ConcurrentHashMap<>();
-    registerAccessControlMBean();
-    logger.info("Started Management interceptor on JMX connector");
-  }
-
-  /**
-   * This method registers an AccessControlMBean which allows any remote JMX Client (for example Pulse) to check for
-   * access allowed for given Operation Code.
-   */
-  private void registerAccessControlMBean() {
-    try {
-      AccessControlMBean acc = new AccessControlMBean(this);
-      ObjectName accessControlMBeanON = new ObjectName(ResourceConstants.OBJECT_NAME_ACCESSCONTROL);
-      MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
-
-      Set<ObjectName> names = platformMBeanServer.queryNames(accessControlMBeanON, null);
-      if (names.isEmpty()) {
-        try {
-          platformMBeanServer.registerMBean(acc, accessControlMBeanON);
-          logger.info("Registered AccessContorlMBean on " + accessControlMBeanON);
-        } catch (InstanceAlreadyExistsException e) {
-          throw new GemFireConfigException("Error while configuring accesscontrol for jmx resource", e);
-        } catch (MBeanRegistrationException e) {
-          throw new GemFireConfigException("Error while configuring accesscontrol for jmx resource", e);
-        } catch (NotCompliantMBeanException e) {
-          throw new GemFireConfigException("Error while configuring accesscontrol for jmx resource", e);
-        }
-      }
-    } catch (MalformedObjectNameException e) {
-      throw new GemFireConfigException("Error while configuring accesscontrol for jmx resource", e);
-    }
-  }
-
-  /**
-   * Delegates authentication to GemFire Authenticator
-   *
-   * @throws SecurityException if authentication fails
-   */
-  @Override
-  public Subject authenticate(Object credentials) {
-    String username = null, password = null;
-    Properties pr = new Properties();
-    if (credentials instanceof String[]) {
-      final String[] aCredentials = (String[]) credentials;
-      username = aCredentials[0];
-      password = aCredentials[1];
-      pr.put(USER_NAME, username);
-      pr.put(PASSWORD, password);
-    } else if (credentials instanceof Properties) {
-      pr = (Properties) credentials;
-    } else {
-      throw new SecurityException(WRONGE_CREDENTIALS_MESSAGE);
-    }
-
-    try {
-      Principal principal = getAuthenticator(sysProps).authenticate(pr);
-      return new Subject(true, Collections.singleton(new JMXPrincipal(principal.getName())), Collections.EMPTY_SET,
-          Collections.EMPTY_SET);
-    } catch (AuthenticationFailedException e) {
-      //wrap inside Security exception. AuthenticationFailedException is gemfire class
-      //which generic JMX client can't serialize
-      throw new SecurityException("Authentication Failed " + e.getMessage());
-    }
-
-  }
-
-  /**
-   * Builds ResourceOperationContext for the given JMX MBean Request for delegates Authorization to
-   * gemfire AccessControl plugin with context as parameter
-   *
-   * @throws SecurityException if access is not granted
-   */
-  public void authorize(ResourceOperationContext context) {
-    if (StringUtils.isBlank(authzFactoryName)) {
-      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 + ": No principal found.");
-    }
-
-    Principal principal = principals.iterator().next();
-
-    AccessControl accessControl = getAccessControl(principal, false);
-
-    if (!accessControl.authorizeOperation(null, context)) {
-      throw new SecurityException(ACCESS_DENIED_MESSAGE + ": Not authorized for " + 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);
-          cachedAuthZCallback.put(principal, authzCallback);
-          return authzCallback;
-        } catch (Exception ex) {
-          throw new AuthenticationFailedException(LocalizedStrings.HandShake_FAILED_TO_ACQUIRE_AUTHENTICATOR_OBJECT.toLocalizedString(), 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);
-          cachedPostAuthZCallback.put(principal, postAuthzCallback);
-          return postAuthzCallback;
-        } catch (Exception ex) {
-          throw new AuthenticationFailedException(LocalizedStrings.HandShake_FAILED_TO_ACQUIRE_AUTHENTICATOR_OBJECT.toLocalizedString(), ex);
-        }
-      }
-    }
-    return null;
-  }
-
-  private Authenticator getAuthenticator(Properties gfSecurityProperties) throws AuthenticationFailedException {
-    Authenticator auth;
-    try {
-      Method instanceGetter = ClassLoadUtil.methodFromName(this.authenticatorFactoryName);
-      auth = (Authenticator) instanceGetter.invoke(null, (Object[]) null);
-    } catch (Exception ex) {
-      throw new AuthenticationFailedException(
-          LocalizedStrings.HandShake_FAILED_TO_ACQUIRE_AUTHENTICATOR_OBJECT.toLocalizedString(), ex);
-    }
-    if (auth == null) {
-      throw new AuthenticationFailedException(
-          LocalizedStrings.HandShake_AUTHENTICATOR_INSTANCE_COULD_NOT_BE_OBTAINED.toLocalizedString());
-    }
-    auth.init(gfSecurityProperties);
-    return auth;
-  }
-
-  public void postAuthorize(ResourceOperationContext 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);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c733f0c2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
index 9ef458d..396cdac 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
@@ -27,24 +27,29 @@ public class ResourceOperationContext extends OperationContext {
   private Object opResult = null;
   private Resource resource = null;
   private OperationCode operation = null;
+  private String regionName = null;
 
   public ResourceOperationContext() {
   }
 
   public ResourceOperationContext(Resource resource, OperationCode operation) {
+    setParts(resource.name()+":"+operation.name(), false);
     this.resource = resource;
     this.operation = operation;
   }
 
-  public ResourceOperationContext(String resource, String operation) {
+  public ResourceOperationContext(String resource, String operation, String regionName) {
+    setParts(resource+":"+operation+":"+regionName, false);
     if (resource != null) this.resource = Resource.valueOf(resource);
     if (operation != null) this.operation = OperationCode.valueOf(operation);
+    this.regionName = regionName;
   }
 
   public void setResourceOperation(ResourceOperation op) {
     if (op != null) {
       resource = op.resource();
       operation = op.operation();
+      setParts(resource.name()+":"+operation.name(), false);
     }
   }
 
@@ -64,6 +69,11 @@ public class ResourceOperationContext extends OperationContext {
   }
 
   @Override
+  public String getRegionName(){
+    return this.regionName;
+  }
+
+  @Override
   public boolean isPostOperation() {
     return isPostOperation;
   }
@@ -77,4 +87,20 @@ public class ResourceOperationContext extends OperationContext {
     return this.opResult;
   }
 
+  public String toString(){
+    if(this.regionName==null)
+      return getResource() + ":"+ getOperationCode();
+    else
+      return getResource() + ":"+ getOperationCode()+ ":" +this.regionName;
+  }
+
+  public boolean equals(Object o){
+    if(! (o instanceof ResourceOperationContext))
+      return false;
+
+    ResourceOperationContext other = (ResourceOperationContext)o;
+    return (this.resource==other.getResource() && this.operation==other.getOperationCode());
+  }
+
+
 }
\ No newline at end of file