You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@santuario.apache.org by co...@apache.org on 2012/07/03 14:46:32 UTC

svn commit: r1356704 - in /santuario/xml-security-java/trunk/src: main/java/org/apache/xml/security/stax/ext/ main/java/org/apache/xml/security/stax/impl/processor/input/ main/resources/messages/ test/java/org/apache/xml/security/test/stax/signature/

Author: coheigea
Date: Tue Jul  3 12:46:31 2012
New Revision: 1356704

URL: http://svn.apache.org/viewvc?rev=1356704&view=rev
Log:
[SANTUARIO-316] - Initial support for streaming signature verification

Added:
    santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureInputHandler.java
    santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureInputProcessor.java
    santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureReferenceVerifyInputProcessor.java
    santuario/xml-security-java/trunk/src/test/java/org/apache/xml/security/test/stax/signature/SignatureVerificationTest.java
Modified:
    santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/ext/XMLSecurityProperties.java
    santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/AbstractSignatureInputHandler.java
    santuario/xml-security-java/trunk/src/main/resources/messages/errors.properties

Modified: santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/ext/XMLSecurityProperties.java
URL: http://svn.apache.org/viewvc/santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/ext/XMLSecurityProperties.java?rev=1356704&r1=1356703&r2=1356704&view=diff
==============================================================================
--- santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/ext/XMLSecurityProperties.java (original)
+++ santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/ext/XMLSecurityProperties.java Tue Jul  3 12:46:31 2012
@@ -467,6 +467,15 @@ public class XMLSecurityProperties {
 
     private Class<? extends MerlinBase> signatureVerificationCryptoClass;
     private KeyStore signatureVerificationKeyStore;
+    private Key signatureVerificationKey;
+    
+    public Key getSignatureVerificationKey() {
+        return signatureVerificationKey;
+    }
+
+    public void setSignatureVerificationKey(Key signatureVerificationKey) {
+        this.signatureVerificationKey = signatureVerificationKey;
+    }
 
     public KeyStore getSignatureVerificationKeyStore() {
         return signatureVerificationKeyStore;
@@ -545,4 +554,5 @@ public class XMLSecurityProperties {
     public void setDisableSchemaValidation(boolean disableSchemaValidation) {
         this.disableSchemaValidation = disableSchemaValidation;
     }
+
 }

Modified: santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/AbstractSignatureInputHandler.java
URL: http://svn.apache.org/viewvc/santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/AbstractSignatureInputHandler.java?rev=1356704&r1=1356703&r2=1356704&view=diff
==============================================================================
--- santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/AbstractSignatureInputHandler.java (original)
+++ santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/AbstractSignatureInputHandler.java Tue Jul  3 12:46:31 2012
@@ -26,7 +26,6 @@ import org.apache.xml.security.stax.ext.
 import org.apache.xml.security.stax.ext.stax.XMLSecEvent;
 import org.apache.xml.security.stax.impl.algorithms.SignatureAlgorithm;
 import org.apache.xml.security.stax.impl.algorithms.SignatureAlgorithmFactory;
-import org.apache.xml.security.stax.impl.securityToken.SecurityTokenFactory;
 import org.apache.xml.security.stax.impl.util.SignerOutputStream;
 
 import javax.xml.bind.JAXBElement;
@@ -151,7 +150,7 @@ public abstract class AbstractSignatureI
     </ds:Signature>
      */
 
-    public class SignatureVerifier {
+    public abstract class SignatureVerifier {
 
         private final SignatureType signatureType;
         private final SecurityToken securityToken;
@@ -165,9 +164,8 @@ public abstract class AbstractSignatureI
             this.signatureType = signatureType;
 
             KeyInfoType keyInfoType = signatureType.getKeyInfo();
-            SecurityToken securityToken = SecurityTokenFactory.getInstance().getSecurityToken(keyInfoType,
-                    securityProperties.getSignatureVerificationCrypto(), securityProperties.getCallbackHandler(),
-                    securityContext);
+            SecurityToken securityToken = 
+                retrieveSecurityToken(keyInfoType, securityProperties, securityContext);
             securityToken.verify();
 
             try {
@@ -178,6 +176,10 @@ public abstract class AbstractSignatureI
             }
             this.securityToken = securityToken;
         }
+        
+        protected abstract SecurityToken retrieveSecurityToken(KeyInfoType keyInfoType,
+                                                 XMLSecurityProperties securityProperties,
+                                                 SecurityContext securityContext) throws XMLSecurityException;
 
         public SecurityToken getSecurityToken() {
             return securityToken;

Added: santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureInputHandler.java
URL: http://svn.apache.org/viewvc/santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureInputHandler.java?rev=1356704&view=auto
==============================================================================
--- santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureInputHandler.java (added)
+++ santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureInputHandler.java Tue Jul  3 12:46:31 2012
@@ -0,0 +1,184 @@
+/**
+ * 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.xml.security.stax.impl.processor.input;
+
+import java.security.Key;
+import java.security.PublicKey;
+import java.security.cert.X509Certificate;
+import java.util.List;
+
+import javax.crypto.SecretKey;
+import javax.xml.namespace.QName;
+
+import org.apache.xml.security.binding.xmldsig.KeyInfoType;
+import org.apache.xml.security.binding.xmldsig.SignatureType;
+import org.apache.xml.security.stax.ext.InputProcessorChain;
+import org.apache.xml.security.stax.ext.SecurityContext;
+import org.apache.xml.security.stax.ext.SecurityToken;
+import org.apache.xml.security.stax.ext.XMLSecurityConstants;
+import org.apache.xml.security.stax.ext.XMLSecurityException;
+import org.apache.xml.security.stax.ext.XMLSecurityProperties;
+import org.apache.xml.security.stax.ext.stax.XMLSecEvent;
+
+/**
+ * An input handler for XML Signature.
+ */
+public class XMLSignatureInputHandler extends AbstractSignatureInputHandler {
+
+    @Override
+    protected SignatureVerifier newSignatureVerifier(final InputProcessorChain inputProcessorChain,
+                                                     final XMLSecurityProperties securityProperties,
+                                                     final SignatureType signatureType) throws XMLSecurityException {
+
+        if (signatureType.getSignedInfo() == null) {
+            throw new XMLSecurityException(XMLSecurityException.ErrorCode.INVALID_SECURITY);
+        }
+        if (signatureType.getSignedInfo().getSignatureMethod() == null) {
+            throw new XMLSecurityException(XMLSecurityException.ErrorCode.INVALID_SECURITY);
+        }
+        if (signatureType.getSignedInfo().getCanonicalizationMethod() == null) {
+            throw new XMLSecurityException(XMLSecurityException.ErrorCode.INVALID_SECURITY);
+        }
+        if (signatureType.getSignatureValue() == null) {
+            throw new XMLSecurityException(XMLSecurityException.ErrorCode.INVALID_SECURITY);
+        }
+        if (signatureType.getKeyInfo() == null) {
+            throw new XMLSecurityException(XMLSecurityException.ErrorCode.INVALID_SECURITY);
+        }
+        return new XMLSignatureVerifier(signatureType, inputProcessorChain.getSecurityContext(), securityProperties);
+    }
+
+    @Override
+    protected void addSignatureReferenceInputProcessorToChain(InputProcessorChain inputProcessorChain,
+                                                              XMLSecurityProperties securityProperties,
+                                                              SignatureType signatureType, SecurityToken securityToken) throws XMLSecurityException {
+        //add processors to verify references
+        inputProcessorChain.addProcessor(new XMLSignatureReferenceVerifyInputProcessor(signatureType, securityToken, securityProperties));
+    }
+    
+    public class XMLSignatureVerifier extends SignatureVerifier {
+        
+        public XMLSignatureVerifier(SignatureType signatureType, SecurityContext securityContext,
+                                    XMLSecurityProperties securityProperties) throws XMLSecurityException {
+            super(signatureType, securityContext, securityProperties);
+        }
+        
+        protected SecurityToken retrieveSecurityToken(KeyInfoType keyInfoType,
+                                                      XMLSecurityProperties securityProperties,
+                                                      SecurityContext securityContext) throws XMLSecurityException {
+            return new SignatureSecurityToken(securityProperties.getSignatureVerificationKey());
+        }
+    }
+    
+    private static class SignatureSecurityToken implements SecurityToken {
+        private Key key;
+        
+        public SignatureSecurityToken(Key key) {
+            this.key = key;
+        }
+
+        public String getId() {
+            return null;
+        }
+
+
+        public Object getProcessor() {
+            return null;
+        }
+
+        public boolean isAsymmetric() {
+            if (key instanceof PublicKey) {
+                return true;
+            }
+            return false;
+        }
+
+        public Key getSecretKey(
+            String algorithmURI, XMLSecurityConstants.KeyUsage keyUsage
+        ) throws XMLSecurityException {
+            if (key instanceof SecretKey) {
+                return key;
+            }
+            return null;
+        }
+
+        public PublicKey getPublicKey(
+            String algorithmURI, XMLSecurityConstants.KeyUsage keyUsage
+        ) throws XMLSecurityException {
+            if (key instanceof PublicKey) {
+                return (PublicKey)key;
+            }
+            return null;
+        }
+
+        public X509Certificate[] getX509Certificates() throws XMLSecurityException {
+            return null;
+        }
+
+        public void verify() throws XMLSecurityException {
+        }
+
+        public SecurityToken getKeyWrappingToken() {
+            return null;
+        }
+
+        public XMLSecurityConstants.TokenType getTokenType() {
+            return null;
+        }
+
+        @Override
+        public List<QName> getElementPath() {
+            // TODO Auto-generated method stub
+            return null;
+        }
+
+        @Override
+        public XMLSecEvent getXMLSecEvent() {
+            // TODO Auto-generated method stub
+            return null;
+        }
+
+        @Override
+        public List<SecurityToken> getWrappedTokens()
+                throws XMLSecurityException {
+            // TODO Auto-generated method stub
+            return null;
+        }
+
+        @Override
+        public void addWrappedToken(SecurityToken securityToken) {
+            // TODO Auto-generated method stub
+            
+        }
+
+        @Override
+        public void addTokenUsage(TokenUsage tokenUsage)
+                throws XMLSecurityException {
+            // TODO Auto-generated method stub
+            
+        }
+
+        @Override
+        public List<TokenUsage> getTokenUsages() {
+            // TODO Auto-generated method stub
+            return null;
+        }
+    };
+
+}

Added: santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureInputProcessor.java
URL: http://svn.apache.org/viewvc/santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureInputProcessor.java?rev=1356704&view=auto
==============================================================================
--- santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureInputProcessor.java (added)
+++ santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureInputProcessor.java Tue Jul  3 12:46:31 2012
@@ -0,0 +1,177 @@
+/**
+ * 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.xml.security.stax.impl.processor.input;
+
+import java.util.ArrayDeque;
+
+import javax.xml.stream.XMLStreamConstants;
+import javax.xml.stream.XMLStreamException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.xml.security.stax.ext.AbstractInputProcessor;
+import org.apache.xml.security.stax.ext.InputProcessorChain;
+import org.apache.xml.security.stax.ext.XMLSecurityConstants;
+import org.apache.xml.security.stax.ext.XMLSecurityException;
+import org.apache.xml.security.stax.ext.XMLSecurityProperties;
+import org.apache.xml.security.stax.ext.stax.XMLSecEndElement;
+import org.apache.xml.security.stax.ext.stax.XMLSecEvent;
+import org.apache.xml.security.stax.ext.stax.XMLSecStartElement;
+
+/**
+ * Processor for XML Signature.
+ *
+ * @author $Author: coheigea $
+ * @version $Revision: 1354898 $ $Date: 2012-06-28 11:19:02 +0100 (Thu, 28 Jun 2012) $
+ */
+public class XMLSignatureInputProcessor extends AbstractInputProcessor {
+
+    protected static final transient Log logger = LogFactory.getLog(XMLSignatureInputProcessor.class);
+
+    private final ArrayDeque<XMLSecEvent> xmlSecEventList = new ArrayDeque<XMLSecEvent>();
+    private int eventCount = 0;
+    private int startIndexForProcessor = 0;
+
+    public XMLSignatureInputProcessor(XMLSecurityProperties securityProperties) {
+        super(securityProperties);
+        setPhase(XMLSecurityConstants.Phase.POSTPROCESSING);
+    }
+
+    @Override
+    public XMLSecEvent processNextHeaderEvent(InputProcessorChain inputProcessorChain)
+            throws XMLStreamException, XMLSecurityException {
+        return null;
+    }
+
+    @Override
+    public XMLSecEvent processNextEvent(InputProcessorChain inputProcessorChain)
+            throws XMLStreamException, XMLSecurityException {
+
+        //buffer all events until the end of the required actions
+        final InputProcessorChain subInputProcessorChain = inputProcessorChain.createSubChain(this);
+        final InternalBufferProcessor internalBufferProcessor
+                = new InternalBufferProcessor(getSecurityProperties());
+        subInputProcessorChain.addProcessor(internalBufferProcessor);
+
+        boolean elementFound = false;
+
+        XMLSecEvent xmlSecEvent;
+        do {
+            subInputProcessorChain.reset();
+            xmlSecEvent = subInputProcessorChain.processHeaderEvent();
+            eventCount++;
+
+            switch (xmlSecEvent.getEventType()) {
+                case XMLStreamConstants.START_ELEMENT:
+                    XMLSecStartElement xmlSecStartElement = xmlSecEvent.asStartElement();
+
+                    if (xmlSecStartElement.getName().equals(XMLSecurityConstants.TAG_dsig_Signature)) {
+                        elementFound = true;
+                        startIndexForProcessor = eventCount - 1;
+                    }
+                    break;
+                case XMLStreamConstants.END_ELEMENT:
+                    XMLSecEndElement xmlSecEndElement = xmlSecEvent.asEndElement();
+                    if (elementFound
+                            && xmlSecEndElement.getName().equals(XMLSecurityConstants.TAG_dsig_Signature)) {
+                        // Handle the signature
+                        XMLSignatureInputHandler inputHandler = new XMLSignatureInputHandler();
+                        inputHandler.handle(inputProcessorChain, getSecurityProperties(), 
+                                            xmlSecEventList, startIndexForProcessor);
+                        
+                        subInputProcessorChain.removeProcessor(internalBufferProcessor);
+                        subInputProcessorChain.addProcessor(
+                                new InternalReplayProcessor(getSecurityProperties()));
+
+                        //remove this processor from chain now. the next events will go directly to the other processors
+                        subInputProcessorChain.removeProcessor(this);
+                        //since we cloned the inputProcessor list we have to add the processors from
+                        //the subChain to the main chain.
+                        inputProcessorChain.getProcessors().clear();
+                        inputProcessorChain.getProcessors().addAll(subInputProcessorChain.getProcessors());
+
+                        //return first event now;
+                        return xmlSecEventList.pollLast();
+                    }
+                    break;
+            }
+
+        } while (!xmlSecEvent.isEndDocument());
+        //if we reach this state we didn't find a signature
+        throw new XMLSecurityException(XMLSecurityException.ErrorCode.FAILURE, "missingSignature");
+    }
+
+    /**
+     * Temporary Processor to buffer all events until the end of the required actions
+     */
+    public class InternalBufferProcessor extends AbstractInputProcessor {
+
+        InternalBufferProcessor(XMLSecurityProperties securityProperties) {
+            super(securityProperties);
+            setPhase(XMLSecurityConstants.Phase.POSTPROCESSING);
+            addBeforeProcessor(XMLSignatureInputProcessor.class.getName());
+        }
+
+        @Override
+        public XMLSecEvent processNextHeaderEvent(InputProcessorChain inputProcessorChain)
+                throws XMLStreamException, XMLSecurityException {
+            XMLSecEvent xmlSecEvent = inputProcessorChain.processHeaderEvent();
+            xmlSecEventList.push(xmlSecEvent);
+            return xmlSecEvent;
+        }
+
+        @Override
+        public XMLSecEvent processNextEvent(InputProcessorChain inputProcessorChain)
+                throws XMLStreamException, XMLSecurityException {
+            //should never be called because we remove this processor before
+            return null;
+        }
+    }
+
+    /**
+     * Temporary processor to replay the buffered events
+     */
+    public class InternalReplayProcessor extends AbstractInputProcessor {
+
+        public InternalReplayProcessor(XMLSecurityProperties securityProperties) {
+            super(securityProperties);
+            setPhase(XMLSecurityConstants.Phase.PREPROCESSING);
+            addBeforeProcessor(XMLSignatureInputProcessor.class.getName());
+            addAfterProcessor(XMLEventReaderInputProcessor.class.getName());
+        }
+
+        @Override
+        public XMLSecEvent processNextHeaderEvent(InputProcessorChain inputProcessorChain)
+                throws XMLStreamException, XMLSecurityException {
+            return null;
+        }
+
+        @Override
+        public XMLSecEvent processNextEvent(InputProcessorChain inputProcessorChain)
+                throws XMLStreamException, XMLSecurityException {
+
+            if (!xmlSecEventList.isEmpty()) {
+                return xmlSecEventList.pollLast();
+            } else {
+                inputProcessorChain.removeProcessor(this);
+                return inputProcessorChain.processEvent();
+            }
+        }
+    }
+}

Added: santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureReferenceVerifyInputProcessor.java
URL: http://svn.apache.org/viewvc/santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureReferenceVerifyInputProcessor.java?rev=1356704&view=auto
==============================================================================
--- santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureReferenceVerifyInputProcessor.java (added)
+++ santuario/xml-security-java/trunk/src/main/java/org/apache/xml/security/stax/impl/processor/input/XMLSignatureReferenceVerifyInputProcessor.java Tue Jul  3 12:46:31 2012
@@ -0,0 +1,36 @@
+/**
+ * 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.xml.security.stax.impl.processor.input;
+
+import org.apache.xml.security.binding.xmldsig.SignatureType;
+import org.apache.xml.security.stax.ext.SecurityToken;
+import org.apache.xml.security.stax.ext.XMLSecurityException;
+import org.apache.xml.security.stax.ext.XMLSecurityProperties;
+
+/**
+ * A processor to verify XML Signature references.
+ */
+public class XMLSignatureReferenceVerifyInputProcessor extends AbstractSignatureReferenceVerifyInputProcessor {
+
+    public XMLSignatureReferenceVerifyInputProcessor(SignatureType signatureType, SecurityToken securityToken, XMLSecurityProperties securityProperties) throws XMLSecurityException {
+        super(signatureType, securityToken, securityProperties);
+        this.addAfterProcessor(XMLSignatureReferenceVerifyInputProcessor.class.getName());
+    }
+
+}

Modified: santuario/xml-security-java/trunk/src/main/resources/messages/errors.properties
URL: http://svn.apache.org/viewvc/santuario/xml-security-java/trunk/src/main/resources/messages/errors.properties?rev=1356704&r1=1356703&r2=1356704&view=diff
==============================================================================
--- santuario/xml-security-java/trunk/src/main/resources/messages/errors.properties (original)
+++ santuario/xml-security-java/trunk/src/main/resources/messages/errors.properties Tue Jul  3 12:46:31 2012
@@ -34,6 +34,7 @@ badTokenType01 = Bad UsernameToken Value
 failedAuthentication = User ({0}) not authenticated
 missingUsernameToken = UsernameToken is missing
 missingSecurityHeader = Security header is missing
+missingSignature = Signature is missing
 missingUsername = Username is missing
 missingPassword = Password is missing
 missingPasswordType =Password attribute type is missing

Added: santuario/xml-security-java/trunk/src/test/java/org/apache/xml/security/test/stax/signature/SignatureVerificationTest.java
URL: http://svn.apache.org/viewvc/santuario/xml-security-java/trunk/src/test/java/org/apache/xml/security/test/stax/signature/SignatureVerificationTest.java?rev=1356704&view=auto
==============================================================================
--- santuario/xml-security-java/trunk/src/test/java/org/apache/xml/security/test/stax/signature/SignatureVerificationTest.java (added)
+++ santuario/xml-security-java/trunk/src/test/java/org/apache/xml/security/test/stax/signature/SignatureVerificationTest.java Tue Jul  3 12:46:31 2012
@@ -0,0 +1,210 @@
+/**
+ * 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.xml.security.test.stax.signature;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.InputStream;
+import java.security.Key;
+import java.security.KeyStore;
+import java.security.Provider;
+import java.security.Security;
+import java.security.cert.X509Certificate;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamReader;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathFactory;
+
+import org.apache.xml.security.signature.XMLSignature;
+import org.apache.xml.security.stax.config.Init;
+import org.apache.xml.security.stax.ext.InputProcessor;
+import org.apache.xml.security.stax.ext.XMLSecurityConstants;
+import org.apache.xml.security.stax.ext.XMLSecurityProperties;
+import org.apache.xml.security.stax.impl.DocumentContextImpl;
+import org.apache.xml.security.stax.impl.InputProcessorChainImpl;
+import org.apache.xml.security.stax.impl.SecurityContextImpl;
+import org.apache.xml.security.stax.impl.XMLSecurityStreamReader;
+import org.apache.xml.security.stax.impl.processor.input.XMLEventReaderInputProcessor;
+import org.apache.xml.security.stax.impl.processor.input.XMLSignatureInputProcessor;
+import org.apache.xml.security.test.dom.DSNamespaceContext;
+import org.apache.xml.security.test.stax.utils.StAX2DOM;
+import org.apache.xml.security.test.stax.utils.XMLSecEventAllocator;
+import org.apache.xml.security.transforms.Transforms;
+import org.apache.xml.security.utils.Constants;
+import org.apache.xml.security.utils.XMLUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+
+/**
+ * A set of test-cases for Signature verification.
+ */
+public class SignatureVerificationTest extends org.junit.Assert {
+
+    private XMLInputFactory xmlInputFactory;
+    private DocumentBuilderFactory documentBuilderFactory;
+    private TransformerFactory transformerFactory = TransformerFactory.newInstance();
+    
+    static {
+        try {
+            Class<?> c =
+                    SignatureVerificationTest.class.getClassLoader().loadClass(
+                            "org.bouncycastle.jce.provider.BouncyCastleProvider"
+                    );
+            if (null == Security.getProvider("BC")) {
+                // Security.addProvider((Provider) c.newInstance());
+                Security.insertProviderAt((Provider) c.newInstance(), 1);
+            }
+        } catch (Throwable e) {
+            throw new RuntimeException("Adding BouncyCastle provider failed", e);
+        }
+    }
+
+
+    @Before
+    public void setUp() throws Exception {
+        Init.init(SignatureVerificationTest.class.getClassLoader().getResource("security-config.xml").toURI());
+        org.apache.xml.security.Init.init();
+        
+        xmlInputFactory = XMLInputFactory.newInstance();
+        xmlInputFactory.setEventAllocator(new XMLSecEventAllocator());
+        
+        documentBuilderFactory = DocumentBuilderFactory.newInstance();
+        documentBuilderFactory.setNamespaceAware(true);
+        documentBuilderFactory.setIgnoringComments(false);
+        documentBuilderFactory.setCoalescing(false);
+        documentBuilderFactory.setIgnoringElementContentWhitespace(false);
+    }
+    
+
+    @Test
+    public void testSignatureVerification() throws Exception {
+        // Read in plaintext document
+        InputStream sourceDocument = 
+                this.getClass().getClassLoader().getResourceAsStream(
+                        "ie/baltimore/merlin-examples/merlin-xmlenc-five/plaintext.xml");
+        DocumentBuilder builder = documentBuilderFactory.newDocumentBuilder();
+        Document document = builder.parse(sourceDocument);
+        
+        // Set up the Key
+        KeyStore keyStore = KeyStore.getInstance("jks");
+        keyStore.load(
+            this.getClass().getClassLoader().getResource("transmitter.jks").openStream(), 
+            "default".toCharArray()
+        );
+        Key key = keyStore.getKey("transmitter", "default".toCharArray());
+        X509Certificate cert = (X509Certificate)keyStore.getCertificate("transmitter");
+        
+        // Sign using DOM
+        signUsingDOM(
+            "http://www.w3.org/2000/09/xmldsig#rsa-sha1", document, cert, key
+        );
+        
+        // XMLUtils.outputDOM(document, System.out);
+        
+        // Convert Document to a Stream Reader
+        javax.xml.transform.Transformer transformer = transformerFactory.newTransformer();
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        transformer.transform(new DOMSource(document), new StreamResult(baos));
+        final XMLStreamReader xmlStreamReader = 
+                xmlInputFactory.createXMLStreamReader(new ByteArrayInputStream(baos.toByteArray()));
+  
+        // Verify signature
+        SecurityContextImpl securityContextImpl = new SecurityContextImpl();
+        securityContextImpl.put(XMLSecurityConstants.XMLINPUTFACTORY, xmlInputFactory);
+        
+        XMLSecurityProperties properties = new XMLSecurityProperties();
+        properties.setSignatureVerificationKey(cert.getPublicKey());
+        
+        final DocumentContextImpl documentContext = new DocumentContextImpl();
+        documentContext.setEncoding("UTF-8");
+        
+        // Set up the processor chain
+        InputProcessorChainImpl processorChain = 
+            new InputProcessorChainImpl(securityContextImpl, documentContext);
+        processorChain.addProcessor(new XMLEventReaderInputProcessor(properties, xmlStreamReader));
+        
+        List<InputProcessor> additionalInputProcessors = properties.getInputProcessorList();
+        if (!additionalInputProcessors.isEmpty()) {
+            Iterator<InputProcessor> inputProcessorIterator = additionalInputProcessors.iterator();
+            while (inputProcessorIterator.hasNext()) {
+                InputProcessor inputProcessor = inputProcessorIterator.next();
+                processorChain.addProcessor(inputProcessor);
+            }
+        }
+        
+        processorChain.addProcessor(new XMLSignatureInputProcessor(properties));
+         
+        XMLStreamReader securityStreamReader = new XMLSecurityStreamReader(processorChain, properties);
+        document = StAX2DOM.readDoc(documentBuilderFactory.newDocumentBuilder(), securityStreamReader);
+        
+        XMLUtils.outputDOM(document, System.out);
+    }
+    
+    /**
+     * Sign the document using DOM
+     */
+    private void signUsingDOM(
+        String algorithm,
+        Document document,
+        X509Certificate cert,
+        Key signingKey
+    ) throws Exception {
+        XMLSignature sig = new XMLSignature(document, "", algorithm);
+        Element root = document.getDocumentElement();
+        root.appendChild(sig.getElement());
+
+        XPathFactory xpf = XPathFactory.newInstance();
+        XPath xpath = xpf.newXPath();
+        xpath.setNamespaceContext(new DSNamespaceContext());
+        
+        String expression = "//*[local-name()='" + "PaymentInfo" + "']";
+        Element elementToSign = 
+            (Element)xpath.evaluate(expression, document, XPathConstants.NODE);
+        Assert.assertNotNull(elementToSign);
+        String id = UUID.randomUUID().toString();
+        elementToSign.setAttributeNS(null, "Id", id);
+        elementToSign.setIdAttributeNS(null, "Id", true);
+        
+        Transforms transforms = new Transforms(document);
+        transforms.addTransform(Transforms.TRANSFORM_C14N_EXCL_OMIT_COMMENTS);
+        sig.addDocument("#" + id, transforms, Constants.ALGO_ID_DIGEST_SHA1);
+
+        sig.addKeyInfo(cert);
+        sig.sign(signingKey);
+        
+        expression = "//ds:Signature[1]";
+        Element sigElement = 
+            (Element) xpath.evaluate(expression, document, XPathConstants.NODE);
+        Assert.assertNotNull(sigElement);
+    }
+    
+}
\ No newline at end of file