You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by je...@apache.org on 2016/03/11 08:27:03 UTC

incubator-geode git commit: GEODE-17: JSONAuthorization permissions are now defined as resource:operation

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


GEODE-17: JSONAuthorization permissions are now defined as resource:operation


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 3040e5afee8172ab058e18a06df9470125bdbb3a
Parents: cdc49fe
Author: Jens Deppe <jd...@pivotal.io>
Authored: Thu Mar 10 23:26:54 2016 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Thu Mar 10 23:26:54 2016 -0800

----------------------------------------------------------------------
 ...rDistributedSystemMXBeanIntegrationTest.java |   4 +-
 .../CacheServerMBeanSecurityJUnitTest.java      |  17 +--
 .../internal/security/JSONAuthorization.java    | 145 ++++++++-----------
 ...JSONAuthorizationDetailsIntegrationTest.java |  80 +---------
 .../security/MemberMBeanSecurityJUnitTest.java  |  30 ++--
 .../management/internal/security/auth1.json     |  28 ++--
 .../management/internal/security/auth3.json     |   4 +-
 .../internal/security/cacheServer.json          |  31 ++--
 .../security/testSimpleUserAndRole.json         |  28 ++--
 .../testUserAndRoleRegionServerGroup.json       |  32 ++--
 .../internal/security/testUserMultipleRole.json |  42 +++---
 11 files changed, 178 insertions(+), 263 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/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 871b74a..3859283 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
@@ -36,11 +36,11 @@ public class AuthorizeOperationForDistributedSystemMXBeanIntegrationTest {
     JSONAuthorization authorization = new JSONAuthorization("auth1.json");
     authorization.init(new JMXPrincipal("tushark"), null, null);
 
-    ResourceOperationContext context = new ResourceOperationContext(null, "QUERY");
+    ResourceOperationContext context = new ResourceOperationContext("QUERY", "EXECUTE");
     boolean result = authorization.authorizeOperation(null, context);
     assertTrue(result);
 
-    context = new ResourceOperationContext(null, "MANAGE");
+    context = new ResourceOperationContext("REGION", "CREATE");
     result = authorization.authorizeOperation(null, context);
     assertFalse(result);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/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 d233d55..dab65ea 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
@@ -51,7 +51,6 @@ public class CacheServerMBeanSecurityJUnitTest {
 
   @Before
   public void setUp() throws Exception {
-    //assertThat(cache.getCacheServers()).hasSize(1);
     cacheServerMXBean = (CacheServerMXBean) connectionRule.getProxyMBean(CacheServerMXBean.class,
         "GemFire:service=CacheServer,*");
     con = connectionRule.getMBeanServerConnection();
@@ -74,14 +73,14 @@ public class CacheServerMBeanSecurityJUnitTest {
   @Test
   @JMXConnectionConfiguration(user = "superuser", password = "1234567")
   public void testAllAccess() throws Exception {
-    cacheServerMXBean.removeIndex("foo"); // "DESTROY_INDEX",
-    cacheServerMXBean.executeContinuousQuery("bar"); //QUERY
-    cacheServerMXBean.fetchLoadProbe(); //LIST_DS
-    cacheServerMXBean.getActiveCQCount(); //LIST_DS
-    cacheServerMXBean.stopContinuousQuery("bar"); //STOP_CONTINUOUS_QUERY
-    cacheServerMXBean.closeAllContinuousQuery("bar"); //STOP_CONTINUOUS_QUERY
-    cacheServerMXBean.isRunning(); //LIST_DS
-    cacheServerMXBean.showClientQueueDetails("foo"); //LIST_DS
+    cacheServerMXBean.removeIndex("foo"); // "INDEX:DESTROY",
+    cacheServerMXBean.executeContinuousQuery("bar"); // CONTNUOUS_QUERY:EXECUTE
+    cacheServerMXBean.fetchLoadProbe(); // DISTRIBUTED_SYSTEM:LIST_DS
+    cacheServerMXBean.getActiveCQCount(); // DISTRIBUTED_SYSTEM:LIST_DS
+    cacheServerMXBean.stopContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
+    cacheServerMXBean.closeAllContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
+    cacheServerMXBean.isRunning(); // DISTRIBUTED_SYSTEM:LIST_DS
+    cacheServerMXBean.showClientQueueDetails("foo"); // DISTRIBUTED_SYSTEM:LIST_DS
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorization.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorization.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorization.java
index 4b8245c..7422d27 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorization.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorization.java
@@ -36,16 +36,50 @@ import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.security.Principal;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
 public class JSONAuthorization implements AccessControl, Authenticator {
 
+  static class Permission {
+
+    private final Resource resource;
+    private final OperationCode operationCode;
+    private final String region;
+
+    Permission(Resource resource, OperationCode operationCode, String region) {
+      this.resource = resource;
+      this.operationCode = operationCode;
+      this.region = region;
+    }
+
+    public Resource getResource() {
+      return resource;
+    }
+
+    public OperationCode getOperationCode() {
+      return operationCode;
+    }
+
+    public String getRegion() {
+      return region;
+    }
+
+    @Override
+    public String toString() {
+      String result = resource.toString() + ":" + operationCode.toString();
+      result += (region != null) ? "[" + region + "]" : "";
+      return result;
+    }
+  }
+
   public static class Role {
-    String[] permissions;
+    List<Permission> permissions = new ArrayList<>();
     String name;
     String regionName;
     String serverGroup;
@@ -53,7 +87,7 @@ public class JSONAuthorization implements AccessControl, Authenticator {
 
   public static class User {
     String name;
-    Role[] roles;
+    Set<Permission> permissions = new HashSet<>();
     String pwd;
   }
 
@@ -77,47 +111,13 @@ public class JSONAuthorization implements AccessControl, Authenticator {
 
   private static void readSecurityDescriptor(String json) throws IOException, JSONException {
     JSONObject jsonBean = new JSONObject(json);
-    acl = new HashMap<String, User>();
+    acl = new HashMap<>();
     Map<String, Role> roleMap = readRoles(jsonBean);
     readUsers(acl, jsonBean, roleMap);
   }
 
-  public static Set<OperationCode> getAuthorizedOps(User user, ResourceOperationContext context) {
-    Set<OperationCode> codeList = new HashSet<OperationCode>();
-    for (Role role : user.roles) {
-      for (String perm : role.permissions) {
-        OperationCode code = OperationCode.valueOf(perm);
-        if (role.regionName == null && role.serverGroup == null) {
-          addPermissions(code, codeList);
-        } else if (role.regionName != null) {
-          LogService.getLogger().info("This role requires region=" + role.regionName);
-          if (context instanceof CLIOperationContext) {
-            CLIOperationContext cliContext = (CLIOperationContext) context;
-            String region = cliContext.getCommandOptions().get("region");
-            if (region != null && region.equals(role.regionName)) {
-              addPermissions(code, codeList);
-            } else {
-              LogService.getLogger()
-                  .info("Not adding permission " + code + " since region=" + region + " does not match");
-            }
-          }
-        }
-        // Same to be implemented for ServerGroup
-      }
-    }
-    LogService.getLogger().info("Final set of permisions " + codeList);
-    return codeList;
-  }
-
-  private static void addPermissions(OperationCode code, Set<OperationCode> codeList) {
-    if (code == null) {
-      return;
-    }
-    codeList.add(code);
-  }
-
-  private static void readUsers(Map<String, User> acl, JSONObject jsonBean,
-      Map<String, Role> roleMap) throws JSONException {
+  private static void readUsers(Map<String, User> acl, JSONObject jsonBean, Map<String, Role> roleMap)
+      throws JSONException {
     JSONArray array = jsonBean.getJSONArray("users");
     for (int i = 0; i < array.length(); i++) {
       JSONObject obj = array.getJSONObject(i);
@@ -130,20 +130,17 @@ public class JSONAuthorization implements AccessControl, Authenticator {
       }
 
       JSONArray ops = obj.getJSONArray("roles");
-      user.roles = new Role[ops.length()];
       for (int j = 0; j < ops.length(); j++) {
         String roleName = ops.getString(j);
-        user.roles[j] = roleMap.get(roleName);
-        if (user.roles[j] == null) {
-          throw new RuntimeException("Role not present " + roleName);
-        }
+
+        user.permissions.addAll(roleMap.get(roleName).permissions);
       }
       acl.put(user.name, user);
     }
   }
 
   private static Map<String, Role> readRoles(JSONObject jsonBean) throws JSONException {
-    Map<String, Role> roleMap = new HashMap<String, Role>();
+    Map<String, Role> roleMap = new HashMap<>();
     JSONArray array = jsonBean.getJSONArray("roles");
     for (int i = 0; i < array.length(); i++) {
       JSONObject obj = array.getJSONObject(i);
@@ -151,10 +148,14 @@ public class JSONAuthorization implements AccessControl, Authenticator {
       role.name = obj.getString("name");
 
       if (obj.has("operationsAllowed")) {
+        // The default region is null and not the empty string
+        String region = obj.optString("region", null);
         JSONArray ops = obj.getJSONArray("operationsAllowed");
-        role.permissions = new String[ops.length()];
         for (int j = 0; j < ops.length(); j++) {
-          role.permissions[j] = ops.getString(j);
+          String[] parts = ops.getString(j).split(":");
+          Resource r = Resource.valueOf(parts[0]);
+          OperationCode op = parts.length > 1 ? OperationCode.valueOf(parts[1]) : OperationCode.ALL;
+          role.permissions.add(new Permission(r, op, region));
         }
       } else {
         if (!obj.has("inherit")) {
@@ -174,40 +175,6 @@ public class JSONAuthorization implements AccessControl, Authenticator {
       }
     }
 
-    for (int i = 0; i < array.length(); i++) {
-      JSONObject obj = array.getJSONObject(i);
-      String name = obj.getString("name");
-      Role role = roleMap.get(name);
-      if (role == null) {
-        throw new RuntimeException("Role not present " + role);
-      }
-      if (obj.has("inherit")) {
-        JSONArray parentRoles = obj.getJSONArray("inherit");
-        for (int m = 0; m < parentRoles.length(); m++) {
-          String parentRoleName = parentRoles.getString(m);
-          Role parentRole = roleMap.get(parentRoleName);
-          if (parentRole == null) {
-            throw new RuntimeException("Role not present " + parentRoleName);
-          }
-          int oldLenth = 0;
-          if (role.permissions != null) oldLenth = role.permissions.length;
-          int newLength = oldLenth + parentRole.permissions.length;
-          String[] str = new String[newLength];
-          int k = 0;
-          if (role.permissions != null) {
-            for (; k < role.permissions.length; k++) {
-              str[k] = role.permissions[k];
-            }
-          }
-
-          for (int l = 0; l < parentRole.permissions.length; l++) {
-            str[k + l] = parentRole.permissions[l];
-          }
-          role.permissions = str;
-        }
-      }
-
-    }
     return roleMap;
   }
 
@@ -230,20 +197,24 @@ public class JSONAuthorization implements AccessControl, Authenticator {
       if (user != null) {
         LogService.getLogger().info("Context received " + context);
         ResourceOperationContext ctx = (ResourceOperationContext) context;
-        LogService.getLogger().info("Checking for code " + ctx.getOperationCode());
+        LogService.getLogger().info("Checking for permission " + ctx.getResource() + ":" + ctx.getOperationCode());
 
         //TODO : This is for un-annotated commands
-        if (ctx.getOperationCode() == null) return true;
+        if (ctx.getOperationCode() == null) {
+          return true;
+        }
 
         boolean found = false;
-        for (OperationCode code : getAuthorizedOps(user, (ResourceOperationContext) context)) {
-          if (ctx.getOperationCode().equals(code)) {
+        for (Permission perm : acl.get(user.name).permissions) {
+          if (ctx.getResource() == perm.getResource() && ctx.getOperationCode() == perm.getOperationCode()) {
             found = true;
-            LogService.getLogger().info("found code " + code.toString());
+            LogService.getLogger().info("Found permission " + perm);
             break;
           }
         }
-        if (found) return true;
+        if (found) {
+          return true;
+        }
         LogService.getLogger().info("Did not find code " + ctx.getOperationCode());
         return false;
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorizationDetailsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorizationDetailsIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorizationDetailsIntegrationTest.java
index 5e89948..acc11ef 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorizationDetailsIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorizationDetailsIntegrationTest.java
@@ -21,14 +21,10 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.util.Arrays;
 import java.util.Map;
 
-import static org.hamcrest.Matchers.hasItems;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertThat;
 
 /**
  * Tests JSONAuthorization with JSON loaded from files.
@@ -44,11 +40,9 @@ public class JSONAuthorizationDetailsIntegrationTest {
     assertEquals(1, acl.size());
     User user = acl.get("tushark");
     assertNotNull(user);
-    assertNotNull(user.roles);
-    assertEquals(1, user.roles.length);
-    assertEquals("jmxReader", user.roles[0].name);
-    assertEquals(1, user.roles[0].permissions.length);
-    assertEquals("QUERY", user.roles[0].permissions[0]);
+    assertEquals(1, user.permissions.size());
+    JSONAuthorization.Permission p = user.permissions.iterator().next();
+    assertEquals("QUERY:EXECUTE", p.toString());
   }
 
   @Test
@@ -60,14 +54,9 @@ public class JSONAuthorizationDetailsIntegrationTest {
     assertEquals(1, acl.size());
     User user = acl.get("tushark");
     assertNotNull(user);
-    assertNotNull(user.roles);
-    assertEquals(1, user.roles.length);
-    assertEquals("jmxReader", user.roles[0].name);
-    assertEquals(1, user.roles[0].permissions.length);
-    assertEquals("QUERY", user.roles[0].permissions[0]);
-
-    assertEquals("secureRegion", user.roles[0].regionName);
-    assertEquals("SG2", user.roles[0].serverGroup);
+    assertEquals(1, user.permissions.size());
+    JSONAuthorization.Permission p = user.permissions.iterator().next();
+    assertEquals("secureRegion", p.getRegion());
   }
 
   @Test
@@ -79,61 +68,6 @@ public class JSONAuthorizationDetailsIntegrationTest {
     assertEquals(1, acl.size());
     User user = acl.get("tushark");
     assertNotNull(user);
-    assertNotNull(user.roles);
-    assertEquals(2, user.roles.length);
-
-    JSONAuthorization.Role role = user.roles[0];
-    assertEquals("jmxReader", role.name);
-
-    assertEquals(1, role.permissions.length);
-    assertEquals("QUERY", role.permissions[0]);
-
-    role = user.roles[1];
-    assertNotEquals("jmxReader", role.name);
-
-    assertEquals(7, role.permissions.length);
-    assertEquals("sysMonitors", role.name);
-    assertThat(Arrays.asList(role.permissions), hasItems(
-        "CMD_EXPORT_LOGS",
-        "CMD_STACK_TRACES",
-        "CMD_GC",
-        "CMD_NETSTAT",
-        "CMD_SHOW_DEADLOCKS",
-        "CMD_SHOW_LOG",
-        "SHOW_METRICS"));
-  }
-
-  @Test
-  public void testInheritRole() throws Exception {
-    new JSONAuthorization("testInheritRole.json");
-    Map<String, User> acl = JSONAuthorization.getAcl();
-
-    assertNotNull(acl);
-    assertEquals(3, acl.size());
-    User user = acl.get("tushark");
-    assertNotNull(user);
-    assertNotNull(user.roles);
-    assertEquals(1, user.roles.length);
-    assertEquals("jmxReader", user.roles[0].name);
-    assertEquals(1, user.roles[0].permissions.length);
-    assertEquals("QUERY", user.roles[0].permissions[0]);
-
-    User admin1 = acl.get("admin1");
-    assertNotNull(admin1);
-    assertNotNull(admin1.roles);
-    assertEquals(1, admin1.roles.length);
-    assertEquals("adminSG1", admin1.roles[0].name);
-    assertEquals("SG1", admin1.roles[0].serverGroup);
-    assertEquals(1, admin1.roles[0].permissions.length);
-    assertEquals("CMD_SHUTDOWN", admin1.roles[0].permissions[0]);
-
-    User admin2 = acl.get("admin2");
-    assertNotNull(admin2);
-    assertNotNull(admin2.roles);
-    assertEquals(1, admin2.roles.length);
-    assertEquals("adminSG2", admin2.roles[0].name);
-    assertEquals("SG2", admin2.roles[0].serverGroup);
-    assertEquals(2, admin2.roles[0].permissions.length);
-    assertThat(Arrays.asList(admin2.roles[0].permissions), hasItems("CHANGE_LOG_LEVEL", "CMD_SHUTDOWN"));
+    assertEquals(3, user.permissions.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/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 2882fcb..22bc25b 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,8 +25,6 @@ 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)
@@ -34,7 +32,6 @@ public class MemberMBeanSecurityJUnitTest {
   private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
 
   private MemberMXBean bean;
-  private MBeanServerConnection con;
 
   @ClassRule
   public static JsonAuthorizationMBeanServerStartRule serverRule = new JsonAuthorizationMBeanServerStartRule(
@@ -46,25 +43,24 @@ public class MemberMBeanSecurityJUnitTest {
   @Before
   public void setUp() throws Exception {
     bean = (MemberMXBean) connectionRule.getProxyMBean(MemberMXBean.class);
-    con = connectionRule.getMBeanServerConnection();
   }
 
   @Test
   @JMXConnectionConfiguration(user = "superuser", password = "1234567")
   public void testAllAccess() throws Exception {
-    bean.shutDownMember();  //SHUTDOWN
-    bean.compactAllDiskStores(); //COMPACT_DISKSTORE
-    bean.createManager(); //CREATE_MANAGER
-    bean.fetchJvmThreads(); //LIST_DS
-    bean.getName(); //LIST_DS
-    bean.getDiskStores(); //LIST_DS
-    bean.hasGatewayReceiver(); //LIST_DS
-    bean.isCacheServer(); //LIST_DS
-    bean.isServer(); //LIST_DS
-    bean.listConnectedGatewayReceivers(); //LIST_DS
-    bean.processCommand("create region --name=Region_A"); //CREATE_REGION
-    bean.showJVMMetrics(); //LIST_DS
-    bean.status(); //LIST_DS
+    bean.shutDownMember();  // MEMBER:SHUTDOWN
+    bean.compactAllDiskStores(); // DISKSTORE:COMPACT
+    bean.createManager(); // MANAGER:CREATE
+    bean.fetchJvmThreads(); // DEFAULT:LIST_DS
+    bean.getName(); // DEFAULT:LIST_DS
+    bean.getDiskStores(); // DEFAULT:LIST_DS
+    bean.hasGatewayReceiver(); // DEFAULT:LIST_DS
+    bean.isCacheServer(); // DEFAULT:LIST_DS
+    bean.isServer(); // DEFAULT:LIST_DS
+    bean.listConnectedGatewayReceivers(); // DEFAULT:LIST_DS
+    bean.processCommand("create region --name=Region_A"); // REGION:CREATE
+    bean.showJVMMetrics(); // DEFAULT:LIST_DS
+    bean.status(); // DEFAULT:LIST_DS
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/auth1.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/auth1.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/auth1.json
index d4318d1..72c165a 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/auth1.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/auth1.json
@@ -1,14 +1,18 @@
 {
-"roles" : [	
-			{
-				"name" : "jmxReader",
-				"operationsAllowed" : ["QUERY"]
-			}
-		],
-users : [
-	 		{
-	 			"name" : "tushark",
-	 			"roles" : ["jmxReader"]
-	 		}
-		]
+  "roles": [
+    {
+      "name": "jmxReader",
+      "operationsAllowed": [
+        "QUERY:EXECUTE"
+      ]
+    }
+  ],
+  "users": [
+    {
+      "name": "tushark",
+      "roles": [
+        "jmxReader"
+      ]
+    }
+  ]
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/auth3.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/auth3.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/auth3.json
index b7e6ee0..cfd43f5 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/auth3.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/auth3.json
@@ -3,13 +3,13 @@
     {
       "name": "dataUsers",
       "operationsAllowed": [
-        "GET"
+        "REGION:GET"
       ]
     },
     {
       "name": "secureDataUsers",
       "operationsAllowed": [
-        "GET"
+        "REGION:GET"
       ],
       "region": "secureRegion"
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
index 2b0466c..deefd97 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
@@ -3,22 +3,19 @@
     {
       "name": "everything",
       "operationsAllowed": [
-        "CREATE",
-        "LIST_DS",
-        "DESTROY",
-        "DESTROY_INDEX",
-        "EXECUTE",
-        "LOCATE_ENTRY",
-        "QUERY",
-        "STOP",
-        "STOP_CONTINUOUS_QUERY",
-        "SHUTDOWN",
-        "COMPACT",
-        "COMPACT_DISKSTORE",
-        "CREATE_MANAGER",
-        "CREATE_REGION",
-        "PROCESS_COMMAND",
-        "STATUS"
+        "DEFAULT:LIST_DS",
+        "DISTRIBUTED_SYSTEM:LIST_DS",
+        "INDEX:DESTROY",
+        "QUERY:EXECUTE",
+        "DEFAULT:LOCATE_ENTRY",
+        "CONTINUOUS_QUERY:EXECUTE",
+        "CONTINUOUS_QUERY:STOP",
+        "MEMBER:SHUTDOWN",
+        "DISKSTORE:COMPACT",
+        "MANAGER:CREATE",
+        "REGION:CREATE",
+        "MEMBER:PROCESS_COMMAND",
+        "MEMBER:STATUS"
       ]
     },
     {
@@ -29,7 +26,7 @@
     {
       "name": "something",
       "operationsAllowed": [
-        "LIST_DS"
+        "DISTRIBUTED_SYSTEM:LIST_DS"
       ]
     }
   ],

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testSimpleUserAndRole.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testSimpleUserAndRole.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testSimpleUserAndRole.json
index 3f81f2d..0542cf4 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testSimpleUserAndRole.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testSimpleUserAndRole.json
@@ -1,14 +1,18 @@
 {
-"roles" : [	
-			{
-				"name" : "jmxReader",
-				"operationsAllowed" : ["QUERY"]				
-			}
-		],
-users : [
-	 		{
-	 			"name" : "tushark",
-	 			"roles" : ["jmxReader"]
-	 		}
-		]
+  "roles": [
+    {
+      "name": "jmxReader",
+      "operationsAllowed": [
+        "QUERY:EXECUTE"
+      ]
+    }
+  ],
+  "users": [
+    {
+      "name": "tushark",
+      "roles": [
+        "jmxReader"
+      ]
+    }
+  ]
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testUserAndRoleRegionServerGroup.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testUserAndRoleRegionServerGroup.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testUserAndRoleRegionServerGroup.json
index 53f4987..6bb28bf 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testUserAndRoleRegionServerGroup.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testUserAndRoleRegionServerGroup.json
@@ -1,16 +1,20 @@
 {
-"roles" : [	
-			{
-				"name" : "jmxReader",
-				"operationsAllowed" : ["QUERY"],
-				"serverGroup" : "SG2",
-				"region" : "secureRegion"
-			}
-		],
-users : [
-	 		{
-	 			"name" : "tushark",
-	 			"roles" : ["jmxReader"]
-	 		}
-		]
+  "roles": [
+    {
+      "name": "jmxReader",
+      "operationsAllowed": [
+        "QUERY:EXECUTE"
+      ],
+      "serverGroup": "SG2",
+      "region": "secureRegion"
+    }
+  ],
+  "users": [
+    {
+      "name": "tushark",
+      "roles": [
+        "jmxReader"
+      ]
+    }
+  ]
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3040e5af/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testUserMultipleRole.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testUserMultipleRole.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testUserMultipleRole.json
index a0f3c90..7a07a21 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testUserMultipleRole.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/testUserMultipleRole.json
@@ -1,20 +1,26 @@
 {
-"roles" : [	
-			{
-				"name" : "jmxReader",
-				"operationsAllowed" : ["QUERY"]				
-			},
-			{
-				"name" : "sysMonitors",
-				"operationsAllowed" : [
-					"CMD_EXPORT_LOGS","CMD_STACK_TRACES","CMD_GC","CMD_NETSTAT","CMD_SHOW_DEADLOCKS", "CMD_SHOW_LOG", "SHOW_METRICS"
-				]
-			}
-		],
-users : [
-	 		{
-	 			"name" : "tushark",
-	 			"roles" : ["jmxReader", "sysMonitors"]
-	 		}
-		]
+  "roles": [
+    {
+      "name": "jmxReader",
+      "operationsAllowed": [
+        "QUERY:EXECUTE"
+      ]
+    },
+    {
+      "name": "sysMonitors",
+      "operationsAllowed": [
+        "MEMBER:EXPORT_LOGS",
+        "MEMBER:GC"
+      ]
+    }
+  ],
+  "users": [
+    {
+      "name": "tushark",
+      "roles": [
+        "jmxReader",
+        "sysMonitors"
+      ]
+    }
+  ]
 }