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/14 22:02:03 UTC

incubator-geode git commit: GEODE-17: Initial CLIOperationContext tests

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-17-2 c55aa9501 -> 5669aefdd


GEODE-17: Initial CLIOperationContext tests


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 5669aefddf14706d4a35fea79eef46e459c7d931
Parents: c55aa95
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon Mar 14 14:01:27 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Mar 14 14:01:27 2016 -0700

----------------------------------------------------------------------
 .../cache/operations/OperationContext.java      |   2 +-
 .../internal/security/MBeanServerWrapper.java   |   6 +
 .../management/internal/security/Resource.java  |   2 -
 ...rDistributedSystemMXBeanIntegrationTest.java |  47 -------
 ...erationForRegionCommandsIntegrationTest.java | 123 -------------------
 .../security/DataCommandsSecurityTest.java      |  62 ++++++++++
 .../internal/security/JSONAuthorization.java    | 100 +++++++--------
 ...JSONAuthorizationDetailsIntegrationTest.java |  14 ++-
 .../security/MemberMBeanSecurityJUnitTest.java  |   1 -
 .../internal/security/cacheServer.json          |  31 +++++
 10 files changed, 158 insertions(+), 230 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5669aefd/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 f950007..f900796 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
@@ -336,7 +336,7 @@ public abstract class OperationContext {
   public abstract OperationCode getOperationCode();
 
   public Resource getResource(){
-    return Resource.DEFAULT;
+    return Resource.CLIENT_SERVER;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5669aefd/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 0cbd23f..ee52070 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
@@ -215,6 +215,12 @@ public class MBeanServerWrapper implements MBeanServerForwarder {
       throws InstanceNotFoundException, MBeanException, ReflectionException {
     ResourceOperationContext ctx = getOperationContext(name, operationName, true);
     doAuthorization(ctx);
+    // further authorize the processCommand call
+    if("processCommand".equals(operationName) && params.length==1){
+      CLIOperationContext cliContext = new CLIOperationContext((String)params[0]);
+      doAuthorization(cliContext);
+    }
+
     Object result = mbs.invoke(name, operationName, params, signature);
     if(ctx!=null)
       ctx.setPostOperationResult(result);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5669aefd/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 75120d0..4570501 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
@@ -22,8 +22,6 @@ public enum Resource {
   CLIENT_SERVER,
   CLUSTER_CONFIGURTION,
   CONTINUOUS_QUERY,
-  DEFAULT,
-  MBEAN,
   DISKSTORE,
   DISKSTORE_OFFLINE,
   DISTRIBUTED_SYSTEM,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5669aefd/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
deleted file mode 100755
index 3859283..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AuthorizeOperationForDistributedSystemMXBeanIntegrationTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.management.internal.security;
-
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-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.
- */
-@Category(IntegrationTest.class)
-public class AuthorizeOperationForDistributedSystemMXBeanIntegrationTest {
-
-  @Test
-  public void returnsFalseForUnauthorizedUser() throws Exception {
-    JSONAuthorization authorization = new JSONAuthorization("auth1.json");
-    authorization.init(new JMXPrincipal("tushark"), null, null);
-
-    ResourceOperationContext context = new ResourceOperationContext("QUERY", "EXECUTE");
-    boolean result = authorization.authorizeOperation(null, context);
-    assertTrue(result);
-
-    context = new ResourceOperationContext("REGION", "CREATE");
-    result = authorization.authorizeOperation(null, context);
-    assertFalse(result);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5669aefd/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AuthorizeOperationForRegionCommandsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AuthorizeOperationForRegionCommandsIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AuthorizeOperationForRegionCommandsIntegrationTest.java
deleted file mode 100644
index 61bf767..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AuthorizeOperationForRegionCommandsIntegrationTest.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.management.internal.security;
-
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.contrib.java.lang.system.RestoreSystemProperties;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-import javax.management.remote.JMXPrincipal;
-import java.lang.management.ManagementFactory;
-import java.util.Properties;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests <code>JSONAuthorization.authorizeOperation(...)</code> for Region commands.
- */
-@Category(IntegrationTest.class)
-@SuppressWarnings("deprecation")
-public class AuthorizeOperationForRegionCommandsIntegrationTest {
-
-  private GemFireCacheImpl cache;
-  private DistributedSystem ds;
-  private int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-
-  @Rule
-  public TestName testName = new TestName();
-
-  @Rule
-  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
-
-  @Before
-  public void setUp() {
-    Properties properties = new Properties();
-    properties.put("name", testName.getMethodName());
-    properties.put(DistributionConfig.LOCATORS_NAME, "");
-    properties.put(DistributionConfig.MCAST_PORT_NAME, "0");
-    properties.put(DistributionConfig.JMX_MANAGER_NAME, "true");
-    properties.put(DistributionConfig.JMX_MANAGER_START_NAME, "true");
-    properties.put(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(this.jmxManagerPort));
-    properties.put(DistributionConfig.HTTP_SERVICE_PORT_NAME, "0");
-    properties.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
-    properties.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME,
-        JSONAuthorization.class.getName() + ".create");
-
-    this.ds = DistributedSystem.connect(properties);
-    this.cache = (GemFireCacheImpl) CacheFactory.create(ds);
-  }
-
-  @After
-  public void tearDown() {
-    if (cache != null) {
-      cache.close();
-      cache = null;
-    }
-    if (ds != null) {
-      ds.disconnect();
-      ds = null;
-    }
-  }
-
-  @Test
-  public void testAuthorizeOperationWithRegionOperations() throws Exception {
-    JSONAuthorization authorization = new JSONAuthorization("auth3.json");
-    authorization.init(new JMXPrincipal("tushark"), null, null);
-
-    checkAccessControlMBean();
-
-    CLIOperationContext cliContext = new CLIOperationContext("locate entry --key=k1 --region=region1");
-    boolean result = authorization.authorizeOperation(null, cliContext);
-    assertTrue("Operation not authorized", result);
-
-    cliContext = new CLIOperationContext("locate entry --key=k1 --region=secureRegion");
-    result = authorization.authorizeOperation(null, cliContext);
-    assertTrue("Operation not authorized", result);
-
-    authorization.init(new JMXPrincipal("avinash"), null, null);
-    result = authorization.authorizeOperation(null, cliContext);
-    assertTrue(result);
-
-    cliContext = new CLIOperationContext("locate entry --key=k1 --region=region1");
-    result = authorization.authorizeOperation(null, cliContext);
-    assertTrue(result);
-  }
-
-  private void checkAccessControlMBean() throws Exception {
-    ObjectName name = new ObjectName(ManagementInterceptor.OBJECT_NAME_ACCESSCONTROL);
-    MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
-    Set<ObjectName> names = platformMBeanServer.queryNames(name, null);
-    assertFalse(names.isEmpty());
-    assertEquals(1, names.size());
-    assertEquals(name, names.iterator().next());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5669aefd/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
new file mode 100644
index 0000000..ab42e58
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.management.internal.security;
+
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.management.MemberMXBean;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+@Category(IntegrationTest.class)
+public class DataCommandsSecurityTest {
+  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+
+  private MemberMXBean bean;
+
+  @ClassRule
+  public static JsonAuthorizationCacheStartRule serverRule = new JsonAuthorizationCacheStartRule(
+      jmxManagerPort, "cacheServer.json");
+
+  @Rule
+  public MBeanServerConnectionRule connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
+
+  @Before
+  public void setUp() throws Exception {
+    bean = connectionRule.getProxyMBean(MemberMXBean.class);
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "dataUser", password = "1234567")
+  public void testDataUser() throws Exception {
+    bean.processCommand("locate entry --key=k1 --region=region1");
+    assertThatThrownBy(() -> bean.processCommand("locate entry --key=k1 --region=secureRegion")).isInstanceOf(SecurityException.class);
+  }
+
+  @JMXConnectionConfiguration(user = "secureDataUser", password = "1234567")
+  @Test
+  public void testSecureDataUser(){
+    bean.processCommand("locate entry --key=k1 --region=region1");
+    bean.processCommand("locate entry --key=k1 --region=secureRegion");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5669aefd/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 7422d27..40497f7 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
@@ -50,12 +50,10 @@ public class JSONAuthorization implements AccessControl, Authenticator {
 
     private final Resource resource;
     private final OperationCode operationCode;
-    private final String region;
 
-    Permission(Resource resource, OperationCode operationCode, String region) {
+    Permission(Resource resource, OperationCode operationCode) {
       this.resource = resource;
       this.operationCode = operationCode;
-      this.region = region;
     }
 
     public Resource getResource() {
@@ -66,14 +64,9 @@ public class JSONAuthorization implements AccessControl, Authenticator {
       return operationCode;
     }
 
-    public String getRegion() {
-      return region;
-    }
-
     @Override
     public String toString() {
       String result = resource.toString() + ":" + operationCode.toString();
-      result += (region != null) ? "[" + region + "]" : "";
       return result;
     }
   }
@@ -81,13 +74,13 @@ public class JSONAuthorization implements AccessControl, Authenticator {
   public static class Role {
     List<Permission> permissions = new ArrayList<>();
     String name;
-    String regionName;
+    List<String> regionNames = null; // when checking, if regionNames is null, that means all regions are allowed.
     String serverGroup;
   }
 
   public static class User {
     String name;
-    Set<Permission> permissions = new HashSet<>();
+    Set<Role> roles = new HashSet<>();
     String pwd;
   }
 
@@ -132,8 +125,7 @@ public class JSONAuthorization implements AccessControl, Authenticator {
       JSONArray ops = obj.getJSONArray("roles");
       for (int j = 0; j < ops.length(); j++) {
         String roleName = ops.getString(j);
-
-        user.permissions.addAll(roleMap.get(roleName).permissions);
+        user.roles.add(roleMap.get(roleName));
       }
       acl.put(user.name, user);
     }
@@ -146,30 +138,33 @@ public class JSONAuthorization implements AccessControl, Authenticator {
       JSONObject obj = array.getJSONObject(i);
       Role role = new Role();
       role.name = obj.getString("name");
+      JSONArray ops = obj.getJSONArray("operationsAllowed");
+      for (int j = 0; j < ops.length(); 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));
+      }
 
-      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");
-        for (int j = 0; j < ops.length(); 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));
+      if(obj.has("region")) {
+        if (role.regionNames == null) {
+          role.regionNames = new ArrayList<>();
         }
-      } else {
-        if (!obj.has("inherit")) {
-          throw new RuntimeException(
-              "Role " + role.name + " does not have any permission neither it inherits any parent role");
+        role.regionNames.add(obj.getString("region"));
+      }
+
+      if(obj.has("regions")) {
+        JSONArray regions = obj.getJSONArray("regions");
+        if (role.regionNames == null) {
+          role.regionNames = new ArrayList<>();
+        }
+        for (int j = 0; j < regions.length(); j++) {
+          role.regionNames.add(regions.getString(j));
         }
       }
 
       roleMap.put(role.name, role);
 
-      if (obj.has("region")) {
-        role.regionName = obj.getString("region");
-      }
-
       if (obj.has("serverGroup")) {
         role.serverGroup = obj.getString("serverGroup");
       }
@@ -191,34 +186,39 @@ public class JSONAuthorization implements AccessControl, Authenticator {
 
   @Override
   public boolean authorizeOperation(String arg0, OperationContext context) {
+    if (principal == null)
+      return false;
 
-    if (principal != null) {
-      User user = acl.get(principal.getName());
-      if (user != null) {
-        LogService.getLogger().info("Context received " + context);
-        ResourceOperationContext ctx = (ResourceOperationContext) context;
-        LogService.getLogger().info("Checking for permission " + ctx.getResource() + ":" + ctx.getOperationCode());
+    User user = acl.get(principal.getName());
+    if(user == null)
+      return false; // this user is not authorized to do anything
 
-        //TODO : This is for un-annotated commands
-        if (ctx.getOperationCode() == null) {
-          return true;
-        }
+    LogService.getLogger().info("Context received " + context);
+    LogService.getLogger().info("Checking for permission " + context.getResource() + ":" + context.getOperationCode());
+
+    // check if the user has this permission defined in the context
+    for(Role role:acl.get(user.name).roles) {
+      for (Permission perm : role.permissions) {
+        if (context.getResource() == perm.getResource() && context.getOperationCode() == perm.getOperationCode()) {
+          LogService.getLogger().info("Found permission " + perm);
 
-        boolean found = false;
-        for (Permission perm : acl.get(user.name).permissions) {
-          if (ctx.getResource() == perm.getResource() && ctx.getOperationCode() == perm.getOperationCode()) {
-            found = true;
-            LogService.getLogger().info("Found permission " + perm);
-            break;
+          //if this is only for JMX aurthorization, we've found the permission needed, i.e, this operation is authorized
+          if(!(context instanceof CLIOperationContext)){
+            return true;
+          }
+
+          // If this is a Command operation context, we need to further check if the region is allowed in this role
+          CLIOperationContext ctx = (CLIOperationContext) context;
+          String region = ctx.getCommandOptions().get("region");
+          if(role.regionNames == null || role.regionNames.contains(region)){
+            // if regionName is null, i.e. all regions are allowed
+            return true;
           }
         }
-        if (found) {
-          return true;
-        }
-        LogService.getLogger().info("Did not find code " + ctx.getOperationCode());
-        return false;
       }
     }
+
+    LogService.getLogger().info("Did not find code " + context.getOperationCode());
     return false;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5669aefd/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 acc11ef..03e26fa 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
@@ -40,8 +40,10 @@ public class JSONAuthorizationDetailsIntegrationTest {
     assertEquals(1, acl.size());
     User user = acl.get("tushark");
     assertNotNull(user);
-    assertEquals(1, user.permissions.size());
-    JSONAuthorization.Permission p = user.permissions.iterator().next();
+    assertEquals(1, user.roles.size());
+    JSONAuthorization.Role r  = user.roles.iterator().next();
+    assertEquals(1, r.permissions.size());
+    JSONAuthorization.Permission p = r.permissions.get(0);
     assertEquals("QUERY:EXECUTE", p.toString());
   }
 
@@ -54,9 +56,9 @@ public class JSONAuthorizationDetailsIntegrationTest {
     assertEquals(1, acl.size());
     User user = acl.get("tushark");
     assertNotNull(user);
-    assertEquals(1, user.permissions.size());
-    JSONAuthorization.Permission p = user.permissions.iterator().next();
-    assertEquals("secureRegion", p.getRegion());
+    assertEquals(1, user.roles.size());
+    JSONAuthorization.Role r  = user.roles.iterator().next();
+    assertEquals("secureRegion", r.regionNames.get(0));
   }
 
   @Test
@@ -68,6 +70,6 @@ public class JSONAuthorizationDetailsIntegrationTest {
     assertEquals(1, acl.size());
     User user = acl.get("tushark");
     assertNotNull(user);
-    assertEquals(3, user.permissions.size());
+    assertEquals(2, user.roles.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5669aefd/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 a4177e6..b310d2d 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
@@ -68,5 +68,4 @@ public class MemberMBeanSecurityJUnitTest {
   public void testNoAccess() throws Exception {
     assertThatThrownBy(() -> bean.shutDownMember()).isInstanceOf(SecurityException.class);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5669aefd/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 deefd97..3385403 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
@@ -28,6 +28,22 @@
       "operationsAllowed": [
         "DISTRIBUTED_SYSTEM:LIST_DS"
       ]
+    },
+    {
+      "name": "dataUsers",
+      "operationsAllowed": [
+        "REGION:GET",
+        "MEMBER:PROCESS_COMMAND"
+      ],
+      "region": "region1"
+    },
+    {
+      "name": "secureDataUsers",
+      "operationsAllowed": [
+        "REGION:GET",
+        "MEMBER:PROCESS_COMMAND"
+      ],
+      "regions": ["region1", "secureRegion"]
     }
   ],
   "users": [
@@ -51,6 +67,21 @@
       "roles": [
         "something"
       ]
+    },
+    {
+      "name": "dataUser",
+      "password": "1234567",
+      "roles": [
+        "dataUsers"
+      ]
+    },
+    {
+      "name": "secureDataUser",
+      "password": "1234567",
+      "roles": [
+        "secureDataUsers",
+        "dataUsers"
+      ]
     }
   ]
 }