You are viewing a plain text version of this content. The canonical link for it is here.
Posted to fx-dev@ws.apache.org by we...@apache.org on 2004/11/01 11:43:09 UTC

cvs commit: ws-fx/wss4j/src/org/apache/ws/axis/security WSDoAllReceiver.java WSDoAllSender.java

werner      2004/11/01 02:43:09

  Modified:    wss4j/src/org/apache/ws/axis/security WSDoAllReceiver.java
                        WSDoAllSender.java
  Log:
  Refactor code to pack all request specific data in a private data class.
  This enables the drivers to run as single instance in a mutli-thread
  enviornment wihout doing additional synchronization thus runs at
  maximum speed.
  
  Revision  Changes    Path
  1.21      +335 -298  ws-fx/wss4j/src/org/apache/ws/axis/security/WSDoAllReceiver.java
  
  Index: WSDoAllReceiver.java
  ===================================================================
  RCS file: /home/cvs/ws-fx/wss4j/src/org/apache/ws/axis/security/WSDoAllReceiver.java,v
  retrieving revision 1.20
  retrieving revision 1.21
  diff -u -r1.20 -r1.21
  --- WSDoAllReceiver.java	21 Oct 2004 00:32:12 -0000	1.20
  +++ WSDoAllReceiver.java	1 Nov 2004 10:43:09 -0000	1.21
  @@ -61,20 +61,36 @@
   
   public class WSDoAllReceiver extends BasicHandler {
       static Log log = LogFactory.getLog(WSDoAllReceiver.class.getName());
  -    static final WSSecurityEngine secEngine = new WSSecurityEngine();
  +    static final WSSecurityEngine secEngine = WSSecurityEngine.getInstance();
   
       private boolean doDebug = false;
   
       private static Hashtable cryptos = new Hashtable(5);
   
  -    Crypto sigCrypto = null;
  -    String sigPropFile = null;
  -
  -    Crypto decCrypto = null;
  -    String decPropFile = null;
  -
  -    protected int timeToLive = 300; // Timestamp: time in seconds the receiver accepts between creation and reception
  -
  +    /**
  +     * This nested private class hold per request data.
  +     * 
  +     * @author wdi
  +     */
  +    private class RequestData {
  +    	MessageContext msgContext = null;
  +    	
  +        Crypto sigCrypto = null;
  +        String sigPropFile = null;
  +
  +        Crypto decCrypto = null;
  +        String decPropFile = null;
  +
  +        int timeToLive = 300; 	// Timestamp: time in seconds between creation 
  +        						// and expiery
  +        void clear() {
  +        	decCrypto = null;
  +        	decPropFile = null;
  +        	msgContext = null;
  +        	sigCrypto = null;
  +        	sigPropFile = null;
  +        }
  +    }
       /**
        * Axis calls invoke to handle a message.
        * <p/>
  @@ -84,277 +100,297 @@
        */
       public void invoke(MessageContext msgContext) throws AxisFault {
   
  -        if (doDebug) {
  -            log.debug("WSDoAllReceiver: enter invoke() with msg type: "
  -                    + msgContext.getCurrentMessage().getMessageType());
  -        }
  -
  -        Vector actions = new Vector();
  -        String action = null;
  -        if ((action = (String) getOption(WSHandlerConstants.ACTION)) == null) {
  -            action = (String) msgContext.getProperty(WSHandlerConstants.ACTION);
  -        }
  -        if (action == null) {
  -            throw new AxisFault("WSDoAllReceiver: No action defined");
  -        }
  -        int doAction = AxisUtil.decodeAction(action, actions);
  -
  -        String actor = (String) getOption(WSHandlerConstants.ACTOR);
  -
  -        Message sm = msgContext.getCurrentMessage();
  -        Document doc = null;
  -        try {
  -            doc = sm.getSOAPEnvelope().getAsDocument();
  -            if (doDebug) {
  -                log.debug("Received SOAP request: ");
  -                log.debug(org.apache.axis.utils.XMLUtils.PrettyDocumentToString(doc));
  -            }
  -        } catch (Exception ex) {
  -            throw new AxisFault("WSDoAllReceiver: cannot convert into document",
  -                    ex);
  -        }
  -        /*
  -         * Check if it's a response and if its a fault. Don't
  -         * process faults.
  -         */
  -        String msgType = sm.getMessageType();
  -        if (msgType != null && msgType.equals(Message.RESPONSE)) {
  -            SOAPConstants soapConstants =
  -                    WSSecurityUtil.getSOAPConstants(doc.getDocumentElement());
  -            if (WSSecurityUtil
  -                    .findElement(doc.getDocumentElement(),
  -                            "Fault",
  -                            soapConstants.getEnvelopeURI())
  -                    != null) {
  -                return;
  -            }
  -        }
  -
  -        /*
  -         * To check a UsernameToken or to decrypt an encrypted message we need 
  -         * a password.
  -         */
  -        CallbackHandler cbHandler = null;
  -        if ((doAction & (WSConstants.ENCR | WSConstants.UT)) != 0) {
  -            cbHandler = getPasswordCB(msgContext);
  -        }
  -
  -        /*
  -         * Get and check the Signature specific parameters first because they 
  -         * may be used for encryption too.
  -         */
  -
  -        if ((doAction & WSConstants.SIGN) == WSConstants.SIGN) {
  -            decodeSignatureParameter(msgContext);
  -        }
  -
  -        if ((doAction & WSConstants.ENCR) == WSConstants.ENCR) {
  -            decodeDecryptionParameter(msgContext);
  -        }
  -
  -        Vector wsResult = null;
  -        try {
  -            wsResult =
  -                    secEngine.processSecurityHeader(doc,
  -                            actor,
  -                            cbHandler,
  -                            sigCrypto,
  -                            decCrypto);
  -        } catch (WSSecurityException ex) {
  -            ex.printStackTrace();
  -            throw new AxisFault("WSDoAllReceiver: security processing failed",
  -                    ex);
  -        }
  -        if (wsResult == null) {            // no security header found
  -            if (doAction == WSConstants.NO_SECURITY) {
  -                return;
  -            } else {
  -                throw new AxisFault("WSDoAllReceiver: Request does not contain required Security header");
  -            }
  -        }
  -
  -        /*
  -         * save the processed-header flags
  -         */ 
  -        ArrayList processedHeaders = new ArrayList();
  -        Iterator iterator = sm.getSOAPEnvelope().getHeaders().iterator();
  -        while (iterator.hasNext()) {
  -            org.apache.axis.message.SOAPHeaderElement tempHeader = (org.apache.axis.message.SOAPHeaderElement) iterator.next();
  -            if (tempHeader.isProcessed()) {
  -                processedHeaders.add(tempHeader.getQName());
  -            }
  -        }        
  -        
  -        /*
  -         * If we had some security processing, get the original
  -         * SOAP part of Axis' message and replace it with new SOAP
  -         * part. This new part may contain decrypted elements.
  -         */
  -        SOAPPart sPart = (org.apache.axis.SOAPPart) sm.getSOAPPart();
  -
  -        ByteArrayOutputStream os = new ByteArrayOutputStream();
  -        XMLUtils.outputDOM(doc, os, true);
  -        sPart.setCurrentMessage(os.toByteArray(), SOAPPart.FORM_BYTES);
  -        if (doDebug) {
  -            log.debug("Processed received SOAP request");
  -            log.debug(org.apache.axis.utils.XMLUtils.PrettyDocumentToString(doc));
  -        }
  -        
  -        /*
  -         * set the original processed-header flags
  -         */
  -        iterator = processedHeaders.iterator();
  -        while (iterator.hasNext()) {
  -            QName qname = (QName) iterator.next();
  -            org.apache.axis.message.SOAPHeaderElement tempHeader = (org.apache.axis.message.SOAPHeaderElement) sm.getSOAPEnvelope().getHeadersByName(qname.getNamespaceURI(), qname.getLocalPart());
  -            tempHeader.setProcessed(true);
  -        }        
  -
  -        /*
  -         * After setting the new current message, probably modified because
  -         * of decryption, we need to locate the security header. That is,
  -         * we force Axis (with getSOAPEnvelope()) to parse the string, build 
  -         * the new header. Then we examine, look up the security header 
  -         * and set the header as processed.
  -         * 
  -         * Please note: find all header elements that contain the same
  -         * actor that was given to processSecurityHeader(). Then
  -         * check if there is a security header with this actor.
  -         */
  -
  -        SOAPHeader sHeader = null;
  -        try {
  -            sHeader = sm.getSOAPEnvelope().getHeader();
  -        } catch (Exception ex) {
  -            throw new AxisFault("WSDoAllReceiver: cannot get SOAP header after security processing", ex);
  -        }
  -
  -        Iterator headers = sHeader.examineHeaderElements(actor);
  -
  -        SOAPHeaderElement headerElement = null;
  -        while (headers.hasNext()) {
  -            SOAPHeaderElement hE = (SOAPHeaderElement) headers.next();
  -            if (hE.getLocalName().equals(WSConstants.WSSE_LN)
  -                    && hE.getNamespaceURI().equals(WSConstants.WSSE_NS)) {
  -                headerElement = hE;
  -                break;
  -            }
  -        }
  -        ((org.apache.axis.message.SOAPHeaderElement) headerElement).setProcessed(true);
  -
  -        /*
  -         * Now we can check the certificate used to sign the message.
  -         * In the following implementation the certificate is only trusted
  -         * if either it itself or the certificate of the issuer is installed
  -         * in the keystore.
  -         * 
  -         * Note: the method verifyTrust(X509Certificate) allows custom
  -         * implementations with other validation algorithms for subclasses.
  -         */
  -
  -        // Extract the signature action result from the action vector
  -        WSSecurityEngineResult actionResult = WSSecurityUtil.fetchActionResult(wsResult, WSConstants.SIGN);
  -
  -        if (actionResult != null) {
  -            X509Certificate returnCert = actionResult.getCertificate();
  -
  -            if (returnCert != null) {
  -                if (!verifyTrust(returnCert)) {
  -                    throw new AxisFault("WSDoAllReceiver: The certificate used for the signature is not trusted");
  -                }
  -            }
  -        }
  -
  -        /*
  -         * Perform further checks on the timestamp that was transmitted in the header.
  -         * In the following implementation the timestamp is valid if it was
  -         * created after (now-ttl), where ttl is set on server side, not by the client. 
  -         * 
  -         * Note: the method verifyTimestamp(Timestamp) allows custom
  -         * implementations with other validation algorithms for subclasses.
  -         */
  -         
  -        // Extract the timestamp action result from the action vector
  -        actionResult = WSSecurityUtil.fetchActionResult(wsResult, WSConstants.TS);
  -
  -        if (actionResult != null) {
  -            Timestamp timestamp = actionResult.getTimestamp();
  -
  -            if (timestamp != null) {
  -                String ttl = null;
  -                if ((ttl =
  -                        (String) getOption(WSHandlerConstants.TTL_TIMESTAMP))
  -                        == null) {
  -                    ttl =
  -                            (String) msgContext.getProperty(WSHandlerConstants.TTL_TIMESTAMP);
  -                }
  -                int ttl_i = 0;
  -                if (ttl != null) {
  -                    try {
  -                        ttl_i = Integer.parseInt(ttl);
  -                    } catch (NumberFormatException e) {
  -                        ttl_i = timeToLive;
  -                    }
  -                }
  -                if (ttl_i <= 0) {
  -                    ttl_i = timeToLive;
  -                }
  -
  -                if (!verifyTimestamp(timestamp, timeToLive)) {
  -                    throw new AxisFault("WSDoAllReceiver: The timestamp could not be validated");
  -                }
  -            }
  -        }
  +		if (doDebug) {
  +			log.debug("WSDoAllReceiver: enter invoke() with msg type: "
  +					+ msgContext.getCurrentMessage().getMessageType());
  +		}
  +
  +		RequestData reqData = new RequestData();
  +		/*
  +		 * The overall try, just to have a finally at the end to perform some
  +		 * housekeeping.
  +		 */
  +		try {
  +			reqData.msgContext = msgContext;
  +
  +			Vector actions = new Vector();
  +			String action = null;
  +			if ((action = (String) getOption(WSHandlerConstants.ACTION)) == null) {
  +				action = (String) msgContext
  +						.getProperty(WSHandlerConstants.ACTION);
  +			}
  +			if (action == null) {
  +				throw new AxisFault("WSDoAllReceiver: No action defined");
  +			}
  +			int doAction = AxisUtil.decodeAction(action, actions);
  +
  +			String actor = (String) getOption(WSHandlerConstants.ACTOR);
  +
  +			Message sm = msgContext.getCurrentMessage();
  +			Document doc = null;
  +			try {
  +				doc = sm.getSOAPEnvelope().getAsDocument();
  +				if (doDebug) {
  +					log.debug("Received SOAP request: ");
  +					log.debug(org.apache.axis.utils.XMLUtils
  +							.PrettyDocumentToString(doc));
  +				}
  +			} catch (Exception ex) {
  +				throw new AxisFault(
  +						"WSDoAllReceiver: cannot convert into document", ex);
  +			}
  +			/*
  +			 * Check if it's a response and if its a fault. Don't process
  +			 * faults.
  +			 */
  +			String msgType = sm.getMessageType();
  +			if (msgType != null && msgType.equals(Message.RESPONSE)) {
  +				SOAPConstants soapConstants = WSSecurityUtil
  +						.getSOAPConstants(doc.getDocumentElement());
  +				if (WSSecurityUtil.findElement(doc.getDocumentElement(),
  +						"Fault", soapConstants.getEnvelopeURI()) != null) {
  +					return;
  +				}
  +			}
  +
  +			/*
  +			 * To check a UsernameToken or to decrypt an encrypted message we
  +			 * need a password.
  +			 */
  +			CallbackHandler cbHandler = null;
  +			if ((doAction & (WSConstants.ENCR | WSConstants.UT)) != 0) {
  +				cbHandler = getPasswordCB(reqData);
  +			}
  +
  +			/*
  +			 * Get and check the Signature specific parameters first because
  +			 * they may be used for encryption too.
  +			 */
  +
  +			if ((doAction & WSConstants.SIGN) == WSConstants.SIGN) {
  +				decodeSignatureParameter(reqData);
  +			}
  +
  +			if ((doAction & WSConstants.ENCR) == WSConstants.ENCR) {
  +				decodeDecryptionParameter(reqData);
  +			}
  +
  +			Vector wsResult = null;
  +			try {
  +				wsResult = secEngine.processSecurityHeader(doc, actor,
  +						cbHandler, reqData.sigCrypto, reqData.decCrypto);
  +			} catch (WSSecurityException ex) {
  +				ex.printStackTrace();
  +				throw new AxisFault(
  +						"WSDoAllReceiver: security processing failed", ex);
  +			}
  +			if (wsResult == null) { // no security header found
  +				if (doAction == WSConstants.NO_SECURITY) {
  +					return;
  +				} else {
  +					throw new AxisFault(
  +							"WSDoAllReceiver: Request does not contain required Security header");
  +				}
  +			}
  +
  +			/*
  +			 * save the processed-header flags
  +			 */
  +			ArrayList processedHeaders = new ArrayList();
  +			Iterator iterator = sm.getSOAPEnvelope().getHeaders().iterator();
  +			while (iterator.hasNext()) {
  +				org.apache.axis.message.SOAPHeaderElement tempHeader = (org.apache.axis.message.SOAPHeaderElement) iterator
  +						.next();
  +				if (tempHeader.isProcessed()) {
  +					processedHeaders.add(tempHeader.getQName());
  +				}
  +			}
  +
  +			/*
  +			 * If we had some security processing, get the original SOAP part of
  +			 * Axis' message and replace it with new SOAP part. This new part
  +			 * may contain decrypted elements.
  +			 */
  +			SOAPPart sPart = (org.apache.axis.SOAPPart) sm.getSOAPPart();
  +
  +			ByteArrayOutputStream os = new ByteArrayOutputStream();
  +			XMLUtils.outputDOM(doc, os, true);
  +			sPart.setCurrentMessage(os.toByteArray(), SOAPPart.FORM_BYTES);
  +			if (doDebug) {
  +				log.debug("Processed received SOAP request");
  +				log.debug(org.apache.axis.utils.XMLUtils
  +						.PrettyDocumentToString(doc));
  +			}
  +
  +			/*
  +			 * set the original processed-header flags
  +			 */
  +			iterator = processedHeaders.iterator();
  +			while (iterator.hasNext()) {
  +				QName qname = (QName) iterator.next();
  +				org.apache.axis.message.SOAPHeaderElement tempHeader = (org.apache.axis.message.SOAPHeaderElement) sm
  +						.getSOAPEnvelope().getHeadersByName(
  +								qname.getNamespaceURI(), qname.getLocalPart());
  +				tempHeader.setProcessed(true);
  +			}
  +
  +			/*
  +			 * After setting the new current message, probably modified because
  +			 * of decryption, we need to locate the security header. That is, we
  +			 * force Axis (with getSOAPEnvelope()) to parse the string, build
  +			 * the new header. Then we examine, look up the security header and
  +			 * set the header as processed.
  +			 * 
  +			 * Please note: find all header elements that contain the same actor
  +			 * that was given to processSecurityHeader(). Then check if there is
  +			 * a security header with this actor.
  +			 */
  +
  +			SOAPHeader sHeader = null;
  +			try {
  +				sHeader = sm.getSOAPEnvelope().getHeader();
  +			} catch (Exception ex) {
  +				throw new AxisFault(
  +						"WSDoAllReceiver: cannot get SOAP header after security processing",
  +						ex);
  +			}
  +
  +			Iterator headers = sHeader.examineHeaderElements(actor);
  +
  +			SOAPHeaderElement headerElement = null;
  +			while (headers.hasNext()) {
  +				SOAPHeaderElement hE = (SOAPHeaderElement) headers.next();
  +				if (hE.getLocalName().equals(WSConstants.WSSE_LN)
  +						&& hE.getNamespaceURI().equals(WSConstants.WSSE_NS)) {
  +					headerElement = hE;
  +					break;
  +				}
  +			}
  +			((org.apache.axis.message.SOAPHeaderElement) headerElement)
  +					.setProcessed(true);
  +
  +			/*
  +			 * Now we can check the certificate used to sign the message. In the
  +			 * following implementation the certificate is only trusted if
  +			 * either it itself or the certificate of the issuer is installed in
  +			 * the keystore.
  +			 * 
  +			 * Note: the method verifyTrust(X509Certificate) allows custom
  +			 * implementations with other validation algorithms for subclasses.
  +			 */
  +
  +			// Extract the signature action result from the action vector
  +			WSSecurityEngineResult actionResult = WSSecurityUtil
  +					.fetchActionResult(wsResult, WSConstants.SIGN);
  +
  +			if (actionResult != null) {
  +				X509Certificate returnCert = actionResult.getCertificate();
  +
  +				if (returnCert != null) {
  +					if (!verifyTrust(returnCert, reqData)) {
  +						throw new AxisFault(
  +								"WSDoAllReceiver: The certificate used for the signature is not trusted");
  +					}
  +				}
  +			}
  +
  +			/*
  +			 * Perform further checks on the timestamp that was transmitted in
  +			 * the header. In the following implementation the timestamp is
  +			 * valid if it was created after (now-ttl), where ttl is set on
  +			 * server side, not by the client.
  +			 * 
  +			 * Note: the method verifyTimestamp(Timestamp) allows custom
  +			 * implementations with other validation algorithms for subclasses.
  +			 */
  +
  +			// Extract the timestamp action result from the action vector
  +			actionResult = WSSecurityUtil.fetchActionResult(wsResult,
  +					WSConstants.TS);
  +
  +			if (actionResult != null) {
  +				Timestamp timestamp = actionResult.getTimestamp();
  +
  +				if (timestamp != null) {
  +					String ttl = null;
  +					if ((ttl = (String) getOption(WSHandlerConstants.TTL_TIMESTAMP)) == null) {
  +						ttl = (String) msgContext
  +								.getProperty(WSHandlerConstants.TTL_TIMESTAMP);
  +					}
  +					int ttl_i = 0;
  +					if (ttl != null) {
  +						try {
  +							ttl_i = Integer.parseInt(ttl);
  +						} catch (NumberFormatException e) {
  +							ttl_i = reqData.timeToLive;
  +						}
  +					}
  +					if (ttl_i <= 0) {
  +						ttl_i = reqData.timeToLive;
  +					}
  +
  +					if (!verifyTimestamp(timestamp, reqData.timeToLive)) {
  +						throw new AxisFault(
  +								"WSDoAllReceiver: The timestamp could not be validated");
  +					}
  +				}
  +			}
  +
  +			/*
  +			 * now check the security actions: do they match, in right order?
  +			 */
  +			int resultActions = wsResult.size();
  +			int size = actions.size();
  +			if (size != resultActions) {
  +				throw new AxisFault(
  +						"WSDoAllReceiver: security processing failed (actions number mismatch)");
  +			}
  +			for (int i = 0; i < size; i++) {
  +				if (((Integer) actions.get(i)).intValue() != ((WSSecurityEngineResult) wsResult
  +						.get(i)).getAction()) {
  +					throw new AxisFault(
  +							"WSDoAllReceiver: security processing failed (actions mismatch)");
  +				}
  +			}
  +
  +			/*
  +			 * All ok up to this point. Now construct and setup the security
  +			 * result structure. The service may fetch this and check it.
  +			 */
  +			Vector results = null;
  +			if ((results = (Vector) msgContext
  +					.getProperty(WSHandlerConstants.RECV_RESULTS)) == null) {
  +				results = new Vector();
  +				msgContext
  +						.setProperty(WSHandlerConstants.RECV_RESULTS, results);
  +			}
  +			WSHandlerResult rResult = new WSHandlerResult(actor, wsResult);
  +			results.add(0, rResult);
  +			if (doDebug) {
  +				log.debug("WSDoAllReceiver: exit invoke()");
  +			}
  +		} finally {
  +			reqData.clear();
  +			reqData = null;
  +		}
   
  -        /*
  -          * now check the security actions: do they match, in right order?
  -          */
  -        int resultActions = wsResult.size();
  -        int size = actions.size();
  -        if (size != resultActions) {
  -            throw new AxisFault("WSDoAllReceiver: security processing failed (actions number mismatch)");
  -        }
  -        for (int i = 0; i < size; i++) {
  -            if (((Integer) actions.get(i)).intValue()
  -                    != ((WSSecurityEngineResult) wsResult.get(i)).getAction()) {
  -                throw new AxisFault("WSDoAllReceiver: security processing failed (actions mismatch)");
  -            }
  -        }
  -        
  -        /*
  -         * All ok up to this point. Now construct and setup the
  -         * security result structure. The service may fetch this
  -         * and check it.
  -         */
  -        Vector results = null;
  -        if ((results = (Vector) msgContext.getProperty(WSHandlerConstants.RECV_RESULTS))
  -                == null) {
  -            results = new Vector();
  -            msgContext.setProperty(WSHandlerConstants.RECV_RESULTS, results);
  -        }
  -        WSHandlerResult rResult =
  -                new WSHandlerResult(actor,
  -                        wsResult);
  -        results.add(0, rResult);
  -        if (doDebug) {
  -            log.debug("WSDoAllReceiver: exit invoke()");
  -        }
  -    }
  +	}
   
       /**
  -     * Hook to allow subclasses to load their Signature Crypto however they see fit.
  -     */
  -    protected Crypto loadSignatureCrypto(final MessageContext msgContext) throws AxisFault {
  +	 * Hook to allow subclasses to load their Signature Crypto however they see
  +	 * fit.
  +	 */
  +    protected Crypto loadSignatureCrypto(RequestData reqData) throws AxisFault {
           Crypto crypto = null;
  -        if ((sigPropFile = (String) getOption(WSHandlerConstants.SIG_PROP_FILE))
  +        if ((reqData.sigPropFile = (String) getOption(WSHandlerConstants.SIG_PROP_FILE))
                   == null) {
  -            sigPropFile =
  -                    (String) msgContext.getProperty(WSHandlerConstants.SIG_PROP_FILE);
  +        	reqData.sigPropFile =
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.SIG_PROP_FILE);
           }
  -        if (sigPropFile != null) {
  -            if ((crypto = (Crypto) cryptos.get(sigPropFile)) == null) {
  -                crypto = CryptoFactory.getInstance(sigPropFile);
  -                cryptos.put(sigPropFile, crypto);
  +        if (reqData.sigPropFile != null) {
  +            if ((crypto = (Crypto) cryptos.get(reqData.sigPropFile)) == null) {
  +                crypto = CryptoFactory.getInstance(reqData.sigPropFile);
  +                cryptos.put(reqData.sigPropFile, crypto);
               }
           } else {
               throw new AxisFault("WSDoAllReceiver: Signature: no crypto property file");
  @@ -363,28 +399,29 @@
       }
   
       /**
  -     * Hook to allow subclasses to load their Decryption Crypto however they see fit.
  -     */
  -    protected Crypto loadDecryptionCrypto(final MessageContext msgContext) throws AxisFault {
  +	 * Hook to allow subclasses to load their Decryption Crypto however they see
  +	 * fit.
  +	 */
  +    protected Crypto loadDecryptionCrypto(RequestData reqData) throws AxisFault {
           Crypto crypto = null;
  -        if ((decPropFile = (String) getOption(WSHandlerConstants.DEC_PROP_FILE))
  +        if ((reqData.decPropFile = (String) getOption(WSHandlerConstants.DEC_PROP_FILE))
                   == null) {
  -            decPropFile =
  -                    (String) msgContext.getProperty(WSHandlerConstants.DEC_PROP_FILE);
  +        	reqData.decPropFile =
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.DEC_PROP_FILE);
           }
  -        if (decPropFile != null) {
  -            if ((crypto = (Crypto) cryptos.get(decPropFile)) == null) {
  -                crypto = CryptoFactory.getInstance(decPropFile);
  -                cryptos.put(decPropFile, crypto);
  +        if (reqData.decPropFile != null) {
  +            if ((crypto = (Crypto) cryptos.get(reqData.decPropFile)) == null) {
  +                crypto = CryptoFactory.getInstance(reqData.decPropFile);
  +                cryptos.put(reqData.decPropFile, crypto);
               }
  -        } else if ((crypto = sigCrypto) == null) {
  +        } else if ((crypto = reqData.sigCrypto) == null) {
               throw new AxisFault("WSDoAllReceiver: Encryption: no crypto property file");
           }
           return crypto;
       }
   
  -    private void decodeSignatureParameter(final MessageContext msgContext) throws AxisFault {
  -        sigCrypto = loadSignatureCrypto(msgContext);
  +    private void decodeSignatureParameter(RequestData reqData) throws AxisFault {
  +    	reqData.sigCrypto = loadSignatureCrypto(reqData);
           /* There are currently no other signature parameters that need to be handled 
            * here, but we call the load crypto hook rather than just changing the visibility
            * of this method to maintain parity with WSDoAllSender.
  @@ -396,8 +433,8 @@
        * take over signatur crypto instance.
        */ 
   
  -    private void decodeDecryptionParameter(final MessageContext msgContext) throws AxisFault {
  -        decCrypto = loadDecryptionCrypto(msgContext);
  +    private void decodeDecryptionParameter(RequestData reqData) throws AxisFault {
  +    	reqData.decCrypto = loadDecryptionCrypto(reqData);
           /* There are currently no other decryption parameters that need to be handled 
            * here, but we call the load crypto hook rather than just changing the visibility
            * of this method to maintain parity with WSDoAllSender.
  @@ -408,14 +445,14 @@
        * Get the password callback class and get an instance
        * <p/>
        */
  -    private CallbackHandler getPasswordCB(final MessageContext msgContext) throws AxisFault {
  +    private CallbackHandler getPasswordCB(RequestData reqData) throws AxisFault {
   
           String callback = null;
           CallbackHandler cbHandler = null;
           if ((callback = (String) getOption(WSHandlerConstants.PW_CALLBACK_CLASS))
                   == null) {
               callback =
  -                    (String) msgContext.getProperty(WSHandlerConstants.PW_CALLBACK_CLASS);
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.PW_CALLBACK_CLASS);
           }
           if (callback != null) {
               Class cbClass = null;
  @@ -435,7 +472,7 @@
               }
           } else {
               cbHandler =
  -                    (CallbackHandler) msgContext.getProperty(WSHandlerConstants.PW_CALLBACK_REF);
  +                    (CallbackHandler) reqData.msgContext.getProperty(WSHandlerConstants.PW_CALLBACK_REF);
               if (cbHandler == null) {
                   throw new AxisFault("WSDoAllReceiver: no reference in callback property");
               }
  @@ -457,7 +494,7 @@
        * @return true if the certificate is trusted, false if not (AxisFault is thrown for exceptions during CertPathValidation)
        * @throws AxisFault
        */
  -    private boolean verifyTrust(X509Certificate cert) throws AxisFault {
  +    private boolean verifyTrust(X509Certificate cert, RequestData reqData) throws AxisFault {
   
           // If no certificate was transmitted, do not trust the signature 
           if (cert == null) {
  @@ -482,7 +519,7 @@
       
           // Search the keystore for the alias of the transmitted certificate
           try {
  -            alias = sigCrypto.getAliasForX509Cert(issuerString, issuerSerial);
  +            alias = reqData.sigCrypto.getAliasForX509Cert(issuerString, issuerSerial);
           } catch (WSSecurityException ex) {
               throw new AxisFault("WSDoAllReceiver: Could not get alias for certificate with " + subjectString, ex);
           }
  @@ -490,7 +527,7 @@
           if (alias != null) {
               // Retrieve the certificate for the alias from the keystore 
               try {
  -                certs = sigCrypto.getCertificates(alias);
  +                certs = reqData.sigCrypto.getCertificates(alias);
               } catch (WSSecurityException ex) {
                   throw new AxisFault("WSDoAllReceiver: Could not get certificates for alias " + alias, ex);
               }
  @@ -514,7 +551,7 @@
   
           // Search the keystore for the alias of the transmitted certificates issuer
           try {
  -            aliases = sigCrypto.getAliasesForDN(issuerString);
  +            aliases = reqData.sigCrypto.getAliasesForDN(issuerString);
           } catch (WSSecurityException ex) {
               throw new AxisFault("WSDoAllReceiver: Could not get alias for certificate with " + issuerString, ex);
           }
  @@ -539,7 +576,7 @@
   
               // Retrieve the certificate(s) for the alias from the keystore 
               try {
  -                certs = sigCrypto.getCertificates(alias);
  +                certs = reqData.sigCrypto.getCertificates(alias);
               } catch (WSSecurityException ex) {
                   throw new AxisFault("WSDoAllReceiver: Could not get certificates for alias " + alias, ex);
               }
  @@ -591,7 +628,7 @@
   
               // Use the validation method from the crypto to check whether the subjects certificate was really signed by the issuer stated in the certificate 
               try {
  -                if (sigCrypto.validateCertPath(certs)) {
  +                if (reqData.sigCrypto.validateCertPath(certs)) {
                       if (doDebug) {
                           log.debug("WSDoAllReceiver: Certificate path has been verified for certificate with subject " + subjectString);
                       }
  
  
  
  1.26      +404 -372  ws-fx/wss4j/src/org/apache/ws/axis/security/WSDoAllSender.java
  
  Index: WSDoAllSender.java
  ===================================================================
  RCS file: /home/cvs/ws-fx/wss4j/src/org/apache/ws/axis/security/WSDoAllSender.java,v
  retrieving revision 1.25
  retrieving revision 1.26
  diff -u -r1.25 -r1.26
  --- WSDoAllSender.java	25 Oct 2004 18:35:46 -0000	1.25
  +++ WSDoAllSender.java	1 Nov 2004 10:43:09 -0000	1.26
  @@ -36,8 +36,6 @@
   import org.apache.ws.security.WSSecurityException;
   import org.apache.ws.security.handler.WSHandlerResult;
   import org.apache.ws.security.handler.WSHandlerConstants;
  -import org.apache.ws.security.saml.SAMLIssuer;
  -import org.apache.ws.security.saml.SAMLIssuerFactory;
   import org.apache.ws.security.components.crypto.Crypto;
   import org.apache.ws.security.components.crypto.CryptoFactory;
   import org.apache.ws.security.message.WSAddTimestamp;
  @@ -54,7 +52,6 @@
   
   import javax.security.auth.callback.Callback;
   import javax.security.auth.callback.CallbackHandler;
  -import javax.xml.rpc.JAXRPCException;
   
   import java.io.ByteArrayOutputStream;
   import java.io.UnsupportedEncodingException;
  @@ -69,60 +66,78 @@
   
       static Log log = LogFactory.getLog(WSDoAllSender.class.getName());
   
  -    static final WSSecurityEngine secEngine = new WSSecurityEngine();
  +    static final WSSecurityEngine secEngine = WSSecurityEngine.getInstance();
   
  -    private boolean doDebug = true;
  +    private static boolean doDebug = true;
   
       private static Hashtable cryptos = new Hashtable(5);
  +    
  +    /**
  +     * This nested private class hold per request data.
  +     * 
  +     * @author wdi
  +     */
  +    private class RequestData {
  +        MessageContext msgContext = null;
   
  -    private MessageContext msgContext = null;
  -
  -    private boolean noSerialization = false;
  +        boolean noSerialization = false;
   
  -    private SOAPConstants soapConstants = null;
  +        SOAPConstants soapConstants = null;
   
  -    String actor = null;
  +        String actor = null;
   
  -    String username = null;
  +        String username = null;
   
  -    String pwType = null;
  +        String pwType = null;
   
  -    String[] utElements = null;
  +        String[] utElements = null;
   
  -    Crypto sigCrypto = null;
  +        Crypto sigCrypto = null;
   
  -    int sigKeyId = 0;
  +        int sigKeyId = 0;
   
  -    String sigAlgorithm = null;
  +        String sigAlgorithm = null;
   
  -    Vector signatureParts = new Vector();
  +        Vector signatureParts = new Vector();
   
  -    Crypto encCrypto = null;
  +        Crypto encCrypto = null;
   
  -    int encKeyId = 0;
  +        int encKeyId = 0;
   
  -    String encSymmAlgo = null;
  +        String encSymmAlgo = null;
   
  -    String encKeyTransport = null;
  +        String encKeyTransport = null;
   
  -    String encUser = null;
  +        String encUser = null;
   
  -    Vector encryptParts = new Vector();
  +        Vector encryptParts = new Vector();
   
  -    X509Certificate encCert = null;
  +        X509Certificate encCert = null;
   
  -    protected int timeToLive = 300; // Timestamp: time in seconds the receiver accepts between creation and reception
  +        int timeToLive = 300; 	// Timestamp: time in seconds between creation 
  +								// and expiery
  +        void clear() {
  +        	soapConstants = null;
  +        	actor = username = pwType = sigAlgorithm = encSymmAlgo = encKeyTransport = encUser = null;
  +        	sigCrypto = encCrypto = null;
  +        	signatureParts.clear();
  +        	encryptParts.clear();
  +        	signatureParts = encryptParts = null;
  +        	encCert = null;
  +        	utElements = null;
  +        }
  +    }
   
       /**
        * Initialize data fields from previous use in case of cached object. Axis
  -     * may cache the handler object, thus weneed to intiailize (reset) some
  +     * may cache the handler object, thus we need to intiailize (reset) some
        * data fields. In particular remove old elements from the vectors. The
        * other fields are initialized implictly using the lookup of the WSDD
        * parameter (getOption()) and properties.
        */
  -    private void initialize() {
  -        signatureParts.removeAllElements();
  -        encryptParts.removeAllElements();
  +    private RequestData initialize() {
  +    	RequestData reqData = new RequestData();
  +    	return reqData;
       }
   
       /**
  @@ -133,339 +148,355 @@
        */
       public void invoke(MessageContext mc) throws AxisFault {
   
  -        doDebug = log.isDebugEnabled();
  -        if (doDebug) {
  -            log.debug("WSDoAllSender: enter invoke() with msg type: "
  -                    + mc.getCurrentMessage().getMessageType());
  -        }
  -
  -        initialize();
  -
  -        noSerialization = false;
  -        msgContext = mc;
  -        /*
  -         * Get the action first.
  -         */
  -        Vector actions = new Vector();
  -        String action = null;
  -        if ((action = (String) getOption(WSHandlerConstants.ACTION)) == null) {
  -            action = (String) msgContext.getProperty(WSHandlerConstants.ACTION);
  -        }
  -        if (action == null) {
  -            throw new AxisFault("WSDoAllReceiver: No action defined");
  -        }
  -        int doAction = AxisUtil.decodeAction(action, actions);
  -        if (doAction == WSConstants.NO_SECURITY) {
  -            return;
  -        }
  -
  -        boolean mu = decodeMustUnderstand();
  -
  -        if ((actor = (String) getOption(WSHandlerConstants.ACTOR)) == null) {
  -            actor = (String) msgContext.getProperty(WSHandlerConstants.ACTOR);
  -        }
  -        /*
  -         * For every action we need a username, so get this now. The username
  -         * defined in the deployment descriptor takes precedence.
  -         */
  -        username = (String) getOption(WSHandlerConstants.USER);
  -        if (username == null || username.equals("")) {
  -            username = msgContext.getUsername();
  -            msgContext.setUsername(null);
  -        }
  -        /*
  -         * Now we perform some set-up for UsernameToken and Signature
  -         * functions. No need to do it for encryption only. Check if username
  -         * is available and then get a passowrd.
  -         */
  -        if ((doAction & (WSConstants.SIGN | WSConstants.UT | WSConstants.UT_SIGN)) != 0) {
  -            /*
  -             * We need a username - if none throw an AxisFault. For encryption
  -             * there is a specific parameter to get a username.
  -             */
  -            if (username == null || username.equals("")) {
  -                throw new AxisFault("WSDoAllSender: Empty username for specified action");
  -            }
  -        }
  -        if (doDebug) {
  -            log.debug("Action: " + doAction);
  -            log.debug("Actor: " + actor + ", mu: " + mu);
  -        }
  -        /*
  -         * Now get the SOAP part from the request message and convert it into a
  -         * Document.
  -         * 
  -         * This forces Axis to serialize the SOAP request into FORM_STRING.
  -         * This string is converted into a document.
  -         * 
  -         * During the FORM_STRING serialization Axis performs multi-ref of
  -         * complex data types (if requested), generates and inserts references
  -         * for attachements and so on. The resulting Document MUST be the
  -         * complete and final SOAP request as Axis would send it over the wire.
  -         * Therefore this must shall be the last (or only) handler in a chain.
  -         * 
  -         * Now we can perform our security operations on this request.
  -         */
  -        Document doc = null;
  -        Message message = msgContext.getCurrentMessage();
  -
  -        /*
  -         * If the message context property conatins a document then this is a
  -         * chained handler.
  -         */
  -        SOAPPart sPart = (org.apache.axis.SOAPPart) message.getSOAPPart();
  -        if ((doc =
  -                (Document) msgContext.getProperty(WSHandlerConstants.SND_SECURITY))
  -                == null) {
  -            try {
  -                doc =
  -                        ((org.apache.axis.message.SOAPEnvelope) sPart
  -                        .getEnvelope())
  -                        .getAsDocument();
  -            } catch (Exception e) {
  -                throw new AxisFault("WSDoAllSender: cannot get SOAP envlope from message" + e);
  -            }
  -        }
  -        soapConstants =
  -                WSSecurityUtil.getSOAPConstants(doc.getDocumentElement());
  -        /*
  -         * Here we have action, username, password, and actor, mustUnderstand.
  -         * Now get the action specific parameters.
  -         */
  -        if ((doAction & WSConstants.UT) == WSConstants.UT) {
  -            decodeUTParameter();
  -        }
  -        /*
  -         * Here we have action, username, password, and actor, mustUnderstand.
  -         * Now get the action specific parameters.
  -         */
  -        if ((doAction & WSConstants.UT_SIGN) == WSConstants.UT_SIGN) {
  -            decodeUTParameter();
  -            decodeSignatureParameter();
  -       }
  -        /*
  -         * Get and check the Signature specific parameters first because they
  -         * may be used for encryption too.
  -         */
  -        if ((doAction & WSConstants.SIGN) == WSConstants.SIGN) {
  -            sigCrypto = loadSignatureCrypto();
  -            decodeSignatureParameter();
  -        }
  -        /*
  -         * If we need to handle signed SAML token then we need may of the
  -         * Signature parameters. The handle procedure loads the signature 
  -         * crypto file on demand, thus don't do it here.
  -         */
  -        if ((doAction & WSConstants.ST_SIGNED) == WSConstants.ST_SIGNED) {
  -            decodeSignatureParameter();
  -        }
  -        /*
  -         * Set and check the encryption specific parameters, if necessary take
  -         * over signature parameters username and crypto instance.
  -         */
  -        if ((doAction & WSConstants.ENCR) == WSConstants.ENCR) {
  -            encCrypto = loadEncryptionCrypto();
  -            decodeEncryptionParameter();
  -        }
  -        /*
  -         * Here we have all necessary information to perform the requested
  -         * action(s).
  -         */
  -        for (int i = 0; i < actions.size(); i++) {
  +		doDebug = log.isDebugEnabled();
  +		if (doDebug) {
  +			log.debug("WSDoAllSender: enter invoke() with msg type: "
  +					+ mc.getCurrentMessage().getMessageType());
  +		}
   
  -            int actionToDo = ((Integer) actions.get(i)).intValue();
  -            if (doDebug) {
  -                log.debug("Performing Action: " + actionToDo);
  -            }
  -
  -            String password = null;
  -            switch (actionToDo) {
  -                case WSConstants.UT:
  -                    performUTAction(actionToDo, mu, doc);
  -                    break;
  -
  -                case WSConstants.ENCR:
  -                    performENCRAction(mu, actionToDo, doc);
  -                    break;
  -
  -                case WSConstants.SIGN:
  -                    performSIGNAction(actionToDo, mu, doc);
  -                    break;
  -
  -                case WSConstants.ST_SIGNED:
  -                    performST_SIGNAction(actionToDo, mu, doc);
  -                    break;
  -
  -                case WSConstants.ST_UNSIGNED:
  -                    performSTAction(actionToDo, mu, doc);
  -                    break;
  -
  -                case WSConstants.TS:
  -                    performTSAction(actionToDo, mu, doc);
  -                    break;
  -
  -    			case WSConstants.UT_SIGN:
  -    				performUT_SIGNAction(actionToDo, mu, doc);
  -    				break;
  -
  -    			case WSConstants.NO_SERIALIZE:
  -                    noSerialization = true;
  -                    break;
  -            }
  -        }
  +		RequestData reqData = initialize();
   
  -        /*
  -         * If required convert the resulting document into a message first. The
  -         * outputDOM() method performs the necessary c14n call. After that we
  -         * extract it as a string for further processing.
  -         * 
  -         * Set the resulting byte array as the new SOAP message.
  -         * 
  -         * If noSerialization is false, this handler shall be the last (or
  -         * only) one in a handler chain. If noSerialization is true, just set
  -         * the processed Document in the transfer property. The next Axis WSS4J
  -         * handler takes it and performs additional security processing steps.
  -         *  
  -         */
  -        if (noSerialization) {
  -            msgContext.setProperty(WSHandlerConstants.SND_SECURITY, doc);
  -        } else {
  -            ByteArrayOutputStream os = new ByteArrayOutputStream();
  -            XMLUtils.outputDOM(doc, os, true);
  -            sPart.setCurrentMessage(os.toByteArray(), SOAPPart.FORM_BYTES);
  -            if (doDebug) {
  -                String osStr = null;
  -                try {
  -                    osStr = os.toString("UTF-8");
  -                } catch (UnsupportedEncodingException e) {
  -                    osStr = os.toString();
  -                }
  -                log.debug("Send request:");
  -                log.debug(osStr);
  -            }
  -            msgContext.setProperty(WSHandlerConstants.SND_SECURITY, null);
  -        }
  -        if (doDebug) {
  -            log.debug("WSDoAllSender: exit invoke()");
  -        }
  -    }
  +		reqData.noSerialization = false;
  +		reqData.msgContext = mc;
  +		/*
  +		 * The overall try, just to have a finally at the end to perform some
  +		 * housekeeping.
  +		 */
  +		try {
  +			/*
  +			 * Get the action first.
  +			 */
  +			Vector actions = new Vector();
  +			String action = null;
  +			if ((action = (String) getOption(WSHandlerConstants.ACTION)) == null) {
  +				action = (String) reqData.msgContext
  +						.getProperty(WSHandlerConstants.ACTION);
  +			}
  +			if (action == null) {
  +				throw new AxisFault("WSDoAllReceiver: No action defined");
  +			}
  +			int doAction = AxisUtil.decodeAction(action, actions);
  +			if (doAction == WSConstants.NO_SECURITY) {
  +				return;
  +			}
  +
  +			boolean mu = decodeMustUnderstand(reqData);
  +
  +			if ((reqData.actor = (String) getOption(WSHandlerConstants.ACTOR)) == null) {
  +				reqData.actor = (String) reqData.msgContext
  +						.getProperty(WSHandlerConstants.ACTOR);
  +			}
  +			/*
  +			 * For every action we need a username, so get this now. The
  +			 * username defined in the deployment descriptor takes precedence.
  +			 */
  +			reqData.username = (String) getOption(WSHandlerConstants.USER);
  +			if (reqData.username == null || reqData.username.equals("")) {
  +				reqData.username = reqData.msgContext.getUsername();
  +				reqData.msgContext.setUsername(null);
  +			}
  +			/*
  +			 * Now we perform some set-up for UsernameToken and Signature
  +			 * functions. No need to do it for encryption only. Check if
  +			 * username is available and then get a passowrd.
  +			 */
  +			if ((doAction & (WSConstants.SIGN | WSConstants.UT | WSConstants.UT_SIGN)) != 0) {
  +				/*
  +				 * We need a username - if none throw an AxisFault. For
  +				 * encryption there is a specific parameter to get a username.
  +				 */
  +				if (reqData.username == null || reqData.username.equals("")) {
  +					throw new AxisFault(
  +							"WSDoAllSender: Empty username for specified action");
  +				}
  +			}
  +			if (doDebug) {
  +				log.debug("Action: " + doAction);
  +				log.debug("Actor: " + reqData.actor + ", mu: " + mu);
  +			}
  +			/*
  +			 * Now get the SOAP part from the request message and convert it
  +			 * into a Document.
  +			 * 
  +			 * This forces Axis to serialize the SOAP request into FORM_STRING.
  +			 * This string is converted into a document.
  +			 * 
  +			 * During the FORM_STRING serialization Axis performs multi-ref of
  +			 * complex data types (if requested), generates and inserts
  +			 * references for attachements and so on. The resulting Document
  +			 * MUST be the complete and final SOAP request as Axis would send it
  +			 * over the wire. Therefore this must shall be the last (or only)
  +			 * handler in a chain.
  +			 * 
  +			 * Now we can perform our security operations on this request.
  +			 */
  +			Document doc = null;
  +			Message message = reqData.msgContext.getCurrentMessage();
  +
  +			/*
  +			 * If the message context property conatins a document then this is
  +			 * a chained handler.
  +			 */
  +			SOAPPart sPart = (org.apache.axis.SOAPPart) message.getSOAPPart();
  +			if ((doc = (Document) reqData.msgContext
  +					.getProperty(WSHandlerConstants.SND_SECURITY)) == null) {
  +				try {
  +					doc = ((org.apache.axis.message.SOAPEnvelope) sPart
  +							.getEnvelope()).getAsDocument();
  +				} catch (Exception e) {
  +					throw new AxisFault(
  +							"WSDoAllSender: cannot get SOAP envlope from message"
  +									+ e);
  +				}
  +			}
  +			reqData.soapConstants = WSSecurityUtil.getSOAPConstants(doc
  +					.getDocumentElement());
  +			/*
  +			 * Here we have action, username, password, and actor,
  +			 * mustUnderstand. Now get the action specific parameters.
  +			 */
  +			if ((doAction & WSConstants.UT) == WSConstants.UT) {
  +				decodeUTParameter(reqData);
  +			}
  +			/*
  +			 * Here we have action, username, password, and actor,
  +			 * mustUnderstand. Now get the action specific parameters.
  +			 */
  +			if ((doAction & WSConstants.UT_SIGN) == WSConstants.UT_SIGN) {
  +				decodeUTParameter(reqData);
  +				decodeSignatureParameter(reqData);
  +			}
  +			/*
  +			 * Get and check the Signature specific parameters first because
  +			 * they may be used for encryption too.
  +			 */
  +			if ((doAction & WSConstants.SIGN) == WSConstants.SIGN) {
  +				reqData.sigCrypto = loadSignatureCrypto(reqData);
  +				decodeSignatureParameter(reqData);
  +			}
  +			/*
  +			 * If we need to handle signed SAML token then we need may of the
  +			 * Signature parameters. The handle procedure loads the signature
  +			 * crypto file on demand, thus don't do it here.
  +			 */
  +			if ((doAction & WSConstants.ST_SIGNED) == WSConstants.ST_SIGNED) {
  +				decodeSignatureParameter(reqData);
  +			}
  +			/*
  +			 * Set and check the encryption specific parameters, if necessary
  +			 * take over signature parameters username and crypto instance.
  +			 */
  +			if ((doAction & WSConstants.ENCR) == WSConstants.ENCR) {
  +				reqData.encCrypto = loadEncryptionCrypto(reqData);
  +				decodeEncryptionParameter(reqData);
  +			}
  +			/*
  +			 * Here we have all necessary information to perform the requested
  +			 * action(s).
  +			 */
  +			for (int i = 0; i < actions.size(); i++) {
  +
  +				int actionToDo = ((Integer) actions.get(i)).intValue();
  +				if (doDebug) {
  +					log.debug("Performing Action: " + actionToDo);
  +				}
  +
  +				String password = null;
  +				switch (actionToDo) {
  +				case WSConstants.UT:
  +					performUTAction(actionToDo, mu, doc, reqData);
  +					break;
  +
  +				case WSConstants.ENCR:
  +					performENCRAction(mu, actionToDo, doc, reqData);
  +					break;
  +
  +				case WSConstants.SIGN:
  +					performSIGNAction(actionToDo, mu, doc, reqData);
  +					break;
  +
  +				case WSConstants.ST_SIGNED:
  +					performST_SIGNAction(actionToDo, mu, doc, reqData);
  +					break;
  +
  +				case WSConstants.ST_UNSIGNED:
  +					performSTAction(actionToDo, mu, doc, reqData);
  +					break;
  +
  +				case WSConstants.TS:
  +					performTSAction(actionToDo, mu, doc, reqData);
  +					break;
  +
  +				case WSConstants.UT_SIGN:
  +					performUT_SIGNAction(actionToDo, mu, doc, reqData);
  +					break;
  +
  +				case WSConstants.NO_SERIALIZE:
  +					reqData.noSerialization = true;
  +					break;
  +				}
  +			}
  +
  +			/*
  +			 * If required convert the resulting document into a message first.
  +			 * The outputDOM() method performs the necessary c14n call. After
  +			 * that we extract it as a string for further processing.
  +			 * 
  +			 * Set the resulting byte array as the new SOAP message.
  +			 * 
  +			 * If noSerialization is false, this handler shall be the last (or
  +			 * only) one in a handler chain. If noSerialization is true, just
  +			 * set the processed Document in the transfer property. The next
  +			 * Axis WSS4J handler takes it and performs additional security
  +			 * processing steps.
  +			 *  
  +			 */
  +			if (reqData.noSerialization) {
  +				reqData.msgContext.setProperty(WSHandlerConstants.SND_SECURITY,
  +						doc);
  +			} else {
  +				ByteArrayOutputStream os = new ByteArrayOutputStream();
  +				XMLUtils.outputDOM(doc, os, true);
  +				sPart.setCurrentMessage(os.toByteArray(), SOAPPart.FORM_BYTES);
  +				if (doDebug) {
  +					String osStr = null;
  +					try {
  +						osStr = os.toString("UTF-8");
  +					} catch (UnsupportedEncodingException e) {
  +						osStr = os.toString();
  +					}
  +					log.debug("Send request:");
  +					log.debug(osStr);
  +				}
  +				reqData.msgContext.setProperty(WSHandlerConstants.SND_SECURITY,
  +						null);
  +			}
  +			if (doDebug) {
  +				log.debug("WSDoAllSender: exit invoke()");
  +			}
  +		} finally {
  +			reqData.clear();
  +			reqData = null;
  +		}
  +	}
   
  -    private void performSIGNAction(int actionToDo, boolean mu, Document doc)
  +    private void performSIGNAction(int actionToDo, boolean mu, Document doc, RequestData reqData)
               throws AxisFault {
           String password;
           password =
  -                getPassword(username,
  +                getPassword(reqData.username,
                           actionToDo,
                           WSHandlerConstants.PW_CALLBACK_CLASS,
  -                        WSHandlerConstants.PW_CALLBACK_REF)
  +                        WSHandlerConstants.PW_CALLBACK_REF, reqData)
                   .getPassword();
   
  -        WSSignEnvelope wsSign = new WSSignEnvelope(actor, mu);
  -        if (sigKeyId != 0) {
  -            wsSign.setKeyIdentifierType(sigKeyId);
  +        WSSignEnvelope wsSign = new WSSignEnvelope(reqData.actor, mu);
  +        if (reqData.sigKeyId != 0) {
  +            wsSign.setKeyIdentifierType(reqData.sigKeyId);
           }
  -        if (sigAlgorithm != null) {
  -            wsSign.setSignatureAlgorithm(sigAlgorithm);
  +        if (reqData.sigAlgorithm != null) {
  +            wsSign.setSignatureAlgorithm(reqData.sigAlgorithm);
           }
   
  -        wsSign.setUserInfo(username, password);
  -        if (signatureParts.size() > 0) {
  -            wsSign.setParts(signatureParts);
  +        wsSign.setUserInfo(reqData.username, password);
  +        if (reqData.signatureParts.size() > 0) {
  +            wsSign.setParts(reqData.signatureParts);
           }
   
           try {
  -            wsSign.build(doc, sigCrypto);
  +            wsSign.build(doc, reqData.sigCrypto);
           } catch (WSSecurityException e) {
               throw new AxisFault("WSDoAllSender: Signature: error during message procesing" + e);
           }
       }
   
  -    private void performENCRAction(boolean mu, int actionToDo, Document doc)
  +    private void performENCRAction(boolean mu, int actionToDo, Document doc, RequestData reqData)
               throws AxisFault {
  -        WSEncryptBody wsEncrypt = new WSEncryptBody(actor, mu);
  -        if (encKeyId != 0) {
  -            wsEncrypt.setKeyIdentifierType(encKeyId);
  +        WSEncryptBody wsEncrypt = new WSEncryptBody(reqData.actor, mu);
  +        if (reqData.encKeyId != 0) {
  +            wsEncrypt.setKeyIdentifierType(reqData.encKeyId);
           }
  -        if (encKeyId == WSConstants.EMBEDDED_KEYNAME) {
  +        if (reqData.encKeyId == WSConstants.EMBEDDED_KEYNAME) {
               String encKeyName = null;
               if ((encKeyName =
                       (String) getOption(WSHandlerConstants.ENC_KEY_NAME))
                       == null) {
                   encKeyName =
  -                        (String) msgContext.getProperty(WSHandlerConstants.ENC_KEY_NAME);
  +                        (String) reqData.msgContext.getProperty(WSHandlerConstants.ENC_KEY_NAME);
               }
               wsEncrypt.setEmbeddedKeyName(encKeyName);
               byte[] embeddedKey =
  -                    getPassword(encUser,
  +                    getPassword(reqData.encUser,
                               actionToDo,
                               WSHandlerConstants.ENC_CALLBACK_CLASS,
  -                            WSHandlerConstants.ENC_CALLBACK_REF)
  +                            WSHandlerConstants.ENC_CALLBACK_REF, reqData)
                       .getKey();
               wsEncrypt.setKey(embeddedKey);
           }
  -        if (encSymmAlgo != null) {
  -            wsEncrypt.setSymmetricEncAlgorithm(encSymmAlgo);
  +        if (reqData.encSymmAlgo != null) {
  +            wsEncrypt.setSymmetricEncAlgorithm(reqData.encSymmAlgo);
           }
  -        if (encKeyTransport != null) {
  -            wsEncrypt.setKeyEnc(encKeyTransport);
  +        if (reqData.encKeyTransport != null) {
  +            wsEncrypt.setKeyEnc(reqData.encKeyTransport);
           }
  -        wsEncrypt.setUserInfo(encUser);
  -        wsEncrypt.setUseThisCert(encCert);
  -        if (encryptParts.size() > 0) {
  -            wsEncrypt.setParts(encryptParts);
  +        wsEncrypt.setUserInfo(reqData.encUser);
  +        wsEncrypt.setUseThisCert(reqData.encCert);
  +        if (reqData.encryptParts.size() > 0) {
  +            wsEncrypt.setParts(reqData.encryptParts);
           }
           try {
  -            wsEncrypt.build(doc, encCrypto);
  +            wsEncrypt.build(doc, reqData.encCrypto);
           } catch (WSSecurityException e) {
               throw new AxisFault("WSDoAllSender: Encryption: error during message processing"
                       + e);
           }
       }
   
  -    private void performUTAction(int actionToDo, boolean mu, Document doc)
  +    private void performUTAction(int actionToDo, boolean mu, Document doc, RequestData reqData)
               throws AxisFault {
           String password;
           password =
  -                getPassword(username,
  +                getPassword(reqData.username,
                           actionToDo,
                           WSHandlerConstants.PW_CALLBACK_CLASS,
  -                        WSHandlerConstants.PW_CALLBACK_REF)
  +                        WSHandlerConstants.PW_CALLBACK_REF, reqData)
                   .getPassword();
   
  -        WSSAddUsernameToken builder = new WSSAddUsernameToken(actor, mu);
  -        builder.setPasswordType(pwType);
  +        WSSAddUsernameToken builder = new WSSAddUsernameToken(reqData.actor, mu);
  +        builder.setPasswordType(reqData.pwType);
           // add the UsernameToken to the SOAP Enevelope
  -        builder.build(doc, username, password);
  +        builder.build(doc, reqData.username, password);
   
  -        if (utElements != null && utElements.length > 0) {
  -            for (int j = 0; j < utElements.length; j++) {
  -                utElements[j].trim();
  -                if (utElements[j].equals("Nonce")) {
  +        if (reqData.utElements != null && reqData.utElements.length > 0) {
  +            for (int j = 0; j < reqData.utElements.length; j++) {
  +            	reqData.utElements[j].trim();
  +                if (reqData.utElements[j].equals("Nonce")) {
                       builder.addNonce(doc);
                   }
  -                if (utElements[j].equals("Created")) {
  +                if (reqData.utElements[j].equals("Created")) {
                       builder.addCreated(doc);
                   }
  +                reqData.utElements[j] = null;
               }
           }
       }
   
  -    private void performUT_SIGNAction(int actionToDo, boolean mu, Document doc)
  +    private void performUT_SIGNAction(int actionToDo, boolean mu, Document doc, RequestData reqData)
   			throws AxisFault {
   		String password;
  -		password = getPassword(username, actionToDo,
  +		password = getPassword(reqData.username, actionToDo,
   				WSHandlerConstants.PW_CALLBACK_CLASS,
  -				WSHandlerConstants.PW_CALLBACK_REF).getPassword();
  +				WSHandlerConstants.PW_CALLBACK_REF, reqData).getPassword();
   
  -		WSSAddUsernameToken builder = new WSSAddUsernameToken(actor, mu);
  +		WSSAddUsernameToken builder = new WSSAddUsernameToken(reqData.actor, mu);
   		builder.setPasswordType(WSConstants.PASSWORD_TEXT);
  -		builder.preSetUsernameToken(doc, username, password);
  +		builder.preSetUsernameToken(doc, reqData.username, password);
   		builder.addCreated(doc);
   		builder.addNonce(doc);
   
  -		WSSignEnvelope sign = new WSSignEnvelope(actor, mu);
  -        if (signatureParts.size() > 0) {
  -            sign.setParts(signatureParts);
  +		WSSignEnvelope sign = new WSSignEnvelope(reqData.actor, mu);
  +        if (reqData.signatureParts.size() > 0) {
  +            sign.setParts(reqData.signatureParts);
           }
   		sign.setUsernameToken(builder);
   		sign.setKeyIdentifierType(WSConstants.UT_SIGNING);
  @@ -480,41 +511,41 @@
   	}
   
   
  -    private void performSTAction(int actionToDo, boolean mu, Document doc)
  +    private void performSTAction(int actionToDo, boolean mu, Document doc, RequestData reqData)
               throws AxisFault {
  -        WSSAddSAMLToken builder = new WSSAddSAMLToken(actor, mu);
  +        WSSAddSAMLToken builder = new WSSAddSAMLToken(reqData.actor, mu);
   
           String samlPropFile = null;
           if ((samlPropFile =
                   (String) getOption(WSHandlerConstants.SAML_PROP_FILE))
                   == null) {
               samlPropFile =
  -                    (String) msgContext.getProperty(WSHandlerConstants.SAML_PROP_FILE);
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.SAML_PROP_FILE);
           }
           SAMLIssuer saml = SAMLIssuerFactory.getInstance(samlPropFile);
  -        saml.setUsername(username);
  +        saml.setUsername(reqData.username);
           SAMLAssertion assertion = saml.newAssertion();
   
           // add the SAMLAssertion Token to the SOAP Enevelope
           builder.build(doc, assertion);
       }
   
  -    private void performST_SIGNAction(int actionToDo, boolean mu, Document doc)
  +    private void performST_SIGNAction(int actionToDo, boolean mu, Document doc, RequestData reqData)
               throws AxisFault {
           String samlPropFile = null;
           if ((samlPropFile =
                   (String) getOption(WSHandlerConstants.SAML_PROP_FILE))
                   == null) {
               samlPropFile =
  -                    (String) msgContext.getProperty(WSHandlerConstants.SAML_PROP_FILE);
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.SAML_PROP_FILE);
           }
           Crypto crypto = null;
           try {
  -            crypto = loadSignatureCrypto();
  +            crypto = loadSignatureCrypto(reqData);
           } catch (AxisFault ex) {
           }
           SAMLIssuer saml = SAMLIssuerFactory.getInstance(samlPropFile);
  -        saml.setUsername(username);
  +        saml.setUsername(reqData.username);
           saml.setUserCrypto(crypto);
           saml.setInstanceDoc(doc);
   
  @@ -526,7 +557,7 @@
           String issuerKeyPW = null;
           Crypto issuerCrypto = null;
   
  -        WSSignEnvelope wsSign = new WSSignEnvelope(actor, mu);
  +        WSSignEnvelope wsSign = new WSSignEnvelope(reqData.actor, mu);
           String password = null;
           if (saml.isSenderVouches()) {
               issuerKeyName = saml.getIssuerKeyName();
  @@ -534,15 +565,15 @@
               issuerCrypto = saml.getIssuerCrypto();
           } else {
               password =
  -                    getPassword(username,
  +                    getPassword(reqData.username,
                               actionToDo,
                               WSHandlerConstants.PW_CALLBACK_CLASS,
  -                            WSHandlerConstants.PW_CALLBACK_REF)
  +                            WSHandlerConstants.PW_CALLBACK_REF, reqData)
                       .getPassword();
  -            wsSign.setUserInfo(username, password);
  +            wsSign.setUserInfo(reqData.username, password);
           }
  -        if (sigKeyId != 0) {
  -            wsSign.setKeyIdentifierType(sigKeyId);
  +        if (reqData.sigKeyId != 0) {
  +            wsSign.setKeyIdentifierType(reqData.sigKeyId);
           }
           try {
               wsSign.build(doc,
  @@ -557,27 +588,27 @@
           }
       }
   
  -    private void performTSAction(int actionToDo, boolean mu, Document doc) throws AxisFault {
  +    private void performTSAction(int actionToDo, boolean mu, Document doc, RequestData reqData) throws AxisFault {
           String ttl = null;
           if ((ttl =
                   (String) getOption(WSHandlerConstants.TTL_TIMESTAMP))
                   == null) {
               ttl =
  -                    (String) msgContext.getProperty(WSHandlerConstants.TTL_TIMESTAMP);
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.TTL_TIMESTAMP);
           }
           int ttl_i = 0;
           if (ttl != null) {
               try {
                   ttl_i = Integer.parseInt(ttl);
               } catch (NumberFormatException e) {
  -                ttl_i = timeToLive;
  +                ttl_i = reqData.timeToLive;
               }
           }
           if (ttl_i <= 0) {
  -            ttl_i = timeToLive;
  +            ttl_i = reqData.timeToLive;
           }
           WSAddTimestamp timeStampBuilder =
  -                new WSAddTimestamp(actor, mu);
  +                new WSAddTimestamp(reqData.actor, mu);
           // add the Timestamp to the SOAP Enevelope
           timeStampBuilder.build(doc, ttl_i);
       }
  @@ -586,7 +617,7 @@
        * Hook to allow subclasses to load their Signature Crypto however they see
        * fit.
        */
  -    protected Crypto loadSignatureCrypto() throws AxisFault {
  +    protected Crypto loadSignatureCrypto(RequestData reqData) throws AxisFault {
           Crypto crypto = null;
           /*
            * Get crypto property file for signature. If none specified throw
  @@ -596,7 +627,7 @@
           if ((sigPropFile = (String) getOption(WSHandlerConstants.SIG_PROP_FILE))
                   == null) {
               sigPropFile =
  -                    (String) msgContext.getProperty(WSHandlerConstants.SIG_PROP_FILE);
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.SIG_PROP_FILE);
           }
           if (sigPropFile != null) {
               if ((crypto = (Crypto) cryptos.get(sigPropFile)) == null) {
  @@ -613,7 +644,7 @@
        * Hook to allow subclasses to load their Encryption Crypto however they
        * see fit.
        */
  -    protected Crypto loadEncryptionCrypto() throws AxisFault {
  +    protected Crypto loadEncryptionCrypto(RequestData reqData) throws AxisFault {
           Crypto crypto = null;
           /*
            * Get encryption crypto property file. If non specified take crypto
  @@ -623,28 +654,28 @@
           if ((encPropFile = (String) getOption(WSHandlerConstants.ENC_PROP_FILE))
                   == null) {
               encPropFile =
  -                    (String) msgContext.getProperty(WSHandlerConstants.ENC_PROP_FILE);
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.ENC_PROP_FILE);
           }
           if (encPropFile != null) {
               if ((crypto = (Crypto) cryptos.get(encPropFile)) == null) {
                   crypto = CryptoFactory.getInstance(encPropFile);
                   cryptos.put(encPropFile, crypto);
               }
  -        } else if ((crypto = sigCrypto) == null) {
  +        } else if ((crypto = reqData.sigCrypto) == null) {
               throw new AxisFault("WSDoAllSender: Encryption: no crypto property file");
           }
           return crypto;
       }
   
  -    private void decodeUTParameter() throws AxisFault {
  -        if ((pwType = (String) getOption(WSHandlerConstants.PASSWORD_TYPE))
  +    private void decodeUTParameter(RequestData reqData) throws AxisFault {
  +        if ((reqData.pwType = (String) getOption(WSHandlerConstants.PASSWORD_TYPE))
                   == null) {
  -            pwType =
  -                    (String) msgContext.getProperty(WSHandlerConstants.PASSWORD_TYPE);
  +        	reqData.pwType =
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.PASSWORD_TYPE);
           }
  -        if (pwType != null) {
  -            pwType =
  -                    pwType.equals(WSConstants.PW_TEXT)
  +        if (reqData.pwType != null) {
  +        	reqData.pwType =
  +        		reqData.pwType.equals(WSConstants.PW_TEXT)
                       ? WSConstants.PASSWORD_TEXT
                       : WSConstants.PASSWORD_DIGEST;
           }
  @@ -652,53 +683,53 @@
           if ((tmpS = (String) getOption(WSHandlerConstants.ADD_UT_ELEMENTS))
                   == null) {
               tmpS =
  -                    (String) msgContext.getProperty(WSHandlerConstants.ADD_UT_ELEMENTS);
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.ADD_UT_ELEMENTS);
           }
           if (tmpS != null) {
  -            utElements = StringUtil.split(tmpS, ' ');
  +        	reqData.utElements = StringUtil.split(tmpS, ' ');
           }
       }
   
  -    private void decodeSignatureParameter() throws AxisFault {
  +    private void decodeSignatureParameter(RequestData reqData) throws AxisFault {
           String tmpS = null;
           if ((tmpS = (String) getOption(WSHandlerConstants.SIG_KEY_ID)) == null) {
  -            tmpS = (String) msgContext.getProperty(WSHandlerConstants.SIG_KEY_ID);
  +            tmpS = (String) reqData.msgContext.getProperty(WSHandlerConstants.SIG_KEY_ID);
           }
           if (tmpS != null) {
               Integer I = (Integer) WSHandlerConstants.keyIdentifier.get(tmpS);
               if (I == null) {
                   throw new AxisFault("WSDoAllSender: Signature: unknown key identification");
               }
  -            sigKeyId = I.intValue();
  -            if (!(sigKeyId == WSConstants.ISSUER_SERIAL
  -                    || sigKeyId == WSConstants.BST_DIRECT_REFERENCE
  -                    || sigKeyId == WSConstants.X509_KEY_IDENTIFIER
  -                    || sigKeyId == WSConstants.SKI_KEY_IDENTIFIER)) {
  +            reqData.sigKeyId = I.intValue();
  +            if (!(reqData.sigKeyId == WSConstants.ISSUER_SERIAL
  +                    || reqData.sigKeyId == WSConstants.BST_DIRECT_REFERENCE
  +                    || reqData.sigKeyId == WSConstants.X509_KEY_IDENTIFIER
  +                    || reqData.sigKeyId == WSConstants.SKI_KEY_IDENTIFIER)) {
                   throw new AxisFault("WSDoAllSender: Signature: illegal key identification");
               }
           }
  -        if ((sigAlgorithm = (String) getOption(WSHandlerConstants.SIG_ALGO))
  +        if ((reqData.sigAlgorithm = (String) getOption(WSHandlerConstants.SIG_ALGO))
                   == null) {
  -            tmpS = (String) msgContext.getProperty(WSHandlerConstants.SIG_ALGO);
  +            tmpS = (String) reqData.msgContext.getProperty(WSHandlerConstants.SIG_ALGO);
           }
           if ((tmpS = (String) getOption(WSHandlerConstants.SIGNATURE_PARTS))
                   == null) {
               tmpS =
  -                    (String) msgContext.getProperty(WSHandlerConstants.SIGNATURE_PARTS);
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.SIGNATURE_PARTS);
           }
           if (tmpS != null) {
  -            splitEncParts(tmpS, signatureParts);
  +            splitEncParts(tmpS, reqData.signatureParts, reqData);
           }
       }
   
  -    private void decodeEncryptionParameter() throws AxisFault {
  -        if ((encUser = (String) getOption(WSHandlerConstants.ENCRYPTION_USER))
  +    private void decodeEncryptionParameter(RequestData reqData) throws AxisFault {
  +        if ((reqData.encUser = (String) getOption(WSHandlerConstants.ENCRYPTION_USER))
                   == null) {
  -            encUser =
  -                    (String) msgContext.getProperty(WSHandlerConstants.ENCRYPTION_USER);
  +        	reqData.encUser =
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.ENCRYPTION_USER);
           }
   
  -        if (encUser == null && (encUser = username) == null) {
  +        if (reqData.encUser == null && (reqData.encUser = reqData.username) == null) {
               throw new AxisFault("WSDoAllSender: Encryption: no username");
           }
           /*
  @@ -706,7 +737,7 @@
            * (msgType != null && msgType.equals(Message.RESPONSE)) {
            * handleSpecialUser(encUser); }
            */
  -        handleSpecialUser(encUser);
  +        handleSpecialUser(reqData);
   
           /*
            * If the following parameters are no used (they return null) then the
  @@ -714,51 +745,51 @@
            */
           String tmpS = null;
           if ((tmpS = (String) getOption(WSHandlerConstants.ENC_KEY_ID)) == null) {
  -            tmpS = (String) msgContext.getProperty(WSHandlerConstants.ENC_KEY_ID);
  +            tmpS = (String) reqData.msgContext.getProperty(WSHandlerConstants.ENC_KEY_ID);
           }
           if (tmpS != null) {
               Integer I = (Integer) WSHandlerConstants.keyIdentifier.get(tmpS);
               if (I == null) {
                   throw new AxisFault("WSDoAllSender: Encryption: unknown key identification");
               }
  -            encKeyId = I.intValue();
  -            if (!(encKeyId == WSConstants.ISSUER_SERIAL
  -                    || encKeyId == WSConstants.X509_KEY_IDENTIFIER
  -                    || encKeyId == WSConstants.SKI_KEY_IDENTIFIER
  -                    || encKeyId == WSConstants.BST_DIRECT_REFERENCE
  -                    || encKeyId == WSConstants.EMBEDDED_KEYNAME)) {
  +            reqData.encKeyId = I.intValue();
  +            if (!(reqData.encKeyId == WSConstants.ISSUER_SERIAL
  +                    || reqData.encKeyId == WSConstants.X509_KEY_IDENTIFIER
  +                    || reqData.encKeyId == WSConstants.SKI_KEY_IDENTIFIER
  +                    || reqData.encKeyId == WSConstants.BST_DIRECT_REFERENCE
  +                    || reqData.encKeyId == WSConstants.EMBEDDED_KEYNAME)) {
                   throw new AxisFault("WSDoAllSender: Encryption: illegal key identification");
               }
           }
  -        if ((encSymmAlgo = (String) getOption(WSHandlerConstants.ENC_SYM_ALGO))
  +        if ((reqData.encSymmAlgo = (String) getOption(WSHandlerConstants.ENC_SYM_ALGO))
                   == null) {
  -            encSymmAlgo =
  -                    (String) msgContext.getProperty(WSHandlerConstants.ENC_SYM_ALGO);
  +        	reqData.encSymmAlgo =
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.ENC_SYM_ALGO);
           }
  -        if ((encKeyTransport =
  +        if ((reqData.encKeyTransport =
                   (String) getOption(WSHandlerConstants.ENC_KEY_TRANSPORT))
                   == null) {
  -            encKeyTransport =
  -                    (String) msgContext.getProperty(WSHandlerConstants.ENC_KEY_TRANSPORT);
  +        	reqData.encKeyTransport =
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.ENC_KEY_TRANSPORT);
           }
           if ((tmpS = (String) getOption(WSHandlerConstants.ENCRYPTION_PARTS))
                   == null) {
               tmpS =
  -                    (String) msgContext.getProperty(WSHandlerConstants.ENCRYPTION_PARTS);
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.ENCRYPTION_PARTS);
           }
           if (tmpS != null) {
  -            splitEncParts(tmpS, encryptParts);
  +            splitEncParts(tmpS, reqData.encryptParts, reqData);
           }
       }
   
  -    private boolean decodeMustUnderstand() throws AxisFault {
  +    private boolean decodeMustUnderstand(RequestData reqData) throws AxisFault {
           boolean mu = true;
           String mustUnderstand = null;
           if ((mustUnderstand =
                   (String) getOption(WSHandlerConstants.MUST_UNDERSTAND))
                   == null) {
               mustUnderstand =
  -                    (String) msgContext.getProperty(WSHandlerConstants.MUST_UNDERSTAND);
  +                    (String) reqData.msgContext.getProperty(WSHandlerConstants.MUST_UNDERSTAND);
           }
           if (mustUnderstand != null) {
               if (mustUnderstand.equals("0") || mustUnderstand.equals("false")) {
  @@ -781,7 +812,8 @@
       private WSPasswordCallback getPassword(String username,
                                              int doAction,
                                              String clsProp,
  -                                           String refProp)
  +                                           String refProp,
  +										   RequestData reqData)
               throws AxisFault {
           WSPasswordCallback pwCb = null;
           String password = null;
  @@ -789,7 +821,7 @@
           CallbackHandler cbHandler = null;
   
           if ((callback = (String) getOption(clsProp)) == null) {
  -            callback = (String) msgContext.getProperty(clsProp);
  +            callback = (String) reqData.msgContext.getProperty(clsProp);
           }
           if (callback != null) { // we have a password callback class
               pwCb = readPwViaCallbackClass(callback, username, doAction);
  @@ -797,16 +829,16 @@
                   throw new AxisFault("WSDoAllSender: password callback class provided null or empty password");
               }
           } else if (
  -                (cbHandler = (CallbackHandler) msgContext.getProperty(refProp))
  +                (cbHandler = (CallbackHandler) reqData.msgContext.getProperty(refProp))
                   != null) {
               pwCb = performCallback(cbHandler, username, doAction);
               if ((pwCb.getPassword() == null) && (pwCb.getKey() == null)) {
                   throw new AxisFault("WSDoAllSender: password callback provided null or empty password");
               }
  -        } else if ((password = msgContext.getPassword()) == null) {
  +        } else if ((password = reqData.msgContext.getPassword()) == null) {
               throw new AxisFault("WSDoAllSender: application provided null or empty password");
           } else {
  -            msgContext.setPassword(null);
  +        	reqData.msgContext.setPassword(null);
               pwCb = new WSPasswordCallback("", WSPasswordCallback.UNKNOWN);
               pwCb.setPassword(password);
           }
  @@ -878,7 +910,7 @@
           return pwCb;
       }
   
  -    private void splitEncParts(String tmpS, Vector encryptParts)
  +    private void splitEncParts(String tmpS, Vector parts, RequestData reqData)
               throws AxisFault {
           WSEncryptionPart encPart = null;
           String[] rawParts = StringUtil.split(tmpS, ';');
  @@ -892,7 +924,7 @@
                   }
                   encPart =
                           new WSEncryptionPart(partDef[0].trim(),
  -                                soapConstants.getEnvelopeURI(),
  +                        		reqData.soapConstants.getEnvelopeURI(),
                                   "Content");
               } else if (partDef.length == 3) {
                   String mode = partDef[0].trim();
  @@ -903,7 +935,7 @@
                   }
                   String nmSpace = partDef[1].trim();
                   if (nmSpace.length() <= 1) {
  -                    nmSpace = soapConstants.getEnvelopeURI();
  +                    nmSpace = reqData.soapConstants.getEnvelopeURI();
                   } else {
                       nmSpace = nmSpace.substring(1);
                   }
  @@ -921,17 +953,17 @@
               } else {
                   throw new AxisFault("WSDoAllSender: wrong part definition: " + tmpS);
               }
  -            encryptParts.add(encPart);
  +            parts.add(encPart);
           }
       }
   
  -    private void handleSpecialUser(String encUser) {
  -        if (!WSHandlerConstants.USE_REQ_SIG_CERT.equals(encUser)) {
  +    private void handleSpecialUser(RequestData reqData) {
  +        if (!WSHandlerConstants.USE_REQ_SIG_CERT.equals(reqData.encUser)) {
               return;
           }
           Vector results = null;
           if ((results =
  -                (Vector) msgContext.getProperty(WSHandlerConstants.RECV_RESULTS))
  +                (Vector) reqData.msgContext.getProperty(WSHandlerConstants.RECV_RESULTS))
                   == null) {
               return;
           }
  @@ -943,7 +975,7 @@
               WSHandlerResult rResult =
                       (WSHandlerResult) results.get(i);
               String hActor = rResult.getActor();
  -            if (!WSSecurityUtil.isActorEqual(actor, hActor)) {
  +            if (!WSSecurityUtil.isActorEqual(reqData.actor, hActor)) {
                   continue;
               }
               Vector wsSecEngineResults = rResult.getResults();
  @@ -956,7 +988,7 @@
                   WSSecurityEngineResult wser =
                           (WSSecurityEngineResult) wsSecEngineResults.get(j);
                   if (wser.getAction() == WSConstants.SIGN) {
  -                    encCert = wser.getCertificate();
  +                	reqData.encCert = wser.getCertificate();
                       return;
                   }
               }