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/05/10 17:16:28 UTC

svn commit: r1481050 - in /cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j: PolicyBasedWSS4JInInterceptor.java PolicyBasedWSS4JStaxInInterceptor.java PolicyStaxActionInInterceptor.java WSS4JStaxInInterceptor.java

Author: coheigea
Date: Fri May 10 15:16:28 2013
New Revision: 1481050

URL: http://svn.apache.org/r1481050
Log:
Enabled policy validation via the ws-security-policy-stax layer in WSS4J

Added:
    cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyStaxActionInInterceptor.java
Modified:
    cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JInInterceptor.java
    cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JStaxInInterceptor.java
    cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/WSS4JStaxInInterceptor.java

Modified: cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JInInterceptor.java
URL: http://svn.apache.org/viewvc/cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JInInterceptor.java?rev=1481050&r1=1481049&r2=1481050&view=diff
==============================================================================
--- cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JInInterceptor.java (original)
+++ cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JInInterceptor.java Fri May 10 15:16:28 2013
@@ -269,10 +269,10 @@ public class PolicyBasedWSS4JInIntercept
         }
         
         if (encrCrypto != null) {
-            message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + encrCrypto.hashCode());
+            message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + encrCrypto.hashCode());
             message.put("RefId-" + encrCrypto.hashCode(), (Crypto)encrCrypto);
         } else if (signCrypto != null) {
-            message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + signCrypto.hashCode());
+            message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + signCrypto.hashCode());
             message.put("RefId-" + signCrypto.hashCode(), (Crypto)signCrypto);
         }
      
@@ -313,10 +313,10 @@ public class PolicyBasedWSS4JInIntercept
         }
         
         if (encrCrypto != null) {
-            message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + encrCrypto.hashCode());
+            message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + encrCrypto.hashCode());
             message.put("RefId-" + encrCrypto.hashCode(), (Crypto)encrCrypto);
         } else if (signCrypto != null) {
-            message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + signCrypto.hashCode());
+            message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + signCrypto.hashCode());
             message.put("RefId-" + signCrypto.hashCode(), (Crypto)signCrypto);
         }
 
@@ -373,7 +373,7 @@ public class PolicyBasedWSS4JInIntercept
                 crypto = signCrypto;
             }
             if (crypto != null) {
-                message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + crypto.hashCode());
+                message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + crypto.hashCode());
                 message.put("RefId-" + crypto.hashCode(), crypto);
             }
             
@@ -391,7 +391,7 @@ public class PolicyBasedWSS4JInIntercept
                 crypto = encrCrypto;
             }
             if (crypto != null) {
-                message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + crypto.hashCode());
+                message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + crypto.hashCode());
                 message.put("RefId-" + crypto.hashCode(), crypto);
             }
             

Modified: cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JStaxInInterceptor.java
URL: http://svn.apache.org/viewvc/cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JStaxInInterceptor.java?rev=1481050&r1=1481049&r2=1481050&view=diff
==============================================================================
--- cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JStaxInInterceptor.java (original)
+++ cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JStaxInInterceptor.java Fri May 10 15:16:28 2013
@@ -22,10 +22,14 @@ package org.apache.cxf.ws.security.wss4j
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URL;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Properties;
 import java.util.logging.Logger;
 
@@ -33,15 +37,20 @@ import javax.xml.namespace.QName;
 
 import org.apache.cxf.Bus;
 import org.apache.cxf.binding.soap.SoapMessage;
+import org.apache.cxf.binding.soap.model.SoapBindingInfo;
+import org.apache.cxf.binding.soap.model.SoapOperationInfo;
 import org.apache.cxf.common.classloader.ClassLoaderUtils;
 import org.apache.cxf.common.logging.LogUtils;
 import org.apache.cxf.endpoint.Endpoint;
 import org.apache.cxf.interceptor.Fault;
 import org.apache.cxf.message.MessageUtils;
 import org.apache.cxf.resource.ResourceManager;
+import org.apache.cxf.service.model.BindingInfo;
+import org.apache.cxf.service.model.BindingOperationInfo;
 import org.apache.cxf.service.model.EndpointInfo;
 import org.apache.cxf.ws.policy.AssertionInfo;
 import org.apache.cxf.ws.policy.AssertionInfoMap;
+import org.apache.cxf.ws.policy.EffectivePolicy;
 import org.apache.cxf.ws.security.SecurityConstants;
 import org.apache.wss4j.common.crypto.Crypto;
 import org.apache.wss4j.common.crypto.CryptoFactory;
@@ -50,6 +59,13 @@ import org.apache.wss4j.dom.handler.WSHa
 import org.apache.wss4j.policy.SP11Constants;
 import org.apache.wss4j.policy.SP12Constants;
 import org.apache.wss4j.policy.SPConstants;
+import org.apache.wss4j.policy.WSSPolicyException;
+import org.apache.wss4j.policy.stax.OperationPolicy;
+import org.apache.wss4j.policy.stax.PolicyEnforcer;
+import org.apache.wss4j.policy.stax.PolicyInputProcessor;
+import org.apache.wss4j.stax.ext.WSSSecurityProperties;
+import org.apache.xml.security.stax.securityEvent.SecurityEvent;
+import org.apache.xml.security.stax.securityEvent.SecurityEventListener;
 
 /**
  * 
@@ -69,6 +85,7 @@ public class PolicyBasedWSS4JStaxInInter
             MessageUtils.isTrue(msg.getContextualProperty(SecurityConstants.ENABLE_STREAMING_SECURITY));
         if (aim != null && enableStax) {
             super.handleMessage(msg);
+            msg.getInterceptorChain().add(new PolicyStaxActionInInterceptor());
         }
     }
     
@@ -165,10 +182,10 @@ public class PolicyBasedWSS4JStaxInInter
         }
         
         if (encrCrypto != null) {
-            message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + encrCrypto.hashCode());
+            message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + encrCrypto.hashCode());
             message.put("RefId-" + encrCrypto.hashCode(), (Crypto)encrCrypto);
         } else if (signCrypto != null) {
-            message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + signCrypto.hashCode());
+            message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + signCrypto.hashCode());
             message.put("RefId-" + signCrypto.hashCode(), (Crypto)signCrypto);
         }
     }
@@ -205,10 +222,10 @@ public class PolicyBasedWSS4JStaxInInter
         }
         
         if (encrCrypto != null) {
-            message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + encrCrypto.hashCode());
+            message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + encrCrypto.hashCode());
             message.put("RefId-" + encrCrypto.hashCode(), (Crypto)encrCrypto);
         } else if (signCrypto != null) {
-            message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + signCrypto.hashCode());
+            message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + signCrypto.hashCode());
             message.put("RefId-" + signCrypto.hashCode(), (Crypto)signCrypto);
         }
     }
@@ -245,7 +262,7 @@ public class PolicyBasedWSS4JStaxInInter
                 crypto = signCrypto;
             }
             if (crypto != null) {
-                message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + crypto.hashCode());
+                message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + crypto.hashCode());
                 message.put("RefId-" + crypto.hashCode(), crypto);
             }
             
@@ -263,7 +280,7 @@ public class PolicyBasedWSS4JStaxInInter
                 crypto = encrCrypto;
             }
             if (crypto != null) {
-                message.put(WSHandlerConstants.SIG_PROP_REF_ID, "RefId-" + crypto.hashCode());
+                message.put(WSHandlerConstants.SIG_VER_PROP_REF_ID, "RefId-" + crypto.hashCode());
                 message.put("RefId-" + crypto.hashCode(), crypto);
             }
             
@@ -326,52 +343,80 @@ public class PolicyBasedWSS4JStaxInInter
     
     @Override
     protected void configureProperties(SoapMessage msg) throws WSSecurityException {
-        super.configureProperties(msg);
-        
         AssertionInfoMap aim = msg.get(AssertionInfoMap.class);
         checkAsymmetricBinding(aim, msg);
         checkSymmetricBinding(aim, msg);
         checkTransportBinding(aim, msg);
+        
+        super.configureProperties(msg);
     }
     
-/*
-    protected void computeAction(SoapMessage message, RequestData data) throws WSSecurityException {
-        AssertionInfoMap aim = message.get(AssertionInfoMap.class);
-        if (aim != null) {
-            // stuff we can default to asserted and un-assert if a condition isn't met
-            assertPolicy(aim, SPConstants.KEY_VALUE_TOKEN);
-            assertPolicy(aim, SPConstants.RSA_KEY_VALUE);
-            assertPolicy(aim, SPConstants.REQUIRE_ISSUER_SERIAL_REFERENCE);
-            assertPolicy(aim, SPConstants.REQUIRE_THUMBPRINT_REFERENCE);
-            assertPolicy(aim, SPConstants.REQUIRE_KEY_IDENTIFIER_REFERENCE);
-            assertPolicy(aim, SPConstants.REQUIRE_EMBEDDED_TOKEN_REFERENCE);
-            assertPolicy(aim, SPConstants.REQUIRE_INTERNAL_REFERENCE);
-            
-            // WSS10
-            assertPolicy(aim, SPConstants.WSS10);
-            assertPolicy(aim, SPConstants.MUST_SUPPORT_REF_KEY_IDENTIFIER);
-            assertPolicy(aim, SPConstants.MUST_SUPPORT_REF_ISSUER_SERIAL);
-            assertPolicy(aim, SPConstants.MUST_SUPPORT_REF_EXTERNAL_URI);
-            assertPolicy(aim, SPConstants.MUST_SUPPORT_REF_EMBEDDED_TOKEN);
-            
-            // Trust 1.0
-            assertPolicy(aim, SPConstants.TRUST_10);
-            assertPolicy(aim, SPConstants.MUST_SUPPORT_CLIENT_CHALLENGE);
-            assertPolicy(aim, SPConstants.MUST_SUPPORT_SERVER_CHALLENGE);
-            assertPolicy(aim, SPConstants.REQUIRE_CLIENT_ENTROPY);
-            assertPolicy(aim, SPConstants.REQUIRE_SERVER_ENTROPY);
-            assertPolicy(aim, SPConstants.MUST_SUPPORT_ISSUED_TOKENS);
-            
-            // Trust 1.3
-            assertPolicy(aim, SPConstants.TRUST_13);
-            assertPolicy(aim, SP12Constants.REQUIRE_REQUEST_SECURITY_TOKEN_COLLECTION);
-            assertPolicy(aim, SP12Constants.REQUIRE_APPLIES_TO);
-            assertPolicy(aim, SP13Constants.SCOPE_POLICY_15);
-            assertPolicy(aim, SP13Constants.MUST_SUPPORT_INTERACTIVE_CHALLENGE);
+    @Override
+    protected SecurityEventListener configureSecurityEventListener(
+        SoapMessage msg, WSSSecurityProperties securityProperties
+    ) throws WSSPolicyException {
+        Endpoint endoint = msg.getExchange().get(Endpoint.class);
+        
+        PolicyEnforcer policyEnforcer = createPolicyEnforcer(endoint.getEndpointInfo(), msg);
+        securityProperties.addInputProcessor(new PolicyInputProcessor(policyEnforcer, securityProperties));
+
+        return policyEnforcer;
+    }
+    
+    private PolicyEnforcer createPolicyEnforcer(
+        EndpointInfo endpointInfo, SoapMessage msg
+    ) throws WSSPolicyException {
+
+        List<OperationPolicy> operationPolicies = new ArrayList<OperationPolicy>();
+        Collection<BindingOperationInfo> bindingOperationInfos = endpointInfo.getBinding().getOperations();
+        for (Iterator<BindingOperationInfo> bindingOperationInfoIterator =
+                     bindingOperationInfos.iterator(); bindingOperationInfoIterator.hasNext();) {
+            BindingOperationInfo bindingOperationInfo = bindingOperationInfoIterator.next();
+            QName operationName = bindingOperationInfo.getName();
+
+            // todo: I'm not sure what the effectivePolicy exactly contains,
+            // a) only the operation policy,
+            // or b) all policies for the service,
+            // or c) all policies which applies for the current operation.
+            // c) is that what we need for stax.
+            EffectivePolicy policy = 
+                (EffectivePolicy)bindingOperationInfo.getProperty("policy-engine-info-serve-request");
+            //PolicyEngineImpl.POLICY_INFO_REQUEST_SERVER);
+            SoapOperationInfo soapOperationInfo = bindingOperationInfo.getExtensor(SoapOperationInfo.class);
+
+            String soapNS;
+            BindingInfo bindingInfo = bindingOperationInfo.getBinding();
+            if (bindingInfo instanceof SoapBindingInfo) {
+                soapNS = ((SoapBindingInfo)bindingInfo).getSoapVersion().getNamespace();
+            } else {
+                //no idea what todo here...
+                //most probably throw an exception:
+                throw new IllegalArgumentException("BindingInfo is not an instance of SoapBindingInfo");
+            }
+
+            //todo: I think its a bug that we handover only the localPart of the operation. 
+            // Needs to be fixed in ws-security-policy-stax
+            OperationPolicy operationPolicy = new OperationPolicy(operationName.getLocalPart());
+            operationPolicy.setPolicy(policy.getPolicy());
+            operationPolicy.setOperationAction(soapOperationInfo.getAction());
+            operationPolicy.setSoapMessageVersionNamespace(soapNS);
             
-            message.put(WSHandlerConstants.ACTION, action.trim());
+            operationPolicies.add(operationPolicy);
         }
+        
+        final List<SecurityEvent> incomingSecurityEventList = new LinkedList<SecurityEvent>();
+        // TODO Soap Action
+        PolicyEnforcer securityEventListener = new PolicyEnforcer(operationPolicies, "") {
+            @Override
+            public void registerSecurityEvent(SecurityEvent securityEvent) throws WSSecurityException {
+                incomingSecurityEventList.add(securityEvent);
+            }
+        };
+        
+        msg.getExchange().put(SecurityEvent.class.getName() + ".in", incomingSecurityEventList);
+        msg.put(SecurityEvent.class.getName() + ".in", incomingSecurityEventList);
+        
+        return securityEventListener;
     }
-    */
-
+    
 }

Added: cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyStaxActionInInterceptor.java
URL: http://svn.apache.org/viewvc/cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyStaxActionInInterceptor.java?rev=1481050&view=auto
==============================================================================
--- cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyStaxActionInInterceptor.java (added)
+++ cxf/branches/wss4j2.0-port/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyStaxActionInInterceptor.java Fri May 10 15:16:28 2013
@@ -0,0 +1,199 @@
+/**
+ * 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.util.Collection;
+import java.util.List;
+
+import javax.xml.namespace.QName;
+
+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.ws.policy.AssertionInfo;
+import org.apache.cxf.ws.policy.AssertionInfoMap;
+import org.apache.wss4j.policy.SP11Constants;
+import org.apache.wss4j.policy.SP12Constants;
+import org.apache.wss4j.policy.SPConstants;
+import org.apache.wss4j.policy.model.AlgorithmSuite;
+import org.apache.wss4j.stax.securityEvent.WSSecurityEventConstants;
+import org.apache.xml.security.stax.securityEvent.SecurityEvent;
+
+/**
+ * This interceptor handles parsing the StaX WS-Security results (events) + marks the 
+ * corresponding CXF AssertionInfos as asserted accordingly. WSS4J 2.0 (StAX) takes care of all
+ * policy validation, so we are just asserting the appropriate AssertionInfo objects in CXF to 
+ * make sure that policy validation passes.
+ */
+public class PolicyStaxActionInInterceptor extends AbstractPhaseInterceptor<SoapMessage> {
+    
+    public PolicyStaxActionInInterceptor() {
+        super(Phase.PRE_PROTOCOL);
+        this.getBefore().add(StaxSecurityContextInInterceptor.class.getName());
+    }
+
+    @Override
+    public void handleMessage(SoapMessage soapMessage) throws Fault {
+        
+        AssertionInfoMap aim = soapMessage.get(AssertionInfoMap.class);
+        @SuppressWarnings("unchecked")
+        final List<SecurityEvent> incomingSecurityEventList = 
+            (List<SecurityEvent>)soapMessage.get(SecurityEvent.class.getName() + ".in");
+        if (aim == null || incomingSecurityEventList == null) {
+            return;
+        }
+        
+        verifyTokens(aim, incomingSecurityEventList);
+        verifyPartsAndElements(aim, incomingSecurityEventList);
+        verifyBindings(aim);
+    }
+    
+    private void verifyPartsAndElements(
+        AssertionInfoMap aim, List<SecurityEvent> incomingSecurityEventList
+    ) {
+        for (SecurityEvent event : incomingSecurityEventList) {
+            if (WSSecurityEventConstants.SignedPart == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.SIGNED_PARTS);
+            } else if (WSSecurityEventConstants.SignedElement == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.SIGNED_ELEMENTS);
+            } else if (WSSecurityEventConstants.EncryptedPart == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.ENCRYPTED_PARTS);
+            } else if (WSSecurityEventConstants.EncryptedElement == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.ENCRYPTED_ELEMENTS);
+            } else if (WSSecurityEventConstants.ContentEncrypted == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.CONTENT_ENCRYPTED_ELEMENTS);
+            } else if (WSSecurityEventConstants.RequiredPart == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.REQUIRED_PARTS);
+            } else if (WSSecurityEventConstants.RequiredElement == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.REQUIRED_ELEMENTS);
+            } 
+        }
+    }
+    
+    private void verifyTokens(
+        AssertionInfoMap aim, List<SecurityEvent> incomingSecurityEventList
+    ) {
+        for (SecurityEvent event : incomingSecurityEventList) {
+            if (WSSecurityEventConstants.Timestamp == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, "Timestamp");
+            } else if (WSSecurityEventConstants.UsernameToken == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.USERNAME_TOKEN);
+                assertAllAssertionsByLocalname(aim, SPConstants.USERNAME_TOKEN10);
+                assertAllAssertionsByLocalname(aim, SPConstants.USERNAME_TOKEN11);
+                assertAllAssertionsByLocalname(aim, SPConstants.HASH_PASSWORD);
+                assertAllAssertionsByLocalname(aim, SPConstants.NO_PASSWORD);
+                assertAllAssertionsByLocalname(aim, SPConstants.NONCE);
+                assertAllAssertionsByLocalname(aim, SPConstants.CREATED);
+            } else if (WSSecurityEventConstants.X509Token == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.X509_TOKEN);
+                assertAllAssertionsByLocalname(aim, SPConstants.WSS_X509_PKCS7_TOKEN10);
+                assertAllAssertionsByLocalname(aim, SPConstants.WSS_X509_PKCS7_TOKEN11);
+                assertAllAssertionsByLocalname(aim, SPConstants.WSS_X509_PKI_PATH_V1_TOKEN10);
+                assertAllAssertionsByLocalname(aim, SPConstants.WSS_X509_PKI_PATH_V1_TOKEN11);
+                assertAllAssertionsByLocalname(aim, SPConstants.WSS_X509_V1_TOKEN10);
+                assertAllAssertionsByLocalname(aim, SPConstants.WSS_X509_V1_TOKEN11);
+                assertAllAssertionsByLocalname(aim, SPConstants.WSS_X509_V3_TOKEN10);
+                assertAllAssertionsByLocalname(aim, SPConstants.WSS_X509_V3_TOKEN11);
+            } else if (WSSecurityEventConstants.SamlToken == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.SAML_TOKEN);
+                assertAllAssertionsByLocalname(aim, "WssSamlV11Token10");
+                assertAllAssertionsByLocalname(aim, "WssSamlV11Token11");
+                assertAllAssertionsByLocalname(aim, "WssSamlV20Token11");
+            } else if (WSSecurityEventConstants.SecurityContextToken == event.getSecurityEventType()) {
+                assertAllAssertionsByLocalname(aim, SPConstants.SECURITY_CONTEXT_TOKEN);
+                assertAllAssertionsByLocalname(aim, SPConstants.REQUIRE_EXTERNAL_URI_REFERENCE);
+            }
+        }
+        
+        assertAllAssertionsByLocalname(aim, SPConstants.SUPPORTING_TOKENS);
+        assertAllAssertionsByLocalname(aim, SPConstants.SIGNED_SUPPORTING_TOKENS);
+        assertAllAssertionsByLocalname(aim, SPConstants.ENCRYPTED_SUPPORTING_TOKENS);
+        assertAllAssertionsByLocalname(aim, SPConstants.ENDORSING_SUPPORTING_TOKENS);
+        assertAllAssertionsByLocalname(aim, SPConstants.SIGNED_ENCRYPTED_SUPPORTING_TOKENS);
+        assertAllAssertionsByLocalname(aim, SPConstants.SIGNED_ENDORSING_SUPPORTING_TOKENS);
+        assertAllAssertionsByLocalname(aim, SPConstants.SIGNED_ENDORSING_SUPPORTING_TOKENS);
+        assertAllAssertionsByLocalname(aim, SPConstants.ENDORSING_ENCRYPTED_SUPPORTING_TOKENS);
+        assertAllAssertionsByLocalname(aim, SPConstants.SIGNED_ENDORSING_ENCRYPTED_SUPPORTING_TOKENS);
+    }
+    
+    private void verifyBindings(AssertionInfoMap aim) {
+        assertAllAssertionsByLocalname(aim, SPConstants.SYMMETRIC_BINDING);
+        assertAllAssertionsByLocalname(aim, SPConstants.ASYMMETRIC_BINDING);
+        assertAllAssertionsByLocalname(aim, SPConstants.TRANSPORT_BINDING);
+        assertAllAssertionsByLocalname(aim, SPConstants.PROTECTION_TOKEN);
+        assertAllAssertionsByLocalname(aim, SPConstants.TRANSPORT_TOKEN);
+        assertAllAssertionsByLocalname(aim, SPConstants.INITIATOR_ENCRYPTION_TOKEN);
+        assertAllAssertionsByLocalname(aim, SPConstants.INITIATOR_SIGNATURE_TOKEN);
+        assertAllAssertionsByLocalname(aim, SPConstants.INITIATOR_TOKEN);
+        assertAllAssertionsByLocalname(aim, SPConstants.RECIPIENT_ENCRYPTION_TOKEN);
+        assertAllAssertionsByLocalname(aim, SPConstants.RECIPIENT_SIGNATURE_TOKEN);
+        assertAllAssertionsByLocalname(aim, SPConstants.RECIPIENT_TOKEN);
+        
+        assertAllAssertionsByLocalname(aim, SPConstants.ONLY_SIGN_ENTIRE_HEADERS_AND_BODY);
+        assertAllAssertionsByLocalname(aim, SPConstants.PROTECT_TOKENS);
+        assertAllAssertionsByLocalname(aim, SPConstants.INCLUDE_TIMESTAMP);
+        assertAllAssertionsByLocalname(aim, SPConstants.ENCRYPT_SIGNATURE);
+        assertAllAssertionsByLocalname(aim, SPConstants.SIGN_BEFORE_ENCRYPTING);
+        assertAllAssertionsByLocalname(aim, SPConstants.ENCRYPT_BEFORE_SIGNING);
+        assertAllAssertionsByLocalname(aim, SPConstants.LAYOUT);
+        assertAllAssertionsByLocalname(aim, SPConstants.LAYOUT_LAX);
+        assertAllAssertionsByLocalname(aim, SPConstants.LAYOUT_LAX_TIMESTAMP_FIRST);
+        assertAllAssertionsByLocalname(aim, SPConstants.LAYOUT_LAX_TIMESTAMP_LAST);
+        assertAllAssertionsByLocalname(aim, SPConstants.LAYOUT_STRICT);
+        assertAllAssertionsByLocalname(aim, SPConstants.REQUIRE_DERIVED_KEYS);
+        assertAllAssertionsByLocalname(aim, SPConstants.REQUIRE_SIGNATURE_CONFIRMATION);
+        
+        assertAllAssertionsByLocalname(aim, SPConstants.ALGORITHM_SUITE);
+        for (String s : AlgorithmSuite.getSupportedAlgorithmSuiteNames()) {
+            assertAllAssertionsByLocalname(aim, s);
+        }
+        
+        assertAllAssertionsByLocalname(aim, SPConstants.REQUIRE_INTERNAL_REFERENCE);
+        assertAllAssertionsByLocalname(aim, SPConstants.REQUIRE_EXTERNAL_REFERENCE);
+        assertAllAssertionsByLocalname(aim, SPConstants.REQUIRE_THUMBPRINT_REFERENCE);
+        assertAllAssertionsByLocalname(aim, SPConstants.REQUIRE_EMBEDDED_TOKEN_REFERENCE);
+        assertAllAssertionsByLocalname(aim, SPConstants.REQUIRE_ISSUER_SERIAL_REFERENCE);
+        assertAllAssertionsByLocalname(aim, SPConstants.REQUIRE_KEY_IDENTIFIER_REFERENCE);
+        
+        assertAllAssertionsByLocalname(aim, SPConstants.MUST_SUPPORT_REF_KEY_IDENTIFIER);
+        assertAllAssertionsByLocalname(aim, SPConstants.MUST_SUPPORT_REF_ISSUER_SERIAL);
+        assertAllAssertionsByLocalname(aim, SPConstants.MUST_SUPPORT_REF_EXTERNAL_URI);
+        assertAllAssertionsByLocalname(aim, SPConstants.MUST_SUPPORT_REF_EMBEDDED_TOKEN);
+
+        assertAllAssertionsByLocalname(aim, SPConstants.MUST_SUPPORT_REF_THUMBPRINT);
+        assertAllAssertionsByLocalname(aim, SPConstants.MUST_SUPPORT_REF_ENCRYPTED_KEY);
+    }
+    
+    private void assertAllAssertionsByLocalname(AssertionInfoMap aim, String localname) {
+        Collection<AssertionInfo> sp11Ais = aim.get(new QName(SP11Constants.SP_NS, localname));
+        if (sp11Ais != null) {
+            for (AssertionInfo ai : sp11Ais) {
+                ai.setAsserted(true);
+            }
+        }
+        Collection<AssertionInfo> sp12Ais = aim.get(new QName(SP12Constants.SP_NS, localname));
+        if (sp12Ais != null) {
+            for (AssertionInfo ai : sp12Ais) {
+                ai.setAsserted(true);
+            }
+        }
+    }
+
+}

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=1481050&r1=1481049&r2=1481050&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 Fri May 10 15:16:28 2013
@@ -39,11 +39,13 @@ import org.apache.cxf.interceptor.StaxIn
 import org.apache.cxf.interceptor.URIMappingInterceptor;
 import org.apache.cxf.message.MessageUtils;
 import org.apache.cxf.phase.Phase;
+import org.apache.cxf.security.transport.TLSSessionInfo;
 import org.apache.cxf.ws.security.SecurityConstants;
 import org.apache.wss4j.common.ConfigurationConstants;
 import org.apache.wss4j.common.cache.ReplayCache;
 import org.apache.wss4j.common.crypto.Crypto;
 import org.apache.wss4j.common.ext.WSSecurityException;
+import org.apache.wss4j.policy.WSSPolicyException;
 import org.apache.wss4j.stax.ConfigurationConverter;
 import org.apache.wss4j.stax.WSSec;
 import org.apache.wss4j.stax.ext.InboundWSSec;
@@ -106,7 +108,7 @@ public class WSS4JStaxInInterceptor exte
         
         try {
             @SuppressWarnings("unchecked")
-            final List<SecurityEvent> requestSecurityEvents = 
+            List<SecurityEvent> requestSecurityEvents = 
                 (List<SecurityEvent>) soapMessage.getExchange().get(SecurityEvent.class.getName() + ".out");
             
             translateProperties(soapMessage);
@@ -114,16 +116,22 @@ public class WSS4JStaxInInterceptor exte
             configureCallbackHandler(soapMessage);
             
             InboundWSSec inboundWSSec = null;
+            WSSSecurityProperties secProps = null;
             if (getSecurityProperties() != null) {
-                inboundWSSec = WSSec.getInboundWSSec(getSecurityProperties());
+                secProps = getSecurityProperties();
             } else {
-                WSSSecurityProperties convertedProperties = 
-                    ConfigurationConverter.convert(getProperties());
-                inboundWSSec = WSSec.getInboundWSSec(convertedProperties);
+                secProps = ConfigurationConverter.convert(getProperties());
             }
             
             SecurityEventListener securityEventListener = 
-                configureSecurityEventListener(soapMessage, inboundWSSec);
+                configureSecurityEventListener(soapMessage, secProps);
+            
+            TLSSessionInfo tlsInfo = soapMessage.get(TLSSessionInfo.class);
+            if (tlsInfo != null) {
+                // TODO HttpsSecurityTokenEvent
+            }
+            
+            inboundWSSec = WSSec.getInboundWSSec(secProps);
             
             newXmlStreamReader = 
                 inboundWSSec.processInMessage(originalXmlStreamReader, requestSecurityEvents, securityEventListener);
@@ -137,12 +145,16 @@ public class WSS4JStaxInInterceptor exte
             // processing in the WS-Stack.
         } catch (WSSecurityException e) {
             throw createSoapFault(soapMessage.getVersion(), e);
+        } catch (WSSPolicyException e) {
+            throw new SoapFault(e.getMessage(), e, soapMessage.getVersion().getSender());
         } catch (XMLStreamException e) {
             throw new SoapFault(new Message("STAX_EX", LOG), e, soapMessage.getVersion().getSender());
         }
     }
     
-    protected SecurityEventListener configureSecurityEventListener(SoapMessage msg, InboundWSSec inboundWSSec) {
+    protected SecurityEventListener configureSecurityEventListener(
+        SoapMessage msg, WSSSecurityProperties securityProperties
+    ) throws WSSPolicyException {
         final List<SecurityEvent> incomingSecurityEventList = new LinkedList<SecurityEvent>();
         SecurityEventListener securityEventListener = new SecurityEventListener() {
             @Override