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:03 UTC

[38/50] [abbrv] incubator-geode git commit: GEODE-17: clean up error messages

GEODE-17: clean up error messages

* clean up authentication/authorization error messages
* Catch Authorization exception later in the command chain to avoid unnecesary parsing of command result
* Add ExceptionHandler in controller to set the http header correctly
* Catch Authorization exception in gfsh execution for better error report


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

Branch: refs/heads/feature/GEODE-835
Commit: 14a548ff22d7055f1da75e5432412472113e5e9d
Parents: 21c0e24
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue May 17 14:48:30 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu May 19 10:40:57 2016 -0700

----------------------------------------------------------------------
 .../internal/security/GeodeSecurityUtil.java    |  55 ++---
 .../internal/beans/MemberMBeanBridge.java       |  18 +-
 .../internal/cli/remote/CommandProcessor.java   |   9 +-
 .../internal/cli/result/ResultBuilder.java      |   3 +-
 .../cli/shell/GfshExecutionStrategy.java        | 230 ++++++++++---------
 .../security/ResourceOperationContext.java      |   2 +-
 .../controllers/AbstractCommandsController.java |  88 ++-----
 .../web/shell/AbstractHttpOperationInvoker.java |  16 +-
 .../security/GfshCommandsSecurityTest.java      |  11 +-
 .../security/MemberMBeanSecurityJUnitTest.java  |   2 +-
 .../WanCommandsControllerJUnitTest.java         |   7 +-
 11 files changed, 199 insertions(+), 242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/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 236b00b..322c59e 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
@@ -33,6 +33,7 @@ import com.gemstone.gemfire.management.internal.security.ResourceOperation;
 import com.gemstone.gemfire.management.internal.security.ResourceOperationContext;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.Logger;
@@ -53,31 +54,6 @@ public class GeodeSecurityUtil {
   private static Logger logger = LogService.getLogger();
 
   /**
-   *
-   * @param username
-   * @param password
-   * @return null if security is not enabled, otherwise return a shiro subject
-   */
-  public static Subject login(String username, String password){
-    if(!isSecured())
-      return null;
-
-    Subject currentUser = SecurityUtils.getSubject();
-
-    UsernamePasswordToken token =
-        new UsernamePasswordToken(username, password);
-    try {
-      logger.info("Logging in "+username+"/"+password);
-      currentUser.login(token);
-    } catch (ShiroException e) {
-      logger.info(e.getMessage(), e);
-      throw new AuthenticationFailedException(e.getMessage(), e);
-    }
-
-    return currentUser;
-  }
-
-  /**
    * It first looks the shiro subject in AccessControlContext since JMX will use multiple threads to process operations from the same client.
    * then it looks into Shiro's thead context.
    *
@@ -114,6 +90,31 @@ public class GeodeSecurityUtil {
     return currentUser;
   }
 
+  /**
+   *
+   * @param username
+   * @param password
+   * @return null if security is not enabled, otherwise return a shiro subject
+   */
+  public static Subject login(String username, String password){
+    if(!isSecured())
+      return null;
+
+    Subject currentUser = SecurityUtils.getSubject();
+
+    UsernamePasswordToken token =
+      new UsernamePasswordToken(username, password);
+    try {
+      logger.info("Logging in "+username+"/"+password);
+      currentUser.login(token);
+    } catch (ShiroException e) {
+      logger.info(e.getMessage(), e);
+      throw new AuthenticationFailedException("Authentication error. Please check your username/password.", e);
+    }
+
+    return currentUser;
+  }
+
   public static void logout(){
     Subject currentUser = getSubject();
     if(currentUser==null)
@@ -125,7 +126,7 @@ public class GeodeSecurityUtil {
     }
     catch(ShiroException e){
       logger.info(e.getMessage(), e);
-      throw new AuthenticationFailedException(e.getMessage(), e);
+      throw new GemFireSecurityException(e.getMessage(), e);
     }
     // clean out Shiro's thread local content
     ThreadContext.remove();
@@ -205,7 +206,7 @@ public class GeodeSecurityUtil {
     }
     catch(ShiroException e){
       logger.info(currentUser.getPrincipal() + " not authorized for " + context);
-      throw new GemFireSecurityException(e.getMessage(), e);
+      throw new NotAuthorizedException(e.getMessage(), e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
index 67ad60d..98258e8 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
@@ -123,10 +123,9 @@ import com.gemstone.gemfire.management.internal.cli.CommandResponseBuilder;
 import com.gemstone.gemfire.management.internal.cli.remote.CommandExecutionContext;
 import com.gemstone.gemfire.management.internal.cli.remote.MemberCommandService;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
-import com.gemstone.gemfire.management.internal.cli.result.ErrorResultData;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
-import com.gemstone.gemfire.security.GemFireSecurityException;
+
 import org.apache.logging.log4j.Logger;
 
 /**
@@ -1738,17 +1737,8 @@ public class MemberMBeanBridge {
     }
 
     if (isGfshRequest) {
-      String responseJson = CommandResponseBuilder.createCommandResponseJson(getMember(), (CommandResult) result);
-  //    System.out.println("responseJson :: "+responseJson);
-      return responseJson;
+      return CommandResponseBuilder.createCommandResponseJson(getMember(), (CommandResult) result);
     } else {
-      // throw GemFireSecurityException is the returned error code is 415
-      if(((CommandResult) result).getResultData() instanceof ErrorResultData){
-        ErrorResultData resultData = (ErrorResultData) ((CommandResult)result).getResultData();
-        if(resultData.getErrorCode()==ResultBuilder.ERRORCODE_UNAUTHORIZED){
-          throw new GemFireSecurityException(resultData.getGfJsonObject().toString());
-        }
-      }
       return ResultBuilder.resultAsString(result);
     }
   }
@@ -1758,14 +1748,12 @@ public class MemberMBeanBridge {
     if (env != null) {
       appName = env.get(Gfsh.ENV_APP_NAME);
     }
-//    System.out.println("appName :: "+appName);
     
     return Gfsh.GFSH_APP_NAME.equals(appName);
   }
   
   public long getTotalDiskUsage() {
-    long diskSpaceUsage = regionMonitor.getDiskSpace();
-    return diskSpaceUsage;
+    return regionMonitor.getDiskSpace();
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
index 7edc3e4..69bc64e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
@@ -21,6 +21,7 @@ import java.lang.reflect.Method;
 import java.util.Map;
 import java.util.Properties;
 
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.cli.CommandProcessingException;
 import com.gemstone.gemfire.management.cli.CommandStatement;
 import com.gemstone.gemfire.management.cli.Result;
@@ -30,8 +31,7 @@ import com.gemstone.gemfire.management.internal.cli.LogWrapper;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.util.CommentSkipHelper;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
-import com.gemstone.gemfire.security.GemFireSecurityException;
-import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 
 import org.springframework.shell.core.Parser;
 import org.springframework.shell.event.ParseResult;
@@ -126,12 +126,13 @@ public class CommandProcessor {
           logWrapper.info("Could not parse \""+cmdStmt.getCommandString()+"\".", e);
         }
         return ResultBuilder.createParsingErrorResult(e.getMessage());
-      } catch (GemFireSecurityException e) {
+      } catch (NotAuthorizedException e) {
         setLastExecutionStatus(1);
         if (logWrapper.infoEnabled()) {
           logWrapper.info("Could not execute \""+cmdStmt.getCommandString()+"\".", e);
         }
-        return ResultBuilder.createGemFireUnAuthorizedErrorResult("Unauthorized while processing command <" +cmdStmt.getCommandString()+"> Reason : " + e.getMessage());
+        // for NotAuthorizedException, will catch this later in the code
+        throw e;
       }catch (RuntimeException e) {
         setLastExecutionStatus(1);
         if (logWrapper.infoEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
index 6b435d3..55f7a89 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
@@ -116,8 +116,7 @@ public class ResultBuilder {
   }
 
   public static Result createGemFireUnAuthorizedErrorResult(String message) {
-    return createErrorResult(ERRORCODE_UNAUTHORIZED,
-        "Could not process command due to GemFire error. " + message);
+    return createErrorResult(ERRORCODE_UNAUTHORIZED, message);
   }
 
   public static Result createUserErrorResult(String message) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategy.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategy.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategy.java
index c5ebe9a..0cfae9c 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategy.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategy.java
@@ -16,17 +16,11 @@
  */
 package com.gemstone.gemfire.management.internal.cli.shell;
 
-import static com.gemstone.gemfire.management.internal.cli.multistep.CLIMultiStepHelper.execCLISteps;
+import static com.gemstone.gemfire.management.internal.cli.multistep.CLIMultiStepHelper.*;
 
 import java.lang.reflect.Method;
 import java.util.Map;
 
-import org.springframework.shell.core.ExecutionStrategy;
-import org.springframework.shell.core.Shell;
-import org.springframework.shell.event.ParseResult;
-import org.springframework.util.Assert;
-import org.springframework.util.ReflectionUtils;
-
 import com.gemstone.gemfire.internal.ClassPathLoader;
 import com.gemstone.gemfire.management.cli.CliMetaData;
 import com.gemstone.gemfire.management.cli.CommandProcessingException;
@@ -42,6 +36,13 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.multistep.MultiStepCommand;
 import com.gemstone.gemfire.management.internal.cli.result.FileResult;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
+import com.gemstone.gemfire.security.NotAuthorizedException;
+
+import org.springframework.shell.core.ExecutionStrategy;
+import org.springframework.shell.core.Shell;
+import org.springframework.shell.event.ParseResult;
+import org.springframework.util.Assert;
+import org.springframework.util.ReflectionUtils;
 
 /**
  * Defines the {@link ExecutionStrategy} for commands that are executed in
@@ -81,43 +82,40 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
     Object result = null;
     Method method = parseResult.getMethod();
     try {
-      
-    //Check if it's a multi-step command
+      //Check if it's a multi-step command
       Method reflectmethod = parseResult.getMethod();
-      MultiStepCommand cmd = reflectmethod.getAnnotation(MultiStepCommand.class);      
-      if(cmd!=null){
-        return execCLISteps(logWrapper, shell,parseResult);
+      MultiStepCommand cmd = reflectmethod.getAnnotation(MultiStepCommand.class);
+      if (cmd != null) {
+        return execCLISteps(logWrapper, shell, parseResult);
       }
 
-//      See #46072 
-//      String commandName = getCommandName(parseResult);
-//      if (commandName != null) {
-//        shell.flashMessage("Executing " + getCommandName(parseResult) + " ... ");
-//      }
-      //Check if it's a remote command
-      if (!isShellOnly(method)) {
-        if (GfshParseResult.class.isInstance(parseResult)) {
-          result = executeOnRemote((GfshParseResult)parseResult);
-        } else {//Remote command means implemented for Gfsh and ParseResult should be GfshParseResult.
-          //TODO - Abhishek: should this message be more specific?
-          throw new IllegalStateException("Configuration error!");
-        }
-      } else {
+      //check if it's a shell only command
+      if(isShellOnly(method)){
         Assert.notNull(parseResult, "Parse result required");
         synchronized (mutex) {
-          //TODO: Remove Assert
           Assert.isTrue(isReadyForCommands(), "ProcessManagerHostedExecutionStrategy not yet ready for commands");
-          result = ReflectionUtils.invokeMethod(parseResult.getMethod(), parseResult.getInstance(), parseResult.getArguments());
+          return ReflectionUtils.invokeMethod(parseResult.getMethod(), parseResult.getInstance(), parseResult.getArguments());
         }
       }
-//    See #46072
-//      shell.flashMessage("");
-    } catch (JMXInvocationException e) {
+
+      //check if it's a GfshParseResult
+      if(!GfshParseResult.class.isInstance(parseResult)){
+        throw new IllegalStateException("Configuration error!");
+      }
+
+      result = executeOnRemote((GfshParseResult) parseResult);
+    }
+    catch(NotAuthorizedException e) {
+      result = ResultBuilder.createGemFireUnAuthorizedErrorResult("Unauthorized. Reason : " + e.getMessage());
+    }
+    catch (JMXInvocationException e) {
       Gfsh.getCurrentInstance().logWarning(e.getMessage(), e);
-    } catch (IllegalStateException e) {
+    }
+    catch (IllegalStateException e) {
       // Shouldn't occur - we are always using GfsParseResult
       Gfsh.getCurrentInstance().logWarning(e.getMessage(), e);
-    } catch (CommandProcessingException e) {
+    }
+    catch (CommandProcessingException e) {
       Gfsh.getCurrentInstance().logWarning(e.getMessage(), null);
       Object errorData = e.getErrorData();
       if (errorData != null && errorData instanceof Throwable) {
@@ -125,16 +123,17 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
       } else {
         logWrapper.warning(e.getMessage());
       }
-    } catch (RuntimeException e) {
+    }
+    catch (RuntimeException e) {
       Gfsh.getCurrentInstance().logWarning("Exception occurred. " + e.getMessage(), e);
       // Log other runtime exception in gfsh log
       logWrapper.warning("Error occurred while executing command : "+((GfshParseResult)parseResult).getUserInput(), e);
-    } catch (Exception e) {
+    }
+    catch (Exception e) {
       Gfsh.getCurrentInstance().logWarning("Unexpected exception occurred. " + e.getMessage(), e);
       // Log other exceptions in gfsh log
       logWrapper.warning("Unexpected error occurred while executing command : "+((GfshParseResult)parseResult).getUserInput(), e);
     }
-    
     return result;
   }
   
@@ -204,90 +203,97 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
   private Result executeOnRemote(GfshParseResult parseResult) {
     Result   commandResult = null;
     Object   response      = null;
-    if (shell.isConnectedAndReady()) {
-      byte[][]             fileData    = null;
-      CliAroundInterceptor interceptor = null;
-      
-      String interceptorClass = getInterceptor(parseResult.getMethod());
-      
-      //1. Pre Remote Execution
-      if (!CliMetaData.ANNOTATION_NULL_VALUE.equals(interceptorClass)) {
-        try {
-          interceptor = (CliAroundInterceptor) ClassPathLoader.getLatest().forName(interceptorClass).newInstance();
-        } catch (InstantiationException e) {
-          shell.logWarning("Configuration error", e);
-        } catch (IllegalAccessException e) {
-          shell.logWarning("Configuration error", e);
-        } catch (ClassNotFoundException e) {
-          shell.logWarning("Configuration error", e);
-        }
-        if (interceptor != null) {
-          Result preExecResult = interceptor.preExecution(parseResult);
-          if (Status.ERROR.equals(preExecResult.getStatus())) {
-            return preExecResult;
-          } else if (preExecResult instanceof FileResult) {            
-            FileResult fileResult = (FileResult) preExecResult;
-            fileData = fileResult.toBytes();
-          }
-        } else {
-          return ResultBuilder.createBadConfigurationErrorResult("Interceptor Configuration Error");
+
+    if(!shell.isConnectedAndReady()){
+      shell.logWarning("Can't execute a remote command without connection. Use 'connect' first to connect.", null);
+      logWrapper.info("Can't execute a remote command \""+parseResult.getUserInput()+"\" without connection. Use 'connect' first to connect to GemFire.");
+      return null;
+    }
+
+    byte[][]             fileData    = null;
+    CliAroundInterceptor interceptor = null;
+
+    String interceptorClass = getInterceptor(parseResult.getMethod());
+
+    //1. Pre Remote Execution
+    if (!CliMetaData.ANNOTATION_NULL_VALUE.equals(interceptorClass)) {
+      try {
+        interceptor = (CliAroundInterceptor) ClassPathLoader.getLatest().forName(interceptorClass).newInstance();
+      } catch (InstantiationException e) {
+        shell.logWarning("Configuration error", e);
+      } catch (IllegalAccessException e) {
+        shell.logWarning("Configuration error", e);
+      } catch (ClassNotFoundException e) {
+        shell.logWarning("Configuration error", e);
+      }
+      if (interceptor != null) {
+        Result preExecResult = interceptor.preExecution(parseResult);
+        if (Status.ERROR.equals(preExecResult.getStatus())) {
+          return preExecResult;
+        } else if (preExecResult instanceof FileResult) {
+          FileResult fileResult = (FileResult) preExecResult;
+          fileData = fileResult.toBytes();
         }
+      } else {
+        return ResultBuilder.createBadConfigurationErrorResult("Interceptor Configuration Error");
       }
+    }
 
-      //2. Remote Execution
-      final Map<String, String> env = shell.getEnv();
+    //2. Remote Execution
+    final Map<String, String> env = shell.getEnv();
+    try {
       response = shell.getOperationInvoker().processCommand(new CommandRequest(parseResult, env, fileData));
+    } catch(NotAuthorizedException e) {
+      return ResultBuilder.createGemFireUnAuthorizedErrorResult("Unauthorized. Reason : " + e.getMessage());
+    }
+    finally {
       env.clear();
-      
-      if (response == null) {
-        shell.logWarning("Response was null for: \""+parseResult.getUserInput()+"\". (gfsh.isConnected="+shell.isConnectedAndReady()+")", null);
-        commandResult = 
-            ResultBuilder.createBadResponseErrorResult(" Error occurred while " + 
-                "executing \""+parseResult.getUserInput()+"\" on manager. " +
-                		"Please check manager logs for error.");
-      } else {
-        if (logWrapper.fineEnabled()) {
-          logWrapper.fine("Received response :: "+response);
-        }
-        CommandResponse commandResponse = CommandResponseBuilder.prepareCommandResponseFromJson((String) response);
-        
-        if (commandResponse.isFailedToPersist()) {
-          shell.printAsSevere(CliStrings.SHARED_CONFIGURATION_FAILED_TO_PERSIST_COMMAND_CHANGES);
-          logWrapper.severe(CliStrings.SHARED_CONFIGURATION_FAILED_TO_PERSIST_COMMAND_CHANGES);
-        }
-        
-        String debugInfo = commandResponse.getDebugInfo();
-        if (debugInfo != null && !debugInfo.trim().isEmpty()) {
-          //TODO - Abhishek When debug is ON, log response in gfsh logs
-          //TODO - Abhishek handle \n better. Is it coming from GemFire formatter
-          debugInfo = debugInfo.replaceAll("\n\n\n", "\n");
-          debugInfo = debugInfo.replaceAll("\n\n", "\n");
-          debugInfo = debugInfo.replaceAll("\n", "\n[From Manager : "+commandResponse.getSender()+"]");
-          debugInfo = "[From Manager : "+commandResponse.getSender()+"]" + debugInfo;
-          LogWrapper.getInstance().info(debugInfo);
-        }
-        commandResult = ResultBuilder.fromJson((String) response);
-        
-        
-        //3. Post Remote Execution
-        if (interceptor != null) {
-          Result postExecResult = interceptor.postExecution(parseResult, commandResult);
-          if (postExecResult != null) {
-            if (Status.ERROR.equals(postExecResult.getStatus())) {
-              if (logWrapper.infoEnabled()) {
-                logWrapper.info("Post execution Result :: "+ResultBuilder.resultAsString(postExecResult));
-              }
-            } else if (logWrapper.fineEnabled()) {
-              logWrapper.fine("Post execution Result :: "+ResultBuilder.resultAsString(postExecResult));
-            }
-            commandResult = postExecResult;
+    }
+
+    if (response == null) {
+      shell.logWarning("Response was null for: \"" + parseResult.getUserInput() + "\". (gfsh.isConnected=" + shell.isConnectedAndReady() + ")", null);
+      return ResultBuilder.createBadResponseErrorResult(" Error occurred while " +
+        "executing \"" + parseResult.getUserInput() + "\" on manager. " +
+        "Please check manager logs for error.");
+    }
+
+    if (logWrapper.fineEnabled()) {
+      logWrapper.fine("Received response :: "+response);
+    }
+    CommandResponse commandResponse = CommandResponseBuilder.prepareCommandResponseFromJson((String) response);
+
+    if (commandResponse.isFailedToPersist()) {
+      shell.printAsSevere(CliStrings.SHARED_CONFIGURATION_FAILED_TO_PERSIST_COMMAND_CHANGES);
+      logWrapper.severe(CliStrings.SHARED_CONFIGURATION_FAILED_TO_PERSIST_COMMAND_CHANGES);
+    }
+
+    String debugInfo = commandResponse.getDebugInfo();
+    if (debugInfo != null && !debugInfo.trim().isEmpty()) {
+      //TODO - Abhishek When debug is ON, log response in gfsh logs
+      //TODO - Abhishek handle \n better. Is it coming from GemFire formatter
+      debugInfo = debugInfo.replaceAll("\n\n\n", "\n");
+      debugInfo = debugInfo.replaceAll("\n\n", "\n");
+      debugInfo = debugInfo.replaceAll("\n", "\n[From Manager : "+commandResponse.getSender()+"]");
+      debugInfo = "[From Manager : "+commandResponse.getSender()+"]" + debugInfo;
+      LogWrapper.getInstance().info(debugInfo);
+    }
+    commandResult = ResultBuilder.fromJson((String) response);
+
+    //3. Post Remote Execution
+    if (interceptor != null) {
+      Result postExecResult = interceptor.postExecution(parseResult, commandResult);
+      if (postExecResult != null) {
+        if (Status.ERROR.equals(postExecResult.getStatus())) {
+          if (logWrapper.infoEnabled()) {
+            logWrapper.info("Post execution Result :: "+ResultBuilder.resultAsString(postExecResult));
           }
+        } else if (logWrapper.fineEnabled()) {
+          logWrapper.fine("Post execution Result :: "+ResultBuilder.resultAsString(postExecResult));
         }
-      }// not null response
-    } else {
-      shell.logWarning("Can't execute a remote command without connection. Use 'connect' first to connect.", null);
-      logWrapper.info("Can't execute a remote command \""+parseResult.getUserInput()+"\" without connection. Use 'connect' first to connect to GemFire.");
+        commandResult = postExecResult;
+      }
     }
+
     return commandResult;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
index ab49270..580b6c0 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
@@ -46,7 +46,7 @@ public class ResourceOperationContext extends OperationContext {
     //for DATA resource, when we construct the lock to guard the operations, there should always be a 3rd part (regionName),
     // if no regionName is specified, we need to add "NULL" to it.
     // this means, for general data operations, or operations that we can't put a regionName on yet, like backup diskstore, query data, create regions
-    // it will require DATA:REAT/WRITE:NULL role
+    // it will require DATA:READ/WRITE:NULL role
     if(this.resource==Resource.DATA && this.regionName==null){
       this.regionName = "NULL";
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
index c411972..1f6c52a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
@@ -38,6 +38,7 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.internal.util.ArrayUtils;
 import com.gemstone.gemfire.management.DistributedSystemMXBean;
 import com.gemstone.gemfire.management.ManagementService;
@@ -48,9 +49,8 @@ import com.gemstone.gemfire.management.internal.SystemManagementService;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
 import com.gemstone.gemfire.management.internal.web.controllers.support.LoginHandlerInterceptor;
-import com.gemstone.gemfire.management.internal.web.controllers.support.MemberMXBeanAdapter;
 import com.gemstone.gemfire.management.internal.web.util.UriUtils;
-import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 
 import org.apache.logging.log4j.Logger;
 import org.springframework.beans.propertyeditors.StringArrayPropertyEditor;
@@ -425,7 +425,6 @@ public abstract class AbstractCommandsController {
    * 
    * @return a proxy instance to the MemberMXBean of the GemFire Manager.
    * @see #getMBeanServer()
-   * @see #createMemberMXBeanForManagerUsingAdapter(javax.management.MBeanServer, javax.management.ObjectName)
    * @see #createMemberMXBeanForManagerUsingProxy(javax.management.MBeanServer, javax.management.ObjectName)
    * @see com.gemstone.gemfire.management.DistributedSystemMXBean
    * @see com.gemstone.gemfire.management.MemberMXBean
@@ -456,21 +455,6 @@ public abstract class AbstractCommandsController {
   }
 
   /**
-   * Creates an Adapter using the Platform MBeanServer and ObjectName to invoke operations on the GemFire Manager's
-   * MemberMXBean.
-   * 
-   * @param server a reference to this JVM's Platform MBeanServer.
-   * @param managingMemberObjectName the ObjectName of the GemFire Manager's MemberMXBean registered in
-   * the Platform MBeanServer.
-   * @return an Adapter for invoking operations on the GemFire Manager's MemberMXBean.
-   * @see com.gemstone.gemfire.management.internal.web.controllers.AbstractCommandsController.MemberMXBeanProxy
-   * @see #createMemberMXBeanForManagerUsingProxy(javax.management.MBeanServer, javax.management.ObjectName)
-   */
-  private MemberMXBean createMemberMXBeanForManagerUsingAdapter(final MBeanServer server, final ObjectName managingMemberObjectName) {
-    return new MemberMXBeanProxy(server, managingMemberObjectName);
-  }
-
-  /**
    * Creates a Proxy using the Platform MBeanServer and ObjectName in order to access attributes and invoke operations
    * on the GemFire Manager's MemberMXBean.
    * 
@@ -478,7 +462,6 @@ public abstract class AbstractCommandsController {
    * @param managingMemberObjectName the ObjectName of the GemFire Manager's MemberMXBean registered in
    * the Platform MBeanServer.
    * @return a Proxy for accessing attributes and invoking operations on the GemFire Manager's MemberMXBean.
-   * @see #createMemberMXBeanForManagerUsingAdapter(javax.management.MBeanServer, javax.management.ObjectName)
    * @see javax.management.JMX#newMXBeanProxy(javax.management.MBeanServerConnection, javax.management.ObjectName, Class)
    */
   private MemberMXBean createMemberMXBeanForManagerUsingProxy(final MBeanServer server, final ObjectName managingMemberObjectName) {
@@ -538,7 +521,7 @@ public abstract class AbstractCommandsController {
   /**
    * Executes the specified command as entered by the user using the GemFire Shell (Gfsh).  Note, Gfsh performs
    * validation of the command during parsing before sending the command to the Manager for processing.
-   * 
+   *
    * @param command a String value containing a valid command String as would be entered by the user in Gfsh.
    * @return a result of the command execution as a String, typically marshalled in JSON to be serialized back to Gfsh.
    * @see com.gemstone.gemfire.management.internal.cli.shell.Gfsh
@@ -549,14 +532,26 @@ public abstract class AbstractCommandsController {
   protected String processCommand(final String command) {
     return processCommand(command, getEnvironment(), null);
   }
+
   protected Callable<ResponseEntity<String>> getProcessCommandCallable(final String command){
     return getProcessCommandCallable(command, null);
   }
 
   protected Callable<ResponseEntity<String>> getProcessCommandCallable(final String command, final byte[][] fileData){
     Callable callable = new Callable<ResponseEntity<String>>() {
-      @Override public ResponseEntity<String> call() throws Exception {
-        return new ResponseEntity<String>(processCommand(command, fileData), HttpStatus.OK);
+      @Override
+      public ResponseEntity<String> call() throws Exception {
+        String result = null;
+        try {
+          result = processCommand(command, fileData);
+        }
+        catch(NotAuthorizedException ex){
+          return new ResponseEntity<String>(ex.getMessage(), HttpStatus.FORBIDDEN);
+        }
+        catch(Exception ex){
+          return new ResponseEntity<String>(ex.getMessage(), HttpStatus.INTERNAL_SERVER_ERROR);
+        }
+        return new ResponseEntity<String>(result, HttpStatus.OK);
       }
     };
     return GeodeSecurityUtil.associateWith(callable);
@@ -614,52 +609,13 @@ public abstract class AbstractCommandsController {
    */
   protected String processCommand(final String command, final Map<String, String> environment, final byte[][] fileData) {
     logger.info(LogMarker.CONFIG, "Processing Command ({}) with Environment ({}) having File Data ({})...", command,
-        environment, (fileData != null));
-    String result =  getManagingMemberMXBean().processCommand(command, environment, ArrayUtils.toByteArray(fileData));
-
-    return result;
+      environment, (fileData != null));
+    return getManagingMemberMXBean().processCommand(command, environment, ArrayUtils.toByteArray(fileData));
   }
 
-
-  /**
-   * The MemberMXBeanProxy class is a proxy for the MemberMXBean interface transforming an operation on the member
-   * MBean into a invocation on the MBeanServer, invoke method.
-   * 
-   * @see com.gemstone.gemfire.management.internal.web.controllers.support.MemberMXBeanAdapter
-   */
-  private static class MemberMXBeanProxy extends MemberMXBeanAdapter {
-
-    private final MBeanServer server;
-
-    private final ObjectName objectName;
-
-    public MemberMXBeanProxy(final MBeanServer server, final ObjectName objectName) {
-      assertNotNull(server, "The connection or reference to the Platform MBeanServer cannot be null!");
-      assertNotNull(objectName, "The JMX ObjectName for the GemFire Manager MemberMXBean cannot be null!");
-      this.server = server;
-      this.objectName = objectName;
-    }
-
-    protected MBeanServer getMBeanServer() {
-      return server;
-    }
-
-    protected ObjectName getObjectName() {
-      return objectName;
-    }
-
-    @Override
-    public String processCommand(final String commandString, final Map<String, String> env) {
-      try {
-        return String.valueOf(getMBeanServer().invoke(getObjectName(), "processCommand",
-          new Object[] { commandString, env }, new String[] { String.class.getName(), Map.class.getName() }));
-      }
-      catch (Exception e) {
-        throw new RuntimeException(String.format(
-          "An error occurred while executing processCommand with command String (%1$s) on the MemberMXBean (%2$s) of the GemFire Manager using environment (%3$s)!",
-            commandString, getObjectName(), env), e);
-      }
-    }
+  @ExceptionHandler(NotAuthorizedException.class)
+  public ResponseEntity<String> handleAppException(NotAuthorizedException ex) {
+    return new ResponseEntity<String>(ex.getMessage(), HttpStatus.FORBIDDEN);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/shell/AbstractHttpOperationInvoker.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/shell/AbstractHttpOperationInvoker.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/shell/AbstractHttpOperationInvoker.java
index b2159d2..944644f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/shell/AbstractHttpOperationInvoker.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/shell/AbstractHttpOperationInvoker.java
@@ -42,7 +42,6 @@ import com.gemstone.gemfire.management.DistributedSystemMXBean;
 import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
 import com.gemstone.gemfire.management.internal.ManagementConstants;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
-import com.gemstone.gemfire.management.internal.security.ResourceConstants;
 import com.gemstone.gemfire.management.internal.web.domain.Link;
 import com.gemstone.gemfire.management.internal.web.domain.QueryParameterSource;
 import com.gemstone.gemfire.management.internal.web.http.ClientHttpRequest;
@@ -51,6 +50,7 @@ import com.gemstone.gemfire.management.internal.web.http.HttpMethod;
 import com.gemstone.gemfire.management.internal.web.http.converter.SerializableObjectHttpMessageConverter;
 import com.gemstone.gemfire.management.internal.web.shell.support.HttpMBeanProxyFactory;
 import com.gemstone.gemfire.management.internal.web.util.UriUtils;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 
 import org.apache.logging.log4j.Logger;
 import org.springframework.http.HttpStatus;
@@ -206,15 +206,17 @@ public abstract class AbstractHttpOperationInvoker implements HttpOperationInvok
 
       @Override
       public void handleError(final ClientHttpResponse response) throws IOException {
-        final String message = String.format("The HTTP request failed with: %1$d - %2$s", response.getRawStatusCode(),
-          response.getStatusText());
-
-        //gfsh.logSevere(message, null);
+        String body = readBody(response);
+        final String message = String.format("The HTTP request failed with: %1$d - %2$s.", response.getRawStatusCode(), body);
 
         if (gfsh.getDebug()) {
-          gfsh.logSevere(readBody(response), null);
+          gfsh.logSevere(body, null);
         }
-        throw new RuntimeException(message);
+
+        if(response.getRawStatusCode()==403)
+          throw new NotAuthorizedException(message);
+        else
+          throw new RuntimeException(message);
       }
 
       private String readBody(final ClientHttpResponse response) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/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 377ab77..b21302e 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
@@ -112,9 +112,11 @@ public class GfshCommandsSecurityTest {
 
 
   private void runCommandsWithAndWithout(String permission) throws Exception{
-    List<TestCommand> permitted = TestCommand.getPermittedCommands(new WildcardPermission(permission, true));
-    for(TestCommand clusterRead:permitted) {
-      LogService.getLogger().info("Processing authorized command: "+clusterRead.getCommand());gfsh.executeCommand(clusterRead.getCommand());
+    List<TestCommand> allPermitted = TestCommand.getPermittedCommands(new WildcardPermission(permission, true));
+    for(TestCommand permitted:allPermitted) {
+      LogService.getLogger().info("Processing authorized command: "+permitted.getCommand());
+
+      gfsh.executeCommand(permitted.getCommand());
       CommandResult result = (CommandResult) gfsh.getResult();
       assertNotNull(result);
 
@@ -127,7 +129,7 @@ public class GfshCommandsSecurityTest {
     }
 
     List<TestCommand> others = TestCommand.getCommands();
-    others.removeAll(permitted);
+    others.removeAll(allPermitted);
     for(TestCommand other:others) {
       // skip no permission commands
       if(other.getPermission()==null)
@@ -135,6 +137,7 @@ public class GfshCommandsSecurityTest {
 
       LogService.getLogger().info("Processing unauthorized command: "+other.getCommand());
       gfsh.executeCommand(other.getCommand());
+
       CommandResult result = (CommandResult) gfsh.getResult();
       int errorCode = ((ErrorResultData) result.getResultData()).getErrorCode();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/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 8261d09..cabf555 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
@@ -103,7 +103,7 @@ public class MemberMBeanSecurityJUnitTest {
     assertThatThrownBy(() -> bean.isCacheServer()).hasMessageContaining(TestCommand.clusterRead.toString());
     assertThatThrownBy(() -> bean.isServer()).hasMessageContaining(TestCommand.clusterRead.toString());
     assertThatThrownBy(() -> bean.listConnectedGatewayReceivers()).hasMessageContaining(TestCommand.clusterRead.toString());
-    //assertThatThrownBy(() -> bean.processCommand("create region --name=Region_A")).hasMessageContaining("DATA:MANAGE");
+    assertThatThrownBy(() -> bean.processCommand("create region --name=Region_A")).hasMessageContaining("DATA:MANAGE");
     assertThatThrownBy(() -> bean.showJVMMetrics()).hasMessageContaining(TestCommand.clusterRead.toString());
     assertThatThrownBy(() -> bean.status()).hasMessageContaining(TestCommand.clusterRead.toString());
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
index 03d39fd..2e1aa65 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
@@ -20,14 +20,15 @@ import static com.gemstone.gemfire.management.internal.cli.i18n.CliStrings.*;
 import static junitparams.JUnitParamsRunner.*;
 import static org.assertj.core.api.Assertions.*;
 
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
 
 /**
  * Unit tests for WanCommandsController.