You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2016/05/20 17:11:14 UTC

[49/50] [abbrv] incubator-geode git commit: GEODE-17: add regionName in some of the commands authorization

GEODE-17: add regionName in some of the commands authorization


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

Branch: refs/heads/feature/GEODE-835
Commit: 8a3c351abd971ba740e77364a431aee7d5cc5b10
Parents: f05e87d
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Thu May 19 10:39:16 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri May 20 09:36:10 2016 -0700

----------------------------------------------------------------------
 .../internal/security/GeodeSecurityUtil.java    |   4 +
 .../CreateAlterDestroyRegionCommands.java       |   5 +-
 .../internal/cli/commands/DataCommands.java     |   1 -
 .../internal/cli/commands/IndexCommands.java    |  32 +-
 .../cli/functions/DataCommandFunction.java      |  11 +-
 .../support/MemberMXBeanAdapter.java            | 654 -------------------
 .../security/CliCommandsSecurityTest.java       |  24 +-
 .../security/GfshCommandsSecurityTest.java      |  22 +-
 .../internal/security/TestCommand.java          |   9 +-
 .../internal/security/cacheServer.json          |  42 ++
 10 files changed, 118 insertions(+), 686 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
index 322c59e..4eafada 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
@@ -173,6 +173,10 @@ public class GeodeSecurityUtil {
     authorize("DATA", "READ");
   }
 
+  public static void authorizeRegionManage(String regionName){
+    authorize("DATA", "MANAGE", regionName);
+  }
+
   public static void authorizeRegionWrite(String regionName){
     authorize("DATA", "WRITE", regionName);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
index 06c096f..1e19861 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
@@ -52,6 +52,7 @@ import com.gemstone.gemfire.internal.ClassPathLoader;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.DistributedRegionMXBean;
 import com.gemstone.gemfire.management.DistributedSystemMXBean;
 import com.gemstone.gemfire.management.ManagementService;
@@ -437,7 +438,6 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
   
   @CliCommand (value = CliStrings.ALTER_REGION, help = CliStrings.ALTER_REGION__HELP)
   @CliMetaData (relatedTopic = CliStrings.TOPIC_GEMFIRE_REGION, writesToSharedConfiguration = true)
-  @ResourceOperation(resource=Resource.DATA, operation = OperationCode.MANAGE)
   public Result alterRegion(
       @CliOption (key = CliStrings.ALTER_REGION__REGION,
                   mandatory = true,
@@ -531,6 +531,8 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     Result result = null;
     XmlEntity xmlEntity = null;
 
+    GeodeSecurityUtil.authorizeRegionManage(regionPath);
+
     try {
       Cache cache = CacheFactory.getAnyInstance();
 
@@ -1003,6 +1005,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
           mandatory = true,
           help = CliStrings.DESTROY_REGION__REGION__HELP)
       String regionPath) {
+
     if (regionPath == null) {
       return ResultBuilder.createInfoResult(CliStrings.DESTROY_REGION__MSG__SPECIFY_REGIONPATH_TO_DESTROY);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
index fafea9a..778089e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
@@ -1173,7 +1173,6 @@ public class DataCommands implements CommandMarker {
       CliStrings.TOPIC_GEMFIRE_DATA, CliStrings.TOPIC_GEMFIRE_REGION })
   @MultiStepCommand
   @CliCommand(value = { CliStrings.QUERY }, help = CliStrings.QUERY__HELP)
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.READ)
   public Object query(
       @CliOption(key = CliStrings.QUERY__QUERY, help = CliStrings.QUERY__QUERY__HELP, mandatory = true) final String query,
       @CliOption(key = CliStrings.QUERY__STEPNAME, mandatory = false, help = "Step name", unspecifiedDefaultValue = CliStrings.QUERY__STEPNAME__DEFAULTVALUE) String stepName,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
index 24f9f71..b863737 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
@@ -16,6 +16,17 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.cache.operations.OperationContext.*;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -26,6 +37,7 @@ import com.gemstone.gemfire.cache.execute.ResultCollector;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.cache.execute.AbstractExecution;
 import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.cli.CliMetaData;
 import com.gemstone.gemfire.management.cli.ConverterHint;
 import com.gemstone.gemfire.management.cli.Result;
@@ -47,22 +59,11 @@ import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.management.internal.configuration.SharedConfigurationWriter;
 import com.gemstone.gemfire.management.internal.configuration.domain.XmlEntity;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
+
 import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
-import static com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
-
 /**
  * The IndexCommands class encapsulates all GemFire shell (Gfsh) commands related to indexes defined in GemFire.
  * </p>
@@ -171,7 +172,6 @@ public class IndexCommands extends AbstractCommandsSupport {
 
   @CliCommand(value = CliStrings.CREATE_INDEX, help = CliStrings.CREATE_INDEX__HELP)
   @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_DATA}, writesToSharedConfiguration=true)
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   //TODO : Add optionContext for indexName
   public Result createIndex(
       @CliOption (key = CliStrings.CREATE_INDEX__NAME,
@@ -205,7 +205,8 @@ public class IndexCommands extends AbstractCommandsSupport {
 
     Result result = null;
     XmlEntity xmlEntity = null;
-    
+
+    GeodeSecurityUtil.authorizeRegionManage(regionPath);
     try {
       final Cache cache = CacheFactory.getAnyInstance();
 
@@ -458,7 +459,6 @@ public class IndexCommands extends AbstractCommandsSupport {
 
   @CliCommand(value = CliStrings.DEFINE_INDEX, help = CliStrings.DEFINE_INDEX__HELP)
   @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_DATA}, writesToSharedConfiguration=true)
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   //TODO : Add optionContext for indexName
   public Result defineIndex(
       @CliOption (key = CliStrings.DEFINE_INDEX_NAME,
@@ -482,6 +482,8 @@ public class IndexCommands extends AbstractCommandsSupport {
 
     Result result = null;
     XmlEntity xmlEntity = null;
+
+    GeodeSecurityUtil.authorizeRegionManage(regionPath);
     
     int idxType = IndexInfo.RANGE_INDEX;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
index b4e7340..ace24f2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
@@ -26,8 +26,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -59,6 +57,7 @@ import com.gemstone.gemfire.internal.InternalEntity;
 import com.gemstone.gemfire.internal.NanoTimer;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.CliUtil;
 import com.gemstone.gemfire.management.internal.cli.commands.DataCommands;
@@ -77,6 +76,8 @@ import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.util.JsonUtil;
 import com.gemstone.gemfire.pdx.PdxInstance;
+
+import org.apache.logging.log4j.Logger;
 import org.json.JSONArray;
 
 /***
@@ -912,6 +913,12 @@ public class DataCommandFunction extends FunctionAdapter implements  InternalEnt
         CompiledValue compiledQuery = compiler.compileQuery(query);
         Set<String> regions = new HashSet<String>();
         compiledQuery.getRegionsInQuery(regions, null);
+
+        // authorize data read on these regions
+        for(String region:regions){
+          GeodeSecurityUtil.authorizeRegionRead(region);
+        }
+
         regionsInQuery = Collections.unmodifiableSet(regions);
         if (regionsInQuery.size() > 0) {
           Set<DistributedMember> members = DataCommands.getQueryRegionsAssociatedMembers(regionsInQuery, cache, false);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java
deleted file mode 100644
index a3d4cd0..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java
+++ /dev/null
@@ -1,654 +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.web.controllers.support;
-
-import java.util.Map;
-
-import com.gemstone.gemfire.management.GemFireProperties;
-import com.gemstone.gemfire.management.JVMMetrics;
-import com.gemstone.gemfire.management.MemberMXBean;
-import com.gemstone.gemfire.management.OSMetrics;
-
-/**
- * The MemberMXBeanAdapter class is an abstract adapter class to the MemberMXBean interface.
- * <p/>
- * @see com.gemstone.gemfire.management.MemberMXBean
- * @since 8.0
- */
-public class MemberMXBeanAdapter implements MemberMXBean {
-
-  @Override
-  public String showLog(final int numberOfLines) {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String viewLicense() { throw new UnsupportedOperationException("Not Implemented!"); }
-
-  @Override
-  public String[] compactAllDiskStores() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean createManager() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public void shutDownMember() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public JVMMetrics showJVMMetrics() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public OSMetrics showOSMetrics() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String processCommand(final String commandString) {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String processCommand(final String commandString, final Map<String, String> env) {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String processCommand(final String commandString, final Map<String, String> env, final Byte[][] binaryData) {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] listDiskStores(final boolean includeRegionOwned) {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-  
-  @Override
-  public GemFireProperties listGemFireProperties() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getHost() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getName() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getId() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getMember() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] getGroups() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getProcessId() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String status() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getVersion() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean isLocator() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getLockTimeout() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getLockLease() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean isServer() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean hasGatewaySender() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean isManager() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean isManagerCreated() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean hasGatewayReceiver() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getClassPath() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getCurrentTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getMemberUpTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getCpuUsage() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  @Deprecated
-  public long getCurrentHeapSize() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  @Deprecated
-  public long getMaximumHeapSize() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  @Deprecated
-  public long getFreeHeapSize() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] fetchJvmThreads() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getFileDescriptorLimit() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getTotalFileDescriptorOpen() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalRegionCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getPartitionRegionCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] listRegions() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] getDiskStores() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] getRootRegionNames() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalRegionEntryCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalBucketCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalPrimaryBucketCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getGetsAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getPutsAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getPutAllAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalMissCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalHitCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getGetsRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getPutsRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getPutAllRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getCreatesRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getDestroysRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getCacheWriterCallsAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getCacheListenerCallsAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalLoadsCompleted() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getLoadsAverageLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalNetLoadsCompleted() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getNetLoadsAverageLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalNetSearchCompleted() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getNetSearchAverageLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalDiskTasksWaiting() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getBytesSentRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getBytesReceivedRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] listConnectedGatewayReceivers() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] listConnectedGatewaySenders() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getNumRunningFunctions() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getFunctionExecutionRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getNumRunningFunctionsHavingResults() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalTransactionsCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getTransactionCommitsAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTransactionCommittedTotalCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTransactionRolledBackTotalCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getTransactionCommitsRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getDiskReadsRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getDiskWritesRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getDiskFlushAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalBackupInProgress() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalBackupCompleted() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getLockWaitsInProgress() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getTotalLockWaitTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalNumberOfLockService() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalNumberOfGrantors() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getLockRequestQueues() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getLruEvictionRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getLruDestroyRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getInitialImagesInProgres() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getInitialImageTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getInitialImageKeysReceived() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getDeserializationAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getDeserializationLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getDeserializationRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getSerializationAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getSerializationLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getSerializationRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getPDXDeserializationRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getPDXDeserializationAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getTotalDiskUsage() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getNumThreads() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public double getLoadAverage() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getGarbageCollectionCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getGarbageCollectionTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getAverageReads() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getAverageWrites() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getJVMPauses() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override 
-    public int getHostCpuUsage() { 
-       throw new UnsupportedOperationException("Not Implemented!"); 
-   }
-
-  @Override
-  public boolean isCacheServer() {
-    throw new UnsupportedOperationException("Not Implemented!"); 
-  }
-
-  @Override
-  public String getRedundancyZone() {
-    throw new UnsupportedOperationException("Not Implemented!"); 
-  }
-
-  @Override
-  public int getRebalancesInProgress() {
-    throw new UnsupportedOperationException("Not Implemented!"); 
-  }
-
-  @Override
-  public int getReplyWaitsInProgress() {
-    throw new UnsupportedOperationException("Not Implemented!"); 
-  }
-
-  @Override
-  public int getReplyWaitsCompleted() {
-    throw new UnsupportedOperationException("Not Implemented!"); 
-  }
-
-  @Override
-  public int getVisibleNodes() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getOffHeapObjects() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getOffHeapMaxMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getOffHeapFreeMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getOffHeapUsedMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getOffHeapFragmentation() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getOffHeapCompactionTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }    
-
-  @Override
-  public long getMaxMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-  
-  @Override
-  public long getFreeMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-  
-  @Override
-  public long getUsedMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CliCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CliCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CliCommandsSecurityTest.java
index 0864e52..3ccd71c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CliCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CliCommandsSecurityTest.java
@@ -16,13 +16,15 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
-import static org.assertj.core.api.AssertionsForClassTypes.*;
+import static org.assertj.core.api.AssertionsForClassTypes.fail;
+import static org.junit.Assert.*;
 
 import java.util.List;
 
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.management.MemberMXBean;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 import org.junit.Before;
@@ -58,14 +60,20 @@ public class CliCommandsSecurityTest {
   @JMXConnectionConfiguration(user = "stranger", password = "1234567")
   public void testNoAccess(){
    for (TestCommand command:commands) {
+     // skip query commands since query commands are only available in client shell
+     if(command.getCommand().startsWith("query"))
+       continue;
       LogService.getLogger().info("processing: "+command.getCommand());
-      // for those commands that don't require any permission, any user can execute them
-      if(command.getPermission()==null){
-        bean.processCommand(command.getCommand());
-      }
-      else {
-        assertThatThrownBy(() -> bean.processCommand(command.getCommand()))
-            .hasMessageContaining(command.getPermission().toString());
+      // for those commands that requires a permission, we expect an exception to be thrown
+      if(command.getPermission()!=null){
+        try{
+          String result = bean.processCommand(command.getCommand());
+          fail(command.getCommand() + " has result: "+ result);
+        }
+        catch(NotAuthorizedException e){
+          assertTrue(e.getMessage()+" should contain "+command.getPermission(),
+            e.getMessage().contains(command.getPermission().toString()));
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
index b21302e..1a15367 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
@@ -110,6 +110,24 @@ public class GfshCommandsSecurityTest {
     runCommandsWithAndWithout("DATA:MANAGE");
   }
 
+  @Test
+  @JMXConnectionConfiguration(user = "regionA-reader", password = "1234567")
+  public void testregionAReader() throws Exception{
+    runCommandsWithAndWithout("DATA:READ:RegionA");
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "regionA-writer", password = "1234567")
+  public void testregionAWriter() throws Exception{
+    runCommandsWithAndWithout("DATA:WRITE:RegionA");
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "regionA-manager", password = "1234567")
+  public void testregionAManager() throws Exception{
+    runCommandsWithAndWithout("DATA:MANAGE:RegionA");
+  }
+
 
   private void runCommandsWithAndWithout(String permission) throws Exception{
     List<TestCommand> allPermitted = TestCommand.getPermittedCommands(new WildcardPermission(permission, true));
@@ -148,7 +166,9 @@ public class GfshCommandsSecurityTest {
       }
 
       assertEquals(ResultBuilder.ERRORCODE_UNAUTHORIZED, ((ErrorResultData) result.getResultData()).getErrorCode());
-      assertTrue(result.getContent().toString().contains(other.getPermission().toString()));
+      String resultMessage = result.getContent().toString();
+      String permString = other.getPermission().toString();
+      assertTrue(resultMessage+" does not contain "+permString,resultMessage.contains(permString));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
index 667330c..4b482a9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
@@ -33,6 +33,7 @@ public class TestCommand {
 
   public static OperationContext regionARead = new ResourceOperationContext("DATA", "READ", "RegionA");
   public static OperationContext regionAWrite = new ResourceOperationContext("DATA", "WRITE", "RegionA");
+  public static OperationContext regionAManage = new ResourceOperationContext("DATA", "MANAGE", "RegionA");
 
   public static OperationContext clusterRead = new ResourceOperationContext("CLUSTER", "READ");
   public static OperationContext clusterWrite = new ResourceOperationContext("CLUSTER", "WRITE");
@@ -91,7 +92,7 @@ public class TestCommand {
     createTestCommand("export config --member=member1", clusterRead);
 
     //CreateAlterDestroyRegionCommands
-    createTestCommand("alter region --name=region1 --eviction-max=5000", dataManage);
+    createTestCommand("alter region --name=RegionA --eviction-max=5000", regionAManage);
     createTestCommand("create region --name=region12 --type=REPLICATE", dataManage);
     createTestCommand("destroy region --name=value", dataManage);
 
@@ -102,7 +103,7 @@ public class TestCommand {
     createTestCommand("put --key=key1 --value=value1 --region=RegionA", regionAWrite);
     createTestCommand("get --key=key1 --region=RegionA", regionARead);
     createTestCommand("remove --region=RegionA", dataManage);
-    createTestCommand("query --query='SELECT * FROM /RegionA'", dataRead);
+    createTestCommand("query --query='SELECT * FROM /RegionA'", regionARead);
     createTestCommand("locate entry --key=k1 --region=RegionA", regionARead);
 
     // Deploy commands
@@ -147,8 +148,8 @@ public class TestCommand {
     //IndexCommands
     createTestCommand("clear defined indexes", dataManage);
     createTestCommand("create defined indexes", dataManage);
-    createTestCommand("create index --name=myKeyIndex --expression=region1.Id --region=region1 --type=key", dataManage);
-    createTestCommand("define index --name=myIndex1 --expression=exp1 --region=/exampleRegion", dataManage);
+    createTestCommand("create index --name=myKeyIndex --expression=region1.Id --region=RegionA --type=key", regionAManage);
+    createTestCommand("define index --name=myIndex1 --expression=exp1 --region=/RegionA", regionAManage);
     createTestCommand("destroy index --member=server2", dataManage);
     createTestCommand("list indexes", clusterRead);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/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 638ae07..fbbda8d 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
@@ -82,6 +82,27 @@
       "regions": "null,region1"
     },
     {
+      "name": "regionA-manage",
+      "operationsAllowed": [
+        "DATA:MANAGE"
+      ],
+      "regions": "RegionA"
+    },
+    {
+      "name": "regionA-write",
+      "operationsAllowed": [
+        "DATA:WRITE"
+      ],
+      "regions": "RegionA"
+    },
+    {
+      "name": "regionA-read",
+      "operationsAllowed": [
+        "DATA:READ"
+      ],
+      "regions": "RegionA"
+    },
+    {
       "name": "secure-use",
       "operationsAllowed": [
         "DATA:READ",
@@ -183,6 +204,27 @@
       "roles": [
         "cluster-manage"
       ]
+    },
+    {
+      "name": "regionA-manager",
+      "password": "1234567",
+      "roles": [
+        "regionA-manage"
+      ]
+    },
+    {
+      "name": "regionA-writer",
+      "password": "1234567",
+      "roles": [
+        "regionA-write"
+      ]
+    },
+    {
+      "name": "regionA-reader",
+      "password": "1234567",
+      "roles": [
+        "regionA-read"
+      ]
     }
   ]
 }