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/30 00:26:19 UTC

[49/50] [abbrv] incubator-geode git commit: GEODE-17: set up Shiro Security Manager in the DS and properly login and logout

GEODE-17: set up Shiro Security Manager in the DS and properly login and logout


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

Branch: refs/heads/feature/GEODE-17-3
Commit: 96e69799aedb201fa65b410d1916323bf63d3da3
Parents: 5a6a636
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue Mar 29 15:16:26 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Tue Mar 29 15:16:26 2016 -0700

----------------------------------------------------------------------
 .../internal/InternalDistributedSystem.java     |  81 ++++++++-----
 .../internal/cache/GemFireCacheImpl.java        | 116 +++++++++----------
 .../management/internal/ManagementAgent.java    |  26 ++---
 .../security/ManagementInterceptor.java         |   8 +-
 .../gemfire/security/CustomAuthRealm.java       |  36 +-----
 .../gemfire/security/JMXShiroAuthenticator.java |  63 ++++++++++
 .../gemfire/tools/pulse/tests/Server.java       |   2 +-
 7 files changed, 187 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96e69799/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
index 8fc884a..0704b1e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
@@ -17,30 +17,6 @@
 
 package com.gemstone.gemfire.distributed.internal;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.Reader;
-import java.lang.reflect.Array;
-import java.lang.reflect.Method;
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.StringTokenizer;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.ForcedDisconnectException;
@@ -94,6 +70,7 @@ import com.gemstone.gemfire.internal.cache.execute.FunctionStats;
 import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheServerCreation;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LogWriterFactory;
@@ -108,7 +85,36 @@ import com.gemstone.gemfire.internal.tcp.ConnectionTable;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableCondition;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantLock;
 import com.gemstone.gemfire.management.ManagementException;
+import com.gemstone.gemfire.security.CustomAuthRealm;
 import com.gemstone.gemfire.security.GemFireSecurityException;
+import org.apache.logging.log4j.Logger;
+import org.apache.shiro.SecurityUtils;
+import org.apache.shiro.config.IniSecurityManagerFactory;
+import org.apache.shiro.mgt.DefaultSecurityManager;
+import org.apache.shiro.mgt.SecurityManager;
+import org.apache.shiro.realm.Realm;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Reader;
+import java.lang.reflect.Array;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.StringTokenizer;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
  * The concrete implementation of {@link DistributedSystem} that
@@ -275,12 +281,27 @@ public class InternalDistributedSystem
     InternalDataSerializer.checkSerializationVersion();
     try {
       SystemFailure.startThreads();
-    InternalDistributedSystem newSystem = new InternalDistributedSystem(config);
-    newSystem.initialize();
-    reconnectAttemptCounter = 0; // reset reconnect count since we just got a new connection
-    notifyConnectListeners(newSystem);
-    success = true;
-    return newSystem;
+      InternalDistributedSystem newSystem = new InternalDistributedSystem(config);
+      newSystem.initialize();
+      reconnectAttemptCounter = 0; // reset reconnect count since we just got a new connection
+      notifyConnectListeners(newSystem);
+      success = true;
+
+      // setup shiro for authentication and authorization if it's desired
+      String shiroConfig = config.getProperty(DistributionConfig.SHIRO_INIT_NAME);
+      String customAuthenticator = config.getProperty(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME);
+      if (!StringUtils.isBlank(shiroConfig)) {
+        IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:"+shiroConfig);
+        SecurityManager securityManager = factory.getInstance();
+        SecurityUtils.setSecurityManager(securityManager);
+      }
+      else if (!StringUtils.isBlank(customAuthenticator)) {
+        Realm realm = new CustomAuthRealm(config);
+        DefaultSecurityManager securityManager = new DefaultSecurityManager(realm);
+        SecurityUtils.setSecurityManager(securityManager);
+      }
+
+      return newSystem;
     } finally {
       if (!success) {
         LoggingThreadGroup.cleanUpThreadGroups(); // bug44365 - logwriters accumulate, causing mem leak

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96e69799/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index 201acc0..08f81c3 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -17,65 +17,6 @@
 
 package com.gemstone.gemfire.internal.cache;
 
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.io.PrintStream;
-import java.io.Reader;
-import java.io.StringBufferInputStream;
-import java.io.StringWriter;
-import java.io.Writer;
-import java.lang.reflect.Method;
-import java.net.InetSocketAddress;
-import java.net.URL;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.ServiceLoader;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.naming.Context;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.ForcedDisconnectException;
@@ -240,6 +181,63 @@ import com.gemstone.gemfire.pdx.internal.TypeRegistry;
 import com.gemstone.gemfire.redis.GemFireRedisServer;
 import com.sun.jna.Native;
 import com.sun.jna.Platform;
+import org.apache.logging.log4j.Logger;
+
+import javax.naming.Context;
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.Reader;
+import java.io.StringBufferInputStream;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Method;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 
 // @todo somebody Come up with more reasonable values for {@link #DEFAULT_LOCK_TIMEOUT}, etc.
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96e69799/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
index 2a57b90..277894d 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
@@ -16,7 +16,6 @@
  */
 package com.gemstone.gemfire.management.internal;
 
-import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -30,8 +29,8 @@ import com.gemstone.gemfire.management.ManagementException;
 import com.gemstone.gemfire.management.ManagementService;
 import com.gemstone.gemfire.management.ManagerMXBean;
 import com.gemstone.gemfire.management.internal.security.MBeanServerWrapper;
-import com.gemstone.gemfire.management.internal.security.ManagementInterceptor;
 import com.gemstone.gemfire.management.internal.unsafe.ReadOpFileAccessController;
+import com.gemstone.gemfire.security.JMXShiroAuthenticator;
 import org.apache.logging.log4j.Logger;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
@@ -81,6 +80,7 @@ public class ManagementAgent {
   private boolean running = false;
   private Registry registry;
   private JMXConnectorServer cs;
+  private JMXShiroAuthenticator shiroAuthenticator;
   private final DistributionConfig config;
   private boolean isHttpServiceRunning = false;
 
@@ -150,6 +150,10 @@ public class ManagementAgent {
       logger.debug("Stopping jmx manager agent");
     }
     try {
+      // log out the shiro user
+      if (shiroAuthenticator!=null) {
+        shiroAuthenticator.logout();
+      }
       cs.stop();
       UnicastRemoteObject.unexportObject(registry, true);
     } catch (IOException e) {
@@ -385,22 +389,10 @@ public class ManagementAgent {
     // Environment map. KIRK: why is this declared as HashMap?
     final HashMap<String, Object> env = new HashMap<String, Object>();
 
-    Cache cache = CacheFactory.getAnyInstance();
     String shiroConfig = this.config.getShiroInit();
-
-    if (!StringUtils.isEmpty(shiroConfig)) {
-      Factory<SecurityManager> factory = new IniSecurityManagerFactory("classpath:"+shiroConfig);
-      SecurityManager securityManager = factory.getInstance();
-      SecurityUtils.setSecurityManager(securityManager);
-      // TODO: how do we use the security manager configured by the shiro.ini to do JMX authentication?
-    }
-    else if (isCustomAuthenticator()) {
-      Properties sysProps = cache.getDistributedSystem().getProperties();
-      Realm realm = new CustomAuthRealm(sysProps);
-      SecurityManager securityManager = new DefaultSecurityManager(realm);
-
-      SecurityUtils.setSecurityManager(securityManager);
-      env.put(JMXConnectorServer.AUTHENTICATOR, realm);
+    if (! StringUtils.isEmpty(shiroConfig) || isCustomAuthenticator()) {
+      shiroAuthenticator = new JMXShiroAuthenticator();
+      env.put(JMXConnectorServer.AUTHENTICATOR, shiroAuthenticator);
     }
     else {
       /* Disable the old authenticator mechanism */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96e69799/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ManagementInterceptor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ManagementInterceptor.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ManagementInterceptor.java
index 639677b..ad67914 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ManagementInterceptor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ManagementInterceptor.java
@@ -58,10 +58,6 @@ import static com.gemstone.gemfire.management.internal.security.ResourceConstant
  *
  */
 public class ManagementInterceptor implements JMXAuthenticator {
-
-  // FIXME: Merged from GEODE-17. Are they necessary?
-	public static final String USER_NAME = "security-username";
-	public static final String PASSWORD = "security-password";
 	public static final String OBJECT_NAME_ACCESSCONTROL = "GemFire:service=AccessControl,type=Distributed";
 
 	private static final Logger logger = LogManager.getLogger(ManagementInterceptor.class);
@@ -126,8 +122,8 @@ public class ManagementInterceptor implements JMXAuthenticator {
 			final String[] aCredentials = (String[]) credentials;
 			username = aCredentials[0];
 			password = aCredentials[1];
-		  pr.put(USER_NAME, username);
-		  pr.put(PASSWORD, password);
+		  pr.put(ResourceConstants.USER_NAME, username);
+		  pr.put(ResourceConstants.PASSWORD, password);
     } else if (credentials instanceof Properties) {
       pr = (Properties) credentials;
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96e69799/geode-core/src/main/java/com/gemstone/gemfire/security/CustomAuthRealm.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/security/CustomAuthRealm.java b/geode-core/src/main/java/com/gemstone/gemfire/security/CustomAuthRealm.java
index 8789d3c..29f34b0 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/security/CustomAuthRealm.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/security/CustomAuthRealm.java
@@ -20,10 +20,10 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.ClassLoadUtil;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.management.internal.security.ResourceConstants;
 import com.gemstone.gemfire.management.internal.security.ResourceOperationContext;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
-import org.apache.shiro.SecurityUtils;
 import org.apache.shiro.authc.AuthenticationException;
 import org.apache.shiro.authc.AuthenticationInfo;
 import org.apache.shiro.authc.AuthenticationToken;
@@ -34,7 +34,6 @@ import org.apache.shiro.authz.Permission;
 import org.apache.shiro.realm.AuthorizingRealm;
 import org.apache.shiro.subject.PrincipalCollection;
 
-import javax.management.remote.JMXAuthenticator;
 import javax.management.remote.JMXPrincipal;
 import javax.security.auth.Subject;
 import java.lang.reflect.Method;
@@ -47,12 +46,9 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import static com.gemstone.gemfire.management.internal.security.ResourceConstants.ACCESS_DENIED_MESSAGE;
-import static com.gemstone.gemfire.management.internal.security.ResourceConstants.WRONGE_CREDENTIALS_MESSAGE;
 
-public class CustomAuthRealm extends AuthorizingRealm implements JMXAuthenticator {
+public class CustomAuthRealm extends AuthorizingRealm{
   public static final String REALM_NAME = "CUSTOMAUTHREALM";
-  public static final String USER_NAME = "security-username";
-  public static final String PASSWORD = "security-password";
 
   private static final Logger logger = LogManager.getLogger(CustomAuthRealm.class);
   private String authzFactoryName;
@@ -81,8 +77,8 @@ public class CustomAuthRealm extends AuthorizingRealm implements JMXAuthenticato
     String password = new String(authToken.getPassword());
 
     Properties credentialProps = new Properties();
-    credentialProps.put(USER_NAME, username);
-    credentialProps.put(PASSWORD, password);
+    credentialProps.put(ResourceConstants.USER_NAME, username);
+    credentialProps.put(ResourceConstants.PASSWORD, password);
 
     Principal principal  = getAuthenticator(securityProps).authenticate(credentialProps);
 
@@ -105,30 +101,6 @@ public class CustomAuthRealm extends AuthorizingRealm implements JMXAuthenticato
     return accessControl.authorizeOperation(null, context);
   }
 
-
-  @Override
-  public Subject authenticate(Object credentials) {
-    String username = null, password = null;
-    if (credentials instanceof String[]) {
-      final String[] aCredentials = (String[]) credentials;
-      username = aCredentials[0];
-      password = aCredentials[1];
-    } else if (credentials instanceof Properties) {
-      username = ((Properties) credentials).getProperty(USER_NAME);
-      password = ((Properties) credentials).getProperty(PASSWORD);
-    } else {
-      throw new SecurityException(WRONGE_CREDENTIALS_MESSAGE);
-    }
-
-    AuthenticationToken token =
-        new UsernamePasswordToken(username, password);
-    org.apache.shiro.subject.Subject currentUser = SecurityUtils.getSubject();
-    currentUser.login(token);
-
-    // we are not using JMX mechanism to do authentication, therefore, this return value does not matter
-    return null;
-  }
-
   public AccessControl getAccessControl(Principal principal, boolean isPost) {
     if (!isPost) {
       if (cachedAuthZCallback.containsKey(principal)) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96e69799/geode-core/src/main/java/com/gemstone/gemfire/security/JMXShiroAuthenticator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/security/JMXShiroAuthenticator.java b/geode-core/src/main/java/com/gemstone/gemfire/security/JMXShiroAuthenticator.java
new file mode 100644
index 0000000..ba76c16
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/security/JMXShiroAuthenticator.java
@@ -0,0 +1,63 @@
+/*
+ * 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.security;
+
+import com.gemstone.gemfire.management.internal.security.ResourceConstants;
+import org.apache.shiro.SecurityUtils;
+import org.apache.shiro.authc.AuthenticationToken;
+import org.apache.shiro.authc.UsernamePasswordToken;
+
+import javax.management.remote.JMXAuthenticator;
+import javax.security.auth.Subject;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.management.internal.security.ResourceConstants.WRONGE_CREDENTIALS_MESSAGE;
+
+/**
+ * this will make JMX authentication to use Shiro for Authentication
+ */
+
+public class JMXShiroAuthenticator implements JMXAuthenticator {
+
+  @Override
+  public Subject authenticate(Object credentials) {
+    String username = null, password = null;
+    if (credentials instanceof String[]) {
+      final String[] aCredentials = (String[]) credentials;
+      username = aCredentials[0];
+      password = aCredentials[1];
+    } else if (credentials instanceof Properties) {
+      username = ((Properties) credentials).getProperty(ResourceConstants.USER_NAME);
+      password = ((Properties) credentials).getProperty(ResourceConstants.PASSWORD);
+    } else {
+      throw new SecurityException(WRONGE_CREDENTIALS_MESSAGE);
+    }
+
+    AuthenticationToken token =
+        new UsernamePasswordToken(username, password);
+    org.apache.shiro.subject.Subject currentUser = SecurityUtils.getSubject();
+    currentUser.login(token);
+
+    // we are not using JMX mechanism to do authentication, therefore, this return value does not matter
+    return null;
+  }
+
+  public void logout(){
+    org.apache.shiro.subject.Subject currentUser = SecurityUtils.getSubject();
+    currentUser.logout();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96e69799/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/tests/Server.java
----------------------------------------------------------------------
diff --git a/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/tests/Server.java b/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/tests/Server.java
index 970eb34..436b6b8 100644
--- a/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/tests/Server.java
+++ b/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/tests/Server.java
@@ -71,7 +71,7 @@ public class Server {
       ManagementInterceptor interceptor = new ManagementInterceptor(props);
       env.put(JMXConnectorServer.AUTHENTICATOR, interceptor);
       cs = JMXConnectorServerFactory.newJMXConnectorServer(url, env, mbs);
-      cs.setMBeanServerForwarder(new MBeanServerWrapper(interceptor));
+      cs.setMBeanServerForwarder(new MBeanServerWrapper());
     } else {
       System.setProperty("spring.profiles.active", "pulse.authentication.default");
       cs = JMXConnectorServerFactory.newJMXConnectorServer(url, null, mbs);