You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cxf.apache.org by co...@apache.org on 2013/04/22 16:01:35 UTC

svn commit: r1470518 - in /cxf/branches/wss4j2.0-port/rt/ws/security/src: main/java/org/apache/cxf/ws/security/wss4j/ test/java/org/apache/cxf/ws/security/wss4j/

Author: coheigea
Date: Mon Apr 22 14:01:35 2013
New Revision: 1470518

URL: http://svn.apache.org/r1470518
Log:
Added functionality to create a CXF SecurityContext from the StaX WSS4J results/events

Added:
    cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/StaxSecurityContextInInterceptor.java
    cxf/branches/wss4j2.0-port/rt/ws/security/src/test/java/org/apache/cxf/ws/security/wss4j/WSS4JPrincipalInterceptor.java
Modified:
    cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/WSS4JStaxInInterceptor.java
    cxf/branches/wss4j2.0-port/rt/ws/security/src/test/java/org/apache/cxf/ws/security/wss4j/StaxRoundTripTest.java

Added: cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/StaxSecurityContextInInterceptor.java
URL: http://svn.apache.org/viewvc/cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/StaxSecurityContextInInterceptor.java?rev=1470518&view=auto
==============================================================================
--- cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/StaxSecurityContextInInterceptor.java (added)
+++ cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/StaxSecurityContextInInterceptor.java Mon Apr 22 14:01:35 2013
@@ -0,0 +1,221 @@
+/**
+ * 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.cxf.ws.security.wss4j;
+
+import java.security.Principal;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import javax.security.auth.Subject;
+
+import org.apache.cxf.binding.soap.SoapFault;
+import org.apache.cxf.binding.soap.SoapMessage;
+import org.apache.cxf.binding.soap.SoapVersion;
+import org.apache.cxf.common.security.SimplePrincipal;
+import org.apache.cxf.interceptor.Fault;
+import org.apache.cxf.interceptor.security.DefaultSecurityContext;
+import org.apache.cxf.interceptor.security.RolePrefixSecurityContextImpl;
+import org.apache.cxf.interceptor.security.SAMLSecurityContext;
+import org.apache.cxf.phase.AbstractPhaseInterceptor;
+import org.apache.cxf.phase.Phase;
+import org.apache.cxf.security.SecurityContext;
+import org.apache.cxf.ws.security.SecurityConstants;
+import org.apache.wss4j.common.ext.WSSecurityException;
+import org.apache.wss4j.common.principal.CustomTokenPrincipal;
+import org.apache.wss4j.common.principal.WSDerivedKeyTokenPrincipal;
+import org.apache.wss4j.stax.securityEvent.KerberosTokenSecurityEvent;
+import org.apache.wss4j.stax.securityEvent.KeyValueTokenSecurityEvent;
+import org.apache.wss4j.stax.securityEvent.SamlTokenSecurityEvent;
+import org.apache.wss4j.stax.securityEvent.UsernameTokenSecurityEvent;
+import org.apache.wss4j.stax.securityEvent.WSSecurityEventConstants;
+import org.apache.wss4j.stax.securityEvent.X509TokenSecurityEvent;
+import org.apache.wss4j.stax.securityToken.SubjectAndPrincipalSecurityToken;
+import org.apache.xml.security.stax.securityEvent.SecurityEvent;
+
+/**
+ * This interceptor handles parsing the StaX WS-Security results (events) + sets up the
+ * security context appropriately.
+ */
+public class StaxSecurityContextInInterceptor extends AbstractPhaseInterceptor<SoapMessage> {
+    
+    /**
+     * This configuration tag specifies the default attribute name where the roles are present
+     * The default is "http://schemas.xmlsoap.org/ws/2005/05/identity/claims/role".
+     */
+    public static final String SAML_ROLE_ATTRIBUTENAME_DEFAULT =
+        "http://schemas.xmlsoap.org/ws/2005/05/identity/claims/role";
+    
+    public StaxSecurityContextInInterceptor() {
+        super(Phase.PRE_PROTOCOL);
+    }
+
+    @Override
+    public void handleMessage(SoapMessage soapMessage) throws Fault {
+        
+        @SuppressWarnings("unchecked")
+        final List<SecurityEvent> incomingSecurityEventList = 
+            (List<SecurityEvent>)soapMessage.get(SecurityEvent.class.getName() + ".in");
+
+        if (incomingSecurityEventList != null) {
+            try {
+                doResults(soapMessage, incomingSecurityEventList);
+            } catch (WSSecurityException e) {
+                throw createSoapFault(soapMessage.getVersion(), e);
+            }
+        }
+    }
+    
+    private void doResults(SoapMessage msg, List<SecurityEvent> incomingSecurityEventList) throws WSSecurityException {
+        for (SecurityEvent event : incomingSecurityEventList) {
+            SubjectAndPrincipalSecurityToken token = getSubjectPrincipalToken(event);
+            if (token != null) {
+                Principal p = token.getPrincipal();
+                Subject subject = token.getSubject();
+                
+                if (subject != null) {
+                    String roleClassifier = 
+                        (String)msg.getContextualProperty(SecurityConstants.SUBJECT_ROLE_CLASSIFIER);
+                    if (roleClassifier != null && !"".equals(roleClassifier)) {
+                        String roleClassifierType = 
+                            (String)msg.getContextualProperty(SecurityConstants.SUBJECT_ROLE_CLASSIFIER_TYPE);
+                        if (roleClassifierType == null || "".equals(roleClassifierType)) {
+                            roleClassifierType = "prefix";
+                        }
+                        msg.put(
+                            SecurityContext.class, 
+                            new RolePrefixSecurityContextImpl(subject, roleClassifier, roleClassifierType)
+                        );
+                    } else {
+                        msg.put(SecurityContext.class, new DefaultSecurityContext(subject));
+                    }
+                    break;
+                } else if (p != null && isSecurityContextPrincipal(p, incomingSecurityEventList)) {
+
+                    Object receivedAssertion = null;
+                    
+                    List<String> roles = null;
+                    if (event.getSecurityEventType() == WSSecurityEventConstants.SamlToken) {
+                        String roleAttributeName = (String)msg.getContextualProperty(
+                                SecurityConstants.SAML_ROLE_ATTRIBUTENAME);
+                        if (roleAttributeName == null || roleAttributeName.length() == 0) {
+                            roleAttributeName = SAML_ROLE_ATTRIBUTENAME_DEFAULT;
+                        }
+                        
+                        SamlTokenSecurityEvent samlEvent = (SamlTokenSecurityEvent)event;
+                        receivedAssertion = samlEvent.getSamlAssertionWrapper();
+                        roles = SAMLUtils.parseRolesInAssertion(receivedAssertion, roleAttributeName);
+                        SAMLSecurityContext context = createSecurityContext(p, roles);
+                        context.setIssuer(SAMLUtils.getIssuer(receivedAssertion));
+                        context.setAssertionElement(SAMLUtils.getAssertionElement(receivedAssertion));
+                        msg.put(SecurityContext.class, context);
+                    } else {
+                        msg.put(SecurityContext.class, createSecurityContext(p));
+                    }
+                    break;
+                }
+            }
+        }
+    }
+    
+
+    /**
+     * Checks if a given WSS4J Principal can be represented as a user principal
+     * inside SecurityContext. Example, UsernameToken or PublicKey principals can
+     * be used to facilitate checking the user roles, etc.
+     */
+    private boolean isSecurityContextPrincipal(Principal p, List<SecurityEvent> incomingSecurityEventList) {
+        
+        boolean derivedKeyPrincipal = p instanceof WSDerivedKeyTokenPrincipal;
+        if (derivedKeyPrincipal || p instanceof CustomTokenPrincipal) {
+            // If it is a derived key principal or a Custom Token Principal then let it 
+            // be a SecurityContext principal only if no other principals are available.
+            return incomingSecurityEventList.size() > 1 ? false : true;
+        } else {
+            return true;
+        }
+    }
+    
+    private SubjectAndPrincipalSecurityToken getSubjectPrincipalToken(SecurityEvent event) {
+        if (event.getSecurityEventType() == WSSecurityEventConstants.UsernameToken) {
+            return ((UsernameTokenSecurityEvent)event).getSecurityToken();
+        } else if (event.getSecurityEventType() == WSSecurityEventConstants.SamlToken) {
+            return ((SamlTokenSecurityEvent)event).getSecurityToken();
+        } else if (event.getSecurityEventType() == WSSecurityEventConstants.X509Token) {
+            return ((X509TokenSecurityEvent)event).getSecurityToken();
+        } else if (event.getSecurityEventType() == WSSecurityEventConstants.KeyValueToken) {
+            return ((KeyValueTokenSecurityEvent)event).getSecurityToken();
+        } else if (event.getSecurityEventType() == WSSecurityEventConstants.KerberosToken) {
+            return ((KerberosTokenSecurityEvent)event).getSecurityToken();
+        }
+        return null;
+    }
+    
+    private SecurityContext createSecurityContext(final Principal p) {
+        return new SecurityContext() {
+
+            public Principal getUserPrincipal() {
+                return p;
+            }
+
+            public boolean isUserInRole(String arg0) {
+                return false;
+            }
+        };
+    }
+    
+    private SAMLSecurityContext createSecurityContext(final Principal p, final List<String> roles) {
+        final Set<Principal> userRoles;
+        if (roles != null) {
+            userRoles = new HashSet<Principal>();
+            for (String role : roles) {
+                userRoles.add(new SimplePrincipal(role));
+            }
+        } else {
+            userRoles = null;
+        }
+        
+        return new SAMLSecurityContext(p, userRoles);
+    }
+    
+    /**
+     * Create a SoapFault from a WSSecurityException, following the SOAP Message Security
+     * 1.1 specification, chapter 12 "Error Handling".
+     * 
+     * When the Soap version is 1.1 then set the Fault/Code/Value from the fault code
+     * specified in the WSSecurityException (if it exists).
+     * 
+     * Otherwise set the Fault/Code/Value to env:Sender and the Fault/Code/Subcode/Value
+     * as the fault code from the WSSecurityException.
+     */
+    private SoapFault 
+    createSoapFault(SoapVersion version, WSSecurityException e) {
+        SoapFault fault;
+        javax.xml.namespace.QName faultCode = e.getFaultCode();
+        if (version.getVersion() == 1.1 && faultCode != null) {
+            fault = new SoapFault(e.getMessage(), e, faultCode);
+        } else {
+            fault = new SoapFault(e.getMessage(), e, version.getSender());
+            if (version.getVersion() != 1.1 && faultCode != null) {
+                fault.setSubCode(faultCode);
+            }
+        }
+        return fault;
+    }
+}

Modified: cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/WSS4JStaxInInterceptor.java
URL: http://svn.apache.org/viewvc/cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/WSS4JStaxInInterceptor.java?rev=1470518&r1=1470517&r2=1470518&view=diff
==============================================================================
--- cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/WSS4JStaxInInterceptor.java (original)
+++ cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/WSS4JStaxInInterceptor.java Mon Apr 22 14:01:35 2013
@@ -20,14 +20,19 @@ package org.apache.cxf.ws.security.wss4j
 
 import java.util.LinkedList;
 import java.util.List;
+import java.util.logging.Logger;
 
 import javax.xml.stream.XMLStreamException;
 import javax.xml.stream.XMLStreamReader;
 
 import org.apache.cxf.binding.soap.SoapFault;
 import org.apache.cxf.binding.soap.SoapMessage;
+import org.apache.cxf.binding.soap.SoapVersion;
+import org.apache.cxf.common.i18n.Message;
+import org.apache.cxf.common.logging.LogUtils;
 import org.apache.cxf.interceptor.Fault;
 import org.apache.cxf.interceptor.StaxInInterceptor;
+import org.apache.cxf.interceptor.URIMappingInterceptor;
 import org.apache.cxf.phase.Phase;
 import org.apache.wss4j.common.ext.WSSecurityException;
 import org.apache.wss4j.stax.WSSec;
@@ -38,6 +43,8 @@ import org.apache.xml.security.stax.secu
 
 public class WSS4JStaxInInterceptor extends AbstractWSS4JStaxInterceptor {
     
+    private static final Logger LOG = LogUtils.getL7dLogger(WSS4JStaxInInterceptor.class);
+    
     private final InboundWSSec inboundWSSec;
     
     public WSS4JStaxInInterceptor(WSSSecurityProperties securityProperties) throws WSSecurityException {
@@ -48,8 +55,23 @@ public class WSS4JStaxInInterceptor exte
         inboundWSSec = WSSec.getInboundWSSec(securityProperties);
     }
 
+    public final boolean isGET(SoapMessage message) {
+        String method = (String)message.get(SoapMessage.HTTP_REQUEST_METHOD);
+        boolean isGet = 
+            "GET".equals(method) && message.getContent(XMLStreamReader.class) == null;
+        if (isGet) {
+            //make sure we skip the URIMapping as we cannot apply security requirements to that
+            message.put(URIMappingInterceptor.URIMAPPING_SKIP, Boolean.TRUE);
+        }
+        return isGet;
+    }
+    
     @Override
     public void handleMessage(SoapMessage soapMessage) throws Fault {
+        
+        if (isGET(soapMessage)) {
+            return;
+        }
 
         XMLStreamReader originalXmlStreamReader = soapMessage.getContent(XMLStreamReader.class);
         XMLStreamReader newXmlStreamReader;
@@ -62,7 +84,9 @@ public class WSS4JStaxInInterceptor exte
             }
         };
         soapMessage.getExchange().put(SecurityEvent.class.getName() + ".in", incomingSecurityEventList);
-
+        soapMessage.put(SecurityEvent.class.getName() + ".in", incomingSecurityEventList);
+        soapMessage.getInterceptorChain().add(new StaxSecurityContextInInterceptor());
+        
         try {
             @SuppressWarnings("unchecked")
             final List<SecurityEvent> requestSecurityEvents = 
@@ -77,11 +101,35 @@ public class WSS4JStaxInInterceptor exte
             // interceptor doesn't handle the ws-security stuff but just setup the relevant stuff
             // for it. Exceptions will be thrown as a wrapped XMLStreamException during further
             // processing in the WS-Stack.
-
         } catch (WSSecurityException e) {
-            throw new SoapFault("unexpected service error", SoapFault.FAULT_CODE_SERVER);
+            throw createSoapFault(soapMessage.getVersion(), e);
         } catch (XMLStreamException e) {
-            throw new SoapFault("unexpected service error", SoapFault.FAULT_CODE_SERVER);
+            throw new SoapFault(new Message("STAX_EX", LOG), e, soapMessage.getVersion().getSender());
+        }
+    }
+
+    /**
+     * Create a SoapFault from a WSSecurityException, following the SOAP Message Security
+     * 1.1 specification, chapter 12 "Error Handling".
+     * 
+     * When the Soap version is 1.1 then set the Fault/Code/Value from the fault code
+     * specified in the WSSecurityException (if it exists).
+     * 
+     * Otherwise set the Fault/Code/Value to env:Sender and the Fault/Code/Subcode/Value
+     * as the fault code from the WSSecurityException.
+     */
+    private SoapFault 
+    createSoapFault(SoapVersion version, WSSecurityException e) {
+        SoapFault fault;
+        javax.xml.namespace.QName faultCode = e.getFaultCode();
+        if (version.getVersion() == 1.1 && faultCode != null) {
+            fault = new SoapFault(e.getMessage(), e, faultCode);
+        } else {
+            fault = new SoapFault(e.getMessage(), e, version.getSender());
+            if (version.getVersion() != 1.1 && faultCode != null) {
+                fault.setSubCode(faultCode);
+            }
         }
+        return fault;
     }
 }

Modified: cxf/branches/wss4j2.0-port/rt/ws/security/src/test/java/org/apache/cxf/ws/security/wss4j/StaxRoundTripTest.java
URL: http://svn.apache.org/viewvc/cxf/branches/wss4j2.0-port/rt/ws/security/src/test/java/org/apache/cxf/ws/security/wss4j/StaxRoundTripTest.java?rev=1470518&r1=1470517&r2=1470518&view=diff
==============================================================================
--- cxf/branches/wss4j2.0-port/rt/ws/security/src/test/java/org/apache/cxf/ws/security/wss4j/StaxRoundTripTest.java (original)
+++ cxf/branches/wss4j2.0-port/rt/ws/security/src/test/java/org/apache/cxf/ws/security/wss4j/StaxRoundTripTest.java Mon Apr 22 14:01:35 2013
@@ -52,7 +52,10 @@ public class StaxRoundTripTest extends A
         WSSSecurityProperties inProperties = new WSSSecurityProperties();
         inProperties.setCallbackHandler(new TestPwdCallback());
         WSS4JStaxInInterceptor inhandler = new WSS4JStaxInInterceptor(inProperties);
+        WSS4JPrincipalInterceptor principalInterceptor = new WSS4JPrincipalInterceptor();
+        principalInterceptor.setPrincipalName("username");
         service.getInInterceptors().add(inhandler);
+        service.getInInterceptors().add(principalInterceptor);
 
         // Create + configure client
         Echo echo = createClientProxy();
@@ -80,7 +83,10 @@ public class StaxRoundTripTest extends A
         WSSSecurityProperties inProperties = new WSSSecurityProperties();
         inProperties.setCallbackHandler(new TestPwdCallback());
         WSS4JStaxInInterceptor inhandler = new WSS4JStaxInInterceptor(inProperties);
+        WSS4JPrincipalInterceptor principalInterceptor = new WSS4JPrincipalInterceptor();
+        principalInterceptor.setPrincipalName("username");
         service.getInInterceptors().add(inhandler);
+        service.getInInterceptors().add(principalInterceptor);
         
         // Create + configure client
         Echo echo = createClientProxy();
@@ -185,7 +191,10 @@ public class StaxRoundTripTest extends A
             CryptoFactory.getProperties("insecurity.properties", this.getClass().getClassLoader());
         inProperties.setSignatureVerificationCryptoProperties(cryptoProperties);
         WSS4JStaxInInterceptor inhandler = new WSS4JStaxInInterceptor(inProperties);
+        WSS4JPrincipalInterceptor principalInterceptor = new WSS4JPrincipalInterceptor();
+        principalInterceptor.setPrincipalName("CN=myAlias");
         service.getInInterceptors().add(inhandler);
+        service.getInInterceptors().add(principalInterceptor);
         
         // Create + configure client
         Echo echo = createClientProxy();
@@ -286,7 +295,10 @@ public class StaxRoundTripTest extends A
             CryptoFactory.getProperties("cxfca.properties", this.getClass().getClassLoader());
         inProperties.setSignatureVerificationCryptoProperties(cryptoProperties);
         WSS4JStaxInInterceptor inhandler = new WSS4JStaxInInterceptor(inProperties);
+        WSS4JPrincipalInterceptor principalInterceptor = new WSS4JPrincipalInterceptor();
+        principalInterceptor.setPrincipalName("CN=alice,OU=eng,O=apache.org");
         service.getInInterceptors().add(inhandler);
+        service.getInInterceptors().add(principalInterceptor);
         
         // Create + configure client
         Echo echo = createClientProxy();

Added: cxf/branches/wss4j2.0-port/rt/ws/security/src/test/java/org/apache/cxf/ws/security/wss4j/WSS4JPrincipalInterceptor.java
URL: http://svn.apache.org/viewvc/cxf/branches/wss4j2.0-port/rt/ws/security/src/test/java/org/apache/cxf/ws/security/wss4j/WSS4JPrincipalInterceptor.java?rev=1470518&view=auto
==============================================================================
--- cxf/branches/wss4j2.0-port/rt/ws/security/src/test/java/org/apache/cxf/ws/security/wss4j/WSS4JPrincipalInterceptor.java (added)
+++ cxf/branches/wss4j2.0-port/rt/ws/security/src/test/java/org/apache/cxf/ws/security/wss4j/WSS4JPrincipalInterceptor.java Mon Apr 22 14:01:35 2013
@@ -0,0 +1,68 @@
+/**
+ * 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.cxf.ws.security.wss4j;
+
+import java.security.Principal;
+
+import org.apache.cxf.binding.soap.SoapFault;
+import org.apache.cxf.binding.soap.SoapMessage;
+import org.apache.cxf.interceptor.Fault;
+import org.apache.cxf.phase.AbstractPhaseInterceptor;
+import org.apache.cxf.phase.Phase;
+import org.apache.cxf.security.SecurityContext;
+
+/**
+ * A test interceptor to check that the Principal is not null + that the name is equal to a 
+ * given name.
+ */
+public class WSS4JPrincipalInterceptor extends AbstractPhaseInterceptor<SoapMessage> {
+    
+    private String principalName;
+
+    public WSS4JPrincipalInterceptor() {
+        super(Phase.PRE_INVOKE);
+    }
+
+    @Override
+    public void handleMessage(SoapMessage message) throws Fault {
+        SecurityContext context = message.get(SecurityContext.class);
+        if (context == null) {
+            throw new SoapFault("No Security Context", SoapFault.FAULT_CODE_SERVER);
+        }
+        
+        Principal principal = context.getUserPrincipal();
+        if (principal == null) {
+            throw new SoapFault("No Security Principal", SoapFault.FAULT_CODE_SERVER);
+        }
+        
+        if (principalName != null && !principalName.equals(principal.getName())) {
+            throw new SoapFault("Security Principal does not match", SoapFault.FAULT_CODE_SERVER);
+        }
+    }
+
+    public String getPrincipalName() {
+        return principalName;
+    }
+
+    public void setPrincipalName(String principalName) {
+        this.principalName = principalName;
+    }
+    
+   
+}