You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by gt...@apache.org on 2017/09/07 13:12:08 UTC

activemq-artemis git commit: ARTEMIS-1373 - support memberOf type query for role mapping and respect roleName attribute AMQ-3064

Repository: activemq-artemis
Updated Branches:
  refs/heads/master 53a9c9b47 -> 99b2e4c0f


ARTEMIS-1373 - support memberOf type query for role mapping and respect roleName attribute AMQ-3064


Project: http://git-wip-us.apache.org/repos/asf/activemq-artemis/repo
Commit: http://git-wip-us.apache.org/repos/asf/activemq-artemis/commit/99b2e4c0
Tree: http://git-wip-us.apache.org/repos/asf/activemq-artemis/tree/99b2e4c0
Diff: http://git-wip-us.apache.org/repos/asf/activemq-artemis/diff/99b2e4c0

Branch: refs/heads/master
Commit: 99b2e4c0fb4ed27215dd3e469ab46073cc36fbb5
Parents: 53a9c9b
Author: gtully <ga...@gmail.com>
Authored: Thu Sep 7 14:11:20 2017 +0100
Committer: gtully <ga...@gmail.com>
Committed: Thu Sep 7 14:11:48 2017 +0100

----------------------------------------------------------------------
 .../spi/core/security/jaas/LDAPLoginModule.java | 67 +++++++++++--------
 docs/user-manual/en/security.md                 | 30 +++++++--
 .../amqp/SaslKrb5LDAPSecurityTest.java          | 68 ++++++++++++++------
 .../src/test/resources/login.config             | 65 ++++++++++++++++++-
 4 files changed, 178 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/99b2e4c0/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/LDAPLoginModule.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/LDAPLoginModule.java b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/LDAPLoginModule.java
index 7338e19..fa40edc 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/LDAPLoginModule.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/LDAPLoginModule.java
@@ -29,6 +29,8 @@ import javax.naming.directory.DirContext;
 import javax.naming.directory.InitialDirContext;
 import javax.naming.directory.SearchControls;
 import javax.naming.directory.SearchResult;
+import javax.naming.ldap.LdapName;
+import javax.naming.ldap.Rdn;
 import javax.security.auth.Subject;
 import javax.security.auth.callback.Callback;
 import javax.security.auth.callback.CallbackHandler;
@@ -91,6 +93,8 @@ public class LDAPLoginModule implements LoginModule {
    private boolean userAuthenticated = false;
    private boolean authenticateUser = true;
    private Subject brokerGssapiIdentity = null;
+   private boolean isRoleAttributeSet = false;
+   private String roleAttributeName = null;
 
    @Override
    public void initialize(Subject subject,
@@ -104,6 +108,8 @@ public class LDAPLoginModule implements LoginModule {
       if (isLoginPropertySet(AUTHENTICATE_USER)) {
          authenticateUser = Boolean.valueOf(getLDAPPropertyValue(AUTHENTICATE_USER));
       }
+      isRoleAttributeSet = isLoginPropertySet(ROLE_NAME);
+      roleAttributeName = getLDAPPropertyValue(ROLE_NAME);
    }
 
    @Override
@@ -338,7 +344,25 @@ public class LDAPLoginModule implements LoginModule {
             throw new FailedLoginException("User found, but LDAP entry malformed: " + username);
          }
          if (isLoginPropertySet(USER_ROLE_NAME)) {
-            addAttributeValues(getLDAPPropertyValue(USER_ROLE_NAME), attrs, roles);
+            Attribute roleNames = attrs.get(getLDAPPropertyValue(USER_ROLE_NAME));
+            if (roleNames != null) {
+               NamingEnumeration<?> e = roleNames.getAll();
+               while (e.hasMore()) {
+                  String roleDnString = (String) e.next();
+                  if (isRoleAttributeSet) {
+                     // parse out the attribute from the group Dn
+                     LdapName ldapRoleName = new LdapName(roleDnString);
+                     for (int i = 0; i < ldapRoleName.size(); i++) {
+                        Rdn candidate = ldapRoleName.getRdn(i);
+                        if (roleAttributeName.equals(candidate.getType())) {
+                           roles.add((String) candidate.getValue());
+                        }
+                     }
+                  } else {
+                     roles.add(roleDnString);
+                  }
+               }
+            }
          }
       } catch (CommunicationException e) {
          closeContext();
@@ -359,6 +383,11 @@ public class LDAPLoginModule implements LoginModule {
                                    String dn,
                                    String username,
                                    List<String> currentRoles) throws NamingException {
+
+      if (!isLoginPropertySet(ROLE_SEARCH_MATCHING)) {
+         return;
+      }
+
       MessageFormat roleSearchMatchingFormat;
       boolean roleSearchSubtreeBool;
       boolean expandRolesBool;
@@ -366,9 +395,6 @@ public class LDAPLoginModule implements LoginModule {
       roleSearchSubtreeBool = Boolean.valueOf(getLDAPPropertyValue(ROLE_SEARCH_SUBTREE)).booleanValue();
       expandRolesBool = Boolean.valueOf(getLDAPPropertyValue(EXPAND_ROLES)).booleanValue();
 
-      if (!isLoginPropertySet(ROLE_NAME)) {
-         return;
-      }
       final String filter = roleSearchMatchingFormat.format(new String[]{doRFC2254Encoding(dn), doRFC2254Encoding(username)});
 
       SearchControls constraints = new SearchControls();
@@ -397,15 +423,11 @@ public class LDAPLoginModule implements LoginModule {
 
       while (results.hasMore()) {
          SearchResult result = results.next();
-         Attributes attrs = result.getAttributes();
          if (expandRolesBool) {
             haveSeenNames.add(result.getNameInNamespace());
             pendingNameExpansion.add(result.getNameInNamespace());
          }
-         if (attrs == null) {
-            continue;
-         }
-         addAttributeValues(getLDAPPropertyValue(ROLE_NAME), attrs, currentRoles);
+         addRoleAttribute(result, currentRoles);
       }
       if (expandRolesBool) {
          MessageFormat expandRolesMatchingFormat = new MessageFormat(getLDAPPropertyValue(EXPAND_ROLES_MATCHING));
@@ -424,8 +446,7 @@ public class LDAPLoginModule implements LoginModule {
                SearchResult result = results.next();
                name = result.getNameInNamespace();
                if (!haveSeenNames.contains(name)) {
-                  Attributes attrs = result.getAttributes();
-                  addAttributeValues(getLDAPPropertyValue(ROLE_NAME), attrs, currentRoles);
+                  addRoleAttribute(result, currentRoles);
                   haveSeenNames.add(name);
                   pendingNameExpansion.add(name);
                }
@@ -497,24 +518,18 @@ public class LDAPLoginModule implements LoginModule {
       return isValid;
    }
 
-   private void addAttributeValues(String attrId,
-                                           Attributes attrs,
-                                           List<String> values) throws NamingException {
-
-      if (attrId == null || attrs == null) {
-         return;
-      }
-      Attribute attr = attrs.get(attrId);
-      if (attr == null) {
-         return;
-      }
-      NamingEnumeration<?> e = attr.getAll();
-      while (e.hasMore()) {
-         String value = (String) e.next();
-         values.add(value);
+   private void addRoleAttribute(SearchResult searchResult, List<String> roles) throws NamingException {
+      if (isRoleAttributeSet) {
+         Attribute roleAttribute = searchResult.getAttributes().get(roleAttributeName);
+         if (roleAttribute != null) {
+            roles.add((String) roleAttribute.get());
+         }
+      } else {
+         roles.add(searchResult.getNameInNamespace());
       }
    }
 
+
    protected void openContext() throws Exception {
       if (context == null) {
          try {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/99b2e4c0/docs/user-manual/en/security.md
----------------------------------------------------------------------
diff --git a/docs/user-manual/en/security.md b/docs/user-manual/en/security.md
index 1d7f314..93da340 100644
--- a/docs/user-manual/en/security.md
+++ b/docs/user-manual/en/security.md
@@ -510,7 +510,7 @@ managed using the X.500 system. It is implemented by `org.apache.activemq.artemi
     the `ou=Group,ou=ActiveMQ,ou=system` node.
 
 -   `roleName` - specifies the attribute type of the role entry that contains the name of the role/group (e.g. C, O,
-    OU, etc.). If you omit this option, the role search feature is effectively disabled.
+    OU, etc.). If you omit this option the full DN of the role is used.
 
 -   `roleSearchMatching` - specifies an LDAP search filter, which is applied to the subtree selected by `roleBase`.
     This works in a similar manner to the `userSearchMatching` option, except that it supports two substitution strings,
@@ -526,7 +526,8 @@ managed using the X.500 system. It is implemented by `org.apache.activemq.artemi
     search filter is applied to the subtree selected by the role base, `ou=Group,ou=ActiveMQ,ou=system`, it matches all
     role entries that have a `member` attribute equal to `uid=jdoe` (the value of a `member` attribute is a DN).
 
-    This option must always be set, even if role searching is disabled, because it has no default value.
+    This option must always be set to enable role searching because it has no default value. Leaving it unset disables
+    role searching and the role information must come from `userRoleName`.
 
     If you use OpenLDAP, the syntax of the search filter is `(member:=uid=jdoe)`.
 
@@ -702,11 +703,26 @@ An example configuration scope for `login.config` that will pick up a Kerberos k
 On the server, the Kerberos authenticated Peer Principal can be added to the Subject's principal set as an Apache ActiveMQ Artemis UserPrincipal
 using the Apache ActiveMQ Artemis `Krb5LoginModule` login module. The [PropertiesLoginModule](#propertiesloginmodule) or
  [LDAPLoginModule](#ldaploginmodule) can then be used to map
-the authenticated Kerberos Peer Principal to an Apache ActiveMQ Artemis [Role](#role-based-security-for-addresses).
-
-Note: the Kerberos Peer Principal does not exist as an Apache ActiveMQ Artemis user.
-
-    org.apache.activemq.artemis.spi.core.security.jaas.Krb5LoginModule optional;
+the authenticated Kerberos Peer Principal to an Apache ActiveMQ Artemis [Role](#role-based-security-for-addresses). Note
+that the Kerberos Peer Principal does not exist as an Apache ActiveMQ Artemis user, only as a role member.
+ 
+    org.apache.activemq.artemis.spi.core.security.jaas.Krb5LoginModule required
+        ;
+    org.apache.activemq.artemis.spi.core.security.jaas.LDAPLoginModule optional
+        debug=true
+        initialContextFactory=com.sun.jndi.ldap.LdapCtxFactory
+        connectionURL="ldap://localhost:1024"
+        authentication=GSSAPI
+        saslLoginConfigScope=broker-sasl-gssapi
+        connectionProtocol=s
+        userBase="ou=users,dc=example,dc=com"
+        userSearchMatching="(krb5PrincipalName={0})"
+        userSearchSubtree=true
+        authenticateUser=false
+        roleBase="ou=system"
+        roleSearchMatching="(member={0})"
+        roleSearchSubtree=false
+        ;
 
 #### TLS Kerberos Cipher Suites
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/99b2e4c0/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/amqp/SaslKrb5LDAPSecurityTest.java
----------------------------------------------------------------------
diff --git a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/amqp/SaslKrb5LDAPSecurityTest.java b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/amqp/SaslKrb5LDAPSecurityTest.java
index 4ede6c6..0012c19 100644
--- a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/amqp/SaslKrb5LDAPSecurityTest.java
+++ b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/amqp/SaslKrb5LDAPSecurityTest.java
@@ -48,7 +48,6 @@ import java.util.Hashtable;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 
 import org.apache.activemq.artemis.api.core.TransportConfiguration;
 import org.apache.activemq.artemis.core.config.Configuration;
@@ -105,7 +104,8 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.activemq.artemis.tests.util.ActiveMQTestBase.NETTY_ACCEPTOR_FACTORY;
 
-@RunWith(FrameworkRunner.class) @CreateDS(name = "Example",
+@RunWith(FrameworkRunner.class)
+@CreateDS(name = "Example",
    partitions = {@CreatePartition(name = "example", suffix = "dc=example,dc=com",
       contextEntry = @ContextEntry(entryLdif = "dn: dc=example,dc=com\n" + "dc: example\n" + "objectClass: top\n" + "objectClass: domain\n\n"),
       indexes = {@CreateIndex(attribute = "objectClass"), @CreateIndex(attribute = "dc"), @CreateIndex(attribute = "ou")})},
@@ -160,7 +160,19 @@ public class SaslKrb5LDAPSecurityTest extends AbstractLdapTestUnit {
 
       testDir = temporaryFolder.getRoot().getAbsolutePath();
 
-      ActiveMQJAASSecurityManager securityManager = new ActiveMQJAASSecurityManager("Krb5PlusLdap");
+      // hard coded match, default_keytab_name in minikdc-krb5.conf template
+      File userKeyTab = new File("target/test.krb5.keytab");
+      createPrincipal(userKeyTab, "client", "amqp/localhost", "ldap/localhost");
+
+      if (debug) {
+         dumpLdapContents();
+      }
+
+      rewriteKerb5Conf();
+   }
+
+   private void createArtemisServer(String securityConfigScope) {
+      ActiveMQJAASSecurityManager securityManager = new ActiveMQJAASSecurityManager(securityConfigScope);
       HashMap<String, Object> params = new HashMap<>();
       params.put(TransportConstants.PORT_PROP_NAME, String.valueOf(5672));
       params.put(TransportConstants.PROTOCOLS_PROP_NAME, "AMQP");
@@ -171,17 +183,6 @@ public class SaslKrb5LDAPSecurityTest extends AbstractLdapTestUnit {
 
       Configuration configuration = new ConfigurationImpl().setSecurityEnabled(true).addAcceptorConfiguration(new TransportConfiguration(NETTY_ACCEPTOR_FACTORY, params, "netty-amqp", amqpParams)).setJournalDirectory(ActiveMQTestBase.getJournalDir(testDir, 0, false)).setBindingsDirectory(ActiveMQTestBase.getBindingsDir(testDir, 0, false)).setPagingDirectory(ActiveMQTestBase.getPageDir(testDir, 0, false)).setLargeMessagesDirectory(ActiveMQTestBase.getLargeMessagesDir(testDir, 0, false));
       server = ActiveMQServers.newActiveMQServer(configuration, ManagementFactory.getPlatformMBeanServer(), securityManager, false);
-
-
-      // hard coded match, default_keytab_name in minikdc-krb5.conf template
-      File userKeyTab = new File("target/test.krb5.keytab");
-      createPrincipal(userKeyTab, "client", "amqp/localhost", "ldap/localhost");
-
-      if (debug) {
-         dumpLdapContents();
-      }
-
-      rewriteKerb5Conf();
    }
 
    private void rewriteKerb5Conf() throws Exception {
@@ -257,7 +258,14 @@ public class SaslKrb5LDAPSecurityTest extends AbstractLdapTestUnit {
 
    public synchronized void createPrincipal(String principal, String password) throws Exception {
       String baseDn = getKdcServer().getSearchBaseDn();
-      String content = "dn: uid=" + principal + "," + baseDn + "\n" + "objectClass: top\n" + "objectClass: person\n" + "objectClass: inetOrgPerson\n" + "objectClass: krb5principal\n" + "objectClass: krb5kdcentry\n" + "cn: " + principal + "\n" + "sn: " + principal + "\n" + "uid: " + principal + "\n" + "userPassword: " + password + "\n" + "krb5PrincipalName: " + principal + "@" + getRealm() + "\n" + "krb5KeyVersionNumber: 0";
+      String content = "dn: uid=" + principal + "," + baseDn + "\n" + "objectClass: top\n" + "objectClass: person\n" + "objectClass: inetOrgPerson\n" + "objectClass: krb5principal\n"
+         + "objectClass: krb5kdcentry\n" + "cn: " + principal + "\n" + "sn: " + principal + "\n"
+         + "uid: " + principal + "\n" + "userPassword: " + password + "\n"
+         // using businessCategory as a proxy for memberoOf attribute pending: https://issues.apache.org/jira/browse/DIRSERVER-1844
+         + "businessCategory: " + "cn=admins,ou=system" + "\n"
+         + "businessCategory: " + "cn=bees,ou=system" + "\n"
+         + "krb5PrincipalName: " + principal + "@" + getRealm() + "\n"
+         + "krb5KeyVersionNumber: 0";
 
       for (LdifEntry ldifEntry : new LdifReader(new StringReader(content))) {
          service.getAdminSession().add(new DefaultEntry(service.getSchemaManager(), ldifEntry.getEntry()));
@@ -265,7 +273,7 @@ public class SaslKrb5LDAPSecurityTest extends AbstractLdapTestUnit {
    }
 
    public void createPrincipal(File keytabFile, String... principals) throws Exception {
-      String generatedPassword = UUID.randomUUID().toString();
+      String generatedPassword = "notSecret!";
       Keytab keytab = new Keytab();
       List<KeytabEntry> entries = new ArrayList<>();
       for (String principal : principals) {
@@ -288,7 +296,9 @@ public class SaslKrb5LDAPSecurityTest extends AbstractLdapTestUnit {
 
    @After
    public void tearDown() throws Exception {
-      server.stop();
+      if (server != null) {
+         server.stop();
+      }
    }
 
    @Test
@@ -358,9 +368,31 @@ public class SaslKrb5LDAPSecurityTest extends AbstractLdapTestUnit {
 
    @Test
    public void testJAASSecurityManagerAuthorizationPositive() throws Exception {
+      dotestJAASSecurityManagerAuthorizationPositive("Krb5PlusLdap", "admins");
+   }
+
+   @Test
+   public void testJAASSecurityManagerAuthorizationPositiveMemberOf() throws Exception {
+      // using businessCategory as a proxy for memberoOf attribute pending: https://issues.apache.org/jira/browse/DIRSERVER-1844
+      dotestJAASSecurityManagerAuthorizationPositive("Krb5PlusLdapMemberOf", "bees");
+   }
+
+   @Test
+   public void testJAASSecurityManagerAuthorizationPositiveNoRoleName() throws Exception {
+      dotestJAASSecurityManagerAuthorizationPositive("Krb5PlusLdapNoRoleName", "cn=admins,ou=system");
+   }
+
+   @Test
+   public void testJAASSecurityManagerAuthorizationPositiveMemberOfNoRoleName() throws Exception {
+      dotestJAASSecurityManagerAuthorizationPositive("Krb5PlusLdapMemberOfNoRoleName", "cn=bees,ou=system");
+   }
+
+   public void dotestJAASSecurityManagerAuthorizationPositive(String jaasConfigScope, String artemisRoleName) throws Exception {
+
+      createArtemisServer(jaasConfigScope);
 
       Set<Role> roles = new HashSet<>();
-      roles.add(new Role("admins", true, true, true, true, true, true, true, true, true, true));
+      roles.add(new Role(artemisRoleName, true, true, true, true, true, true, true, true, true, true));
       server.getConfiguration().putSecurityRoles(QUEUE_NAME, roles);
       server.start();
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/99b2e4c0/tests/integration-tests/src/test/resources/login.config
----------------------------------------------------------------------
diff --git a/tests/integration-tests/src/test/resources/login.config b/tests/integration-tests/src/test/resources/login.config
index 1fceada..a5e40c1 100644
--- a/tests/integration-tests/src/test/resources/login.config
+++ b/tests/integration-tests/src/test/resources/login.config
@@ -151,7 +151,7 @@ Krb5Plus {
 
 Krb5PlusLdap {
 
-    org.apache.activemq.artemis.spi.core.security.jaas.Krb5LoginModule optional
+    org.apache.activemq.artemis.spi.core.security.jaas.Krb5LoginModule required
         debug=true;
 
     org.apache.activemq.artemis.spi.core.security.jaas.LDAPLoginModule optional
@@ -172,6 +172,69 @@ Krb5PlusLdap {
         ;
 };
 
+Krb5PlusLdapNoRoleName {
+
+    org.apache.activemq.artemis.spi.core.security.jaas.Krb5LoginModule required
+        debug=true;
+
+    org.apache.activemq.artemis.spi.core.security.jaas.LDAPLoginModule optional
+        debug=true
+        initialContextFactory=com.sun.jndi.ldap.LdapCtxFactory
+        connectionURL="ldap://localhost:1024"
+        authentication=GSSAPI
+        saslLoginConfigScope=broker-sasl-gssapi
+        connectionProtocol=s
+        userBase="ou=users,dc=example,dc=com"
+        userSearchMatching="(krb5PrincipalName={0})"
+        userSearchSubtree=true
+        authenticateUser=false
+        roleBase="ou=system"
+        roleSearchMatching="(member={0})"
+        roleSearchSubtree=false
+        ;
+};
+
+Krb5PlusLdapMemberOf {
+
+    org.apache.activemq.artemis.spi.core.security.jaas.Krb5LoginModule required
+        debug=true;
+
+    org.apache.activemq.artemis.spi.core.security.jaas.LDAPLoginModule optional
+        debug=true
+        initialContextFactory=com.sun.jndi.ldap.LdapCtxFactory
+        connectionURL="ldap://localhost:1024"
+        authentication=GSSAPI
+        saslLoginConfigScope=broker-sasl-gssapi
+        connectionProtocol=s
+        userBase="ou=users,dc=example,dc=com"
+        userSearchMatching="(krb5PrincipalName={0})"
+        userSearchSubtree=true
+        authenticateUser=false
+        userRoleName=businessCategory
+        roleName=cn
+        ;
+};
+
+Krb5PlusLdapMemberOfNoRoleName {
+
+    org.apache.activemq.artemis.spi.core.security.jaas.Krb5LoginModule required
+        debug=true;
+
+    org.apache.activemq.artemis.spi.core.security.jaas.LDAPLoginModule optional
+        debug=true
+        initialContextFactory=com.sun.jndi.ldap.LdapCtxFactory
+        connectionURL="ldap://localhost:1024"
+        authentication=GSSAPI
+        saslLoginConfigScope=broker-sasl-gssapi
+        connectionProtocol=s
+        userBase="ou=users,dc=example,dc=com"
+        userSearchMatching="(krb5PrincipalName={0})"
+        userSearchSubtree=true
+        authenticateUser=false
+        userRoleName=businessCategory
+        ;
+};
+
 core-tls-krb5-server {
     com.sun.security.auth.module.Krb5LoginModule required
     isInitiator=false