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/09 06:55:45 UTC

[2/2] incubator-geode git commit: GEODE-17: refactor ManagementIntercepter and MBeanServerWrapper

GEODE-17: refactor ManagementIntercepter and MBeanServerWrapper


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

Branch: refs/heads/feature/GEODE-17-3
Commit: c7680e2b72688c1b00ca5eff791c2d3631136205
Parents: dc7d73e
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue Mar 8 21:54:34 2016 -0800
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Tue Mar 8 21:54:34 2016 -0800

----------------------------------------------------------------------
 .../cache/operations/OperationContext.java      | 112 +++-
 .../internal/security/AccessControlContext.java |  48 --
 .../internal/security/AccessControlMBean.java   |   8 +-
 .../internal/security/CLIOperationContext.java  | 126 +---
 .../internal/security/JMXOperationContext.java  | 262 --------
 .../internal/security/MBeanServerWrapper.java   | 140 ++--
 .../security/ManagementInterceptor.java         | 136 +---
 .../management/internal/security/Resource.java  |   2 +
 .../internal/security/ResourceOperation.java    |   4 +-
 .../security/ResourceOperationContext.java      | 410 +-----------
 .../security/SetAttributesOperationContext.java |  55 --
 .../controllers/AbstractCommandsController.java |   2 -
 ...rDistributedSystemMXBeanIntegrationTest.java |  21 +-
 ...horizeOperationForMBeansIntegrationTest.java | 664 +++++++++----------
 ...erationForRegionCommandsIntegrationTest.java |  33 +-
 .../CacheServerMBeanSecurityJUnitTest.java      |   5 +-
 .../internal/security/JSONAuthorization.java    |  80 +--
 ...tionCodesForDataCommandsIntegrationTest.java |  49 +-
 ...tionCodesForDistributedSystemMXBeanTest.java |  69 +-
 .../management/internal/security/auth1.json     |   2 +-
 .../internal/security/cacheServer.json          |   1 -
 21 files changed, 692 insertions(+), 1537 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/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 fe8fbfb..fad9ff5 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 com.gemstone.gemfire.management.internal.security.Resource;
+
 /**
  * Encapsulates a cache operation and the data associated with it for both the
  * pre-operation and post-operation cases. Implementations for specific
@@ -39,15 +41,12 @@ public abstract class OperationContext {
    * @since 5.5
    */
   public enum OperationCode {
-    GET,
-    PUT,
     DESTROY,
     INVALIDATE,
     REGISTER_INTEREST,
     UNREGISTER_INTEREST,
     CONTAINS_KEY,
     KEY_SET,
-    QUERY,
     EXECUTE_CQ,
     STOP_CQ,
     CLOSE_CQ,
@@ -55,42 +54,84 @@ public abstract class OperationContext {
     REGION_CREATE,
     REGION_DESTROY,
     PUTALL,
-    EXECUTE_FUNCTION,
     GET_DURABLE_CQS,
     REMOVEALL,
     RESOURCE,
+    MANAGE,
     LIST,
     CREATE_QUERY,
     UPDATE_QUERY,
-    DELETE_QUERY;
-
-    /**
-     * Returns true if this is a list operation for regions, functions and queries.
-     */
-    public boolean isList() {
-      return (this == LIST);
-    }
-
-    /**
-     * Returns true if this is a create named query operation.
-     */
-    public boolean isCreateQuery() {
-      return (this == CREATE_QUERY);
-    }
-
-    /**
-     * Returns true if this is a update named query operation.
-     */
-    public boolean isUpdateQuery() {
-      return (this == UPDATE_QUERY);
-    }
-
-    /**
-     * Returns true if this is a delete named query operation.
-     */
-    public boolean isDestroyQuery() {
-      return (this == DELETE_QUERY);
-    }
+    DELETE_QUERY,
+    ALTER_REGION,
+    ALTER_RUNTIME,
+    BACKUP_DISKSTORE,
+    CHANGE_ALERT_LEVEL,
+    CLOSE_DURABLE_CLIENT,
+    CLOSE_DURABLE_CQ,
+    COMPACT_DISKSTORE,
+    CONFIGURE_PDX,
+    CREATE_AEQ,
+    CREATE_DISKSTORE,
+    CREATE_GW_RECEIVER,
+    CREATE_GW_SENDER,
+    CREATE_INDEX,
+    CREATE_REGION,
+    DEPLOY,
+    DESTROY_DISKSTORE,
+    DESTROY_FUNCTION,
+    DESTROY_INDEX,
+    DESTROY_REGION,
+    EXECUTE_FUNCTION,
+    EXPORT_CONFIG,
+    EXPORT_DATA,
+    EXPORT_LOGS,
+    EXPORT_OFFLINE_DISKSTORE,
+    EXPORT_STACKTRACE,
+    GC,
+    GET,
+    IMPORT_CONFIG,
+    IMPORT_DATA,
+    LIST_DS,
+    LOAD_BALANCE_GW_SENDER,
+    LOCATE_ENTRY,
+    NETSTAT,
+    PAUSE_GW_SENDER,
+    PUT,
+    QUERY,
+    REBALANCE,
+    REMOVE,
+    RENAME_PDX,
+    RESUME_GW_SENDER,
+    REVOKE_MISSING_DISKSTORE,
+    SHOW_DEADLOCKS,
+    SHOW_LOG,
+    SHOW_METRICS,
+    SHOW_MISSING_DISKSTORES,
+    SHOW_SUBSCRIPTION_QUEUE_SIZE,
+    SHUTDOWN,
+    STOP_GW_RECEIVER,
+    STOP_GW_SENDER,
+    UNDEPLOY,
+    BACKUP_MEMBERS,
+    ROLL_DISKSTORE,
+    FORCE_COMPACTION,
+    FORCE_ROLL,
+    FLUSH_DISKSTORE,
+    START_GW_RECEIVER,
+    START_GW_SENDER,
+    BECOME_LOCK_GRANTOR,
+    START_MANAGER,
+    STOP_MANAGER,
+    CREATE_MANAGER,
+    STOP_CONTINUOUS_QUERY,
+    SET_DISK_USAGE,
+    CREATE_HDFS_STORE,
+    ALTER_HDFS_STORE,
+    DESTROY_HDFS_STORE,
+    PULSE_DASHBOARD,
+    PULSE_DATABROWSER,
+    PULSE_WEBGFSH,
+    PULSE_ADMIN;
 
     /**
      * Returns true if this is a entry get operation.
@@ -236,6 +277,11 @@ public abstract class OperationContext {
    */
   public abstract OperationCode getOperationCode();
 
+  public Resource getResource(){
+    return Resource.DEFAULT;
+  }
+
+
   /**
    * True if the context is for post-operation.
    *

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlContext.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlContext.java
deleted file mode 100644
index b8ad6e5..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlContext.java
+++ /dev/null
@@ -1,48 +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;
-
-/**
- *
- * ResourceOperationContext passed to AccessControlMBean for Authorization calls made
- * from AccessControlMBean
- *
- * @author tushark
- * @since 9.0
- *
- */
-public class AccessControlContext extends ResourceOperationContext {
-  
-  private ResourceOperationCode code;
-  
-  public AccessControlContext(String code){
-    this.code = ResourceOperationCode.parse(code);
-  }
-
-  @Override
-  public ResourceOperationCode getResourceOperationCode() {
-    return code;
-  }
-
-  @Override
-  public OperationCode getOperationCode() {   
-    return OperationCode.RESOURCE;
-  }  
-
-  public static AccessControlContext ACCESS_GRANTED_CONTEXT = new AccessControlContext(ResourceConstants.LIST_DS);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/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
index a525416..0153c07 100644
--- 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
@@ -16,16 +16,16 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
+import com.gemstone.gemfire.cache.operations.OperationContext;
 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;
 
-import javax.management.remote.JMXPrincipal;
-import javax.security.auth.Subject;
-
 /**
  * AccessControlMBean Implementation. This retrieves JMXPrincipal from AccessController
  * and performs authorization for given role using gemfire AccessControl Plugin
@@ -52,7 +52,7 @@ public class AccessControlMBean implements AccessControlMXBean {
     Principal principal = principals.iterator().next();
     AccessControl gemAccControl = interceptor.getAccessControl(principal, false);
     boolean authorized = gemAccControl.authorizeOperation(null,
-        new com.gemstone.gemfire.management.internal.security.AccessControlContext(role));
+        new ResourceOperationContext(Resource.DEFAULT, OperationContext.OperationCode.valueOf(role)));
     return authorized;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/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
index 4c83fa9..84bb338 100644
--- 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
@@ -16,21 +16,17 @@
  */
 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 org.springframework.shell.event.ParseResult;
-
-import com.gemstone.gemfire.GemFireConfigException;
 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.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.parser.CommandTarget;
-import static com.gemstone.gemfire.management.internal.security.ResourceConstants.*;
+import org.springframework.shell.event.ParseResult;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * It represents command being executed and all passed options and option-values.
@@ -41,70 +37,19 @@ import static com.gemstone.gemfire.management.internal.security.ResourceConstant
  * @since 9.0
  */
 public class CLIOperationContext extends ResourceOperationContext {
-	
-	private OperationCode code = OperationCode.RESOURCE;
-	private ResourceOperationCode resourceCode = null;
+
 	private Map<String,String> commandOptions = null;
 	
-	private static Map<String,ResourceOperationCode> commandToCodeMapping = new HashMap<String,ResourceOperationCode>();
+	private static Map<String,ResourceOperation> commandToCodeMapping = new HashMap<String,ResourceOperation>();
 	private static CommandManager commandManager = null;
 	private static GfshParser parser = null;	
 	
 	public CLIOperationContext(String commandString) throws CommandProcessingException, IllegalStateException{
-		GfshParseResult parseResult = (GfshParseResult) parseCommand(commandString);		
-		this.commandOptions = parseResult.getParamValueStrings();		
-    this.resourceCode = findResourceCode(parseResult.getCommandName());
-    this.code = findOperationCode(parseResult.getCommandName());
+		GfshParseResult parseResult = (GfshParseResult) parseCommand(commandString);
+		ResourceOperation op = findResourceCode(parseResult.getCommandName());
+		setResourceOperation(op);
+		this.commandOptions = parseResult.getParamValueStrings();
   }
-
-  /**
-   * This method returns OperationCode for command. Some commands perform data
-   * operations, for such commands OperationCode returned is not RESOURCE but
-   * corresponding data operation as defined in OperationCode
-   *
-   * @param commandName
-   * @return OperationCode
-   */
-  private OperationCode findOperationCode(String commandName) {
-
-    if(CliStrings.GET.equals(commandName) || CliStrings.LOCATE_ENTRY.equals(commandName))
-      return OperationCode.GET;
-
-    if(CliStrings.PUT.equals(commandName))
-      return OperationCode.PUT;
-
-    if(CliStrings.QUERY.equals(commandName))
-      return OperationCode.QUERY;
-
-    if (CliStrings.REMOVE.equals(commandName)) {
-      if (commandOptions.containsKey(CliStrings.REMOVE__ALL)
-          && "true".equals(commandOptions.get(CliStrings.REMOVE__ALL))) {
-        return OperationCode.REMOVEALL;
-      } else
-        return OperationCode.DESTROY;
-    }
-
-    if(CliStrings.CLOSE_DURABLE_CQS.equals(commandName)) {
-      return OperationCode.CLOSE_CQ;
-    }
-
-    if(CliStrings.CREATE_REGION.equals(commandName)) {
-      return OperationCode.REGION_CREATE;
-    }
-
-    if(CliStrings.DESTROY_REGION.equals(commandName)) {
-      return OperationCode.REGION_DESTROY;
-    }
-
-    if(CliStrings.EXECUTE_FUNCTION.equals(commandName)) {
-      return OperationCode.EXECUTE_FUNCTION;
-    }
-
-    //"stop cq"
-    //"removeall",
-    //"get durable cqs",
-    return OperationCode.RESOURCE;
-	}
 	
 	private static ParseResult parseCommand(String commentLessLine) throws CommandProcessingException, IllegalStateException {
     if (commentLessLine != null) {
@@ -118,43 +63,16 @@ public class CLIOperationContext extends ResourceOperationContext {
 	    commandManager = cmdManager;
 	    parser = new GfshParser(cmdManager);
 	  }
-	  
-		boolean found=false;
 		Annotation ans[] = method.getDeclaredAnnotations();
 		for(Annotation an : ans){
 			if(an instanceof ResourceOperation) {
-				cache(commandTarget.getCommandName(),(ResourceOperation)an);
-				found=true;
+				commandToCodeMapping.put(commandTarget.getCommandName(), (ResourceOperation)an);
 			}
 		}
-		if(!found)
-			cache(commandTarget.getCommandName(),null);
 	}
 
 	private static void cache(String commandName, ResourceOperation op) {
-    ResourceOperationCode resourceOpCode = null;
-		
-		if (op != null) {
-			String opString = op.operation();
-			if (opString != null)
-        resourceOpCode = ResourceOperationCode.parse(opString);
-		}
-		
-    if(resourceOpCode==null){
-      if (commandName.startsWith(GETTER_DESCRIBE) || commandName.startsWith(GETTER_LIST)
-          || commandName.startsWith(GETTER_STATUS)) {
-        resourceOpCode = ResourceOperationCode.LIST_DS;
-			} 
-		}
-
-		
-    if(resourceOpCode!=null) {
-      commandToCodeMapping.put(commandName, resourceOpCode);
-		} else {			
-      throw new GemFireConfigException(
-          "Error while configuring authorization for gfsh commands. No opCode defined for command " + commandName);
-
-		}
+		commandToCodeMapping.put(commandName, op);
 		
 	}
 
@@ -162,25 +80,13 @@ public class CLIOperationContext extends ResourceOperationContext {
 		return commandOptions;
 	}
 
-	private static ResourceOperationCode findResourceCode(String commandName) {		
+	private static ResourceOperation findResourceCode(String commandName) {
 		return commandToCodeMapping.get(commandName);
 	}
-
-
-	@Override
-	public OperationCode getOperationCode() {		
-		return code;
-	}
-
-	@Override
-	public ResourceOperationCode getResourceOperationCode() {
-		return resourceCode;
-	}
-	
 	
 	public String toString(){
 	  String str;
-	  str = "CLIOperationContext(resourceCode=" + resourceCode + ") options=" + commandOptions+")";
+	  str = "CLIOperationContext(resourceCode=" + getOperationCode() + ") options=" + commandOptions+")";
 	  return str;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/JMXOperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/JMXOperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/JMXOperationContext.java
deleted file mode 100644
index 85dca8c..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/JMXOperationContext.java
+++ /dev/null
@@ -1,262 +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.io.IOException;
-import java.lang.annotation.Annotation;
-import java.lang.reflect.Method;
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.management.ObjectName;
-
-import com.gemstone.gemfire.GemFireConfigException;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.management.AsyncEventQueueMXBean;
-import com.gemstone.gemfire.management.CacheServerMXBean;
-import com.gemstone.gemfire.management.DiskStoreMXBean;
-import com.gemstone.gemfire.management.DistributedLockServiceMXBean;
-import com.gemstone.gemfire.management.DistributedRegionMXBean;
-import com.gemstone.gemfire.management.DistributedSystemMXBean;
-import com.gemstone.gemfire.management.GatewayReceiverMXBean;
-import com.gemstone.gemfire.management.GatewaySenderMXBean;
-import com.gemstone.gemfire.management.LocatorMXBean;
-import com.gemstone.gemfire.management.LockServiceMXBean;
-import com.gemstone.gemfire.management.ManagerMXBean;
-import com.gemstone.gemfire.management.MemberMXBean;
-import com.gemstone.gemfire.management.RegionMXBean;
-import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
-import com.gemstone.gemfire.management.internal.cli.util.ClasspathScanLoadHelper;
-import static com.gemstone.gemfire.management.internal.security.ResourceConstants.*;
-
-/**
- * It describes current JMX MBean Method call and its parameters.
- * OpCode returned by JMXOperationContext is retrieved from ResourceOperation annotation
- * on the target methodName
- *
- * @author tushark
- * @since 9.0
- *
- */
-public class JMXOperationContext  extends ResourceOperationContext {
-	
-	private OperationCode code = OperationCode.RESOURCE;
-	private ResourceOperationCode resourceCode = null;
-  private ObjectName name;
-  private String methodName;
-
-  private static Map<Class<?>,Map<String,ResourceOperationCode>> cachedResourceOpsMapping = new HashMap<Class<?>,Map<String,ResourceOperationCode>>();
-  private static Map<String,ResourceOperationCode> distributedSystemMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> diskStoreMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> cacheServerMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> gatewayReceiverMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> gatewaySenderMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> lockServiceMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> managerMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> memberMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> regionMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> locatorMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> distributedLockServiceMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> distributedRegionMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> asyncEventQueueMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-  private static Map<String,ResourceOperationCode> accessControlMXBeanResourceOps = new HashMap<String,ResourceOperationCode>();
-
-	
-	static {
-		readJMXAnnotations();		
-		
-	}	
-
-	private static void readJMXAnnotations() {
-
-    cachedResourceOpsMapping.put(DistributedSystemMXBean.class, distributedSystemMXBeanResourceOps);
-    cachedResourceOpsMapping.put(DiskStoreMXBean.class, diskStoreMXBeanResourceOps);
-    cachedResourceOpsMapping.put(CacheServerMXBean.class, cacheServerMXBeanResourceOps);
-    cachedResourceOpsMapping.put(GatewayReceiverMXBean.class, gatewayReceiverMXBeanResourceOps);
-    cachedResourceOpsMapping.put(GatewaySenderMXBean.class, gatewaySenderMXBeanResourceOps);
-    cachedResourceOpsMapping.put(LockServiceMXBean.class, lockServiceMXBeanResourceOps);
-    cachedResourceOpsMapping.put(ManagerMXBean.class, managerMXBeanResourceOps);
-    cachedResourceOpsMapping.put(MemberMXBean.class, memberMXBeanResourceOps);
-    cachedResourceOpsMapping.put(RegionMXBean.class, regionMXBeanResourceOps);
-    cachedResourceOpsMapping.put(LocatorMXBean.class, locatorMXBeanResourceOps);
-    cachedResourceOpsMapping.put(DistributedLockServiceMXBean.class, distributedLockServiceMXBeanResourceOps);
-    cachedResourceOpsMapping.put(DistributedRegionMXBean.class, distributedRegionMXBeanResourceOps);
-    cachedResourceOpsMapping.put(AsyncEventQueueMXBean.class, asyncEventQueueMXBeanResourceOps);
-    cachedResourceOpsMapping.put(AccessControlMXBean.class, accessControlMXBeanResourceOps);
-
-		try {
-      Class<?>[] klassList = ClasspathScanLoadHelper.getClasses(MANAGEMENT_PACKAGE);
-      for(Class<?> klass : klassList) {
-				if(klass.getName().endsWith("MXBean")) {
-					Method[] methods = klass.getMethods();
-					for(Method method : methods) {
-						String name = method.getName();
-						boolean found=false;
-						Annotation ans[] = method.getDeclaredAnnotations();
-						for(Annotation an : ans){
-							if(an instanceof ResourceOperation) {
-								cache(klass,name,(ResourceOperation)an);
-								found=true;
-							}
-						}
-						if(!found)
-							cache(klass,name,null);
-					}
-				}
-			}
-		} catch (ClassNotFoundException e) {			
-			throw new GemFireConfigException(
-					"Error while configuring authorization for jmx - ", e);
-		} catch (IOException e) {
-			throw new GemFireConfigException(
-					"Error while configuring authorization for jmx - ", e);
-		}
-		
-	}
-	
-  private static void cache(Class<?> klass, String name, ResourceOperation op) {
-		ResourceOperationCode code = null;
-		
-		if (op != null) {
-			String opString = op.operation();
-			if (opString != null)
-				code = ResourceOperationCode.parse(opString);
-		}
-		
-    if(code==null && isGetterSetter(name)){
-				code = ResourceOperationCode.LIST_DS;
-		}
-
-		
-    if (code == null && cachedResourceOpsMapping.keySet().contains(klass) && !isGetterSetter(name)) {
-      throw new GemFireConfigException("Error while configuring authorization for jmx. No opCode defined for "
-					+ klass.getCanonicalName() + " method " + name);
-				}
-
-    final Map<String,ResourceOperationCode> resourceOpsMap = cachedResourceOpsMapping.get(klass);
-    if(resourceOpsMap==null) {
-      if (cachedResourceOpsMapping.keySet().contains(klass))
-        throw new GemFireConfigException("Unknown MBean " + klass.getCanonicalName());
-      else {
-        LogService.getLogger().warn("Unsecured mbean " + klass);
-			}
-		}			
-    else {
-      resourceOpsMap.put(name, code);
-    }
-	}
-
-  public static boolean isGetterSetter(String name) {
-    if(name.startsWith(GETTER_IS) || name.startsWith(GETTER_GET) ||  name.startsWith(GETTER_FETCH)
-      ||  name.startsWith(GETTER_LIST) ||  name.startsWith(GETTER_VIEW) ||  name.startsWith(GETTER_SHOW) ||  name.startsWith(GETTER_HAS))
-		return true;
-		else return false;
-	}
-
-	public JMXOperationContext(ObjectName name , String methodName){
-		code = OperationCode.RESOURCE;
-    Class<?> klass = getMbeanClass(name);
-    Map<String,ResourceOperationCode> resourceOpsMap = cachedResourceOpsMapping.get(klass);
-    resourceCode = resourceOpsMap.get(methodName);
-    this.methodName = methodName;
-    this.name = name;
-
-    //If getAttr is not found try for isAttr ie. boolean getter
-    if(resourceCode==null) {
-      if(this.methodName.startsWith(GET_PREFIX)) {
-        String methodNameBooleanGetter = GET_IS_PREFIX + this.methodName.substring(GET_PREFIX.length());
-        if(resourceOpsMap.containsKey(methodNameBooleanGetter)){
-          resourceCode = resourceOpsMap.get(methodNameBooleanGetter);
-          this.methodName = methodNameBooleanGetter;
-        }
-		}
-	}
-	
-    //If resourceCode is still null most likely its wrong method name so just allow it pass
-    if(resourceCode==null) {
-      resourceCode = ResourceOperationCode.LIST_DS;
-    }
-  }
-
-
-
-
-  private Class<?> getMbeanClass(ObjectName name) {
-    if (name.equals(MBeanJMXAdapter.getDistributedSystemName()))
-      return DistributedSystemMXBean.class;
-    else {
-      String service = name.getKeyProperty(MBEAN_KEY_SERVICE);
-      String mbeanType = name.getKeyProperty(MBEAN_KEY_TYPE);
-
-      if (MBEAN_TYPE_DISTRIBUTED.equals(mbeanType)) {
-        if (MBEAN_SERVICE_SYSTEM.equals(service)) {
-          return DistributedSystemMXBean.class;
-        } else if (MBEAN_SERVICE_REGION.equals(service)) {
-          return DistributedRegionMXBean.class;
-        } else if (MBEAN_SERVICE_LOCKSERVICE.equals(service)) {
-          return DistributedLockServiceMXBean.class;
-        } else {
-          throw new RuntimeException("Unknown mbean type " + name);
-        }
-      } else if (MBEAN_TYPE_MEMBER.equals(mbeanType)) {
-        if (service == null) {
-          return MemberMXBean.class;
-        } else {
-          if (MBEAN_SERVICE_MANAGER.equals(service)) {
-            return ManagerMXBean.class;
-          } else if (MBEAN_SERVICE_CACHESERVER.equals(service)) {
-            return CacheServerMXBean.class;
-          } else if (MBEAN_SERVICE_REGION.equals(service)) {
-            return RegionMXBean.class;
-          } else if (MBEAN_SERVICE_LOCKSERVICE.equals(service)) {
-            return LockServiceMXBean.class;
-          } else if (MBEAN_SERVICE_DISKSTORE.equals(service)) {
-            return DiskStoreMXBean.class;
-          } else if (MBEAN_SERVICE_GATEWAY_RECEIVER.equals(service)) {
-            return GatewayReceiverMXBean.class;
-          } else if (MBEAN_SERVICE_GATEWAY_SENDER.equals(service)) {
-            return GatewaySenderMXBean.class;
-          } else if (MBEAN_SERVICE_ASYNCEVENTQUEUE.equals(service)) {
-            return AsyncEventQueueMXBean.class;
-          } else if (MBEAN_SERVICE_LOCATOR.equals(service)) {
-            return LocatorMXBean.class;
-          } else {
-            throw new RuntimeException("Unknown mbean type " + name);
-          }
-        }
-      } else {
-        throw new RuntimeException("Unknown mbean type " + name);
-      }
-    }
-  }
-
-	@Override
-	public OperationCode getOperationCode() {		
-		return code;
-	}
-
-	@Override
-	public ResourceOperationCode getResourceOperationCode() {
-		return resourceCode;
-	}
-
-  public String toString(){
-    return "JMXOpCtx(on="+name+",method="+methodName+")";
-	}
-
-	}
-	

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/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 6c25102..13a9ab4 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,20 +16,22 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
-import java.io.ObjectInputStream;
-import java.util.HashSet;
-import java.util.Set;
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.security.GemFireSecurityException;
 
 import javax.management.Attribute;
 import javax.management.AttributeList;
 import javax.management.AttributeNotFoundException;
+import javax.management.Descriptor;
 import javax.management.InstanceAlreadyExistsException;
 import javax.management.InstanceNotFoundException;
 import javax.management.IntrospectionException;
 import javax.management.InvalidAttributeValueException;
 import javax.management.ListenerNotFoundException;
+import javax.management.MBeanAttributeInfo;
 import javax.management.MBeanException;
 import javax.management.MBeanInfo;
+import javax.management.MBeanOperationInfo;
 import javax.management.MBeanRegistrationException;
 import javax.management.MBeanServer;
 import javax.management.NotCompliantMBeanException;
@@ -42,6 +44,10 @@ 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.*;
 
 /**
@@ -62,67 +68,50 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
     this.interceptor = interceptor;
   }
   
-  private ResourceOperationContext doAuthorization(ObjectName name, String methodName, Object[] methodParams){
-    return interceptor.authorize(name,methodName, methodParams);
+  private void doAuthorization(ResourceOperationContext context){
+    interceptor.authorize(context);
   }
 
-  private void doAuthorizationPost(ObjectName name, String methodName, ResourceOperationContext context, Object result){
-    interceptor.postAuthorize(name,methodName,context,result);
+  private void doAuthorizationPost(ResourceOperationContext context){
+    interceptor.postAuthorize(context);
   }
 
   @Override
   public ObjectInstance createMBean(String className, ObjectName name) throws ReflectionException,
       InstanceAlreadyExistsException, MBeanRegistrationException, MBeanException, NotCompliantMBeanException {
-    ResourceOperationContext ctx = doAuthorization(name, CREATE_MBEAN, new Object[]{name});
-    ObjectInstance result = mbs.createMBean(className, name);
-    doAuthorizationPost(name, CREATE_MBEAN, ctx, result);
-    return result;
+    throw new SecurityException(ACCESS_DENIED_MESSAGE);
   }
 
   @Override
   public ObjectInstance createMBean(String className, ObjectName name, ObjectName loaderName)
       throws ReflectionException, InstanceAlreadyExistsException, MBeanRegistrationException, MBeanException,
       NotCompliantMBeanException, InstanceNotFoundException {
-    ResourceOperationContext ctx = doAuthorization(name, CREATE_MBEAN, new Object[]{name});
-    ObjectInstance result = mbs.createMBean(className, name, loaderName);
-    doAuthorizationPost(name, CREATE_MBEAN, ctx, result);
-    return result;
+    throw new SecurityException(ACCESS_DENIED_MESSAGE);
   }
 
   @Override
   public ObjectInstance createMBean(String className, ObjectName name, Object[] params, String[] signature)
       throws ReflectionException, InstanceAlreadyExistsException, MBeanRegistrationException, MBeanException,
       NotCompliantMBeanException {
-    ResourceOperationContext ctx = doAuthorization(name, CREATE_MBEAN, new Object[]{name, params});
-    ObjectInstance result = mbs.createMBean(className,name,params,signature);
-    doAuthorizationPost(name, CREATE_MBEAN, ctx, result);
-    return result;
+    throw new SecurityException(ACCESS_DENIED_MESSAGE);
   }
 
   @Override
   public ObjectInstance createMBean(String className, ObjectName name, ObjectName loaderName, Object[] params,
       String[] signature) throws ReflectionException, InstanceAlreadyExistsException, MBeanRegistrationException,
       MBeanException, NotCompliantMBeanException, InstanceNotFoundException {
-    ResourceOperationContext ctx = doAuthorization(name, CREATE_MBEAN, new Object[]{name});
-    ObjectInstance result = mbs.createMBean(className, name, loaderName, params, signature);
-    doAuthorizationPost(name, CREATE_MBEAN, ctx, result);
-    return result;
+    throw new SecurityException(ACCESS_DENIED_MESSAGE);
   }
 
   @Override
   public ObjectInstance registerMBean(Object object, ObjectName name) throws InstanceAlreadyExistsException,
       MBeanRegistrationException, NotCompliantMBeanException {
-    ResourceOperationContext ctx = doAuthorization(name, REGISTER_MBEAN, new Object[]{name});
-    ObjectInstance result = mbs.registerMBean(object, name);
-    doAuthorizationPost(name, REGISTER_MBEAN, ctx, result);
-    return result;
+    throw new SecurityException(ACCESS_DENIED_MESSAGE);
   }
 
   @Override
   public void unregisterMBean(ObjectName name) throws InstanceNotFoundException, MBeanRegistrationException {
-    ResourceOperationContext ctx = doAuthorization(name, UNREGISTER_MBEAN, new Object[]{});
-    mbs.unregisterMBean(name);
-    doAuthorizationPost(name, UNREGISTER_MBEAN, ctx, null);
+    throw new SecurityException(ACCESS_DENIED_MESSAGE);
   }
 
   @Override
@@ -132,6 +121,7 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
 
   @Override
   public Set<ObjectInstance> queryMBeans(ObjectName name, QueryExp query) {
+    ResourceOperationContext ctx = new ResourceOperationContext(Resource.MBEAN, OperationCode.QUERY);
     return filterAccessControlMBeanInstance(mbs.queryMBeans(name, query));
   }
 
@@ -173,44 +163,106 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
   @Override
   public Object getAttribute(ObjectName name, String attribute) throws MBeanException, AttributeNotFoundException,
       InstanceNotFoundException, ReflectionException {
-    ResourceOperationContext ctx = doAuthorization(name, GET_ATTRIBUTE,  new Object[]{attribute});
+    ResourceOperationContext ctx = getAttributeContext(name, attribute);
+    doAuthorization(ctx);
     Object result = mbs.getAttribute(name, attribute);
-    doAuthorizationPost(name, GET_ATTRIBUTE, ctx, result);
+    ctx.setPostOperationResult(result);
+    doAuthorizationPost(ctx);
     return result;
   }
 
   @Override
   public AttributeList getAttributes(ObjectName name, String[] attributes) throws InstanceNotFoundException,
       ReflectionException {
-    ResourceOperationContext ctx = doAuthorization(name, GET_ATTRIBUTES, new Object[]{attributes});
-    AttributeList result = mbs.getAttributes(name, attributes);
-    doAuthorizationPost(name, GET_ATTRIBUTES, ctx, result);
-    return result;
+    AttributeList results = new AttributeList();
+    for(String attribute:attributes){
+      try {
+        Object value = getAttribute(name, attribute);
+        Attribute att = new Attribute(attribute, value);
+        results.add(att);
+      } catch (Exception e) {
+        throw new GemFireSecurityException("error getting value of "+attribute+" from "+name, e);
+      }
+    }
+    return results;
   }
 
   @Override
   public void setAttribute(ObjectName name, Attribute attribute) throws InstanceNotFoundException,
       AttributeNotFoundException, InvalidAttributeValueException, MBeanException, ReflectionException {
-    ResourceOperationContext ctx = doAuthorization(name, SET_ATTRIBUTE, new Object[]{attribute});
+    ResourceOperationContext ctx = getAttributeContext(name, attribute.getName());
+    doAuthorization(ctx);
     mbs.setAttribute(name, attribute);
-    doAuthorizationPost(name, SET_ATTRIBUTE, ctx, null);
+  }
+
+  private ResourceOperationContext getAttributeContext(ObjectName name, String attribute)
+      throws InstanceNotFoundException, ReflectionException {
+    MBeanInfo beanInfo = null;
+    try {
+      beanInfo = mbs.getMBeanInfo(name);
+    } catch (IntrospectionException e) {
+      throw new GemFireSecurityException("error getting beanInfo of "+name);
+    }
+    MBeanAttributeInfo[] attributeInfos = beanInfo.getAttributes();
+    for(MBeanAttributeInfo attributeInfo:attributeInfos){
+      if(attributeInfo.getName().equals(attribute)){
+        // found the operationInfo of this method on the bean
+        Descriptor descriptor = attributeInfo.getDescriptor();
+        Resource resource = (Resource)descriptor.getFieldValue("resource");
+        OperationCode operationCode = (OperationCode)descriptor.getFieldValue("operation");
+        if(resource!=null && operationCode!=null){
+          return new ResourceOperationContext(resource, operationCode);
+        }
+      }
+    }
+    return null;
+  }
+
+  private ResourceOperationContext getOperationContext(ObjectName name, String operationName)
+      throws InstanceNotFoundException, ReflectionException {
+    MBeanInfo beanInfo = null;
+    try {
+      beanInfo = mbs.getMBeanInfo(name);
+    } catch (IntrospectionException e) {
+      throw new GemFireSecurityException("error getting beanInfo of "+name);
+    }
+    MBeanOperationInfo[] opInfos = beanInfo.getOperations();
+    for(MBeanOperationInfo opInfo:opInfos){
+      if(opInfo.getName().equals(operationName)){
+        // found the operationInfo of this method on the bean
+        Descriptor descriptor = opInfo.getDescriptor();
+        String resource = (String)descriptor.getFieldValue("resource");
+        String operationCode = (String)descriptor.getFieldValue("operation");
+        if(resource!=null && operationCode!=null){
+          return new ResourceOperationContext(resource, operationCode);
+        }
+      }
+    }
+    return null;
   }
 
   @Override
   public AttributeList setAttributes(ObjectName name, AttributeList attributes) throws InstanceNotFoundException,
       ReflectionException {
-    ResourceOperationContext ctx = doAuthorization(name, SET_ATTRIBUTES, new Object[]{attributes});
-    AttributeList result = mbs.setAttributes(name, attributes);
-    doAuthorizationPost(name, SET_ATTRIBUTES, ctx, result);
-    return result;
+    // call setAttribute instead to use the authorization logic
+    for(Attribute attribute:attributes.asList()){
+      try {
+        setAttribute(name, attribute);
+      } catch (Exception e) {
+        throw new GemFireSecurityException("error setting attribute "+attribute+" of "+name);
+      }
+    }
+    return attributes;
   }
 
   @Override
   public Object invoke(ObjectName name, String operationName, Object[] params, String[] signature)
       throws InstanceNotFoundException, MBeanException, ReflectionException {
-    ResourceOperationContext ctx = doAuthorization(name, operationName, new Object[]{params, signature});
+    ResourceOperationContext ctx = getOperationContext(name, operationName);
+    doAuthorization(ctx);
     Object result = mbs.invoke(name, operationName, params, signature);
-    doAuthorizationPost(name, operationName, ctx, result);
+    ctx.setPostOperationResult(result);
+    doAuthorizationPost(ctx);
     return result;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/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 7b285f6..0edc812 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
@@ -24,14 +24,11 @@ import com.gemstone.gemfire.internal.ClassLoadUtil;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
-import com.gemstone.gemfire.management.internal.ManagementConstants;
 import com.gemstone.gemfire.security.AccessControl;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.Authenticator;
 import org.apache.logging.log4j.Logger;
 
-import javax.management.Attribute;
-import javax.management.AttributeList;
 import javax.management.InstanceAlreadyExistsException;
 import javax.management.MBeanRegistrationException;
 import javax.management.MBeanServer;
@@ -48,8 +45,6 @@ import java.security.AccessControlContext;
 import java.security.AccessController;
 import java.security.Principal;
 import java.util.Collections;
-import java.util.List;
-import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
@@ -66,7 +61,7 @@ import static com.gemstone.gemfire.management.internal.security.ResourceConstant
  * @since 9.0
  *
  */
-public class ManagementInterceptor implements JMXAuthenticator {
+public class ManagementInterceptor implements JMXAuthenticator{
 
   // FIXME: Merged from GEODE-17. Are they necessary?
 	public static final String USER_NAME = "security-username";
@@ -85,7 +80,7 @@ public class ManagementInterceptor implements JMXAuthenticator {
 
   public ManagementInterceptor(Cache gemFireCacheImpl, Logger logger) {
     this.cache = gemFireCacheImpl;
-		this.logger = logger;		
+		this.logger = logger;
 		this.mBeanServerForwarder = new MBeanServerWrapper(this);
     DistributedSystem system = cache.getDistributedSystem();
     Properties sysProps = system.getProperties();
@@ -107,6 +102,7 @@ public class ManagementInterceptor implements JMXAuthenticator {
       AccessControlMBean acc = new AccessControlMBean(this);
       accessControlMBeanON = new ObjectName(ResourceConstants.OBJECT_NAME_ACCESSCONTROL);
       MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
+
       Set<ObjectName> names = platformMBeanServer.queryNames(accessControlMBeanON, null);
       if(names.isEmpty()) {
         try {
@@ -159,101 +155,43 @@ public class ManagementInterceptor implements JMXAuthenticator {
 	}
 
   }
-	  
+
   /**
    * Builds ResourceOperationContext for the given JMX MBean Request for delegates Authorization to
    * gemfire AccessControl plugin with context as parameter
    *
-   *
-   * @param name
-   * @param methodName
-   * @param params
-   *
    * @throws SecurityException
    *           if access is not granted
    */
-  public ResourceOperationContext authorize(ObjectName name, final String methodName, Object[] params) {
+  public void authorize(ResourceOperationContext context) {
+    if(context==null){
+      return;
+    }
 
     if (StringUtils.isBlank(authzFactoryName)){
-      return com.gemstone.gemfire.management.internal.security.AccessControlContext.ACCESS_GRANTED_CONTEXT;
-      }
-
-    if (name.equals(accessControlMBeanON)) {
-      return com.gemstone.gemfire.management.internal.security.AccessControlContext.ACCESS_GRANTED_CONTEXT;
+      return;
     }
-	  
-    if (!ManagementConstants.OBJECTNAME__DEFAULTDOMAIN.equals(name.getDomain()))
-      return com.gemstone.gemfire.management.internal.security.AccessControlContext.ACCESS_GRANTED_CONTEXT;
 
-		AccessControlContext acc = AccessController.getContext();		
+		AccessControlContext acc = AccessController.getContext();
 		Subject subject = Subject.getSubject(acc);
 
     // Allow operations performed locally on behalf of the connector server itself
 		if (subject == null) {
-      return com.gemstone.gemfire.management.internal.security.AccessControlContext.ACCESS_GRANTED_CONTEXT;
-		}
-
-    if (methodName.equals(ResourceConstants.CREATE_MBEAN) || methodName.equals(ResourceConstants.UNREGISTER_MBEAN)) {
-      throw new SecurityException(ACCESS_DENIED_MESSAGE);
+      return;
 		}
 
     Set<JMXPrincipal> principals = subject.getPrincipals(JMXPrincipal.class);
-		
+
     if (principals == null || principals.isEmpty()) {
       throw new SecurityException(ACCESS_DENIED_MESSAGE);
-		}		
-	
+		}
+
 		Principal principal = principals.iterator().next();
 
-		
-    if (logger.isDebugEnabled()) {
-      logger.debug("Name=" + name + " methodName=" + methodName + " principal=" + principal.getName());
-    }
-		
     AccessControl accessControl = getAccessControl(principal, false);
-    String method = methodName;
-    if (methodName.equals(GET_ATTRIBUTE)) {
-      method = GET_PREFIX + (String) params[0];
-    } else if(methodName.equals(GET_ATTRIBUTES)) {
-      //Pass to first attribute getter
-      String[] attrs = (String[]) params[0];
-      method = GET_PREFIX + attrs[0];
-    } else if(methodName.equals(SET_ATTRIBUTE)) {
-      Attribute attribute = (Attribute) params[0];
-      method = SET_PREFIX + attribute.getName();
-    }
-
-    if (methodName.equals(SET_ATTRIBUTES)) {
-      AttributeList attrList = (AttributeList) params[0];
-      List<Attribute> list = attrList.asList();
-      ResourceOperationContext setterContext = null;
-      SetAttributesOperationContext resourceContext = new SetAttributesOperationContext();
-      for (int i = 0; i < list.size(); i++) {
-        Attribute attribute = list.get(i);
-        String setter = SET_PREFIX + attribute.getName();
-        setterContext = buildContext(name, setter, null);
-        boolean authorized = accessControl.authorizeOperation(null, setterContext);
-        if (logger.isDebugEnabled()) {
-          logger.debug(
-              "Name=" + name + " methodName=" + methodName + " result=" + authorized + " principal=" + principal.getName());
-        }
-        if (!authorized) {
-          throw new SecurityException(ACCESS_DENIED_MESSAGE);
-        } else {
-          resourceContext.addAttribute(attribute.getName(), setterContext);
-        }
-      }
-      return resourceContext;
-    } else {
-      ResourceOperationContext resourceContext = buildContext(name, method, params);
-      boolean authorized = accessControl.authorizeOperation(null, resourceContext);
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "Name=" + name + " methodName=" + methodName + " result=" + authorized + " principal=" + principal.getName());
-      }
 
-      if (!authorized) throw new SecurityException(ACCESS_DENIED_MESSAGE);
-      return resourceContext;
+    if (!accessControl.authorizeOperation(null, context)) {
+      throw new SecurityException(ACCESS_DENIED_MESSAGE);
     }
   }
 
@@ -313,33 +251,20 @@ public class ManagementInterceptor implements JMXAuthenticator {
     return auth;
 	}
 
-  private ResourceOperationContext buildContext(ObjectName name, String methodName, Object[] params) {
-    String service = name.getKeyProperty("service");
-    if (service == null && PROCESS_COMMAND.equals(methodName)) {
-      Object[] array = (Object[]) params[0];
-      String command = (String) array[0];
-      CLIOperationContext context = new CLIOperationContext(command);
-      return context;
-    } else {
-      ResourceOperationContext context = new JMXOperationContext(name, methodName);
-      return context;
-    }
-  }
-
   public ObjectName getAccessControlMBeanON() {
     return accessControlMBeanON;
     }
 
-  public void postAuthorize(ObjectName name, final String methodName, ResourceOperationContext context, Object result) {
+  public void postAuthorize(ResourceOperationContext context) {
+
+    if(context==null){
+      return;
+    }
 
     if (StringUtils.isBlank(postAuthzFactoryName)){
       return ;
     }
 
-    context.setPostOperationResult(result);
-
-    if (context.equals(com.gemstone.gemfire.management.internal.security.AccessControlContext.ACCESS_GRANTED_CONTEXT))
-      return;
 
     AccessControlContext acc = AccessController.getContext();
     Subject subject = Subject.getSubject(acc);
@@ -349,23 +274,8 @@ public class ManagementInterceptor implements JMXAuthenticator {
     }
     Principal principal = principals.iterator().next();
     AccessControl accessControl = getAccessControl(principal, true);
-    if (context instanceof SetAttributesOperationContext) {
-      SetAttributesOperationContext setterContext = (SetAttributesOperationContext) context;
-      for (Entry<String, ResourceOperationContext> e : setterContext.getAttributesContextMap().entrySet()) {
-        //TODO : Retrieve proper values from AttributeList and set to its jmxContext
-        e.getValue().setPostOperationResult(result);
-        boolean authorized = accessControl.authorizeOperation(null, e.getValue());
-        if (!authorized)
-          throw new SecurityException(ACCESS_DENIED_MESSAGE);
-      }
-    } else {
-      boolean authorized = accessControl.authorizeOperation(null, context);
-      if (logger.isDebugEnabled()) {
-        logger.debug("postAuthorize: Name=" + name + " methodName=" + methodName + " result=" + authorized
-            + " principal=" + principal.getName());
-      }
-      if (!authorized)
-        throw new SecurityException(ACCESS_DENIED_MESSAGE);
+    if (!accessControl.authorizeOperation(null, context)) {
+      throw new SecurityException(ACCESS_DENIED_MESSAGE);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/Resource.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/Resource.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/Resource.java
index d5f30fd..02ae64d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/Resource.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/Resource.java
@@ -17,6 +17,8 @@
 package com.gemstone.gemfire.management.internal.security;
 
 public enum Resource {
+  DEFAULT,
+  MBEAN,
   DISTRIBUTED_SYSTEM,
   MEMBER,
   REGION,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/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 ceebd6f..a0cde33 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
@@ -16,6 +16,7 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
+import javax.management.DescriptorKey;
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Inherited;
 import java.lang.annotation.Retention;
@@ -26,9 +27,10 @@ import java.lang.annotation.Target;
 @Retention(RetentionPolicy.RUNTIME)
 @Inherited
 public @interface ResourceOperation {
-  
+  @DescriptorKey("resource")
   Resource resource();
   String label() default ResourceConstants.DEFAULT_LABEL;
+  @DescriptorKey("operation")
   String operation() default ResourceConstants.LIST_DS;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/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 d53b253..ead3430 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
@@ -16,407 +16,57 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import com.gemstone.gemfire.cache.operations.OperationContext;
 
 /**
  * This is base class for OperationContext for resource (JMX and CLI) operations
  *
  */
-public abstract class ResourceOperationContext extends OperationContext {
+public class ResourceOperationContext extends OperationContext {
 	
   private boolean isPostOperation=false;
   private Object opResult = null;
+  private Resource resource = Resource.DEFAULT;
+  private OperationCode operation = OperationCode.MANAGE;
 
-	 public static class ResourceOperationCode {
-		 
-    private static final int OP_ALTER_REGION = 1;
-    private static final int OP_ALTER_RUNTIME = 2;
-    private static final int OP_BACKUP_DISKSTORE = 3;
-    private static final int OP_CHANGE_ALERT_LEVEL = 4;
-    private static final int OP_CLOSE_DURABLE_CLIENT = 5;
-    private static final int OP_CLOSE_DURABLE_CQ = 6;
-    private static final int OP_COMPACT_DISKSTORE = 7;
-    private static final int OP_CONFIGURE_PDX = 8;
-    private static final int OP_CREATE_AEQ = 9;
-    private static final int OP_CREATE_DISKSTORE = 10;
-    private static final int OP_CREATE_GW_RECEIVER = 11;
-    private static final int OP_CREATE_GW_SENDER = 12;
-    private static final int OP_CREATE_INDEX = 13;
-    private static final int OP_CREATE_REGION = 14;
-    private static final int OP_DEPLOY = 15;
-    private static final int OP_DESTROY_DISKSTORE = 16;
-    private static final int OP_DESTROY_FUNCTION = 17;
-    private static final int OP_DESTROY_INDEX = 18;
-    private static final int OP_DESTROY_REGION = 19;
-    private static final int OP_EXECUTE_FUNCTION = 20;
-    private static final int OP_EXPORT_CONFIG = 21;
-    private static final int OP_EXPORT_DATA = 22;
-    private static final int OP_EXPORT_LOGS = 23;
-    private static final int OP_EXPORT_OFFLINE_DISKSTORE = 24;
-    private static final int OP_EXPORT_STACKTRACE = 25;
-    private static final int OP_GC = 26;
-    private static final int OP_GET = 27;
-    private static final int OP_IMPORT_CONFIG = 28;
-    private static final int OP_IMPORT_DATA = 29;
-    private static final int OP_LIST_DS = 30;
-    private static final int OP_LOAD_BALANCE_GW_SENDER = 31;
-    private static final int OP_LOCATE_ENTRY = 32;
-    private static final int OP_NETSTAT = 33;
-    private static final int OP_PAUSE_GW_SENDER = 34;
-    private static final int OP_PUT = 35;
-    private static final int OP_QUERY = 36;
-    private static final int OP_REBALANCE = 37;
-    private static final int OP_REMOVE = 38;
-    private static final int OP_RENAME_PDX = 39;
-    private static final int OP_RESUME_GW_SENDER = 40;
-    private static final int OP_REVOKE_MISSING_DISKSTORE = 41;
-    private static final int OP_SHOW_DEADLOCKS = 42;
-    private static final int OP_SHOW_LOG = 43;
-    private static final int OP_SHOW_METRICS = 44;
-    private static final int OP_SHOW_MISSING_DISKSTORES = 45;
-    private static final int OP_SHOW_SUBSCRIPTION_QUEUE_SIZE = 46;
-    private static final int OP_SHUTDOWN = 47;
-    private static final int OP_STOP_GW_RECEIVER = 48;
-    private static final int OP_STOP_GW_SENDER = 49;
-    private static final int OP_UNDEPLOY = 50;
-    private static final int OP_BACKUP_MEMBERS = 51;
-    private static final int OP_ROLL_DISKSTORE = 52;
-    private static final int OP_FORCE_COMPACTION = 53;
-    private static final int OP_FORCE_ROLL = 54;
-    private static final int OP_FLUSH_DISKSTORE = 55;
-    private static final int OP_START_GW_RECEIVER = 56;
-    private static final int OP_START_GW_SENDER = 57;
-    private static final int OP_BECOME_LOCK_GRANTOR = 58;
-    private static final int OP_START_MANAGER = 59;
-    private static final int OP_STOP_MANAGER = 60;
-    private static final int OP_CREATE_MANAGER = 61;
-    private static final int OP_STOP_CONTINUOUS_QUERY = 62;
-    private static final int OP_SET_DISK_USAGE = 63;
-    private static final int OP_CREATE_HDFS_STORE = 64;
-    private static final int OP_ALTER_HDFS_STORE = 65;
-    private static final int OP_DESTROY_HDFS_STORE = 66;
-	    
-    private static final int OP_PULSE_DASHBOARD = 92;
-    private static final int OP_PULSE_DATABROWSER = 93;
-    private static final int OP_PULSE_WEBGFSH = 94;
-    private static final int OP_PULSE_ADMIN_V1 = 95;
-	    
-    private static final int OP_DATA_READ = 96;
-    private static final int OP_DATA_WRITE = 97;
-    private static final int OP_MONITOR = 98;
-    private static final int OP_ADMIN = 99;
-	    
-    private static final ResourceOperationCode[] VALUES = new ResourceOperationCode[100];
-    private static final Map<String, ResourceOperationCode> OperationNameMap = new HashMap<String, ResourceOperationCode>();
-
-	    
-    public static final ResourceOperationCode ALTER_REGION  = new ResourceOperationCode(ResourceConstants.ALTER_REGION, OP_ALTER_REGION);
-    public static final ResourceOperationCode ALTER_RUNTIME = new ResourceOperationCode(ResourceConstants.ALTER_RUNTIME, OP_ALTER_RUNTIME);
-    public static final ResourceOperationCode BACKUP_DISKSTORE = new ResourceOperationCode(ResourceConstants.BACKUP_DISKSTORE, OP_BACKUP_DISKSTORE);
-    public static final ResourceOperationCode CHANGE_ALERT_LEVEL = new ResourceOperationCode(ResourceConstants.CHANGE_ALERT_LEVEL, OP_CHANGE_ALERT_LEVEL);
-    public static final ResourceOperationCode CLOSE_DURABLE_CLIENT = new ResourceOperationCode(ResourceConstants.CLOSE_DURABLE_CLIENT, OP_CLOSE_DURABLE_CLIENT);
-    public static final ResourceOperationCode CLOSE_DURABLE_CQ = new ResourceOperationCode(ResourceConstants.CLOSE_DURABLE_CQ, OP_CLOSE_DURABLE_CQ);
-    public static final ResourceOperationCode COMPACT_DISKSTORE = new ResourceOperationCode(ResourceConstants.COMPACT_DISKSTORE, OP_COMPACT_DISKSTORE);
-    public static final ResourceOperationCode CONFIGURE_PDX = new ResourceOperationCode(ResourceConstants.CONFIGURE_PDX, OP_CONFIGURE_PDX);
-    public static final ResourceOperationCode CREATE_AEQ = new ResourceOperationCode(ResourceConstants.CREATE_AEQ, OP_CREATE_AEQ);
-    public static final ResourceOperationCode CREATE_DISKSTORE = new ResourceOperationCode(ResourceConstants.CREATE_DISKSTORE, OP_CREATE_DISKSTORE);
-    public static final ResourceOperationCode CREATE_GW_RECEIVER = new ResourceOperationCode(ResourceConstants.CREATE_GW_RECEIVER, OP_CREATE_GW_RECEIVER);
-    public static final ResourceOperationCode CREATE_GW_SENDER = new ResourceOperationCode(ResourceConstants.CREATE_GW_SENDER, OP_CREATE_GW_SENDER);
-    public static final ResourceOperationCode CREATE_INDEX = new ResourceOperationCode(ResourceConstants.CREATE_INDEX, OP_CREATE_INDEX);
-    public static final ResourceOperationCode CREATE_REGION = new ResourceOperationCode(ResourceConstants.CREATE_REGION, OP_CREATE_REGION);
-    public static final ResourceOperationCode DEPLOY = new ResourceOperationCode(ResourceConstants.DEPLOY, OP_DEPLOY);
-    public static final ResourceOperationCode DESTROY_DISKSTORE = new ResourceOperationCode(ResourceConstants.DESTROY_DISKSTORE, OP_DESTROY_DISKSTORE);
-    public static final ResourceOperationCode DESTROY_FUNCTION = new ResourceOperationCode(ResourceConstants.DESTROY_FUNCTION, OP_DESTROY_FUNCTION);
-    public static final ResourceOperationCode DESTROY_INDEX = new ResourceOperationCode(ResourceConstants.DESTROY_INDEX, OP_DESTROY_INDEX);
-    public static final ResourceOperationCode DESTROY_REGION = new ResourceOperationCode(ResourceConstants.DESTROY_REGION, OP_DESTROY_REGION);
-    public static final ResourceOperationCode EXECUTE_FUNCTION = new ResourceOperationCode(ResourceConstants.EXECUTE_FUNCTION, OP_EXECUTE_FUNCTION);
-    public static final ResourceOperationCode EXPORT_CONFIG = new ResourceOperationCode(ResourceConstants.EXPORT_CONFIG, OP_EXPORT_CONFIG);
-    public static final ResourceOperationCode EXPORT_DATA = new ResourceOperationCode(ResourceConstants.EXPORT_DATA, OP_EXPORT_DATA);
-    public static final ResourceOperationCode EXPORT_LOGS = new ResourceOperationCode(ResourceConstants.EXPORT_LOGS, OP_EXPORT_LOGS);
-    public static final ResourceOperationCode EXPORT_OFFLINE_DISKSTORE = new ResourceOperationCode(ResourceConstants.EXPORT_OFFLINE_DISKSTORE, OP_EXPORT_OFFLINE_DISKSTORE);
-    public static final ResourceOperationCode EXPORT_STACKTRACE = new ResourceOperationCode(ResourceConstants.EXPORT_STACKTRACE, OP_EXPORT_STACKTRACE);
-    public static final ResourceOperationCode GC = new ResourceOperationCode(ResourceConstants.GC, OP_GC);
-    public static final ResourceOperationCode GET = new ResourceOperationCode(ResourceConstants.GET, OP_GET);
-    public static final ResourceOperationCode IMPORT_CONFIG = new ResourceOperationCode(ResourceConstants.IMPORT_CONFIG, OP_IMPORT_CONFIG);
-    public static final ResourceOperationCode IMPORT_DATA = new ResourceOperationCode(ResourceConstants.IMPORT_DATA, OP_IMPORT_DATA);
-	    public static final ResourceOperationCode LIST_DS = new ResourceOperationCode(ResourceConstants.LIST_DS, OP_LIST_DS);
-    public static final ResourceOperationCode LOAD_BALANCE_GW_SENDER = new ResourceOperationCode(ResourceConstants.LOAD_BALANCE_GW_SENDER, OP_LOAD_BALANCE_GW_SENDER);
-    public static final ResourceOperationCode LOCATE_ENTRY = new ResourceOperationCode(ResourceConstants.LOCATE_ENTRY, OP_LOCATE_ENTRY);
-    public static final ResourceOperationCode NETSTAT = new ResourceOperationCode(ResourceConstants.NETSTAT, OP_NETSTAT);
-    public static final ResourceOperationCode PAUSE_GW_SENDER = new ResourceOperationCode(ResourceConstants.PAUSE_GW_SENDER, OP_PAUSE_GW_SENDER);
-    public static final ResourceOperationCode PUT = new ResourceOperationCode(ResourceConstants.PUT, OP_PUT);
-    public static final ResourceOperationCode QUERY = new ResourceOperationCode(ResourceConstants.QUERY, OP_QUERY);
-    public static final ResourceOperationCode REBALANCE = new ResourceOperationCode(ResourceConstants.REBALANCE, OP_REBALANCE);
-    public static final ResourceOperationCode REMOVE = new ResourceOperationCode(ResourceConstants.REMOVE, OP_REMOVE);
-    public static final ResourceOperationCode RENAME_PDX = new ResourceOperationCode(ResourceConstants.RENAME_PDX, OP_RENAME_PDX);
-    public static final ResourceOperationCode RESUME_GW_SENDER = new ResourceOperationCode(ResourceConstants.RESUME_GW_SENDER, OP_RESUME_GW_SENDER);
-    public static final ResourceOperationCode REVOKE_MISSING_DISKSTORE = new ResourceOperationCode(ResourceConstants.REVOKE_MISSING_DISKSTORE, OP_REVOKE_MISSING_DISKSTORE);
-    public static final ResourceOperationCode SHOW_DEADLOCKS = new ResourceOperationCode(ResourceConstants.SHOW_DEADLOCKS, OP_SHOW_DEADLOCKS);
-    public static final ResourceOperationCode SHOW_LOG = new ResourceOperationCode(ResourceConstants.SHOW_LOG, OP_SHOW_LOG);
-    public static final ResourceOperationCode SHOW_METRICS = new ResourceOperationCode(ResourceConstants.SHOW_METRICS, OP_SHOW_METRICS);
-    public static final ResourceOperationCode SHOW_MISSING_DISKSTORES = new ResourceOperationCode(ResourceConstants.SHOW_MISSING_DISKSTORES, OP_SHOW_MISSING_DISKSTORES);
-    public static final ResourceOperationCode SHOW_SUBSCRIPTION_QUEUE_SIZE = new ResourceOperationCode(ResourceConstants.SHOW_SUBSCRIPTION_QUEUE_SIZE, OP_SHOW_SUBSCRIPTION_QUEUE_SIZE);
-    public static final ResourceOperationCode SHUTDOWN = new ResourceOperationCode(ResourceConstants.SHUTDOWN, OP_SHUTDOWN);
-    public static final ResourceOperationCode STOP_GW_RECEIVER = new ResourceOperationCode(ResourceConstants.STOP_GW_RECEIVER, OP_STOP_GW_RECEIVER);
-    public static final ResourceOperationCode STOP_GW_SENDER = new ResourceOperationCode(ResourceConstants.STOP_GW_SENDER, OP_STOP_GW_SENDER);
-    public static final ResourceOperationCode UNDEPLOY = new ResourceOperationCode(ResourceConstants.UNDEPLOY, OP_UNDEPLOY);
-    public static final ResourceOperationCode BACKUP_MEMBERS = new ResourceOperationCode(ResourceConstants.BACKUP_MEMBERS, OP_BACKUP_MEMBERS);
-    public static final ResourceOperationCode ROLL_DISKSTORE = new ResourceOperationCode(ResourceConstants.ROLL_DISKSTORE, OP_ROLL_DISKSTORE);
-    public static final ResourceOperationCode FORCE_COMPACTION = new ResourceOperationCode(ResourceConstants.FORCE_COMPACTION, OP_FORCE_COMPACTION);
-    public static final ResourceOperationCode FORCE_ROLL = new ResourceOperationCode(ResourceConstants.FORCE_ROLL, OP_FORCE_ROLL);
-    public static final ResourceOperationCode FLUSH_DISKSTORE = new ResourceOperationCode(ResourceConstants.FLUSH_DISKSTORE, OP_FLUSH_DISKSTORE);
-    public static final ResourceOperationCode START_GW_RECEIVER = new ResourceOperationCode(ResourceConstants.START_GW_RECEIVER, OP_START_GW_RECEIVER);
-    public static final ResourceOperationCode START_GW_SENDER = new ResourceOperationCode(ResourceConstants.START_GW_SENDER, OP_START_GW_SENDER);
-    public static final ResourceOperationCode BECOME_LOCK_GRANTOR = new ResourceOperationCode(ResourceConstants.BECOME_LOCK_GRANTOR, OP_BECOME_LOCK_GRANTOR);
-    public static final ResourceOperationCode START_MANAGER = new ResourceOperationCode(ResourceConstants.START_MANAGER, OP_START_MANAGER);
-    public static final ResourceOperationCode STOP_MANAGER = new ResourceOperationCode(ResourceConstants.STOP_MANAGER, OP_STOP_MANAGER);
-    public static final ResourceOperationCode CREATE_MANAGER = new ResourceOperationCode(ResourceConstants.CREATE_MANAGER, OP_CREATE_MANAGER);
-    public static final ResourceOperationCode STOP_CONTINUOUS_QUERY = new ResourceOperationCode(ResourceConstants.STOP_CONTINUOUS_QUERY, OP_STOP_CONTINUOUS_QUERY);
-    public static final ResourceOperationCode SET_DISK_USAGE = new ResourceOperationCode(ResourceConstants.SET_DISK_USAGE, OP_SET_DISK_USAGE);
-    public static final ResourceOperationCode CREATE_HDFS_STORE = new ResourceOperationCode(ResourceConstants.CREATE_HDFS_STORE, OP_CREATE_HDFS_STORE);
-    public static final ResourceOperationCode ALTER_HDFS_STORE = new ResourceOperationCode(ResourceConstants.ALTER_HDFS_STORE, OP_ALTER_HDFS_STORE);
-    public static final ResourceOperationCode DESTROY_HDFS_STORE = new ResourceOperationCode(ResourceConstants.DESTROY_HDFS_STORE, OP_DESTROY_HDFS_STORE);
-
-	    
-    public static final ResourceOperationCode PULSE_DASHBOARD = new ResourceOperationCode(
-        ResourceConstants.PULSE_DASHBOARD, OP_PULSE_DASHBOARD);
-    public static final ResourceOperationCode PULSE_DATABROWSER = new ResourceOperationCode(
-        ResourceConstants.PULSE_DATABROWSER, OP_PULSE_DATABROWSER);
-    public static final ResourceOperationCode PULSE_WEBGFSH = new ResourceOperationCode(
-        ResourceConstants.PULSE_WEBGFSH, OP_PULSE_WEBGFSH);
-    public static final ResourceOperationCode PULSE_ADMIN_V1 = new ResourceOperationCode(
-        ResourceConstants.PULSE_ADMIN_V1, OP_PULSE_ADMIN_V1);
-	    
-    public static final ResourceOperationCode DATA_READ = new ResourceOperationCode(ResourceConstants.DATA_READ,
-        OP_DATA_READ,
-	    		new ResourceOperationCode[]{
-          LIST_DS,
-          PULSE_DASHBOARD
-    });
-
-    public static final ResourceOperationCode DATA_WRITE = new ResourceOperationCode(ResourceConstants.DATA_WRITE,
-        OP_DATA_WRITE,
-        new ResourceOperationCode[]{
-          DATA_READ,
-          QUERY,
-          BECOME_LOCK_GRANTOR,
-          PUT,
-          REMOVE,
-          EXECUTE_FUNCTION,
-          PULSE_DATABROWSER
-    });
-
-    public static final ResourceOperationCode MONITOR = new ResourceOperationCode(ResourceConstants.MONITOR,
-        OP_MONITOR,
-        new ResourceOperationCode[] {
-          DATA_READ,
-          EXPORT_CONFIG,
-          EXPORT_DATA,
-          EXPORT_LOGS,
-          EXPORT_OFFLINE_DISKSTORE,
-          EXPORT_STACKTRACE,
-          SHOW_DEADLOCKS,
-          SHOW_LOG,
-          SHOW_METRICS,
-          SHOW_MISSING_DISKSTORES,
-          SHOW_SUBSCRIPTION_QUEUE_SIZE
-    });
-
-    public static final ResourceOperationCode ADMIN = new ResourceOperationCode(ResourceConstants.ADMIN,
-        OP_ADMIN,
-        new ResourceOperationCode[] {
-          DATA_WRITE,
-          MONITOR,
-          ALTER_REGION,
-          ALTER_RUNTIME,
-          BACKUP_DISKSTORE,
-          CHANGE_ALERT_LEVEL,
-          CLOSE_DURABLE_CLIENT,
-          CLOSE_DURABLE_CQ,
-          COMPACT_DISKSTORE,
-          CONFIGURE_PDX,
-          CREATE_AEQ,
-          CREATE_DISKSTORE,
-          CREATE_GW_RECEIVER,
-          CREATE_GW_SENDER,
-          CREATE_INDEX,
-          CREATE_REGION,
-          DEPLOY,
-          DESTROY_DISKSTORE,
-          DESTROY_FUNCTION,
-          DESTROY_INDEX,
-          DESTROY_REGION,
-          GC,
-          GET,
-          IMPORT_CONFIG,
-          IMPORT_DATA,
-          LIST_DS,
-          LOAD_BALANCE_GW_SENDER,
-          LOCATE_ENTRY,
-          NETSTAT,
-          PAUSE_GW_SENDER,
-          REBALANCE,
-          RENAME_PDX,
-          RESUME_GW_SENDER,
-          REVOKE_MISSING_DISKSTORE,
-          SHUTDOWN,
-          STOP_GW_RECEIVER,
-          STOP_GW_SENDER,
-          UNDEPLOY,
-          BACKUP_MEMBERS,
-          ROLL_DISKSTORE,
-          FORCE_COMPACTION,
-          FORCE_ROLL,
-          FLUSH_DISKSTORE,
-          START_GW_RECEIVER,
-          START_GW_SENDER,
-          START_MANAGER,
-          STOP_MANAGER,
-          CREATE_MANAGER,
-          STOP_CONTINUOUS_QUERY,
-          SET_DISK_USAGE,
-	    			PULSE_WEBGFSH, 
-	    			PULSE_ADMIN_V1
-	    		});
-		
-	    
-	    private final String name;
-    private final int opCode;
-    private final List<ResourceOperationCode> children;
-	    
-    private ResourceOperationCode(String name, int opCode) {
-	      this.name = name;
-	      this.opCode = opCode;
-	      VALUES[opCode] = this;
-	      OperationNameMap.put(name, this);
-	      this.children = null;
-	    }
-	    
-    private ResourceOperationCode(String name, int opCode, ResourceOperationCode[] children) {
-		      this.name = name;
-		      this.opCode = opCode;
-		      VALUES[opCode] = this;
-		      OperationNameMap.put(name, this);
-      this.children = new ArrayList<ResourceOperationCode>();
-      for(ResourceOperationCode code : children) {
-        this.children.add(code);
-      }
-		}
-	    
-    public List<ResourceOperationCode> getChildren() {
-      return children != null ? Collections.unmodifiableList(children) : null;
-    }
-
-    public void addChild(ResourceOperationCode code) {
-      this.children.add(code);
-      }
-
-      /**
-     * Returns the <code>OperationCode</code> represented by specified int.
-	     */
-    public static ResourceOperationCode fromOrdinal(int opCode) {
-	      return VALUES[opCode];
-	    }
-
-	    /**
-	     * Returns the <code>OperationCode</code> represented by specified string.
-	     */
-	    public static ResourceOperationCode parse(String operationName) {
-      return OperationNameMap.get(operationName);
-	    }
-
-	    /**
-     * Returns the int representing this operation code.
-	     * 
-     * @return a int representing this operation.
-	     */
-    public int toOrdinal() {
-	      return this.opCode;
-	    }
-
-	    /**
-	     * Returns a string representation for this operation.
-	     * 
-	     * @return the name of this operation.
-	     */
-	    @Override
-	    final public String toString() {
-	      return this.name;
-	    }
-
-	    /**
-	     * Indicates whether other object is same as this one.
-	     * 
-	     * @return true if other object is same as this one.
-	     */
-	    @Override
-	    final public boolean equals(final Object obj) {
-	      if (obj == this) {
-	        return true;
-	      }
-	      if (!(obj instanceof ResourceOperationCode)) {
-	        return false;
-	      }
-	      final ResourceOperationCode other = (ResourceOperationCode)obj;
-	      return (other.opCode == this.opCode);
-	    }
+  public ResourceOperationContext(){
+  }
 
-	    /**
-	     * Indicates whether other <code>OperationCode</code> is same as this one.
-	     * 
-	     * @return true if other <code>OperationCode</code> is same as this one.
-	     */
-	    final public boolean equals(final ResourceOperationCode opCode) {
-	      return (opCode != null && opCode.opCode == this.opCode);
-	    }
+  public ResourceOperationContext(Resource resource, OperationCode operation) {
+    this.resource = resource;
+    this.operation = operation;
+  }
 
-	    /**
-	     * Returns a hash code value for this <code>OperationCode</code> which is
-     * the same as the int representing its operation type.
-	     * 
-	     * @return the hashCode of this operation.
-	     */
-	    @Override
-	    final public int hashCode() {
-	      return this.opCode;
-	    }
+  public ResourceOperationContext(String resource, String operation){
+    if(resource!=null)
+      this.resource = Resource.valueOf(resource);
+    if(operation!=null)
+      this.operation = OperationCode.valueOf(operation);
+  }
 
-    /**
-     * Returns true if passed operation is same or any one of its
-     * children
-     *
-     * @param op
-     * @return true if  <code>OperationCode</code> matches
-     */
-    public boolean allowedOp(ResourceOperationCode op) {
-      if(this.equals(op))
-        return true;
-      else {
-        if(children!=null) {
-          for(ResourceOperationCode child : children) {
-            if(child.allowedOp(op))
-              return true;
-	 }
-        }
-      }
-      return false;
+  public void setResourceOperation(ResourceOperation op){
+    if(op!=null){
+      resource = op.resource();
+      operation = OperationCode.valueOf(op.operation());
     }
   }
 
-	 public abstract ResourceOperationCode getResourceOperationCode();
-
 	@Override
   public boolean isClientUpdate() {
     return false;
   }
 
-	@Override
+  @Override
+  public OperationCode getOperationCode() {
+    return operation;
+  }
+
+  @Override
+  public Resource getResource(){
+    return resource;
+  }
+
+  @Override
 	public boolean isPostOperation() {
     return isPostOperation;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/SetAttributesOperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/SetAttributesOperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/SetAttributesOperationContext.java
deleted file mode 100644
index 068ac3c..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/SetAttributesOperationContext.java
+++ /dev/null
@@ -1,55 +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.util.HashMap;
-import java.util.Map;
-
-/**
- * Used to encapsulate Context passed AccessControl Plugin for each of the
- * attributes in attribute list passed to setAttributes call on given MBean
- *
- * @author tushark
- * @since 9.0
- */
-public class SetAttributesOperationContext extends ResourceOperationContext {
-
-  private Map<String,ResourceOperationContext> contextMap = null;
-
-  public SetAttributesOperationContext(){
-    contextMap = new HashMap<String,ResourceOperationContext>();
-  }
-
-  public void addAttribute(String attr, ResourceOperationContext setterContext) {
-    this.contextMap.put(attr, setterContext);
-  }
-
-  public Map<String,ResourceOperationContext> getAttributesContextMap(){
-    return contextMap;
-  }
-
-  @Override
-  public ResourceOperationCode getResourceOperationCode() {
-    return null;
-  }
-
-  @Override
-  public OperationCode getOperationCode() {
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
index ce91b30..6702bc0 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
@@ -631,8 +631,6 @@ public abstract class AbstractCommandsController {
   }
 
   protected ResourceOperationContext authorize(final String command) {
-
-
     SystemManagementService service = (SystemManagementService) ManagementService
         .getExistingManagementService(CacheFactory.getAnyInstance());
     Properties credentials = EnvironmentVariablesHandlerInterceptor.CREDENTIALS.get();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c7680e2b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AuthorizeOperationForDistributedSystemMXBeanIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AuthorizeOperationForDistributedSystemMXBeanIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AuthorizeOperationForDistributedSystemMXBeanIntegrationTest.java
index 4ae0107..b42c784 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AuthorizeOperationForDistributedSystemMXBeanIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AuthorizeOperationForDistributedSystemMXBeanIntegrationTest.java
@@ -16,16 +16,14 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
-import static org.junit.Assert.*;
-
-import javax.management.remote.JMXPrincipal;
-
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import com.gemstone.gemfire.util.test.TestUtil;
+import javax.management.remote.JMXPrincipal;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests <code>JSONAuthorization.authorizeOperation(...)</code> for <code>DistributedSystemMXBean</code> operations.
@@ -35,15 +33,14 @@ public class AuthorizeOperationForDistributedSystemMXBeanIntegrationTest {
 
   @Test
   public void returnsFalseForUnauthorizedUser() throws Exception {    
-    System.setProperty("resource.secDescriptor", TestUtil.getResourcePath(getClass(), "auth1.json")); 
-    JSONAuthorization authorization = JSONAuthorization.create();        
+    JSONAuthorization authorization = new JSONAuthorization("auth1.json");
     authorization.init(new JMXPrincipal("tushark"), null, null);
     
-    JMXOperationContext context = new JMXOperationContext(MBeanJMXAdapter.getDistributedSystemName(), "queryData");
+    ResourceOperationContext context = new ResourceOperationContext(null, "QUERY");
     boolean result = authorization.authorizeOperation(null, context);
-    //assertTrue(result); TODO: why is this commented out? looks like this should be true but it isn't
+    assertTrue(result);
     
-    context = new JMXOperationContext(MBeanJMXAdapter.getDistributedSystemName(), "changeAlertLevel");
+    context = new ResourceOperationContext(null, "MANAGE");
     result = authorization.authorizeOperation(null,context);
     assertFalse(result);
   }