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/03/11 23:20:44 UTC

incubator-geode git commit: GEODE-17: more cleanup

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-17-2 3040e5afe -> 44b248cbe


GEODE-17: more cleanup

* break the cirucular dependency in MBeanServerWrapper and ManagementInterceptor
* Now custom authenticator and authorizer can be set independant of each other
* re-arrange the tests
* add test to test local JMX calls to make sure they don't go through the MBeanServerWrapper

Merge branch 'feature/GEODE-17-2' of https://git-wip-us.apache.org/repos/asf/incubator-geode into feature/GEODE-17-2


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 44b248cbe727e88fd078fedbf806a7a83f616fea
Parents: 3040e5a
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Thu Mar 10 21:35:13 2016 -0800
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri Mar 11 14:20:02 2016 -0800

----------------------------------------------------------------------
 .../management/internal/ManagementAgent.java    |  87 ++++++++--------
 .../internal/security/MBeanServerWrapper.java   |  87 ++++++++--------
 .../security/ManagementInterceptor.java         |  39 ++-----
 .../internal/security/ResourceOperation.java    |   4 -
 .../security/ResourceOperationContext.java      |   4 +-
 .../CacheServerMBeanSecurityJUnitTest.java      |  35 -------
 .../JsonAuthorizationMBeanServerStartRule.java  |   4 +
 .../security/MBeanSecurityJUnitTest.java        | 102 +++++++++++++++++++
 .../security/MBeanServerConnectionRule.java     |  17 ++--
 .../security/MemberMBeanSecurityJUnitTest.java  |   2 +
 ...tionCodesForDistributedSystemMXBeanTest.java |  69 -------------
 11 files changed, 210 insertions(+), 240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44b248cb/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 fecbb4c..b1c9b46 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
@@ -16,33 +16,6 @@
  */
 package com.gemstone.gemfire.management.internal;
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.net.UnknownHostException;
-import java.rmi.AlreadyBoundException;
-import java.rmi.registry.LocateRegistry;
-import java.rmi.registry.Registry;
-import java.rmi.server.RMIClientSocketFactory;
-import java.rmi.server.RMIServerSocketFactory;
-import java.rmi.server.UnicastRemoteObject;
-import java.util.HashMap;
-
-import javax.management.MBeanServer;
-import javax.management.remote.JMXConnectorServer;
-import javax.management.remote.JMXServiceURL;
-import javax.management.remote.rmi.RMIConnectorServer;
-import javax.management.remote.rmi.RMIJRMPServerImpl;
-import javax.management.remote.rmi.RMIServerImpl;
-import javax.rmi.ssl.SslRMIClientSocketFactory;
-
-import org.apache.logging.log4j.Logger;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.ServerConnector;
-
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -55,8 +28,34 @@ import com.gemstone.gemfire.internal.tcp.TCPConduit;
 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 org.apache.logging.log4j.Logger;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+
+import javax.management.MBeanServer;
+import javax.management.remote.JMXConnectorServer;
+import javax.management.remote.JMXServiceURL;
+import javax.management.remote.rmi.RMIConnectorServer;
+import javax.management.remote.rmi.RMIJRMPServerImpl;
+import javax.management.remote.rmi.RMIServerImpl;
+import javax.rmi.ssl.SslRMIClientSocketFactory;
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.management.ManagementFactory;
+import java.net.InetAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.rmi.AlreadyBoundException;
+import java.rmi.registry.LocateRegistry;
+import java.rmi.registry.Registry;
+import java.rmi.server.RMIClientSocketFactory;
+import java.rmi.server.RMIServerSocketFactory;
+import java.rmi.server.UnicastRemoteObject;
+import java.util.HashMap;
 
 /**
  * Agent implementation that controls the JMX server end points for JMX clients
@@ -83,7 +82,6 @@ public class ManagementAgent {
   private JMXConnectorServer cs;
   private final DistributionConfig config;
   private boolean isHttpServiceRunning = false;
-  private ManagementInterceptor securityInterceptor;
 
   /**
    * This system property is set to true when the embedded HTTP server is
@@ -386,11 +384,12 @@ public class ManagementAgent {
     // Environment map. KIRK: why is this declared as HashMap?
     final HashMap<String, Object> env = new HashMap<String, Object>();
 
-    boolean integratedSecEnabled = isIntegratedSecEnabled();
-    if (integratedSecEnabled) {
+    ManagementInterceptor securityInterceptor = null;
+    if (isCustomAuthenticator()) {
       securityInterceptor = new ManagementInterceptor((GemFireCacheImpl)CacheFactory.getAnyInstance(), logger);
       env.put(JMXConnectorServer.AUTHENTICATOR, securityInterceptor);
-    } else {
+    }
+    else {
       /* Disable the old authenticator mechanism */
       String pwFile = this.config.getJmxManagerPasswordFile();
       if (pwFile != null && pwFile.length() > 0) {
@@ -460,12 +459,13 @@ public class ManagementAgent {
         super.start();
       }
     };
-    // This may be the 1.6 way of doing it but the problem is it does not use
-    // our "stub".
-    // cs = JMXConnectorServerFactory.newJMXConnectorServer(url, env, mbs);
 
-    if (integratedSecEnabled) {
-      cs.setMBeanServerForwarder(securityInterceptor.getMBeanServerForwarder());
+    if (isCustomAuthorizer()) {
+      if(securityInterceptor==null){
+        securityInterceptor = new ManagementInterceptor((GemFireCacheImpl)CacheFactory.getAnyInstance(), logger);
+      }
+      MBeanServerWrapper mBeanServerWrapper = new MBeanServerWrapper(securityInterceptor);
+      cs.setMBeanServerForwarder(mBeanServerWrapper);
       logger.info("Starting RMI Connector with Security Interceptor");
     }
 
@@ -473,17 +473,16 @@ public class ManagementAgent {
     if (logger.isDebugEnabled()) {
       logger.debug("Finished starting jmx manager agent.");
     }
-    // System.out.println("Server started at: "+cs.getAddress());
+  }
 
-    // Start the CleanThread daemon... KIRK: not sure what CleanThread is...
-    //
-    // final Thread clean = new CleanThread(cs);
-    // clean.start();
+  private boolean isCustomAuthenticator() {
+    String factoryName = config.getSecurityClientAuthenticator();
+    return factoryName != null && !factoryName.isEmpty();
   }
 
-  private boolean isIntegratedSecEnabled() {
-    String authenticatorFactoryName = config.getSecurityClientAuthenticator();
-    return authenticatorFactoryName != null && !authenticatorFactoryName.isEmpty();
+  private boolean isCustomAuthorizer() {
+    String factoryName = config.getSecurityClientAccessor();
+    return factoryName != null && !factoryName.isEmpty();
   }
 
   private static class GemFireRMIClientSocketFactory implements RMIClientSocketFactory,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44b248cb/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 f2030c0..d12a5de 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
@@ -16,7 +16,6 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
-import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
 import com.gemstone.gemfire.security.GemFireSecurityException;
 
 import javax.management.Attribute;
@@ -39,78 +38,80 @@ import javax.management.NotificationListener;
 import javax.management.ObjectInstance;
 import javax.management.ObjectName;
 import javax.management.OperationsException;
+import javax.management.Query;
 import javax.management.QueryExp;
 import javax.management.ReflectionException;
 import javax.management.loading.ClassLoaderRepository;
 import javax.management.remote.MBeanServerForwarder;
 import java.io.ObjectInputStream;
-import java.util.HashSet;
 import java.util.Set;
 
-import static com.gemstone.gemfire.management.internal.security.ResourceConstants.ACCESS_DENIED_MESSAGE;
-
 /**
  * This class intercepts all MBean requests for GemFire MBeans and passed it to
  * ManagementInterceptor for authorization
- *
- *
- * @author tushark
  * @since 9.0
  *
  */
 public class MBeanServerWrapper implements MBeanServerForwarder {
-  
   private MBeanServer mbs;
   private ManagementInterceptor interceptor;
+
   
   public MBeanServerWrapper(ManagementInterceptor interceptor){
     this.interceptor = interceptor;
   }
-  
+
   private void doAuthorization(ResourceOperationContext context){
+    // allow operations which requires no permissions
+    if(context == null)
+      return;
+
     interceptor.authorize(context);
   }
 
   private void doAuthorizationPost(ResourceOperationContext context){
+    if(context == null)
+      return;
+
     interceptor.postAuthorize(context);
   }
 
   @Override
   public ObjectInstance createMBean(String className, ObjectName name) throws ReflectionException,
-      InstanceAlreadyExistsException, MBeanRegistrationException, MBeanException, NotCompliantMBeanException {
-    throw new SecurityException(ACCESS_DENIED_MESSAGE);
+      InstanceAlreadyExistsException, MBeanException, NotCompliantMBeanException {
+    throw new SecurityException(ResourceConstants.ACCESS_DENIED_MESSAGE);
   }
 
   @Override
   public ObjectInstance createMBean(String className, ObjectName name, ObjectName loaderName)
-      throws ReflectionException, InstanceAlreadyExistsException, MBeanRegistrationException, MBeanException,
+      throws ReflectionException, InstanceAlreadyExistsException, MBeanException,
       NotCompliantMBeanException, InstanceNotFoundException {
-    throw new SecurityException(ACCESS_DENIED_MESSAGE);
+    throw new SecurityException(ResourceConstants.ACCESS_DENIED_MESSAGE);
   }
 
   @Override
   public ObjectInstance createMBean(String className, ObjectName name, Object[] params, String[] signature)
-      throws ReflectionException, InstanceAlreadyExistsException, MBeanRegistrationException, MBeanException,
+      throws ReflectionException, InstanceAlreadyExistsException, MBeanException,
       NotCompliantMBeanException {
-    throw new SecurityException(ACCESS_DENIED_MESSAGE);
+    throw new SecurityException(ResourceConstants.ACCESS_DENIED_MESSAGE);
   }
 
   @Override
   public ObjectInstance createMBean(String className, ObjectName name, ObjectName loaderName, Object[] params,
-      String[] signature) throws ReflectionException, InstanceAlreadyExistsException, MBeanRegistrationException,
+      String[] signature) throws ReflectionException, InstanceAlreadyExistsException,
       MBeanException, NotCompliantMBeanException, InstanceNotFoundException {
-    throw new SecurityException(ACCESS_DENIED_MESSAGE);
+    throw new SecurityException(ResourceConstants.ACCESS_DENIED_MESSAGE);
   }
 
   @Override
   public ObjectInstance registerMBean(Object object, ObjectName name) throws InstanceAlreadyExistsException,
       MBeanRegistrationException, NotCompliantMBeanException {
-    throw new SecurityException(ACCESS_DENIED_MESSAGE);
+    throw new SecurityException(ResourceConstants.ACCESS_DENIED_MESSAGE);
   }
 
   @Override
   public void unregisterMBean(ObjectName name) throws InstanceNotFoundException, MBeanRegistrationException {
-    throw new SecurityException(ACCESS_DENIED_MESSAGE);
+    throw new SecurityException(ResourceConstants.ACCESS_DENIED_MESSAGE);
   }
 
   @Override
@@ -118,35 +119,22 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
     return mbs.getObjectInstance(name);
   }
 
+  private static QueryExp notAccessControlMBean = Query.not(Query.isInstanceOf(Query.value(AccessControlMXBean.class.getName())));
   @Override
   public Set<ObjectInstance> queryMBeans(ObjectName name, QueryExp query) {
-    ResourceOperationContext ctx = new ResourceOperationContext(Resource.MBEAN, OperationCode.QUERY);
-    return filterAccessControlMBeanInstance(mbs.queryMBeans(name, query));
-  }
-
-  private Set<ObjectInstance> filterAccessControlMBeanInstance(Set<ObjectInstance> queryMBeans) {
-    Set<ObjectInstance> set = new HashSet<ObjectInstance>();
-    for(ObjectInstance oi : queryMBeans) {
-      if(!oi.getObjectName().equals(interceptor.getAccessControlMBeanON())){
-        set.add(oi);
-      }
-    }
-    return set;
+    // We need to filter out the AccessControlMXBean so that the clients wouldn't see it
+    if(query!=null)
+      return mbs.queryMBeans(name, Query.and(query, notAccessControlMBean));
+    else
+      return mbs.queryMBeans(name,notAccessControlMBean);
   }
 
   @Override
   public Set<ObjectName> queryNames(ObjectName name, QueryExp query) {
-    return filterAccessControlMBean(mbs.queryNames(name, query));
-  }
-
-  private Set<ObjectName> filterAccessControlMBean(Set<ObjectName> queryNames) {
-    Set<ObjectName> set = new HashSet<ObjectName>();
-    for(ObjectName oi : queryNames) {
-      if(!oi.equals(interceptor.getAccessControlMBeanON())){
-        set.add(oi);
-      }
-    }
-    return set;
+    if(query!=null)
+      return mbs.queryNames(name, Query.and(query, notAccessControlMBean));
+    else
+      return mbs.queryNames(name,notAccessControlMBean);
   }
 
   @Override
@@ -165,7 +153,9 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
     ResourceOperationContext ctx = getOperationContext(name, attribute, false);
     doAuthorization(ctx);
     Object result = mbs.getAttribute(name, attribute);
-    ctx.setPostOperationResult(result);
+    if(ctx != null) {
+      ctx.setPostOperationResult(result);
+    }
     doAuthorizationPost(ctx);
     return result;
   }
@@ -214,7 +204,8 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
     ResourceOperationContext ctx = getOperationContext(name, operationName, true);
     doAuthorization(ctx);
     Object result = mbs.invoke(name, operationName, params, signature);
-    ctx.setPostOperationResult(result);
+    if(ctx!=null)
+      ctx.setPostOperationResult(result);
     doAuthorizationPost(ctx);
     return result;
   }
@@ -228,8 +219,8 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
     } catch (IntrospectionException e) {
       throw new GemFireSecurityException("error getting beanInfo of "+objectName);
     }
-    // Initialize the context with the default value
-    ResourceOperationContext result = new ResourceOperationContext(Resource.DEFAULT, OperationCode.LIST_DS);
+    // If there is no annotation defined either in the class level or method level, we should consider this operation/attribute freely accessible
+    ResourceOperationContext result = null;
 
     // find the context in the beanInfo if defined in the class level
     result = getOperationContext(beanInfo.getDescriptor(), result);
@@ -382,6 +373,10 @@ 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/44b248cb/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
index d7c1474..2362f4d 100644
--- 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
@@ -37,7 +37,6 @@ import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
 import javax.management.remote.JMXAuthenticator;
 import javax.management.remote.JMXPrincipal;
-import javax.management.remote.MBeanServerForwarder;
 import javax.security.auth.Subject;
 import java.lang.management.ManagementFactory;
 import java.lang.reflect.Method;
@@ -50,7 +49,8 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import static com.gemstone.gemfire.management.internal.security.ResourceConstants.*;
+import static com.gemstone.gemfire.management.internal.security.ResourceConstants.ACCESS_DENIED_MESSAGE;
+import static com.gemstone.gemfire.management.internal.security.ResourceConstants.WRONGE_CREDENTIALS_MESSAGE;
 
 /**
  *
@@ -66,9 +66,7 @@ public class ManagementInterceptor implements JMXAuthenticator{
 	public static final String PASSWORD = "security-password";
 	public static final String OBJECT_NAME_ACCESSCONTROL = "GemFire:service=AccessControl,type=Distributed";
 
-	private MBeanServerWrapper mBeanServerForwarder;
-	private Logger logger;  
-  private ObjectName accessControlMBeanON;
+	private Logger logger;
   private Cache cache;
   private String authzFactoryName;
   private String postAuthzFactoryName;
@@ -79,7 +77,6 @@ public class ManagementInterceptor implements JMXAuthenticator{
   public ManagementInterceptor(Cache gemFireCacheImpl, Logger logger) {
     this.cache = gemFireCacheImpl;
 		this.logger = logger;
-		this.mBeanServerForwarder = new MBeanServerWrapper(this);
     DistributedSystem system = cache.getDistributedSystem();
     Properties sysProps = system.getProperties();
     this.authzFactoryName = sysProps.getProperty(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME);
@@ -98,7 +95,7 @@ public class ManagementInterceptor implements JMXAuthenticator{
 	private void registerAccessContorlMbean() {
     try {
       AccessControlMBean acc = new AccessControlMBean(this);
-      accessControlMBeanON = new ObjectName(ResourceConstants.OBJECT_NAME_ACCESSCONTROL);
+      ObjectName accessControlMBeanON = new ObjectName(ResourceConstants.OBJECT_NAME_ACCESSCONTROL);
       MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
 
       Set<ObjectName> names = platformMBeanServer.queryNames(accessControlMBeanON, null);
@@ -162,22 +159,12 @@ public class ManagementInterceptor implements JMXAuthenticator{
    *           if access is not granted
    */
   public void authorize(ResourceOperationContext context) {
-    if(context==null){
-      return;
-    }
-
     if (StringUtils.isBlank(authzFactoryName)){
       return;
     }
 
-		AccessControlContext acc = AccessController.getContext();
-		Subject subject = Subject.getSubject(acc);
-
-    // Allow operations performed locally on behalf of the connector server itself
-		if (subject == null) {
-      return;
-		}
-
+    AccessControlContext acc = AccessController.getContext();
+    Subject subject = Subject.getSubject(acc);
     Set<JMXPrincipal> principals = subject.getPrincipals(JMXPrincipal.class);
 
     if (principals == null || principals.isEmpty()) {
@@ -193,10 +180,6 @@ public class ManagementInterceptor implements JMXAuthenticator{
     }
   }
 
-	public MBeanServerForwarder getMBeanServerForwarder() {
-		return mBeanServerForwarder;
-	}
-
   public AccessControl getAccessControl(Principal principal, boolean isPost) {
     if (!isPost) {
       if (cachedAuthZCallback.containsKey(principal)) {
@@ -249,21 +232,11 @@ public class ManagementInterceptor implements JMXAuthenticator{
     return auth;
 	}
 
-  public ObjectName getAccessControlMBeanON() {
-    return accessControlMBeanON;
-    }
-
   public void postAuthorize(ResourceOperationContext context) {
-
-    if(context==null){
-      return;
-    }
-
     if (StringUtils.isBlank(postAuthzFactoryName)){
       return ;
     }
 
-
     AccessControlContext acc = AccessController.getContext();
     Subject subject = Subject.getSubject(acc);
     Set<JMXPrincipal> principals = subject.getPrincipals(JMXPrincipal.class);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44b248cb/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperation.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperation.java
index 47fd79c..4adc017 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperation.java
@@ -22,16 +22,12 @@ import java.lang.annotation.Inherited;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
-
-
 import static com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
 
 @Target({ElementType.METHOD, ElementType.TYPE})
-
 @Retention(RetentionPolicy.RUNTIME)
 @Inherited
 public @interface ResourceOperation {
-
   @DescriptorKey("resource")
   Resource resource();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44b248cb/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 2dd3456..9ef458d 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
@@ -25,8 +25,8 @@ public class ResourceOperationContext extends OperationContext {
 
   private boolean isPostOperation = false;
   private Object opResult = null;
-  private Resource resource = Resource.DEFAULT;
-  private OperationCode operation = OperationCode.MANAGE;
+  private Resource resource = null;
+  private OperationCode operation = null;
 
   public ResourceOperationContext() {
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44b248cb/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
index dab65ea..50bda62 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
@@ -26,13 +26,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import javax.management.MBeanServerConnection;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectInstance;
-import javax.management.ObjectName;
-import java.io.IOException;
-import java.util.Set;
 
-import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 @Category(IntegrationTest.class)
@@ -53,23 +47,8 @@ public class CacheServerMBeanSecurityJUnitTest {
   public void setUp() throws Exception {
     cacheServerMXBean = (CacheServerMXBean) connectionRule.getProxyMBean(CacheServerMXBean.class,
         "GemFire:service=CacheServer,*");
-    con = connectionRule.getMBeanServerConnection();
   }
 
-  /**
-   * No user can call createBean or unregisterBean
-   */
-  @Test
-  @JMXConnectionConfiguration(user = "superuser", password = "1234567")
-  public void testNoAccessWithWhoever() throws Exception {
-    assertThatThrownBy(() -> con.createMBean("FakeClassName", new ObjectName("GemFire", "name", "foo")))
-        .isInstanceOf(SecurityException.class);
-
-    assertThatThrownBy(() -> con.unregisterMBean(new ObjectName("GemFire", "name", "foo")))
-        .isInstanceOf(SecurityException.class);
-  }
-
-
   @Test
   @JMXConnectionConfiguration(user = "superuser", password = "1234567")
   public void testAllAccess() throws Exception {
@@ -103,18 +82,4 @@ public class CacheServerMBeanSecurityJUnitTest {
     assertThatThrownBy(() -> cacheServerMXBean.isRunning()).isInstanceOf(SecurityException.class);
     assertThatThrownBy(() -> cacheServerMXBean.showClientQueueDetails("bar")).isInstanceOf(SecurityException.class);
   }
-
-  /*
-   * looks like everyone can query for beans, but the AccessControlMXBean is filtered from the result
-   */
-  @Test
-  @JMXConnectionConfiguration(user = "stranger", password = "1234567")
-  public void testQueryBean() throws MalformedObjectNameException, IOException {
-    Set<ObjectInstance> objects = con.queryMBeans(ObjectName.getInstance(ResourceConstants.OBJECT_NAME_ACCESSCONTROL),
-        null);
-    assertThat(objects.size()).isEqualTo(0); // no AccessControlMBean in the query result
-
-    objects = con.queryMBeans(ObjectName.getInstance("GemFire:service=CacheServer,*"), null);
-    assertThat(objects.size()).isEqualTo(1);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44b248cb/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java
index 933b27c..1f4c592 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java
@@ -51,6 +51,10 @@ public class JsonAuthorizationMBeanServerStartRule extends ExternalResource {
     cache.addCacheServer().start();
   }
 
+  public Cache getCache(){
+    return cache;
+  }
+
   /**
    * Override to tear down your specific external resource.
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44b248cb/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanSecurityJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanSecurityJUnitTest.java
new file mode 100644
index 0000000..fb2c10c
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanSecurityJUnitTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.internal.AvailablePort;
+import com.gemstone.gemfire.management.ManagementException;
+import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+
+import javax.management.DynamicMBean;
+import javax.management.MBeanServer;
+import javax.management.MBeanServerConnection;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectInstance;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import java.io.IOException;
+import java.util.Set;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+
+public class MBeanSecurityJUnitTest {
+
+  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+
+  @ClassRule
+  public static JsonAuthorizationMBeanServerStartRule serverRule = new JsonAuthorizationMBeanServerStartRule(jmxManagerPort, "cacheServer.json");
+
+  @Rule
+  public MBeanServerConnectionRule connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
+
+
+  /**
+   * No user can call createBean or unregisterBean of any domain
+   */
+  @Test
+  @JMXConnectionConfiguration(user = "superuser", password = "1234567")
+  public void testNoAccessWithWhoever() throws Exception{
+    MBeanServerConnection con = connectionRule.getMBeanServerConnection();
+    assertThatThrownBy(
+        () -> con.createMBean("FakeClassName", new ObjectName("GemFire", "name", "foo"))
+    ).isInstanceOf(SecurityException.class);
+
+    assertThatThrownBy(
+        () -> con.unregisterMBean(new ObjectName("GemFire", "name", "foo"))
+    ).isInstanceOf(SecurityException.class);
+
+    // user is not allowed to create beans of other domains either
+    assertThatThrownBy(
+        () -> con.createMBean("FakeClassName", new ObjectName("OtherDomain", "name", "foo"))
+    ).isInstanceOf(SecurityException.class);
+  }
+
+  /*
+  * looks like everyone can query for beans, but the AccessControlMXBean is filtered from the result
+  */
+  @Test
+  @JMXConnectionConfiguration(user = "stranger", password = "1234567")
+  public void testQueryBean() throws MalformedObjectNameException, IOException {
+    MBeanServerConnection con = connectionRule.getMBeanServerConnection();
+    Set<ObjectInstance> objects = con.queryMBeans(ObjectName.getInstance(ResourceConstants.OBJECT_NAME_ACCESSCONTROL), null);
+    assertThat(objects.size()).isEqualTo(0); // no AccessControlMBean in the query result
+
+    objects = con.queryMBeans(ObjectName.getInstance("GemFire:service=CacheServer,*"), null);
+    assertThat(objects.size()).isEqualTo(1);
+  }
+
+  /*
+  * These calls does not go through the MBeanServerWrapper authentication, therefore is not throwing the SecurityExceptions
+   */
+  @Test
+  public void testLocalCalls() throws Exception{
+    MBeanServer server = MBeanJMXAdapter.mbeanServer;
+    assertThatThrownBy(
+        () -> server.createMBean("FakeClassName", new ObjectName("GemFire", "name", "foo"))
+    ).isInstanceOf(ReflectionException.class);
+
+    MBeanJMXAdapter adapter = new MBeanJMXAdapter();
+    assertThatThrownBy(
+        () -> adapter.registerMBean(mock(DynamicMBean.class), new ObjectName("MockDomain", "name", "mock"), false)
+    ).isInstanceOf(ManagementException.class);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44b248cb/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
index ed4661c..85c7cec 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
@@ -37,6 +37,7 @@ import java.util.Set;
 /**
  * Class which eases the creation of MBeans for security testing. When combined with {@link JMXConnectionConfiguration}
  * it allows for the creation of per-test connections with different user/password combinations.
+ *
  */
 public class MBeanServerConnectionRule extends DescribedExternalResource {
 
@@ -103,20 +104,22 @@ public class MBeanServerConnectionRule extends DescribedExternalResource {
     if (config != null) {
       String user = config.user();
       String password = config.password();
-      env.put(JMXConnector.CREDENTIALS, new String[]{user, password});
-    }
-    JMXServiceURL url = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://:" + jmxServerPort + "/jmxrmi");
+      env.put(JMXConnector.CREDENTIALS, new String[] { user, password });
 
-    jmxConnector = JMXConnectorFactory.connect(url, env);
-    con = jmxConnector.getMBeanServerConnection();
+      JMXServiceURL url = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://:" + jmxServerPort + "/jmxrmi");
+      jmxConnector = JMXConnectorFactory.connect(url, env);
+      con = jmxConnector.getMBeanServerConnection();
+    }
   }
 
   /**
    * Override to tear down your specific external resource.
    */
   protected void after(Description description) throws Throwable {
-    jmxConnector.close();
-    jmxConnector = null;
+    if (jmxConnector != null) {
+      jmxConnector.close();
+      jmxConnector = null;
+    }
     con = null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44b248cb/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
index 22bc25b..147dd9a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
@@ -25,6 +25,8 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import javax.management.MBeanServerConnection;
+
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 @Category(IntegrationTest.class)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44b248cb/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/OperationCodesForDistributedSystemMXBeanTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/OperationCodesForDistributedSystemMXBeanTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/OperationCodesForDistributedSystemMXBeanTest.java
deleted file mode 100755
index 253b2e7..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/OperationCodesForDistributedSystemMXBeanTest.java
+++ /dev/null
@@ -1,69 +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.cache.operations.OperationContext.OperationCode;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Tests operation codes for DistributedSystemMXBean operations.
- */
-@Category(UnitTest.class)
-public class OperationCodesForDistributedSystemMXBeanTest {
-
-  private final String[] distributedSystemMXBeanOperations = {
-      "listCacheServerObjectNames",
-      "viewRemoteClusterStatus",
-      "getTotalHeapSize",
-      "setQueryCollectionsDepth",
-      "getQueryCollectionsDepth",
-      "changeAlertLevel",
-      "backupAllMembers",
-      "revokeMissingDiskStores",
-      "shutDownAllMembers",
-      "queryData",
-      "queryDataForCompressedResult",
-      "setQueryResultSetLimit"
-  };
-
-  private final OperationCode[] distributedSystemResourceOperationCodes = {
-      OperationCode.LIST_DS,
-      OperationCode.LIST_DS,
-      OperationCode.LIST_DS,
-      OperationCode.QUERY,
-      OperationCode.LIST_DS,
-//      OperationCode.CHANGE_ALERT_LEVEL,
-//      OperationCode.BACKUP_MEMBERS,
-//      OperationCode.REVOKE_MISSING_DISKSTORE,
-      OperationCode.SHUTDOWN,
-      OperationCode.QUERY,
-      OperationCode.QUERY,
-      OperationCode.QUERY
-  };
-
-  @Test
-  public void operationsShouldMapToCodes() {
-//    ObjectName objectName = MBeanJMXAdapter.getDistributedSystemName();
-//    for (int i = 0; i < distributedSystemMXBeanOperations.length; i++) {
-//      JMXOperationContext context = new JMXOperationContext(objectName, distributedSystemMXBeanOperations[i]);
-//      assertEquals(distributedSystemResourceOperationCodes[i], context.getResourceOperationCode());
-//      assertEquals(OperationCode.RESOURCE, context.getOperationCode());
-//    }
-  }
-}