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/04/22 01:22:01 UTC

incubator-geode git commit: GEODE-17: make sure PulseAuthTest runs successfully

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-17-2 832ae353d -> 5757ca4b4


GEODE-17: make sure PulseAuthTest runs successfully

* revive the AccessControlMBean since its needed by pulse to do authorization check
* the server can be configured with only authentication


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 5757ca4b4afca5f82acf601bc3f0463f519cb9a9
Parents: 832ae35
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Thu Apr 21 16:21:40 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu Apr 21 16:21:40 2016 -0700

----------------------------------------------------------------------
 .../management/internal/ManagementAgent.java    | 44 ++++++++++++++++++--
 .../internal/security/AccessControlMBean.java   | 41 ++++++++++++++++++
 .../gemfire/security/CustomAuthRealm.java       |  4 +-
 .../gemfire/security/JMXShiroAuthenticator.java |  6 ++-
 .../gemstone/gemfire/security/ShiroUtil.java    |  1 +
 .../security/AccessControlMBeanJUnitTest.java   |  8 ++--
 .../security/GemFireAuthentication.java         | 41 +++++++-----------
 .../gemfire/tools/pulse/tests/Server.java       | 28 +++++++++++--
 8 files changed, 133 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5757ca4b/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 0ba62e7..d6c18df 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
@@ -30,7 +30,13 @@ import java.rmi.server.RMIClientSocketFactory;
 import java.rmi.server.RMIServerSocketFactory;
 import java.rmi.server.UnicastRemoteObject;
 import java.util.HashMap;
+import java.util.Set;
+import javax.management.InstanceAlreadyExistsException;
+import javax.management.MBeanRegistrationException;
 import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
 import javax.management.remote.JMXConnectorServer;
 import javax.management.remote.JMXServiceURL;
 import javax.management.remote.rmi.RMIConnectorServer;
@@ -38,6 +44,7 @@ import javax.management.remote.rmi.RMIJRMPServerImpl;
 import javax.management.remote.rmi.RMIServerImpl;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
+import com.gemstone.gemfire.GemFireConfigException;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -50,9 +57,12 @@ import com.gemstone.gemfire.internal.tcp.TCPConduit;
 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.AccessControlMBean;
 import com.gemstone.gemfire.management.internal.security.MBeanServerWrapper;
+import com.gemstone.gemfire.management.internal.security.ResourceConstants;
 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;
@@ -442,10 +452,11 @@ public class ManagementAgent {
       shiroAuthenticator = new JMXShiroAuthenticator();
       env.put(JMXConnectorServer.AUTHENTICATOR, shiroAuthenticator);
       cs.addNotificationListener(shiroAuthenticator, null, cs.getAttributes());
-      if(! StringUtils.isBlank(shiroConfig) || isCustomAuthorizer()) {
-        MBeanServerWrapper mBeanServerWrapper = new MBeanServerWrapper();
-        cs.setMBeanServerForwarder(mBeanServerWrapper);
-      }
+      // always going to assume authorization is needed as well, if no custom AccessControl, then the CustomAuthRealm
+      // should take care of that
+      MBeanServerWrapper mBeanServerWrapper = new MBeanServerWrapper();
+      cs.setMBeanServerForwarder(mBeanServerWrapper);
+      registerAccessControlMBean();
     }
 
     else {
@@ -472,6 +483,31 @@ public class ManagementAgent {
     }
   }
 
+  private void registerAccessControlMBean() {
+    try {
+      AccessControlMBean acc = new AccessControlMBean();
+      ObjectName accessControlMBeanON = new ObjectName(ResourceConstants.OBJECT_NAME_ACCESSCONTROL);
+      MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
+
+      Set<ObjectName> names = platformMBeanServer.queryNames(accessControlMBeanON, null);
+      if (names.isEmpty()) {
+        try {
+          platformMBeanServer.registerMBean(acc, accessControlMBeanON);
+          logger.info("Registered AccessContorlMBean on " + accessControlMBeanON);
+        } catch (InstanceAlreadyExistsException e) {
+          throw new GemFireConfigException("Error while configuring accesscontrol for jmx resource", e);
+        } catch (MBeanRegistrationException e) {
+          throw new GemFireConfigException("Error while configuring accesscontrol for jmx resource", e);
+        } catch (NotCompliantMBeanException e) {
+          throw new GemFireConfigException("Error while configuring accesscontrol for jmx resource", e);
+        }
+      }
+    } catch (MalformedObjectNameException e) {
+      throw new GemFireConfigException("Error while configuring accesscontrol for jmx resource", e);
+    }
+  }
+
+
   private boolean isCustomAuthenticator() {
     String factoryName = config.getSecurityClientAuthenticator();
     return factoryName != null && !factoryName.isEmpty();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5757ca4b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlMBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlMBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlMBean.java
new file mode 100644
index 0000000..9921538
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/AccessControlMBean.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.management.internal.security;
+
+import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.security.ShiroUtil;
+
+/**
+ * AccessControlMBean Implementation. This retrieves JMXPrincipal from AccessController
+ * and performs authorization for given role using gemfire AccessControl Plugin
+ *
+ * @since 9.0
+ */
+public class AccessControlMBean implements AccessControlMXBean {
+
+  @Override
+  public boolean authorize(String resource, String permission) {
+    try {
+      ShiroUtil.authorize(resource, permission);
+      return true;
+    }
+    catch (GemFireSecurityException e){
+      return false;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5757ca4b/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 3028f0b..76fc852 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
@@ -95,7 +95,9 @@ public class CustomAuthRealm extends AuthorizingRealm{
   public boolean isPermitted(PrincipalCollection principals, Permission permission) {
     ResourceOperationContext context =(ResourceOperationContext)permission;
     Principal principal = (Principal)principals.getPrimaryPrincipal();
-
+    // if no access control is specified, then we allow all
+    if(StringUtils.isBlank(authzFactoryName))
+      return true;
     AccessControl accessControl = getAccessControl(principal, false);
     return accessControl.authorizeOperation(null, context);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5757ca4b/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
index 7151dc5..8f86c38 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/security/JMXShiroAuthenticator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/security/JMXShiroAuthenticator.java
@@ -18,11 +18,13 @@ package com.gemstone.gemfire.security;
 
 import static com.gemstone.gemfire.management.internal.security.ResourceConstants.*;
 
+import java.util.Collections;
 import java.util.Properties;
 import javax.management.Notification;
 import javax.management.NotificationListener;
 import javax.management.remote.JMXAuthenticator;
 import javax.management.remote.JMXConnectionNotification;
+import javax.management.remote.JMXPrincipal;
 import javax.security.auth.Subject;
 
 import com.gemstone.gemfire.management.internal.security.ResourceConstants;
@@ -49,8 +51,8 @@ public class JMXShiroAuthenticator implements JMXAuthenticator, NotificationList
 
     ShiroUtil.login(username, password);
 
-    // we are not using JMX mechanism to do authentication, therefore, this return value does not matter
-    return null;
+    return new Subject(true, Collections.singleton(new JMXPrincipal(username)), Collections.EMPTY_SET,
+      Collections.EMPTY_SET);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5757ca4b/geode-core/src/main/java/com/gemstone/gemfire/security/ShiroUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/security/ShiroUtil.java b/geode-core/src/main/java/com/gemstone/gemfire/security/ShiroUtil.java
index d19ff00..1a46d41 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/security/ShiroUtil.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/security/ShiroUtil.java
@@ -76,6 +76,7 @@ public class ShiroUtil {
       currentUser.checkPermission(permission);
     }
     catch(ShiroException e){
+      LogService.getLogger().info(currentUser.getPrincipal() + " not authorized for "+resource+":"+operation+":"+regionName);
       throw new GemFireSecurityException(e.getMessage(), e);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5757ca4b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AccessControlMBeanJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AccessControlMBeanJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AccessControlMBeanJUnitTest.java
index 35bb3f2..ac5c65a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AccessControlMBeanJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/AccessControlMBeanJUnitTest.java
@@ -16,17 +16,17 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
+import static org.assertj.core.api.Assertions.*;
+
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
 import org.junit.Before;
 import org.junit.ClassRule;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.assertj.core.api.Assertions.assertThat;
-
 @Category(IntegrationTest.class)
 public class AccessControlMBeanJUnitTest {
   private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
@@ -50,11 +50,9 @@ public class AccessControlMBeanJUnitTest {
    * @throws Exception
    */
   @Test
-  @Ignore("No AccessControlMBean")
   @JMXConnectionConfiguration(user = "stranger", password = "1234567")
   public void testAnyAccess() throws Exception {
     assertThat(bean.authorize("DATA", "READ")).isEqualTo(false);
     assertThat(bean.authorize("CLUSTER", "READ")).isEqualTo(false);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5757ca4b/geode-pulse/src/main/java/com/vmware/gemfire/tools/pulse/internal/security/GemFireAuthentication.java
----------------------------------------------------------------------
diff --git a/geode-pulse/src/main/java/com/vmware/gemfire/tools/pulse/internal/security/GemFireAuthentication.java b/geode-pulse/src/main/java/com/vmware/gemfire/tools/pulse/internal/security/GemFireAuthentication.java
index fb40a20..5253f2f 100644
--- a/geode-pulse/src/main/java/com/vmware/gemfire/tools/pulse/internal/security/GemFireAuthentication.java
+++ b/geode-pulse/src/main/java/com/vmware/gemfire/tools/pulse/internal/security/GemFireAuthentication.java
@@ -16,24 +16,21 @@
  */
 package com.vmware.gemfire.tools.pulse.internal.security;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+
 import com.vmware.gemfire.tools.pulse.internal.data.PulseConstants;
 import com.vmware.gemfire.tools.pulse.internal.log.PulseLogWriter;
+
 import org.springframework.security.authentication.UsernamePasswordAuthenticationToken;
 import org.springframework.security.core.GrantedAuthority;
 import org.springframework.security.core.SpringSecurityCoreVersion;
 import org.springframework.security.core.authority.SimpleGrantedAuthority;
 
-import javax.management.InstanceNotFoundException;
-import javax.management.MBeanException;
-import javax.management.MBeanServerConnection;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.ReflectionException;
-import javax.management.remote.JMXConnector;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-
 /**
  * Spring security authentication object for GemFire
  * 
@@ -62,7 +59,7 @@ import java.util.Collection;
  */
 public class GemFireAuthentication extends UsernamePasswordAuthenticationToken {	
 
-  private final static PulseLogWriter LOGGER = PulseLogWriter.getLogger();
+  private final static PulseLogWriter logger = PulseLogWriter.getLogger();
   
 	private JMXConnector jmxc=null;	
 	
@@ -92,31 +89,25 @@ public class GemFireAuthentication extends UsernamePasswordAuthenticationToken {
 
 	public static ArrayList<GrantedAuthority> populateAuthorities(JMXConnector jmxc) {
 		ObjectName name;
+		ArrayList<GrantedAuthority> authorities = new ArrayList<>();
 		try {
 			name = new ObjectName(PulseConstants.OBJECT_NAME_ACCESSCONTROL_MBEAN);
 			MBeanServerConnection mbeanServer = jmxc.getMBeanServerConnection();
-			ArrayList<GrantedAuthority> authorities = new ArrayList<>();
+
 			for(String role : PulseConstants.PULSE_ROLES){
 				Object[] params = role.split(":");
 				String[] signature = new String[] {String.class.getCanonicalName(), String.class.getCanonicalName()};
 				boolean result = (Boolean)mbeanServer.invoke(name, "authorize", params, signature);
 				if(result){
-				  //spring sec require ROLE_ prefix
 					authorities.add(new SimpleGrantedAuthority(role));
 				}
 			}
-			return authorities;
-		} catch (MalformedObjectNameException e) {
-			throw new RuntimeException(e);
-		} catch (InstanceNotFoundException e) {
-			throw new RuntimeException(e);
-		} catch (MBeanException e) {
-			throw new RuntimeException(e);
-		} catch (ReflectionException e) {
-			throw new RuntimeException(e);
-		} catch (IOException e) {
-			throw new RuntimeException(e);
+		}catch (Exception e){
+			throw new RuntimeException(e.getMessage(), e);
 		}
+
+		return authorities;
+
 	}
 
 	public JMXConnector getJmxc() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5757ca4b/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 bd34820..0ae5d26 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
@@ -38,11 +38,18 @@ import javax.management.remote.JMXConnectorServerFactory;
 import javax.management.remote.JMXServiceURL;
 
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.management.internal.security.AccessControlMBean;
 import com.gemstone.gemfire.management.internal.security.JSONAuthorization;
 import com.gemstone.gemfire.management.internal.security.MBeanServerWrapper;
+import com.gemstone.gemfire.management.internal.security.ResourceConstants;
+import com.gemstone.gemfire.security.CustomAuthRealm;
 import com.gemstone.gemfire.security.JMXShiroAuthenticator;
 import com.vmware.gemfire.tools.pulse.internal.data.PulseConstants;
-import org.json.JSONException;
+
+import org.apache.shiro.SecurityUtils;
+import org.apache.shiro.mgt.DefaultSecurityManager;
+import org.apache.shiro.mgt.SecurityManager;
+import org.apache.shiro.realm.Realm;
 
 public class Server {
   private static final String DEFAULT_HOST = "127.0.0.1"; //"localhost"
@@ -52,7 +59,7 @@ public class Server {
   private JMXConnectorServer cs;
   private String propFile = null;
 
-  public Server(int port, String properties, String jsonAuthFile) throws IOException, JSONException {
+  public Server(int port, String properties, String jsonAuthFile) throws Exception {
     this.propFile = properties;
     mbs = ManagementFactory.getPlatformMBeanServer();
     url = new JMXServiceURL(formJMXServiceURLString(DEFAULT_HOST, port));
@@ -64,14 +71,29 @@ public class Server {
       System.setProperty("spring.profiles.active", "pulse.authentication.gemfire");
       Properties props = new Properties();
       props.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, JSONAuthorization.class.getName() + ".create");
-      props.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
+      //props.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
       JSONAuthorization.setUpWithJsonFile(jsonAuthFile);
       Map<String, Object> env = new HashMap<String, Object>();
 
+      // set up Shiro Security Manager
+      Realm realm = new CustomAuthRealm(props);
+      SecurityManager securityManager = new DefaultSecurityManager(realm);
+      SecurityUtils.setSecurityManager(securityManager);
+
+      // register the AccessControll bean
+      AccessControlMBean acc = new AccessControlMBean();
+      ObjectName accessControlMBeanON = new ObjectName(ResourceConstants.OBJECT_NAME_ACCESSCONTROL);
+      MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
+      platformMBeanServer.registerMBean(acc, accessControlMBeanON);
+
+      // wire in the authenticator and authorizaton
       JMXShiroAuthenticator interceptor = new JMXShiroAuthenticator();
       env.put(JMXConnectorServer.AUTHENTICATOR, interceptor);
       cs = JMXConnectorServerFactory.newJMXConnectorServer(url, env, mbs);
       cs.setMBeanServerForwarder(new MBeanServerWrapper());
+
+      //set up the AccessControlMXBean
+
     } else {
       System.setProperty("spring.profiles.active", "pulse.authentication.default");
       cs = JMXConnectorServerFactory.newJMXConnectorServer(url, null, mbs);