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 2011/07/27 17:49:26 UTC

svn commit: r1151508 - in /cxf/branches/2.4.x-fixes: rt/ws/security/src/main/java/org/apache/cxf/ws/security/ rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/ rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/ rt/ws/securi...

Author: coheigea
Date: Wed Jul 27 15:49:23 2011
New Revision: 1151508

URL: http://svn.apache.org/viewvc?rev=1151508&view=rev
Log:
[CXF-3674] - Changed to use an InterceptorProvider for the Kerberos client case.

Added:
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/KerberosClient.java
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/KerberosUtils.java
      - copied, changed from r1151323, cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/server/DoubleItImpl.java
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/interceptors/KerberosTokenInterceptorProvider.java
    cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/kerberos.jaas
Removed:
    cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/client/KerberosCallbackHandler.java
    cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/server/KerberosTokenValidator.java
Modified:
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/SecurityConstants.java
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/SP11Constants.java
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/WSSecurityPolicyLoader.java
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JInInterceptor.java
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyhandlers/AbstractBindingBuilder.java
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyhandlers/TransportBindingHandler.java
    cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyvalidators/KerberosTokenPolicyValidator.java
    cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/KerberosTokenTest.java
    cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/server/DoubleItImpl.java
    cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/logging.properties
    cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/org/apache/cxf/systest/ws/kerberos/client/client.xml
    cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/org/apache/cxf/systest/ws/kerberos/server/server.xml
    cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/wsdl_systest_wssec/kerberos/DoubleItKerberos.wsdl

Modified: cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/SecurityConstants.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/SecurityConstants.java?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/SecurityConstants.java (original)
+++ cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/SecurityConstants.java Wed Jul 27 15:49:23 2011
@@ -40,7 +40,6 @@ public final class SecurityConstants {
     
     public static final String CALLBACK_HANDLER = "ws-security.callback-handler";
     public static final String SAML_CALLBACK_HANDLER = "ws-security.saml-callback-handler";
-    public static final String BST_CALLBACK_HANDLER = "ws-security.bst-callback-handler";
     
     public static final String SIGNATURE_USERNAME = "ws-security.signature.username";
     public static final String SIGNATURE_PROPERTIES = "ws-security.signature.properties";
@@ -108,7 +107,9 @@ public final class SecurityConstants {
     public static final String STS_TOKEN_ACT_AS = "ws-security.sts.token.act-as";
     
     public static final String STS_TOKEN_ON_BEHALF_OF = "ws-security.sts.token.on-behalf-of";
-
+    
+    public static final String KERBEROS_CLIENT = "ws-security.kerberos.client";
+    
     public static final Set<String> ALL_PROPERTIES;
     
     static {
@@ -122,7 +123,7 @@ public final class SecurityConstants {
             SAML1_TOKEN_VALIDATOR, SAML2_TOKEN_VALIDATOR, TIMESTAMP_TOKEN_VALIDATOR,
             SIGNATURE_TOKEN_VALIDATOR, IS_BSP_COMPLIANT, TIMESTAMP_FUTURE_TTL,
             BST_TOKEN_VALIDATOR, SAML_CALLBACK_HANDLER, STS_TOKEN_ON_BEHALF_OF,
-            BST_CALLBACK_HANDLER
+            KERBEROS_CLIENT
         }));
         ALL_PROPERTIES = Collections.unmodifiableSet(s);
     }

Added: cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/KerberosClient.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/KerberosClient.java?rev=1151508&view=auto
==============================================================================
--- cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/KerberosClient.java (added)
+++ cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/KerberosClient.java Wed Jul 27 15:49:23 2011
@@ -0,0 +1,123 @@
+/**
+ * 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.kerberos;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.security.auth.callback.CallbackHandler;
+
+import org.apache.cxf.Bus;
+import org.apache.cxf.common.logging.LogUtils;
+import org.apache.cxf.configuration.Configurable;
+import org.apache.cxf.helpers.DOMUtils;
+import org.apache.cxf.ws.security.tokenstore.SecurityToken;
+import org.apache.ws.security.WSSConfig;
+import org.apache.ws.security.message.token.KerberosSecurity;
+
+/**
+ * A class that obtains a ticket from a KDC and wraps it in a SecurityToken object.
+ */
+public class KerberosClient implements Configurable {
+    private static final Logger LOG = LogUtils.getL7dLogger(KerberosClient.class);
+    
+    Bus bus;
+    String name = "default.kerberos-client";
+    
+    private String serviceName;
+    private CallbackHandler callbackHandler;
+    private String jaasLoginModuleName;
+    private WSSConfig wssConfig = WSSConfig.getNewInstance();
+    
+    public KerberosClient(Bus b) {
+        bus = b;
+    }
+
+    public String getBeanName() {
+        return name;
+    }
+    
+    /**
+     * Get the JAAS Login module name to use.
+     * @return the JAAS Login module name to use
+     */
+    public String getJaasLoginModuleName() {
+        return jaasLoginModuleName;
+    }
+
+    /**
+     * Set the JAAS Login module name to use.
+     * @param jaasLoginModuleName the JAAS Login module name to use
+     */
+    public void setJaasLoginModuleName(String jaasLoginModuleName) {
+        this.jaasLoginModuleName = jaasLoginModuleName;
+    }
+
+    /**
+     * Get the CallbackHandler to use with the LoginContext
+     * @return the CallbackHandler to use with the LoginContext
+     */
+    public CallbackHandler getCallbackHandler() {
+        return callbackHandler;
+    }
+
+    /**
+     * Set the CallbackHandler to use with the LoginContext. It can be null.
+     * @param callbackHandler the CallbackHandler to use with the LoginContext
+     */
+    public void setCallbackHandler(CallbackHandler callbackHandler) {
+        this.callbackHandler = callbackHandler;
+    }
+
+    /**
+     * The name of the service to use when contacting the KDC.
+     * @param serviceName the name of the service to use when contacting the KDC
+     */
+    public void setServiceName(String serviceName) {
+        this.serviceName = serviceName;
+    }
+    
+    /**
+     * Get the name of the service to use when contacting the KDC.
+     * @return the name of the service to use when contacting the KDC
+     */
+    public String getServiceName() {
+        return serviceName;
+    }
+
+    public SecurityToken requestSecurityToken() throws Exception {
+        if (LOG.isLoggable(Level.FINE)) {
+            LOG.fine("Requesting Kerberos ticket for " + serviceName 
+                    + " using JAAS Login Module: " + jaasLoginModuleName);
+        }
+        KerberosSecurity bst = new KerberosSecurity(DOMUtils.createDocument());
+        bst.retrieveServiceTicket(jaasLoginModuleName, callbackHandler, serviceName);
+        bst.addWSUNamespace();
+        bst.setID(wssConfig.getIdAllocator().createSecureId("BST-", bst));
+        
+        SecurityToken token = new SecurityToken(bst.getID());
+        token.setToken(bst.getElement());
+        //token.setSecret(bst.getToken());
+        token.setTokenType(bst.getValueType());
+
+        return token;
+    }
+
+}

Copied: cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/KerberosUtils.java (from r1151323, cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/server/DoubleItImpl.java)
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/KerberosUtils.java?p2=cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/KerberosUtils.java&p1=cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/server/DoubleItImpl.java&r1=1151323&r2=1151508&rev=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/server/DoubleItImpl.java (original)
+++ cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/kerberos/KerberosUtils.java Wed Jul 27 15:49:23 2011
@@ -17,24 +17,33 @@
  * under the License.
  */
 
-package org.apache.cxf.systest.ws.kerberos.server;
+package org.apache.cxf.ws.security.kerberos;
 
-import java.math.BigInteger;
+import org.apache.cxf.Bus;
+import org.apache.cxf.message.Message;
+import org.apache.cxf.ws.security.SecurityConstants;
 
-import javax.jws.WebService;
-
-import org.apache.cxf.feature.Features;
-
-import wssec.kerberos.DoubleItPortType;
+/**
+ * 
+ */
+public final class KerberosUtils {
 
-@WebService(targetNamespace = "http://WSSec/kerberos", 
-            serviceName = "DoubleItService", 
-            endpointInterface = "wssec.kerberos.DoubleItPortType")
-@Features(features = "org.apache.cxf.feature.LoggingFeature")              
-public class DoubleItImpl implements DoubleItPortType {
+    private KerberosUtils() {
+        //utility class
+    }
     
-    public java.math.BigInteger doubleIt(java.math.BigInteger numberToDouble) {
-        return numberToDouble.multiply(BigInteger.valueOf(2));
+    public static KerberosClient getClient(Message message, String type) {
+        if (type == null) {
+            type = "";
+        } else {
+            type = "." + type + "-client";
+        }
+        KerberosClient client = (KerberosClient)message
+            .getContextualProperty(SecurityConstants.KERBEROS_CLIENT);
+        if (client == null) {
+            client = new KerberosClient(message.getExchange().get(Bus.class));
+        }
+        return client;
     }
     
 }

Modified: cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/SP11Constants.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/SP11Constants.java?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/SP11Constants.java (original)
+++ cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/SP11Constants.java Wed Jul 27 15:49:23 2011
@@ -105,7 +105,7 @@ public final class SP11Constants extends
     public static final QName SAML_TOKEN = new QName(SP11Constants.SP_NS,
             SPConstants.SAML_TOKEN , SP11Constants.SP_PREFIX);
     
-    public static final QName KERBEROS_TOKEN = new QName(SP12Constants.SP_NS,
+    public static final QName KERBEROS_TOKEN = new QName(SP11Constants.SP_NS,
             SPConstants.KERBEROS_TOKEN, SP11Constants.SP_PREFIX);
 
     public static final QName WSS_USERNAME_TOKEN10 = new QName(SP11Constants.SP_NS,

Modified: cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/WSSecurityPolicyLoader.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/WSSecurityPolicyLoader.java?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/WSSecurityPolicyLoader.java (original)
+++ cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/WSSecurityPolicyLoader.java Wed Jul 27 15:49:23 2011
@@ -65,6 +65,7 @@ import org.apache.cxf.ws.security.policy
 import org.apache.cxf.ws.security.policy.builders.X509TokenBuilder;
 import org.apache.cxf.ws.security.policy.interceptors.HttpsTokenInterceptorProvider;
 import org.apache.cxf.ws.security.policy.interceptors.IssuedTokenInterceptorProvider;
+import org.apache.cxf.ws.security.policy.interceptors.KerberosTokenInterceptorProvider;
 import org.apache.cxf.ws.security.policy.interceptors.SecureConversationTokenInterceptorProvider;
 import org.apache.cxf.ws.security.policy.interceptors.UsernameTokenInterceptorProvider;
 import org.apache.cxf.ws.security.policy.interceptors.WSSecurityInterceptorProvider;
@@ -165,6 +166,7 @@ public final class WSSecurityPolicyLoade
         reg.register(new WSSecurityPolicyInterceptorProvider());
         reg.register(new WSSecurityInterceptorProvider());
         reg.register(new HttpsTokenInterceptorProvider());
+        reg.register(new KerberosTokenInterceptorProvider());
         reg.register(new IssuedTokenInterceptorProvider());
         reg.register(new UsernameTokenInterceptorProvider(bus));
         reg.register(new SecureConversationTokenInterceptorProvider());

Added: cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/interceptors/KerberosTokenInterceptorProvider.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/interceptors/KerberosTokenInterceptorProvider.java?rev=1151508&view=auto
==============================================================================
--- cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/interceptors/KerberosTokenInterceptorProvider.java (added)
+++ cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/policy/interceptors/KerberosTokenInterceptorProvider.java Wed Jul 27 15:49:23 2011
@@ -0,0 +1,217 @@
+/**
+ * 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.policy.interceptors;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.cxf.endpoint.Endpoint;
+import org.apache.cxf.helpers.CastUtils;
+import org.apache.cxf.interceptor.Fault;
+import org.apache.cxf.message.Message;
+import org.apache.cxf.phase.AbstractPhaseInterceptor;
+import org.apache.cxf.phase.Phase;
+import org.apache.cxf.ws.policy.AbstractPolicyInterceptorProvider;
+import org.apache.cxf.ws.policy.AssertionInfo;
+import org.apache.cxf.ws.policy.AssertionInfoMap;
+import org.apache.cxf.ws.security.SecurityConstants;
+import org.apache.cxf.ws.security.kerberos.KerberosClient;
+import org.apache.cxf.ws.security.kerberos.KerberosUtils;
+import org.apache.cxf.ws.security.policy.SP11Constants;
+import org.apache.cxf.ws.security.policy.SP12Constants;
+import org.apache.cxf.ws.security.tokenstore.MemoryTokenStore;
+import org.apache.cxf.ws.security.tokenstore.SecurityToken;
+import org.apache.cxf.ws.security.tokenstore.TokenStore;
+import org.apache.cxf.ws.security.wss4j.PolicyBasedWSS4JInInterceptor;
+import org.apache.cxf.ws.security.wss4j.PolicyBasedWSS4JOutInterceptor;
+import org.apache.cxf.ws.security.wss4j.WSS4JInInterceptor;
+import org.apache.cxf.ws.security.wss4j.policyvalidators.KerberosTokenPolicyValidator;
+import org.apache.ws.security.WSConstants;
+import org.apache.ws.security.WSSecurityEngineResult;
+import org.apache.ws.security.handler.WSHandlerConstants;
+import org.apache.ws.security.handler.WSHandlerResult;
+import org.apache.ws.security.message.token.BinarySecurity;
+import org.apache.ws.security.message.token.KerberosSecurity;
+
+/**
+ * 
+ */
+public class KerberosTokenInterceptorProvider extends AbstractPolicyInterceptorProvider {
+
+    public KerberosTokenInterceptorProvider() {
+        super(Arrays.asList(SP11Constants.KERBEROS_TOKEN, SP12Constants.KERBEROS_TOKEN));
+        
+        this.getOutInterceptors().add(PolicyBasedWSS4JOutInterceptor.INSTANCE);
+        this.getOutFaultInterceptors().add(PolicyBasedWSS4JOutInterceptor.INSTANCE);
+        this.getInInterceptors().add(PolicyBasedWSS4JInInterceptor.INSTANCE);
+        this.getInFaultInterceptors().add(PolicyBasedWSS4JInInterceptor.INSTANCE);
+        
+        this.getOutInterceptors().add(new KerberosTokenOutInterceptor());
+        this.getOutFaultInterceptors().add(new KerberosTokenOutInterceptor());
+        this.getInInterceptors().add(new KerberosTokenInInterceptor());
+        this.getInFaultInterceptors().add(new KerberosTokenInInterceptor());
+    }
+    
+    
+    static final TokenStore getTokenStore(Message message) {
+        TokenStore tokenStore = (TokenStore)message.getContextualProperty(TokenStore.class.getName());
+        if (tokenStore == null) {
+            tokenStore = new MemoryTokenStore();
+            message.getExchange().get(Endpoint.class).getEndpointInfo()
+                .setProperty(TokenStore.class.getName(), tokenStore);
+        }
+        return tokenStore;
+    }
+
+    static class KerberosTokenOutInterceptor extends AbstractPhaseInterceptor<Message> {
+        public KerberosTokenOutInterceptor() {
+            super(Phase.PREPARE_SEND);
+        }
+        public void handleMessage(Message message) throws Fault {
+            AssertionInfoMap aim = message.get(AssertionInfoMap.class);
+            // extract Assertion information
+            if (aim != null) {
+                Collection<AssertionInfo> ais = aim.get(SP12Constants.KERBEROS_TOKEN);
+                if (ais == null || ais.isEmpty()) {
+                    return;
+                }
+                if (isRequestor(message)) {
+                    SecurityToken tok = (SecurityToken)message.getContextualProperty(SecurityConstants.TOKEN);
+                    if (tok == null) {
+                        String tokId = (String)message.getContextualProperty(SecurityConstants.TOKEN_ID);
+                        if (tokId != null) {
+                            tok = getTokenStore(message).getToken(tokId);
+                        }
+                    }
+                    if (tok == null) {
+                        try {
+                            KerberosClient client = KerberosUtils.getClient(message, "kerberos");
+                            synchronized (client) {
+                                tok = client.requestSecurityToken();
+                            }
+                        } catch (RuntimeException e) {
+                            throw e;
+                        } catch (Exception e) {
+                            throw new Fault(e);
+                        }
+                    }
+                    if (tok != null) {
+                        for (AssertionInfo ai : ais) {
+                            ai.setAsserted(true);
+                        }
+                        message.getExchange().get(Endpoint.class).put(SecurityConstants.TOKEN_ID, 
+                                                                      tok.getId());
+                        message.getExchange().put(SecurityConstants.TOKEN_ID, 
+                                                  tok.getId());
+                        getTokenStore(message).add(tok);
+                    }
+                } else {
+                    //server side should be checked on the way in
+                    for (AssertionInfo ai : ais) {
+                        ai.setAsserted(true);
+                    }                    
+                }
+            }
+        }
+        
+    }
+    
+    static class KerberosTokenInInterceptor extends AbstractPhaseInterceptor<Message> {
+        public KerberosTokenInInterceptor() {
+            super(Phase.PRE_PROTOCOL);
+            addAfter(WSS4JInInterceptor.class.getName());
+            addAfter(PolicyBasedWSS4JInInterceptor.class.getName());
+        }
+
+        public void handleMessage(Message message) throws Fault {
+            AssertionInfoMap aim = message.get(AssertionInfoMap.class);
+            // extract Assertion information
+            if (aim != null) {
+                Collection<AssertionInfo> ais = aim.get(SP12Constants.KERBEROS_TOKEN);
+                if (ais == null) {
+                    return;
+                }
+                if (!isRequestor(message)) {
+                    List<WSHandlerResult> results = 
+                        CastUtils.cast((List<?>)message.get(WSHandlerConstants.RECV_RESULTS));
+                    if (results != null) {
+                        parseHandlerResults(results, message, aim);
+                    }
+                } else {
+                    //client side should be checked on the way out
+                    for (AssertionInfo ai : ais) {
+                        ai.setAsserted(true);
+                    }                    
+                }
+            }
+        }
+        
+        private void parseHandlerResults(
+            List<WSHandlerResult> results,
+            Message message,
+            AssertionInfoMap aim
+        ) {
+            if (results != null) {
+                for (WSHandlerResult rResult : results) {
+                    List<KerberosSecurity> kerberosResults = findKerberosResults(rResult.getResults());
+                    for (KerberosSecurity kerberosToken : kerberosResults) {
+                        KerberosTokenPolicyValidator kerberosValidator = 
+                            new KerberosTokenPolicyValidator(message);
+                        boolean valid = kerberosValidator.validatePolicy(aim, kerberosToken);
+                        if (valid) {
+                            SecurityToken token = createSecurityToken(kerberosToken);
+                            message.getExchange().put(SecurityConstants.TOKEN, token);
+                            return;
+                        }
+                    }
+                }
+            }
+        }
+        
+        private List<KerberosSecurity> findKerberosResults(
+            List<WSSecurityEngineResult> wsSecEngineResults
+        ) {
+            List<KerberosSecurity> results = new ArrayList<KerberosSecurity>();
+            for (WSSecurityEngineResult wser : wsSecEngineResults) {
+                Integer actInt = (Integer)wser.get(WSSecurityEngineResult.TAG_ACTION);
+                if (actInt.intValue() == WSConstants.BST) {
+                    BinarySecurity binarySecurity = 
+                        (BinarySecurity)wser.get(WSSecurityEngineResult.TAG_BINARY_SECURITY_TOKEN);
+                    if (binarySecurity instanceof KerberosSecurity) {
+                        results.add((KerberosSecurity)binarySecurity);
+                    }
+                }
+            }
+            return results;
+        }
+    }
+    
+    private static SecurityToken createSecurityToken(BinarySecurity binarySecurityToken) {
+        SecurityToken token = new SecurityToken(binarySecurityToken.getID());
+        token.setToken(binarySecurityToken.getElement());
+        token.setSecret(binarySecurityToken.getToken());
+        token.setTokenType(binarySecurityToken.getValueType());
+
+        return token;
+    }
+        
+}

Modified: cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JInInterceptor.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JInInterceptor.java?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JInInterceptor.java (original)
+++ cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/PolicyBasedWSS4JInInterceptor.java Wed Jul 27 15:49:23 2011
@@ -74,7 +74,6 @@ import org.apache.cxf.ws.security.policy
 import org.apache.cxf.ws.security.wss4j.CryptoCoverageUtil.CoverageScope;
 import org.apache.cxf.ws.security.wss4j.CryptoCoverageUtil.CoverageType;
 import org.apache.cxf.ws.security.wss4j.policyvalidators.EndorsingTokenPolicyValidator;
-import org.apache.cxf.ws.security.wss4j.policyvalidators.KerberosTokenPolicyValidator;
 import org.apache.cxf.ws.security.wss4j.policyvalidators.SamlTokenPolicyValidator;
 import org.apache.cxf.ws.security.wss4j.policyvalidators.UsernameTokenPolicyValidator;
 import org.apache.cxf.ws.security.wss4j.policyvalidators.X509TokenPolicyValidator;
@@ -625,10 +624,6 @@ public class PolicyBasedWSS4JInIntercept
         X509TokenPolicyValidator x509Validator = new X509TokenPolicyValidator(msg, results);
         x509Validator.validatePolicy(aim);
         
-        KerberosTokenPolicyValidator kerberosValidator = 
-            new KerberosTokenPolicyValidator(msg, results);
-        kerberosValidator.validatePolicy(aim);
-        
         //REVISIT - probably can verify some of these like if UT is encrypted and/or signed, etc...
         assertPolicy(aim, SP12Constants.SIGNED_SUPPORTING_TOKENS);
         assertPolicy(aim, SP12Constants.SIGNED_ENCRYPTED_SUPPORTING_TOKENS);

Modified: cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyhandlers/AbstractBindingBuilder.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyhandlers/AbstractBindingBuilder.java?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyhandlers/AbstractBindingBuilder.java (original)
+++ cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyhandlers/AbstractBindingBuilder.java Wed Jul 27 15:49:23 2011
@@ -486,13 +486,16 @@ public abstract class AbstractBindingBui
                 }
             } else if (isRequestor() 
                 && (token instanceof IssuedToken
-                    || token instanceof SecureConversationToken)) {
+                    || token instanceof SecureConversationToken
+                    || token instanceof KerberosToken)) {
                 //ws-trust/ws-sc stuff.......
                 SecurityToken secToken = getSecurityToken();
                 if (secToken == null) {
                     policyNotAsserted(token, "Could not find IssuedToken");
                 }
-                addSupportingElement(cloneElement(secToken.getToken()));
+                Element clone = cloneElement(secToken.getToken());
+                secToken.setToken(clone);
+                addSupportingElement(clone);
         
                 if (suppTokens.isEncryptedToken()) {
                     this.encryptedTokensIdList.add(secToken.getId());
@@ -569,11 +572,6 @@ public abstract class AbstractBindingBui
                     addSupportingElement(assertionWrapper.toDOM(saaj.getSOAPPart()));
                     ret.put(token, assertionWrapper);
                 }
-            } else if (token instanceof KerberosToken) {
-                BinarySecurity binarySecurity = addKerberosToken((KerberosToken)token);
-                Element clone = cloneElement(binarySecurity.getElement());
-                addSupportingElement(clone);
-                ret.put(token, new BinarySecurity(clone));
             }
         }
         return ret;
@@ -671,7 +669,8 @@ public abstract class AbstractBindingBui
                     part.setId(secRef.getID());
                     part.setElement(clone);
                 } else {
-                    policyNotAsserted(entry.getKey(), "UnsupportedTokenInSupportingToken: " + tempTok);  
+                    part = new WSEncryptionPart(token.getId());
+                    part.setElement(token.getToken());
                 }
             } else {
                 policyNotAsserted(entry.getKey(), "UnsupportedTokenInSupportingToken: " + tempTok);  
@@ -858,42 +857,6 @@ public abstract class AbstractBindingBui
         return assertion;
     }
     
-    protected BinarySecurity addKerberosToken(KerberosToken token) throws WSSecurityException {
-        AssertionInfo info = null;
-        Collection<AssertionInfo> ais = aim.getAssertionInfo(token.getName());
-        for (AssertionInfo ai : ais) {
-            if (ai.getAssertion() == token) {
-                info = ai;
-                if (!isRequestor()) {
-                    info.setAsserted(true);
-                    return null;
-                }
-            }
-        }
-        
-        //
-        // Get the BST (Kerberos) CallbackHandler
-        //
-        Object o = message.getContextualProperty(SecurityConstants.BST_CALLBACK_HANDLER);
-    
-        CallbackHandler handler = null;
-        if (o instanceof CallbackHandler) {
-            handler = (CallbackHandler)o;
-        } else if (o instanceof String) {
-            try {
-                handler = (CallbackHandler)ClassLoaderUtils
-                    .loadClass((String)o, this.getClass()).newInstance();
-            } catch (Exception e) {
-                handler = null;
-            }
-        }
-        if (handler == null) {
-            policyNotAsserted(token, "No BST CallbackHandler available");
-            return null;
-        }
-        return new BinarySecurity(handler);
-    }
-    
     public String getPassword(String userName, Assertion info, int type) {
         //Then try to get the password from the given callback handler
         Object o = message.getContextualProperty(SecurityConstants.CALLBACK_HANDLER);

Modified: cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyhandlers/TransportBindingHandler.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyhandlers/TransportBindingHandler.java?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyhandlers/TransportBindingHandler.java (original)
+++ cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyhandlers/TransportBindingHandler.java Wed Jul 27 15:49:23 2011
@@ -63,7 +63,6 @@ import org.apache.ws.security.message.WS
 import org.apache.ws.security.message.WSSecSignature;
 import org.apache.ws.security.message.WSSecTimestamp;
 import org.apache.ws.security.message.WSSecUsernameToken;
-import org.apache.ws.security.message.token.BinarySecurity;
 import org.apache.ws.security.message.token.SecurityTokenReference;
 import org.apache.ws.security.saml.ext.AssertionWrapper;
 
@@ -92,7 +91,7 @@ public class TransportBindingHandler ext
                     utBuilder.prepare(saaj.getSOAPPart());
                     utBuilder.appendToHeader(secHeader);
                 }
-            } else if (token instanceof IssuedToken) {
+            } else if (token instanceof IssuedToken || token instanceof KerberosToken) {
                 SecurityToken secTok = getSecurityToken();
                 
                 if (includeToken(token.getInclusion())) {
@@ -104,9 +103,6 @@ public class TransportBindingHandler ext
                 if (assertionWrapper != null) {
                     addSupportingElement(assertionWrapper.toDOM(saaj.getSOAPPart()));
                 }
-            } else if (token instanceof KerberosToken) {
-                BinarySecurity binarySecurity = addKerberosToken((KerberosToken)token);
-                addSupportingElement(cloneElement(binarySecurity.getElement()));
             } else {
                 //REVISIT - not supported for signed.  Exception?
             }

Modified: cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyvalidators/KerberosTokenPolicyValidator.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyvalidators/KerberosTokenPolicyValidator.java?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyvalidators/KerberosTokenPolicyValidator.java (original)
+++ cxf/branches/2.4.x-fixes/rt/ws/security/src/main/java/org/apache/cxf/ws/security/wss4j/policyvalidators/KerberosTokenPolicyValidator.java Wed Jul 27 15:49:23 2011
@@ -19,20 +19,14 @@
 
 package org.apache.cxf.ws.security.wss4j.policyvalidators;
 
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.List;
 
 import org.apache.cxf.message.Message;
 import org.apache.cxf.ws.policy.AssertionInfo;
 import org.apache.cxf.ws.policy.AssertionInfoMap;
 import org.apache.cxf.ws.security.policy.SP12Constants;
 import org.apache.cxf.ws.security.policy.model.KerberosToken;
-import org.apache.ws.security.WSConstants;
-import org.apache.ws.security.WSSecurityEngineResult;
-import org.apache.ws.security.message.token.BinarySecurity;
 import org.apache.ws.security.message.token.KerberosSecurity;
-import org.apache.ws.security.util.WSSecurityUtil;
 
 /**
  * Validate a WSSecurityEngineResult corresponding to the processing of a Kerberos Token
@@ -40,20 +34,17 @@ import org.apache.ws.security.util.WSSec
  */
 public class KerberosTokenPolicyValidator extends AbstractTokenPolicyValidator {
     
-    private List<WSSecurityEngineResult> bstResults;
     private Message message;
 
     public KerberosTokenPolicyValidator(
-        Message message,
-        List<WSSecurityEngineResult> results
+        Message message
     ) {
         this.message = message;
-        bstResults = new ArrayList<WSSecurityEngineResult>();
-        WSSecurityUtil.fetchAllActionResults(results, WSConstants.BST, bstResults);
     }
     
     public boolean validatePolicy(
-        AssertionInfoMap aim
+        AssertionInfoMap aim,
+        KerberosSecurity kerberosToken
     ) {
         Collection<AssertionInfo> krbAis = aim.get(SP12Constants.KERBEROS_TOKEN);
         if (krbAis != null && !krbAis.isEmpty()) {
@@ -65,14 +56,7 @@ public class KerberosTokenPolicyValidato
                     continue;
                 }
                 
-                if (bstResults.isEmpty()) {
-                    ai.setNotAsserted(
-                        "The received token does not match the token inclusion requirement"
-                    );
-                    return false;
-                }
-                
-                if (!checkToken(kerberosTokenPolicy)) {
+                if (!checkToken(kerberosTokenPolicy, kerberosToken)) {
                     ai.setNotAsserted("An incorrect Kerberos Token Type is detected");
                     return false;
                 }
@@ -81,25 +65,16 @@ public class KerberosTokenPolicyValidato
         return true;
     }
     
-    private boolean checkToken(KerberosToken kerberosTokenPolicy) {
-        if (!bstResults.isEmpty()) {
-            boolean isV5ApReq = kerberosTokenPolicy.isV5ApReqToken11();
-            boolean isGssV5ApReq = kerberosTokenPolicy.isGssV5ApReqToken11();
-            
-            for (WSSecurityEngineResult result : bstResults) {
-                BinarySecurity binarySecurityToken = 
-                    (BinarySecurity)result.get(WSSecurityEngineResult.TAG_BINARY_SECURITY_TOKEN);
-                if (binarySecurityToken instanceof KerberosSecurity) {
-                    if (isV5ApReq && ((KerberosSecurity)binarySecurityToken).isV5ApReq()) {
-                        return true;
-                    } else if (isGssV5ApReq 
-                        && ((KerberosSecurity)binarySecurityToken).isGssV5ApReq()) {
-                        return true;
-                    } else if (!isV5ApReq && !isGssV5ApReq) {
-                        return true;
-                    }
-                }
-            }
+    private boolean checkToken(KerberosToken kerberosTokenPolicy, KerberosSecurity kerberosToken) {
+        boolean isV5ApReq = kerberosTokenPolicy.isV5ApReqToken11();
+        boolean isGssV5ApReq = kerberosTokenPolicy.isGssV5ApReqToken11();
+
+        if (isV5ApReq && kerberosToken.isV5ApReq()) {
+            return true;
+        } else if (isGssV5ApReq && kerberosToken.isGssV5ApReq()) {
+            return true;
+        } else if (!(isV5ApReq || isGssV5ApReq)) {
+            return true;
         }
         return false;
     }

Modified: cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/KerberosTokenTest.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/KerberosTokenTest.java?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/KerberosTokenTest.java (original)
+++ cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/KerberosTokenTest.java Wed Jul 27 15:49:23 2011
@@ -25,14 +25,11 @@ import java.net.URL;
 import javax.crypto.Cipher;
 import javax.crypto.SecretKey;
 import javax.crypto.spec.SecretKeySpec;
-import javax.xml.ws.BindingProvider;
 
 import org.apache.cxf.Bus;
 import org.apache.cxf.bus.spring.SpringBusFactory;
-import org.apache.cxf.systest.ws.kerberos.client.KerberosCallbackHandler;
 import org.apache.cxf.systest.ws.kerberos.server.Server;
 import org.apache.cxf.testutil.common.AbstractBusClientServerTestBase;
-import org.apache.ws.security.WSConstants;
 
 import org.junit.BeforeClass;
 
@@ -40,7 +37,13 @@ import wssec.kerberos.DoubleItPortType;
 import wssec.kerberos.DoubleItService;
 
 /**
- * A set of tests for Kerberos Tokens.
+ * A set of tests for Kerberos Tokens. The tests are @Ignore'd, as they require a running KDC. To run the
+ * tests, set up a KDC of realm "WS.APACHE.ORG", with principal "alice" and service principal 
+ * "bob/service.ws.apache.org". Create keytabs for both principals in "/etc/alice.keytab" and
+ * "/etc/bob.keytab" (this can all be edited in src/test/resource/kerberos.jaas". Then disable the
+ * @Ignore annotations and run the tests with:
+ *  
+ * mvn test -Dtest=KerberosTokenTest -Djava.security.auth.login.config=src/test/resources/kerberos.jaas
  */
 public class KerberosTokenTest extends AbstractBusClientServerTestBase {
     static final String PORT = allocatePort(Server.class);
@@ -59,6 +62,7 @@ public class KerberosTokenTest extends A
     }
 
     @org.junit.Test
+    @org.junit.Ignore
     public void testKerberosOverTransport() throws Exception {
 
         SpringBusFactory bf = new SpringBusFactory();
@@ -72,35 +76,12 @@ public class KerberosTokenTest extends A
         
         DoubleItPortType kerberosPort = service.getDoubleItKerberosTransportPort();
         updateAddressPort(kerberosPort, PORT2);
-        
-        try {
-            kerberosPort.doubleIt(BigInteger.valueOf(25));
-            fail("Expected failure on an invocation with no Kerberos Token");
-        } catch (javax.xml.ws.soap.SOAPFaultException ex) {
-            assertTrue(ex.getMessage().contains("No BST CallbackHandler available"));
-        }
-        
-        try {
-            KerberosCallbackHandler handler = new KerberosCallbackHandler();
-            handler.setValueType(WSConstants.WSS_KRB_V5_AP_REQ);
-            ((BindingProvider)kerberosPort).getRequestContext().put(
-                "ws-security.bst-callback-handler", handler
-            );
-            kerberosPort.doubleIt(BigInteger.valueOf(25));
-            fail("Expected failure on an invocation with the wrong Kerberos Token");
-        } catch (javax.xml.ws.soap.SOAPFaultException ex) {
-            assertTrue(ex.getMessage().contains("incorrect Kerberos Token Type"));
-        }
-        
-        ((BindingProvider)kerberosPort).getRequestContext().put(
-            "ws-security.bst-callback-handler", new KerberosCallbackHandler()
-        );
-
         BigInteger result = kerberosPort.doubleIt(BigInteger.valueOf(25));
         assertTrue(result.equals(BigInteger.valueOf(50)));
     }
     
     @org.junit.Test
+    @org.junit.Ignore
     public void testKerberosOverSymmetric() throws Exception {
         
         if (!unrestrictedPoliciesInstalled) {
@@ -119,34 +100,12 @@ public class KerberosTokenTest extends A
         DoubleItPortType kerberosPort = service.getDoubleItKerberosSymmetricPort();
         updateAddressPort(kerberosPort, PORT);
         
-        try {
-            kerberosPort.doubleIt(BigInteger.valueOf(25));
-            fail("Expected failure on an invocation with no Kerberos Token");
-        } catch (javax.xml.ws.soap.SOAPFaultException ex) {
-            assertTrue(ex.getMessage().contains("No BST CallbackHandler available"));
-        }
-        
-        try {
-            KerberosCallbackHandler handler = new KerberosCallbackHandler();
-            handler.setToken("123456566");
-            ((BindingProvider)kerberosPort).getRequestContext().put(
-                "ws-security.bst-callback-handler", handler
-            );
-            kerberosPort.doubleIt(BigInteger.valueOf(25));
-            fail("Expected failure on an invocation with the wrong Kerberos Token");
-        } catch (javax.xml.ws.soap.SOAPFaultException ex) {
-            // expected
-        }
-
-        ((BindingProvider)kerberosPort).getRequestContext().put(
-            "ws-security.bst-callback-handler", new KerberosCallbackHandler()
-        );
-
         BigInteger result = kerberosPort.doubleIt(BigInteger.valueOf(25));
         assertTrue(result.equals(BigInteger.valueOf(50)));
     }
     
     @org.junit.Test
+    @org.junit.Ignore
     public void testKerberosOverSymmetricSupporting() throws Exception {
         
         if (!unrestrictedPoliciesInstalled) {
@@ -165,34 +124,12 @@ public class KerberosTokenTest extends A
         DoubleItPortType kerberosPort = service.getDoubleItKerberosSymmetricSupportingPort();
         updateAddressPort(kerberosPort, PORT);
         
-        try {
-            kerberosPort.doubleIt(BigInteger.valueOf(25));
-            fail("Expected failure on an invocation with no Kerberos Token");
-        } catch (javax.xml.ws.soap.SOAPFaultException ex) {
-            assertTrue(ex.getMessage().contains("No BST CallbackHandler available"));
-        }
-        
-        try {
-            KerberosCallbackHandler handler = new KerberosCallbackHandler();
-            handler.setToken("123456566");
-            ((BindingProvider)kerberosPort).getRequestContext().put(
-                "ws-security.bst-callback-handler", handler
-            );
-            kerberosPort.doubleIt(BigInteger.valueOf(25));
-            fail("Expected failure on an invocation with the wrong Kerberos Token");
-        } catch (javax.xml.ws.soap.SOAPFaultException ex) {
-            // expected
-        }
-        
-        ((BindingProvider)kerberosPort).getRequestContext().put(
-            "ws-security.bst-callback-handler", new KerberosCallbackHandler()
-        );
-
         BigInteger result = kerberosPort.doubleIt(BigInteger.valueOf(25));
         assertTrue(result.equals(BigInteger.valueOf(50)));
     }
     
     @org.junit.Test
+    @org.junit.Ignore
     public void testKerberosOverAsymmetric() throws Exception {
         
         if (!unrestrictedPoliciesInstalled) {
@@ -211,29 +148,6 @@ public class KerberosTokenTest extends A
         DoubleItPortType kerberosPort = service.getDoubleItKerberosAsymmetricPort();
         updateAddressPort(kerberosPort, PORT);
         
-        try {
-            kerberosPort.doubleIt(BigInteger.valueOf(25));
-            fail("Expected failure on an invocation with no Kerberos Token");
-        } catch (javax.xml.ws.soap.SOAPFaultException ex) {
-            assertTrue(ex.getMessage().contains("No BST CallbackHandler available"));
-        }
-        
-        try {
-            ((BindingProvider)kerberosPort).getRequestContext().put(
-                "ws-security.bst-callback-handler", new KerberosCallbackHandler()
-            );
-            kerberosPort.doubleIt(BigInteger.valueOf(25));
-            fail("Expected failure on an invocation with the wrong Kerberos Token");
-        } catch (javax.xml.ws.soap.SOAPFaultException ex) {
-            assertTrue(ex.getMessage().contains("incorrect Kerberos Token Type"));
-        }
-        
-        KerberosCallbackHandler handler = new KerberosCallbackHandler();
-        handler.setValueType(WSConstants.WSS_KRB_V5_AP_REQ);
-        ((BindingProvider)kerberosPort).getRequestContext().put(
-            "ws-security.bst-callback-handler", handler
-        );
-
         BigInteger result = kerberosPort.doubleIt(BigInteger.valueOf(25));
         assertTrue(result.equals(BigInteger.valueOf(50)));
     }

Modified: cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/server/DoubleItImpl.java
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/server/DoubleItImpl.java?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/server/DoubleItImpl.java (original)
+++ cxf/branches/2.4.x-fixes/systests/ws-security/src/test/java/org/apache/cxf/systest/ws/kerberos/server/DoubleItImpl.java Wed Jul 27 15:49:23 2011
@@ -20,10 +20,14 @@
 package org.apache.cxf.systest.ws.kerberos.server;
 
 import java.math.BigInteger;
+import java.security.Principal;
 
+import javax.annotation.Resource;
 import javax.jws.WebService;
+import javax.xml.ws.WebServiceContext;
 
 import org.apache.cxf.feature.Features;
+import org.junit.Assert;
 
 import wssec.kerberos.DoubleItPortType;
 
@@ -33,7 +37,15 @@ import wssec.kerberos.DoubleItPortType;
 @Features(features = "org.apache.cxf.feature.LoggingFeature")              
 public class DoubleItImpl implements DoubleItPortType {
     
+    @Resource
+    WebServiceContext wsContext;
+    
     public java.math.BigInteger doubleIt(java.math.BigInteger numberToDouble) {
+        Principal pr = wsContext.getUserPrincipal();
+        
+        Assert.assertNotNull("Principal must not be null", pr);
+        Assert.assertNotNull("Principal.getName() must not return null", pr.getName());
+        
         return numberToDouble.multiply(BigInteger.valueOf(2));
     }
     

Added: cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/kerberos.jaas
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/kerberos.jaas?rev=1151508&view=auto
==============================================================================
--- cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/kerberos.jaas (added)
+++ cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/kerberos.jaas Wed Jul 27 15:49:23 2011
@@ -0,0 +1,8 @@
+
+alice {
+    com.sun.security.auth.module.Krb5LoginModule required refreshKrb5Config=true useKeyTab=true keyTab="/etc/alice.keytab" principal="alice";
+};
+
+bob {
+    com.sun.security.auth.module.Krb5LoginModule required refreshKrb5Config=true useKeyTab=true storeKey=true keyTab="/etc/bob.keytab" principal="bob/service.ws.apache.org";
+};

Modified: cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/logging.properties
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/logging.properties?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/logging.properties (original)
+++ cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/logging.properties Wed Jul 27 15:49:23 2011
@@ -35,7 +35,7 @@
 # Note that these classes must be on the system classpath.
 # By default we only configure a ConsoleHandler, which will only
 # show messages at the INFO and above levels.
-#handlers= java.util.logging.ConsoleHandler
+handlers= java.util.logging.ConsoleHandler
 
 # To also add the FileHandler, use the following line instead.
 #handlers= java.util.logging.FileHandler, java.util.logging.ConsoleHandler

Modified: cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/org/apache/cxf/systest/ws/kerberos/client/client.xml
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/org/apache/cxf/systest/ws/kerberos/client/client.xml?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/org/apache/cxf/systest/ws/kerberos/client/client.xml (original)
+++ cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/org/apache/cxf/systest/ws/kerberos/client/client.xml Wed Jul 27 15:49:23 2011
@@ -52,12 +52,32 @@
         </http:tlsClientParameters>
     </http:conduit>  
     
+    <jaxws:client name="{http://WSSec/kerberos}DoubleItKerberosTransportPort" 
+                  createdFromAPI="true">
+       <jaxws:properties>
+           <entry key="ws-security.kerberos.client">
+               <bean class="org.apache.cxf.ws.security.kerberos.KerberosClient">
+                   <constructor-arg ref="cxf"/>
+                   <property name="jaasLoginModuleName" value="alice"/>
+                   <property name="serviceName" value="bob@service.ws.apache.org"/>
+               </bean>            
+           </entry> 
+       </jaxws:properties>
+    </jaxws:client>
+    
     <jaxws:client name="{http://WSSec/kerberos}DoubleItKerberosSymmetricPort" 
                   createdFromAPI="true">
        <jaxws:properties>
            <entry key="ws-security.encryption.properties" 
                   value="org/apache/cxf/systest/ws/wssec10/client/bob.properties"/> 
            <entry key="ws-security.encryption.username" value="bob"/>
+           <entry key="ws-security.kerberos.client">
+               <bean class="org.apache.cxf.ws.security.kerberos.KerberosClient">
+                   <constructor-arg ref="cxf"/>
+                   <property name="jaasLoginModuleName" value="alice"/>
+                   <property name="serviceName" value="bob@service.ws.apache.org"/>
+               </bean>            
+           </entry> 
        </jaxws:properties>
     </jaxws:client>
     
@@ -67,6 +87,13 @@
            <entry key="ws-security.encryption.properties" 
                   value="org/apache/cxf/systest/ws/wssec10/client/bob.properties"/> 
            <entry key="ws-security.encryption.username" value="bob"/>
+           <entry key="ws-security.kerberos.client">
+               <bean class="org.apache.cxf.ws.security.kerberos.KerberosClient">
+                   <constructor-arg ref="cxf"/>
+                   <property name="jaasLoginModuleName" value="alice"/>
+                   <property name="serviceName" value="bob@service.ws.apache.org"/>
+               </bean>            
+           </entry> 
        </jaxws:properties>
     </jaxws:client>
     
@@ -81,6 +108,13 @@
            <entry key="ws-security.signature.properties" 
                   value="org/apache/cxf/systest/ws/wssec10/client/alice.properties"/>
            <entry key="ws-security.signature.username" value="alice"/> 
+           <entry key="ws-security.kerberos.client">
+               <bean class="org.apache.cxf.ws.security.kerberos.KerberosClient">
+                   <constructor-arg ref="cxf"/>
+                   <property name="jaasLoginModuleName" value="alice"/>
+                   <property name="serviceName" value="bob@service.ws.apache.org"/>
+               </bean>            
+           </entry> 
        </jaxws:properties>
     </jaxws:client>
     

Modified: cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/org/apache/cxf/systest/ws/kerberos/server/server.xml
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/org/apache/cxf/systest/ws/kerberos/server/server.xml?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/org/apache/cxf/systest/ws/kerberos/server/server.xml (original)
+++ cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/org/apache/cxf/systest/ws/kerberos/server/server.xml Wed Jul 27 15:49:23 2011
@@ -71,6 +71,12 @@
         </httpj:engine>
     </httpj:engine-factory>
     
+    <bean id="kerberosValidator"
+        class="org.apache.ws.security.validate.KerberosTokenValidator">
+        <property name="jaasLoginModuleName" value="bob"/>
+        <property name="serviceName" value="bob@service.ws.apache.org"/>
+    </bean> 
+    
     <jaxws:endpoint 
        id="KerberosOverTransport"
        address="https://localhost:${testutil.ports.Server.2}/DoubleItKerberosTransport" 
@@ -83,6 +89,7 @@
         
        <jaxws:properties>
            <entry key="ws-security.is-bsp-compliant" value="false"/>
+           <entry key="ws-security.bst.validator" value-ref="kerberosValidator"/>
        </jaxws:properties> 
      
     </jaxws:endpoint> 
@@ -101,8 +108,7 @@
                   value="org.apache.cxf.systest.ws.wssec10.client.KeystorePasswordCallback"/>
            <entry key="ws-security.signature.properties" 
                   value="org/apache/cxf/systest/ws/wssec10/client/bob.properties"/> 
-           <entry key="ws-security.bst.validator" 
-                  value="org.apache.cxf.systest.ws.kerberos.server.KerberosTokenValidator"/>
+           <entry key="ws-security.bst.validator" value-ref="kerberosValidator"/>
        </jaxws:properties> 
      
     </jaxws:endpoint> 
@@ -121,8 +127,7 @@
                   value="org.apache.cxf.systest.ws.wssec10.client.KeystorePasswordCallback"/>
            <entry key="ws-security.signature.properties" 
                   value="org/apache/cxf/systest/ws/wssec10/client/bob.properties"/> 
-           <entry key="ws-security.bst.validator" 
-                  value="org.apache.cxf.systest.ws.kerberos.server.KerberosTokenValidator"/>
+           <entry key="ws-security.bst.validator" value-ref="kerberosValidator"/>
        </jaxws:properties> 
      
     </jaxws:endpoint> 
@@ -145,8 +150,7 @@
            <entry key="ws-security.encryption.properties" 
                   value="org/apache/cxf/systest/ws/wssec10/client/alice.properties"/> 
            <entry key="ws-security.encryption.username" value="alice"/>
-           <entry key="ws-security.bst.validator" 
-                  value="org.apache.cxf.systest.ws.kerberos.server.KerberosTokenValidator"/>
+           <entry key="ws-security.bst.validator" value-ref="kerberosValidator"/>
            <entry key="ws-security.is-bsp-compliant" value="false"/>
        </jaxws:properties> 
      

Modified: cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/wsdl_systest_wssec/kerberos/DoubleItKerberos.wsdl
URL: http://svn.apache.org/viewvc/cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/wsdl_systest_wssec/kerberos/DoubleItKerberos.wsdl?rev=1151508&r1=1151507&r2=1151508&view=diff
==============================================================================
--- cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/wsdl_systest_wssec/kerberos/DoubleItKerberos.wsdl (original)
+++ cxf/branches/2.4.x-fixes/systests/ws-security/src/test/resources/wsdl_systest_wssec/kerberos/DoubleItKerberos.wsdl Wed Jul 27 15:49:23 2011
@@ -364,7 +364,7 @@
                   <sp:KerberosToken
                       sp:IncludeToken="http://docs.oasis-open.org/ws-sx/ws-securitypolicy/200702/IncludeToken/Once">
                       <wsp:Policy>
-                          <sp:WssKerberosV5ApReqToken11/>
+                          <sp:WssGssKerberosV5ApReqToken11/>
                       </wsp:Policy>
                   </sp:KerberosToken>
               </wsp:Policy>