You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by cl...@apache.org on 2017/07/07 03:41:10 UTC

[2/2] activemq-artemis git commit: ARTEMIS-1264 allow role mapping via chained login modules

ARTEMIS-1264 allow role mapping via chained login modules

Add krb5sslloginmodule that will populate userPrincipal that can be mapped to roles independently
Generalised callback handlers to take a connection and pull certs or peerprincipal based on
callback. This bubbled up into api change in securitystore and security manager


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

Branch: refs/heads/master
Commit: d4150be0d9ce815a4a19a77e8e6ca92eaabd6a90
Parents: 15c2f09
Author: gtully <ga...@gmail.com>
Authored: Thu Jul 6 16:54:57 2017 +0100
Committer: Clebert Suconic <cl...@apache.org>
Committed: Thu Jul 6 23:41:02 2017 -0400

----------------------------------------------------------------------
 .../artemis/core/remoting/CertificateUtil.java  |  55 +++++++++
 .../openwire/OpenWireProtocolManager.java       |  10 +-
 .../core/protocol/stomp/StompConnection.java    |   5 +-
 .../protocol/stomp/StompProtocolManager.java    |   8 +-
 .../stomp/v10/StompFrameHandlerV10.java         |  10 +-
 .../stomp/v11/StompFrameHandlerV11.java         |  10 +-
 .../artemis/core/security/SecurityStore.java    |   5 +-
 .../core/security/impl/SecurityStoreImpl.java   |  11 +-
 .../core/server/impl/ActiveMQServerImpl.java    |   9 +-
 .../security/ActiveMQJAASSecurityManager.java   |  27 ++---
 .../core/security/ActiveMQSecurityManager3.java |   7 +-
 .../core/security/jaas/JaasCallbackHandler.java |  18 ++-
 .../spi/core/security/jaas/Krb5SslCallback.java |  46 ++++++++
 .../core/security/jaas/Krb5SslLoginModule.java  | 112 +++++++++++++++++++
 .../security/jaas/PropertiesLoginModule.java    |  14 ++-
 .../TextFileCertificateLoginModuleTest.java     |  18 ++-
 .../integration/security/SecurityTest.java      |   7 +-
 .../ssl/CoreClientOverOneWaySSLKerb5Test.java   |  55 +++++++--
 .../dual-authentication-roles.properties        |   3 +-
 .../src/test/resources/login.config             |  11 ++
 20 files changed, 349 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/remoting/CertificateUtil.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/remoting/CertificateUtil.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/remoting/CertificateUtil.java
new file mode 100644
index 0000000..598fcf3
--- /dev/null
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/remoting/CertificateUtil.java
@@ -0,0 +1,55 @@
+/**
+ * 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 org.apache.activemq.artemis.core.remoting;
+
+import io.netty.channel.ChannelHandler;
+import io.netty.handler.ssl.SslHandler;
+import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnection;
+import org.apache.activemq.artemis.spi.core.remoting.Connection;
+
+import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.security.cert.X509Certificate;
+import java.security.Principal;
+
+public class CertificateUtil {
+
+   public static X509Certificate[] getCertsFromConnection(Connection connection) {
+      X509Certificate[] certificates = null;
+      if (connection instanceof NettyConnection) {
+         certificates = org.apache.activemq.artemis.utils.CertificateUtil.getCertsFromChannel(((NettyConnection) connection).getChannel());
+      }
+      return certificates;
+   }
+
+   public static Principal getPeerPrincipalFromConnection(Connection connection) {
+      Principal result = null;
+      if (connection instanceof NettyConnection) {
+         NettyConnection nettyConnection = (NettyConnection) connection;
+         ChannelHandler channelHandler = nettyConnection.getChannel().pipeline().get("ssl");
+         if (channelHandler != null && channelHandler instanceof SslHandler) {
+            SslHandler sslHandler = (SslHandler) channelHandler;
+            try {
+               result = sslHandler.engine().getSession().getPeerPrincipal();
+            } catch (SSLPeerUnverifiedException ignored) {
+            }
+         }
+      }
+
+      return result;
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/OpenWireProtocolManager.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/OpenWireProtocolManager.java b/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/OpenWireProtocolManager.java
index 09c6509..a4acdeb 100644
--- a/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/OpenWireProtocolManager.java
+++ b/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/OpenWireProtocolManager.java
@@ -17,7 +17,6 @@
 package org.apache.activemq.artemis.core.protocol.openwire;
 
 import javax.jms.InvalidClientIDException;
-import javax.security.cert.X509Certificate;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -44,7 +43,6 @@ import org.apache.activemq.artemis.api.core.client.TopologyMember;
 import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQConnectionContext;
 import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQProducerBrokerExchange;
 import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQSession;
-import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnection;
 import org.apache.activemq.artemis.core.remoting.impl.netty.NettyServerConnection;
 import org.apache.activemq.artemis.core.server.ActiveMQServer;
 import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
@@ -57,7 +55,6 @@ import org.apache.activemq.artemis.spi.core.protocol.ProtocolManagerFactory;
 import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
 import org.apache.activemq.artemis.spi.core.remoting.Acceptor;
 import org.apache.activemq.artemis.spi.core.remoting.Connection;
-import org.apache.activemq.artemis.utils.CertificateUtil;
 import org.apache.activemq.artemis.utils.DataConstants;
 import org.apache.activemq.command.ActiveMQMessage;
 import org.apache.activemq.command.ActiveMQTopic;
@@ -462,12 +459,7 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
    }
 
    public void validateUser(String login, String passcode, OpenWireConnection connection) throws Exception {
-      X509Certificate[] certificates = null;
-      if (connection.getTransportConnection() instanceof NettyConnection) {
-         certificates = CertificateUtil.getCertsFromChannel(((NettyConnection) connection.getTransportConnection()).getChannel());
-      }
-
-      server.getSecurityStore().authenticate(login, passcode, certificates);
+      server.getSecurityStore().authenticate(login, passcode, connection.getTransportConnection());
    }
 
    public void sendBrokerInfo(OpenWireConnection connection) throws Exception {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
index cabd049..db5dfd6 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
@@ -16,7 +16,6 @@
  */
 package org.apache.activemq.artemis.core.protocol.stomp;
 
-import javax.security.cert.X509Certificate;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
@@ -561,8 +560,8 @@ public final class StompConnection implements RemotingConnection {
       manager.sendReply(this, frame);
    }
 
-   public boolean validateUser(final String login, final String pass, final X509Certificate[] certificates) {
-      this.valid = manager.validateUser(login, pass, certificates);
+   public boolean validateUser(final String login, final String pass, final Connection connection) {
+      this.valid = manager.validateUser(login, pass, connection);
       if (valid) {
          this.login = login;
          this.passcode = pass;

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompProtocolManager.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompProtocolManager.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompProtocolManager.java
index 181891a..fb60847 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompProtocolManager.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompProtocolManager.java
@@ -16,7 +16,6 @@
  */
 package org.apache.activemq.artemis.core.protocol.stomp;
 
-import javax.security.cert.X509Certificate;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -36,6 +35,7 @@ import org.apache.activemq.artemis.core.io.IOCallback;
 import org.apache.activemq.artemis.core.message.impl.CoreMessage;
 import org.apache.activemq.artemis.core.remoting.impl.netty.NettyServerConnection;
 import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
+import org.apache.activemq.artemis.core.remoting.CertificateUtil;
 import org.apache.activemq.artemis.core.server.ActiveMQServer;
 import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
 import org.apache.activemq.artemis.core.server.ServerSession;
@@ -320,16 +320,16 @@ public class StompProtocolManager extends AbstractProtocolManager<StompFrame, St
       return "activemq";
    }
 
-   public boolean validateUser(String login, String passcode, X509Certificate[] certificates) {
+   public boolean validateUser(String login, String passcode, Connection connection) {
       boolean validated = true;
 
       ActiveMQSecurityManager sm = server.getSecurityManager();
 
       if (sm != null && server.getConfiguration().isSecurityEnabled()) {
          if (sm instanceof ActiveMQSecurityManager3) {
-            validated = ((ActiveMQSecurityManager3) sm).validateUser(login, passcode, certificates) != null;
+            validated = ((ActiveMQSecurityManager3) sm).validateUser(login, passcode, connection) != null;
          } else if (sm instanceof ActiveMQSecurityManager2) {
-            validated = ((ActiveMQSecurityManager2) sm).validateUser(login, passcode, certificates);
+            validated = ((ActiveMQSecurityManager2) sm).validateUser(login, passcode, CertificateUtil.getCertsFromConnection(connection));
          } else {
             validated = sm.validateUser(login, passcode);
          }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
index 4a0cfa1..51abf50 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
@@ -16,7 +16,6 @@
  */
 package org.apache.activemq.artemis.core.protocol.stomp.v10;
 
-import javax.security.cert.X509Certificate;
 import java.util.Map;
 import java.util.concurrent.ScheduledExecutorService;
 
@@ -28,9 +27,7 @@ import org.apache.activemq.artemis.core.protocol.stomp.StompDecoder;
 import org.apache.activemq.artemis.core.protocol.stomp.StompFrame;
 import org.apache.activemq.artemis.core.protocol.stomp.StompVersions;
 import org.apache.activemq.artemis.core.protocol.stomp.VersionedStompFrameHandler;
-import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnection;
 import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
-import org.apache.activemq.artemis.utils.CertificateUtil;
 import org.apache.activemq.artemis.utils.ExecutorFactory;
 
 import static org.apache.activemq.artemis.core.protocol.stomp.ActiveMQStompProtocolMessageBundle.BUNDLE;
@@ -55,12 +52,7 @@ public class StompFrameHandlerV10 extends VersionedStompFrameHandler implements
       String clientID = headers.get(Stomp.Headers.Connect.CLIENT_ID);
       String requestID = headers.get(Stomp.Headers.Connect.REQUEST_ID);
 
-      X509Certificate[] certificates = null;
-      if (connection.getTransportConnection() instanceof NettyConnection) {
-         certificates = CertificateUtil.getCertsFromChannel(((NettyConnection) connection.getTransportConnection()).getChannel());
-      }
-
-      if (connection.validateUser(login, passcode, certificates)) {
+      if (connection.validateUser(login, passcode, connection.getTransportConnection())) {
          connection.setClientID(clientID);
          connection.setValid(true);
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
index c6831cd..9e013f8 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
@@ -16,7 +16,6 @@
  */
 package org.apache.activemq.artemis.core.protocol.stomp.v11;
 
-import javax.security.cert.X509Certificate;
 import java.util.Map;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ScheduledExecutorService;
@@ -31,13 +30,11 @@ import org.apache.activemq.artemis.core.protocol.stomp.StompConnection;
 import org.apache.activemq.artemis.core.protocol.stomp.StompDecoder;
 import org.apache.activemq.artemis.core.protocol.stomp.StompFrame;
 import org.apache.activemq.artemis.core.protocol.stomp.VersionedStompFrameHandler;
-import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnection;
 import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
 import org.apache.activemq.artemis.core.remoting.server.impl.RemotingServiceImpl;
 import org.apache.activemq.artemis.core.server.ActiveMQScheduledComponent;
 import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
 import org.apache.activemq.artemis.spi.core.protocol.ConnectionEntry;
-import org.apache.activemq.artemis.utils.CertificateUtil;
 import org.apache.activemq.artemis.utils.ExecutorFactory;
 
 import static org.apache.activemq.artemis.core.protocol.stomp.ActiveMQStompProtocolMessageBundle.BUNDLE;
@@ -70,13 +67,8 @@ public class StompFrameHandlerV11 extends VersionedStompFrameHandler implements
       String clientID = headers.get(Stomp.Headers.Connect.CLIENT_ID);
       String requestID = headers.get(Stomp.Headers.Connect.REQUEST_ID);
 
-      X509Certificate[] certificates = null;
-      if (connection.getTransportConnection() instanceof NettyConnection) {
-         certificates = CertificateUtil.getCertsFromChannel(((NettyConnection) connection.getTransportConnection()).getChannel());
-      }
-
       try {
-         if (connection.validateUser(login, passcode, certificates)) {
+         if (connection.validateUser(login, passcode, connection.getTransportConnection())) {
             connection.setClientID(clientID);
             connection.setValid(true);
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-server/src/main/java/org/apache/activemq/artemis/core/security/SecurityStore.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/security/SecurityStore.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/security/SecurityStore.java
index 87ba380..1f65443 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/security/SecurityStore.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/security/SecurityStore.java
@@ -16,13 +16,12 @@
  */
 package org.apache.activemq.artemis.core.security;
 
-import javax.security.cert.X509Certificate;
-
 import org.apache.activemq.artemis.api.core.SimpleString;
+import org.apache.activemq.artemis.spi.core.remoting.Connection;
 
 public interface SecurityStore {
 
-   String authenticate(String user, String password, X509Certificate[] certificates) throws Exception;
+   String authenticate(String user, String password, Connection transportConnection) throws Exception;
 
    void check(SimpleString address, CheckType checkType, SecurityAuth session) throws Exception;
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-server/src/main/java/org/apache/activemq/artemis/core/security/impl/SecurityStoreImpl.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/security/impl/SecurityStoreImpl.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/security/impl/SecurityStoreImpl.java
index 2f5f0d5..957a318 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/security/impl/SecurityStoreImpl.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/security/impl/SecurityStoreImpl.java
@@ -16,7 +16,6 @@
  */
 package org.apache.activemq.artemis.core.security.impl;
 
-import javax.security.cert.X509Certificate;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -24,6 +23,7 @@ import java.util.concurrent.ConcurrentMap;
 import org.apache.activemq.artemis.api.core.SimpleString;
 import org.apache.activemq.artemis.api.core.management.CoreNotificationType;
 import org.apache.activemq.artemis.api.core.management.ManagementHelper;
+import org.apache.activemq.artemis.core.remoting.CertificateUtil;
 import org.apache.activemq.artemis.core.security.CheckType;
 import org.apache.activemq.artemis.core.security.Role;
 import org.apache.activemq.artemis.core.security.SecurityAuth;
@@ -33,6 +33,7 @@ import org.apache.activemq.artemis.core.server.management.Notification;
 import org.apache.activemq.artemis.core.server.management.NotificationService;
 import org.apache.activemq.artemis.core.settings.HierarchicalRepository;
 import org.apache.activemq.artemis.core.settings.HierarchicalRepositoryChangeListener;
+import org.apache.activemq.artemis.spi.core.remoting.Connection;
 import org.apache.activemq.artemis.spi.core.security.ActiveMQSecurityManager;
 import org.apache.activemq.artemis.spi.core.security.ActiveMQSecurityManager2;
 import org.apache.activemq.artemis.spi.core.security.ActiveMQSecurityManager3;
@@ -102,7 +103,7 @@ public class SecurityStoreImpl implements SecurityStore, HierarchicalRepositoryC
    @Override
    public String authenticate(final String user,
                               final String password,
-                              X509Certificate[] certificates) throws Exception {
+                              Connection connection) throws Exception {
       if (securityEnabled) {
 
          if (managementClusterUser.equals(user)) {
@@ -125,9 +126,9 @@ public class SecurityStoreImpl implements SecurityStore, HierarchicalRepositoryC
          boolean userIsValid = false;
 
          if (securityManager instanceof ActiveMQSecurityManager3) {
-            validatedUser = ((ActiveMQSecurityManager3) securityManager).validateUser(user, password, certificates);
+            validatedUser = ((ActiveMQSecurityManager3) securityManager).validateUser(user, password, connection);
          } else if (securityManager instanceof ActiveMQSecurityManager2) {
-            userIsValid = ((ActiveMQSecurityManager2) securityManager).validateUser(user, password, certificates);
+            userIsValid = ((ActiveMQSecurityManager2) securityManager).validateUser(user, password, CertificateUtil.getCertsFromConnection(connection));
          } else {
             userIsValid = securityManager.validateUser(user, password);
          }
@@ -177,7 +178,7 @@ public class SecurityStoreImpl implements SecurityStore, HierarchicalRepositoryC
          final boolean validated;
          if (securityManager instanceof ActiveMQSecurityManager3) {
             final ActiveMQSecurityManager3 securityManager3 = (ActiveMQSecurityManager3) securityManager;
-            validated = securityManager3.validateUserAndRole(user, session.getPassword(), roles, checkType, saddress, session.getRemotingConnection()) != null;
+            validated = securityManager3.validateUserAndRole(user, session.getPassword(), roles, checkType, saddress, session.getRemotingConnection().getTransportConnection()) != null;
          } else if (securityManager instanceof ActiveMQSecurityManager2) {
             final ActiveMQSecurityManager2 securityManager2 = (ActiveMQSecurityManager2) securityManager;
             validated = securityManager2.validateUserAndRole(user, session.getPassword(), roles, checkType, saddress, session.getRemotingConnection());

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
index be42ea6..dec338b 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
@@ -17,7 +17,6 @@
 package org.apache.activemq.artemis.core.server.impl;
 
 import javax.management.MBeanServer;
-import javax.security.cert.X509Certificate;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -98,7 +97,6 @@ import org.apache.activemq.artemis.core.postoffice.QueueBinding;
 import org.apache.activemq.artemis.core.postoffice.impl.DivertBinding;
 import org.apache.activemq.artemis.core.postoffice.impl.LocalQueueBinding;
 import org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl;
-import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnection;
 import org.apache.activemq.artemis.core.remoting.server.RemotingService;
 import org.apache.activemq.artemis.core.remoting.server.impl.RemotingServiceImpl;
 import org.apache.activemq.artemis.core.replication.ReplicationEndpoint;
@@ -164,7 +162,6 @@ import org.apache.activemq.artemis.spi.core.protocol.SessionCallback;
 import org.apache.activemq.artemis.spi.core.security.ActiveMQSecurityManager;
 import org.apache.activemq.artemis.utils.ActiveMQThreadFactory;
 import org.apache.activemq.artemis.utils.ActiveMQThreadPoolExecutor;
-import org.apache.activemq.artemis.utils.CertificateUtil;
 import org.apache.activemq.artemis.utils.CompositeAddress;
 import org.apache.activemq.artemis.utils.ExecutorFactory;
 import org.apache.activemq.artemis.utils.OrderedExecutorFactory;
@@ -1308,11 +1305,7 @@ public class ActiveMQServerImpl implements ActiveMQServer {
       String validatedUser = "";
 
       if (securityStore != null) {
-         X509Certificate[] certificates = null;
-         if (connection.getTransportConnection() instanceof NettyConnection) {
-            certificates = CertificateUtil.getCertsFromChannel(((NettyConnection) connection.getTransportConnection()).getChannel());
-         }
-         validatedUser = securityStore.authenticate(username, password, certificates);
+         validatedUser = securityStore.authenticate(username, password, connection.getTransportConnection());
       }
 
       checkSessionLimit(validatedUser);

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/ActiveMQJAASSecurityManager.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/ActiveMQJAASSecurityManager.java b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/ActiveMQJAASSecurityManager.java
index 973ea96..dc4e3a8 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/ActiveMQJAASSecurityManager.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/ActiveMQJAASSecurityManager.java
@@ -19,7 +19,6 @@ package org.apache.activemq.artemis.spi.core.security;
 import javax.security.auth.Subject;
 import javax.security.auth.login.LoginContext;
 import javax.security.auth.login.LoginException;
-import javax.security.cert.X509Certificate;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
 import java.security.Principal;
@@ -28,16 +27,16 @@ import java.util.Iterator;
 import java.util.Set;
 
 import org.apache.activemq.artemis.core.config.impl.SecurityConfiguration;
-import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnection;
 import org.apache.activemq.artemis.core.security.CheckType;
 import org.apache.activemq.artemis.core.security.Role;
-import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
+import org.apache.activemq.artemis.spi.core.remoting.Connection;
 import org.apache.activemq.artemis.spi.core.security.jaas.JaasCallbackHandler;
 import org.apache.activemq.artemis.spi.core.security.jaas.RolePrincipal;
 import org.apache.activemq.artemis.spi.core.security.jaas.UserPrincipal;
-import org.apache.activemq.artemis.utils.CertificateUtil;
 import org.jboss.logging.Logger;
 
+import static org.apache.activemq.artemis.core.remoting.CertificateUtil.getCertsFromConnection;
+
 /**
  * This implementation delegates to the JAAS security interfaces.
  *
@@ -89,9 +88,9 @@ public class ActiveMQJAASSecurityManager implements ActiveMQSecurityManager3 {
    }
 
    @Override
-   public String validateUser(final String user, final String password, X509Certificate[] certificates) {
+   public String validateUser(final String user, final String password, Connection connection) {
       try {
-         return getUserFromSubject(getAuthenticatedSubject(user, password, certificates));
+         return getUserFromSubject(getAuthenticatedSubject(user, password, connection));
       } catch (LoginException e) {
          if (logger.isDebugEnabled()) {
             logger.debug("Couldn't validate user", e);
@@ -122,14 +121,10 @@ public class ActiveMQJAASSecurityManager implements ActiveMQSecurityManager3 {
                                      final Set<Role> roles,
                                      final CheckType checkType,
                                      final String address,
-                                     final RemotingConnection connection) {
-      X509Certificate[] certificates = null;
-      if (connection != null && connection.getTransportConnection() instanceof NettyConnection) {
-         certificates = CertificateUtil.getCertsFromChannel(((NettyConnection) connection.getTransportConnection()).getChannel());
-      }
+                                     final Connection connection) {
       Subject localSubject;
       try {
-         localSubject = getAuthenticatedSubject(user, password, certificates);
+         localSubject = getAuthenticatedSubject(user, password, connection);
       } catch (LoginException e) {
          if (logger.isDebugEnabled()) {
             logger.debug("Couldn't validate user", e);
@@ -175,7 +170,7 @@ public class ActiveMQJAASSecurityManager implements ActiveMQSecurityManager3 {
 
    private Subject getAuthenticatedSubject(final String user,
                                            final String password,
-                                           final X509Certificate[] certificates) throws LoginException {
+                                           final Connection connection) throws LoginException {
       LoginContext lc;
       ClassLoader currentLoader = Thread.currentThread().getContextClassLoader();
       ClassLoader thisLoader = this.getClass().getClassLoader();
@@ -183,10 +178,10 @@ public class ActiveMQJAASSecurityManager implements ActiveMQSecurityManager3 {
          if (thisLoader != currentLoader) {
             Thread.currentThread().setContextClassLoader(thisLoader);
          }
-         if (certificateConfigurationName != null && certificateConfigurationName.length() > 0 && certificates != null) {
-            lc = new LoginContext(certificateConfigurationName, null, new JaasCallbackHandler(user, password, certificates), certificateConfiguration);
+         if (certificateConfigurationName != null && certificateConfigurationName.length() > 0 && getCertsFromConnection(connection) != null) {
+            lc = new LoginContext(certificateConfigurationName, null, new JaasCallbackHandler(user, password, connection), certificateConfiguration);
          } else {
-            lc = new LoginContext(configurationName, null, new JaasCallbackHandler(user, password, certificates), configuration);
+            lc = new LoginContext(configurationName, null, new JaasCallbackHandler(user, password, connection), configuration);
          }
          lc.login();
          return lc.getSubject();

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/ActiveMQSecurityManager3.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/ActiveMQSecurityManager3.java b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/ActiveMQSecurityManager3.java
index d565da4..336e812 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/ActiveMQSecurityManager3.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/ActiveMQSecurityManager3.java
@@ -16,12 +16,11 @@
  */
 package org.apache.activemq.artemis.spi.core.security;
 
-import javax.security.cert.X509Certificate;
 import java.util.Set;
 
 import org.apache.activemq.artemis.core.security.CheckType;
 import org.apache.activemq.artemis.core.security.Role;
-import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
+import org.apache.activemq.artemis.spi.core.remoting.Connection;
 
 /**
  * Used to validate whether a user is authorized to connect to the
@@ -43,7 +42,7 @@ public interface ActiveMQSecurityManager3 extends ActiveMQSecurityManager {
     * @param password the users password
     * @return the name of the validated user or null if the user isn't validated
     */
-   String validateUser(String user, String password, X509Certificate[] certificates);
+   String validateUser(String user, String password, Connection connection);
 
    /**
     * Determine whether the given user is valid and whether they have
@@ -65,5 +64,5 @@ public interface ActiveMQSecurityManager3 extends ActiveMQSecurityManager {
                               Set<Role> roles,
                               CheckType checkType,
                               String address,
-                              RemotingConnection connection);
+                              Connection connection);
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/JaasCallbackHandler.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/JaasCallbackHandler.java b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/JaasCallbackHandler.java
index ca4edbe..a02d237 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/JaasCallbackHandler.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/JaasCallbackHandler.java
@@ -16,14 +16,18 @@
  */
 package org.apache.activemq.artemis.spi.core.security.jaas;
 
+import org.apache.activemq.artemis.spi.core.remoting.Connection;
+
 import javax.security.auth.callback.Callback;
 import javax.security.auth.callback.CallbackHandler;
 import javax.security.auth.callback.NameCallback;
 import javax.security.auth.callback.PasswordCallback;
 import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.cert.X509Certificate;
 import java.io.IOException;
 
+import static org.apache.activemq.artemis.core.remoting.CertificateUtil.getCertsFromConnection;
+import static org.apache.activemq.artemis.core.remoting.CertificateUtil.getPeerPrincipalFromConnection;
+
 /**
  * A JAAS username password CallbackHandler.
  */
@@ -31,12 +35,12 @@ public class JaasCallbackHandler implements CallbackHandler {
 
    private final String username;
    private final String password;
-   final X509Certificate[] certificates;
+   final Connection connection;
 
-   public JaasCallbackHandler(String username, String password, X509Certificate[] certs) {
+   public JaasCallbackHandler(String username, String password, Connection connection) {
       this.username = username;
       this.password = password;
-      this.certificates = certs;
+      this.connection = connection;
    }
 
    @Override
@@ -59,7 +63,11 @@ public class JaasCallbackHandler implements CallbackHandler {
          } else if (callback instanceof CertificateCallback) {
             CertificateCallback certCallback = (CertificateCallback) callback;
 
-            certCallback.setCertificates(certificates);
+            certCallback.setCertificates(getCertsFromConnection(connection));
+         } else if (callback instanceof Krb5SslCallback) {
+            Krb5SslCallback krb5SslCallback = (Krb5SslCallback) callback;
+
+            krb5SslCallback.setPeerPrincipal(getPeerPrincipalFromConnection(connection));
          } else {
             throw new UnsupportedCallbackException(callback);
          }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/Krb5SslCallback.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/Krb5SslCallback.java b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/Krb5SslCallback.java
new file mode 100644
index 0000000..62c80db
--- /dev/null
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/Krb5SslCallback.java
@@ -0,0 +1,46 @@
+/*
+ * 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 org.apache.activemq.artemis.spi.core.security.jaas;
+
+import javax.security.auth.callback.Callback;
+import java.security.Principal;
+
+/**
+ * A Callback for SSL kerberos peer principal.
+ */
+public class Krb5SslCallback implements Callback {
+
+   Principal peerPrincipal;
+
+   /**
+    * Setter for peer Principal.
+    *
+    * @param principal The certificates to be returned.
+    */
+   public void setPeerPrincipal(Principal principal) {
+      peerPrincipal = principal;
+   }
+
+   /**
+    * Getter for peer Principal.
+    *
+    * @return The principal being carried.
+    */
+   public Principal getPeerPrincipal() {
+      return peerPrincipal;
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/Krb5SslLoginModule.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/Krb5SslLoginModule.java b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/Krb5SslLoginModule.java
new file mode 100644
index 0000000..1f6b5b1
--- /dev/null
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/Krb5SslLoginModule.java
@@ -0,0 +1,112 @@
+/*
+ * 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 org.apache.activemq.artemis.spi.core.security.jaas;
+
+import org.jboss.logging.Logger;
+
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.LoginException;
+import javax.security.auth.spi.LoginModule;
+import java.io.IOException;
+import java.security.Principal;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * populate a subject with kerberos and UserPrincipal from SSLContext peerPrincipal
+ */
+public class Krb5SslLoginModule implements LoginModule {
+
+   private static final Logger logger = Logger.getLogger(Krb5SslLoginModule.class);
+
+   private Subject subject;
+   private final List<Principal> principals = new LinkedList<>();
+   private CallbackHandler callbackHandler;
+   private boolean loginSucceeded;
+
+   @Override
+   public void initialize(Subject subject,
+                          CallbackHandler callbackHandler,
+                          Map<String, ?> sharedState,
+                          Map<String, ?> options) {
+      this.subject = subject;
+      this.callbackHandler = callbackHandler;
+   }
+
+   @Override
+   public boolean login() throws LoginException {
+      Callback[] callbacks = new Callback[1];
+
+      callbacks[0] = new Krb5SslCallback();
+      try {
+         callbackHandler.handle(callbacks);
+      } catch (IOException ioe) {
+         throw new LoginException(ioe.getMessage());
+      } catch (UnsupportedCallbackException uce) {
+         throw new LoginException(uce.getMessage() + " not available to obtain information from user");
+      }
+      principals.add(((Krb5SslCallback)callbacks[0]).getPeerPrincipal());
+      if (!principals.isEmpty()) {
+         loginSucceeded = true;
+      }
+      logger.debug("login " + principals);
+      return loginSucceeded;
+   }
+
+   @Override
+   public boolean commit() throws LoginException {
+      boolean result = loginSucceeded;
+      if (result) {
+         principals.add(new UserPrincipal(principals.get(0).getName()));
+         subject.getPrincipals().addAll(principals);
+      }
+
+      clear();
+
+      logger.debug("commit, result: " + result);
+
+      return result;
+   }
+
+   @Override
+   public boolean abort() throws LoginException {
+      clear();
+
+      logger.debug("abort");
+
+      return true;
+   }
+
+   private void clear() {
+      loginSucceeded = false;
+   }
+
+   @Override
+   public boolean logout() throws LoginException {
+      subject.getPrincipals().removeAll(principals);
+      principals.clear();
+      clear();
+
+      logger.debug("logout");
+
+      return true;
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/PropertiesLoginModule.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/PropertiesLoginModule.java b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/PropertiesLoginModule.java
index cbe5e4f..a516a9b 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/PropertiesLoginModule.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/spi/core/security/jaas/PropertiesLoginModule.java
@@ -114,19 +114,25 @@ public class PropertiesLoginModule extends PropertiesLoader implements LoginModu
    @Override
    public boolean commit() throws LoginException {
       boolean result = loginSucceeded;
+      Set<UserPrincipal> authenticatedUsers = subject.getPrincipals(UserPrincipal.class);
       if (result) {
-         principals.add(new UserPrincipal(user));
+         UserPrincipal userPrincipal = new UserPrincipal(user);
+         principals.add(userPrincipal);
+         authenticatedUsers.add(userPrincipal);
+      }
 
-         Set<String> matchedRoles = roles.get(user);
+      // populate roles for UserPrincipal from other login modules too
+      for (UserPrincipal userPrincipal : authenticatedUsers) {
+         Set<String> matchedRoles = roles.get(userPrincipal.getName());
          if (matchedRoles != null) {
             for (String entry : matchedRoles) {
                principals.add(new RolePrincipal(entry));
             }
          }
-
-         subject.getPrincipals().addAll(principals);
       }
 
+      subject.getPrincipals().addAll(principals);
+
       // will whack loginSucceeded
       clear();
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/artemis-server/src/test/java/org/apache/activemq/artemis/core/security/jaas/TextFileCertificateLoginModuleTest.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/test/java/org/apache/activemq/artemis/core/security/jaas/TextFileCertificateLoginModuleTest.java b/artemis-server/src/test/java/org/apache/activemq/artemis/core/security/jaas/TextFileCertificateLoginModuleTest.java
index 0b6f65c..2eebd06 100644
--- a/artemis-server/src/test/java/org/apache/activemq/artemis/core/security/jaas/TextFileCertificateLoginModuleTest.java
+++ b/artemis-server/src/test/java/org/apache/activemq/artemis/core/security/jaas/TextFileCertificateLoginModuleTest.java
@@ -18,8 +18,11 @@ package org.apache.activemq.artemis.core.security.jaas;
 
 import javax.management.remote.JMXPrincipal;
 import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
 import javax.security.auth.login.LoginException;
 import javax.security.cert.X509Certificate;
+import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.net.URL;
 import java.net.URLDecoder;
@@ -27,6 +30,7 @@ import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 
 import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
+import org.apache.activemq.artemis.spi.core.security.jaas.CertificateCallback;
 import org.apache.activemq.artemis.spi.core.security.jaas.CertificateLoginModule;
 import org.apache.activemq.artemis.spi.core.security.jaas.JaasCallbackHandler;
 import org.apache.activemq.artemis.spi.core.security.jaas.PropertiesLoader;
@@ -121,7 +125,19 @@ public class TextFileCertificateLoginModuleTest {
    private JaasCallbackHandler getJaasCertificateCallbackHandler(String user) {
       JMXPrincipal principal = new JMXPrincipal(user);
       X509Certificate cert = new StubX509Certificate(principal);
-      return new JaasCallbackHandler(null, null, new X509Certificate[]{cert});
+      return new JaasCallbackHandler(null, null, null) {
+         @Override
+         public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+            for (Callback callback : callbacks) {
+               if (callback instanceof CertificateCallback) {
+                  CertificateCallback certCallback = (CertificateCallback) callback;
+                  certCallback.setCertificates(new X509Certificate[]{cert});
+               } else {
+                  throw new UnsupportedCallbackException(callback);
+               }
+            }
+         }
+      };
    }
 
    private Subject doAuthenticate(HashMap<String, ?> options,

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/security/SecurityTest.java
----------------------------------------------------------------------
diff --git a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/security/SecurityTest.java b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/security/SecurityTest.java
index ae2efd0..31acfca 100644
--- a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/security/SecurityTest.java
+++ b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/security/SecurityTest.java
@@ -54,6 +54,7 @@ import org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl;
 import org.apache.activemq.artemis.core.server.impl.AddressInfo;
 import org.apache.activemq.artemis.core.settings.HierarchicalRepository;
 import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
+import org.apache.activemq.artemis.spi.core.remoting.Connection;
 import org.apache.activemq.artemis.spi.core.security.ActiveMQJAASSecurityManager;
 import org.apache.activemq.artemis.spi.core.security.ActiveMQSecurityManager;
 import org.apache.activemq.artemis.spi.core.security.ActiveMQSecurityManager2;
@@ -1935,7 +1936,7 @@ public class SecurityTest extends ActiveMQTestBase {
          @Override
          public String validateUser(final String username,
                                     final String password,
-                                    final X509Certificate[] certificates) {
+                                    final Connection connection) {
             if ((username.equals("foo") || username.equals("bar") || username.equals("all")) && password.equals("frobnicate")) {
                return username;
             } else {
@@ -1959,9 +1960,9 @@ public class SecurityTest extends ActiveMQTestBase {
                                            final Set<Role> requiredRoles,
                                            final CheckType checkType,
                                            final String address,
-                                           final RemotingConnection connection) {
+                                           final Connection connection) {
 
-            if (!(connection.getTransportConnection() instanceof InVMConnection)) {
+            if (!(connection instanceof InVMConnection)) {
                return null;
             }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/ssl/CoreClientOverOneWaySSLKerb5Test.java
----------------------------------------------------------------------
diff --git a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/ssl/CoreClientOverOneWaySSLKerb5Test.java b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/ssl/CoreClientOverOneWaySSLKerb5Test.java
index 6c26667..1dd238f 100644
--- a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/ssl/CoreClientOverOneWaySSLKerb5Test.java
+++ b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/ssl/CoreClientOverOneWaySSLKerb5Test.java
@@ -16,6 +16,7 @@
  */
 package org.apache.activemq.artemis.tests.integration.ssl;
 
+import org.apache.activemq.artemis.api.core.ActiveMQSecurityException;
 import org.apache.activemq.artemis.api.core.RoutingType;
 import org.apache.activemq.artemis.api.core.SimpleString;
 import org.apache.activemq.artemis.api.core.TransportConfiguration;
@@ -32,6 +33,8 @@ import org.apache.activemq.artemis.core.security.Role;
 import org.apache.activemq.artemis.core.server.ActiveMQServer;
 import org.apache.activemq.artemis.core.settings.HierarchicalRepository;
 import org.apache.activemq.artemis.spi.core.security.ActiveMQJAASSecurityManager;
+import org.apache.activemq.artemis.core.server.ActiveMQServers;
+import org.apache.activemq.artemis.spi.core.security.ActiveMQSecurityManager;
 import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
 import org.apache.activemq.artemis.utils.RandomUtil;
 import org.apache.hadoop.minikdc.MiniKdc;
@@ -41,6 +44,8 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
+import java.lang.management.ManagementFactory;
+import java.net.URL;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -53,10 +58,23 @@ public class CoreClientOverOneWaySSLKerb5Test extends ActiveMQTestBase {
    public static final String SNI_HOST = "sni.host";
    public static final String SERVICE_PRINCIPAL = "host/" + SNI_HOST;
 
+   static {
+      String path = System.getProperty("java.security.auth.login.config");
+      if (path == null) {
+         URL resource = CoreClientOverOneWaySSLKerb5Test.class.getClassLoader().getResource("login.config");
+         if (resource != null) {
+            path = resource.getFile();
+            System.setProperty("java.security.auth.login.config", path);
+         }
+      }
+   }
+
    private MiniKdc kdc;
    private ActiveMQServer server;
 
    private TransportConfiguration tc;
+   private TransportConfiguration inVMTc;
+   private String userPrincipal;
 
    @Test
    public void testOneWaySSLWithGoodClientCipherSuite() throws Exception {
@@ -103,6 +121,23 @@ public class CoreClientOverOneWaySSLKerb5Test extends ActiveMQTestBase {
          }
          locator.close();
       }
+
+      // validate only ssl creds work, try and fake the principal w/o ssl
+      final ServerLocator inVmLocator = addServerLocator(ActiveMQClient.createServerLocatorWithoutHA(inVMTc));
+      ClientSessionFactory inVmSf = null;
+      try {
+         inVmSf = createSessionFactory(inVmLocator);
+         inVmSf.createSession(userPrincipal, "", false, false, false, false, 10);
+
+         fail("supposed to throw exception");
+      } catch (ActiveMQSecurityException e) {
+         // expected
+      } finally {
+         if (inVmSf != null) {
+            inVmSf.close();
+         }
+         inVmLocator.close();
+      }
    }
 
 
@@ -142,22 +177,26 @@ public class CoreClientOverOneWaySSLKerb5Test extends ActiveMQTestBase {
       config.setPopulateValidatedUser(true); // so we can verify the kerb5 id is present
       config.setSecurityEnabled(true);
 
-      server = createServer(false, config);
-      server.start();
-      waitForServerToStart(server);
+      config.addAcceptorConfiguration(new TransportConfiguration(INVM_ACCEPTOR_FACTORY));
+
+      ActiveMQSecurityManager securityManager = new ActiveMQJAASSecurityManager("Krb5SslPlus");
+      server = addServer(ActiveMQServers.newActiveMQServer(config, ManagementFactory.getPlatformMBeanServer(), securityManager, false));
+      HierarchicalRepository<Set<Role>> securityRepository = server.getSecurityRepository();
+
 
       final String roleName = "ALLOW_ALL";
       Role role = new Role(roleName, true, true, true, true, true, true, true, true, true, true);
       Set<Role> roles = new HashSet<>();
       roles.add(role);
-      HierarchicalRepository<Set<Role>> securityRepository = server.getSecurityRepository();
       securityRepository.addMatch(QUEUE.toString(), roles);
-      ActiveMQJAASSecurityManager securityManager = (ActiveMQJAASSecurityManager) server.getSecurityManager();
 
-      final String user = CLIENT_PRINCIPAL + "@" + kdc.getRealm();
-      securityManager.getConfiguration().addUser(user, "");
-      securityManager.getConfiguration().addRole(user, roleName);
+      server.start();
+      waitForServerToStart(server);
+
+      // note kerberos user does not exist on the broker save as a role member in dual-authentication-roles.properties
+      userPrincipal = CLIENT_PRINCIPAL + "@" + kdc.getRealm();
 
       tc = new TransportConfiguration(NETTY_CONNECTOR_FACTORY);
+      inVMTc = new TransportConfiguration(INVM_CONNECTOR_FACTORY);
    }
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/tests/integration-tests/src/test/resources/dual-authentication-roles.properties
----------------------------------------------------------------------
diff --git a/tests/integration-tests/src/test/resources/dual-authentication-roles.properties b/tests/integration-tests/src/test/resources/dual-authentication-roles.properties
index 9c93173..cbc215a 100644
--- a/tests/integration-tests/src/test/resources/dual-authentication-roles.properties
+++ b/tests/integration-tests/src/test/resources/dual-authentication-roles.properties
@@ -15,4 +15,5 @@
 # limitations under the License.
 #
 
-consumers=consumer
\ No newline at end of file
+consumers=consumer
+ALLOW_ALL=client@EXAMPLE.COM
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/d4150be0/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 6bc3498..a8ce3e0 100644
--- a/tests/integration-tests/src/test/resources/login.config
+++ b/tests/integration-tests/src/test/resources/login.config
@@ -137,3 +137,14 @@ DualAuthenticationPropertiesLogin {
         org.apache.activemq.jaas.properties.user="dual-authentication-users.properties"
         org.apache.activemq.jaas.properties.role="dual-authentication-roles.properties";
 };
+
+Krb5SslPlus {
+
+    org.apache.activemq.artemis.spi.core.security.jaas.Krb5SslLoginModule optional
+        debug=true;
+
+    org.apache.activemq.artemis.spi.core.security.jaas.PropertiesLoginModule optional
+        debug=true
+        org.apache.activemq.jaas.properties.user="dual-authentication-users.properties"
+        org.apache.activemq.jaas.properties.role="dual-authentication-roles.properties";
+};