You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2021/11/22 16:55:45 UTC

[nifi] branch main updated: NIFI-9399 This closes #5542. Apply Secure Processing to TransformXml XSLT

This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new c033deb  NIFI-9399 This closes #5542. Apply Secure Processing to TransformXml XSLT
c033deb is described below

commit c033debdf3077ef465b62e76c3878df325e1203b
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Fri Nov 19 13:30:26 2021 -0600

    NIFI-9399 This closes #5542. Apply Secure Processing to TransformXml XSLT
    
    - Added XML Stream Reader processing for XSLT with external entities disabled
    - Removed unused XsltValidator
    - Upgraded Saxon-HE from 9.6.0-5 to 10.6
    
    Signed-off-by: Joe Witt <jo...@apache.org>
---
 nifi-assembly/NOTICE                               |   2 +-
 .../src/main/resources/META-INF/NOTICE             |   2 +-
 .../nifi/processors/standard/TransformXml.java     | 180 ++++++++++-----------
 .../nifi/processors/standard/TestTransformXml.java |  46 +++++-
 .../test/resources/TestTransformXml/doc-node.xsl   |  22 +++
 .../TestTransformXml/doctype-entity-file-uri.xsl   |  19 +++
 .../src/test/resources/TestTransformXml/math.html  |   2 +-
 nifi-nar-bundles/nifi-standard-bundle/pom.xml      |   2 +-
 8 files changed, 164 insertions(+), 111 deletions(-)

diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE
index 095913f..baa8552 100644
--- a/nifi-assembly/NOTICE
+++ b/nifi-assembly/NOTICE
@@ -2108,7 +2108,7 @@ Mozilla Public License v2.0
 
 The following binary components are provided under the Mozilla Public License v2.0.  See project link for details.
 
-    (MPL 2.0) Saxon HE (net.sf.saxon:Saxon-HE:jar:9.6.0-5 - http://www.saxonica.com/)
+    (MPL 2.0) Saxon HE (net.sf.saxon:Saxon-HE:jar:10.6 - http://www.saxonica.com/)
     (MPL 2.0) Rhino (org.mozilla:rhino:jar:1.7R5 - https://developer.mozilla.org/en-US/docs/Mozilla/Projects/Rhino/Download_Rhino)
 
 *****************
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
index bd1941f..550196c 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
@@ -272,7 +272,7 @@ Mozilla Public License v2.0
 
 The following binary components are provided under the Mozilla Public License v2.0.  See project link for details.
 
-    (MPL 2.0) Saxon HE (net.sf.saxon:Saxon-HE:jar:9.6.0-5 - http://www.saxonica.com/)
+    (MPL 2.0) Saxon HE (net.sf.saxon:Saxon-HE:jar:10.6 - http://www.saxonica.com/)
 
 *****************
 Public Domain
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
index fd20b0c..b55d62a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
@@ -33,7 +33,6 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
 import org.apache.nifi.components.resource.ResourceCardinality;
 import org.apache.nifi.components.resource.ResourceType;
 import org.apache.nifi.expression.AttributeExpression;
@@ -49,26 +48,28 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.xml.XmlUtils;
 import org.apache.nifi.util.StopWatch;
-import org.apache.nifi.util.Tuple;
 
 import javax.xml.XMLConstants;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamReader;
 import javax.xml.transform.OutputKeys;
+import javax.xml.transform.Result;
+import javax.xml.transform.Source;
 import javax.xml.transform.Templates;
 import javax.xml.transform.Transformer;
 import javax.xml.transform.TransformerConfigurationException;
 import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.stax.StAXSource;
 import javax.xml.transform.stream.StreamResult;
 import javax.xml.transform.stream.StreamSource;
 import java.io.BufferedInputStream;
-import java.io.ByteArrayInputStream;
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.charset.StandardCharsets;
+import java.io.Reader;
+import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -78,14 +79,13 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
 
 @EventDriven
 @SideEffectFree
 @SupportsBatching
 @Tags({"xml", "xslt", "transform"})
 @InputRequirement(Requirement.INPUT_REQUIRED)
-@CapabilityDescription("Applies the provided XSLT file to the flowfile XML payload. A new FlowFile is created "
+@CapabilityDescription("Applies the provided XSLT file to the FlowFile XML payload. A new FlowFile is created "
         + "with transformed content and is routed to the 'success' relationship. If the XSL transform "
         + "fails, the original FlowFile is routed to the 'failure' relationship")
 @DynamicProperty(name = "An XSLT transform parameter name", value = "An XSLT transform parameter value",
@@ -95,7 +95,7 @@ public class TransformXml extends AbstractProcessor {
 
     public static final PropertyDescriptor XSLT_FILE_NAME = new PropertyDescriptor.Builder()
             .name("XSLT file name")
-            .description("Provides the name (including full path) of the XSLT file to apply to the flowfile XML content."
+            .description("Provides the name (including full path) of the XSLT file to apply to the FlowFile XML content."
                     + "One of the 'XSLT file name' and 'XSLT Lookup' properties must be defined.")
             .required(false)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
@@ -174,8 +174,6 @@ public class TransformXml extends AbstractProcessor {
     private Set<Relationship> relationships;
     private LoadingCache<String, Templates> cache;
 
-    private static AtomicReference<LookupService<String>> lookupService = new AtomicReference<LookupService<String>>(null);
-
     @Override
     protected void init(final ProcessorInitializationContext context) {
         final List<PropertyDescriptor> properties = new ArrayList<>();
@@ -212,7 +210,7 @@ public class TransformXml extends AbstractProcessor {
         PropertyValue controller = validationContext.getProperty(XSLT_CONTROLLER);
         PropertyValue key = validationContext.getProperty(XSLT_CONTROLLER_KEY);
 
-        if((filename.isSet() && controller.isSet())
+        if ((filename.isSet() && controller.isSet())
                 || (!filename.isSet() && !controller.isSet())) {
             results.add(new ValidationResult.Builder()
                     .valid(false)
@@ -221,7 +219,7 @@ public class TransformXml extends AbstractProcessor {
                     .build());
         }
 
-        if(controller.isSet() && !key.isSet()) {
+        if (controller.isSet() && !key.isSet()) {
             results.add(new ValidationResult.Builder()
                     .valid(false)
                     .subject(XSLT_CONTROLLER_KEY.getDisplayName())
@@ -229,7 +227,7 @@ public class TransformXml extends AbstractProcessor {
                     .build());
         }
 
-        if(controller.isSet()) {
+        if (controller.isSet()) {
             final LookupService<String> lookupService = validationContext.getProperty(XSLT_CONTROLLER).asControllerService(StringLookupService.class);
             final Set<String> requiredKeys = lookupService.getRequiredKeys();
             if (requiredKeys == null || requiredKeys.size() != 1) {
@@ -256,31 +254,6 @@ public class TransformXml extends AbstractProcessor {
                 .build();
     }
 
-    private Templates newTemplates(final ProcessContext context, final String path) throws TransformerConfigurationException, LookupFailureException {
-        final Boolean secureProcessing = context.getProperty(SECURE_PROCESSING).asBoolean();
-        TransformerFactory factory = TransformerFactory.newInstance();
-        final boolean isFilename = context.getProperty(XSLT_FILE_NAME).isSet();
-
-        if (secureProcessing) {
-            factory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
-            // don't be overly DTD-unfriendly forcing http://apache.org/xml/features/disallow-doctype-decl
-            factory.setFeature("http://saxon.sf.net/feature/parserFeature?uri=http://xml.org/sax/features/external-parameter-entities", false);
-            factory.setFeature("http://saxon.sf.net/feature/parserFeature?uri=http://xml.org/sax/features/external-general-entities", false);
-        }
-
-        if(isFilename) {
-            return factory.newTemplates(new StreamSource(path));
-        } else {
-            final String coordinateKey = lookupService.get().getRequiredKeys().iterator().next();
-            final Optional<String> attributeValue = lookupService.get().lookup(Collections.singletonMap(coordinateKey, path));
-            if (attributeValue.isPresent() && StringUtils.isNotBlank(attributeValue.get())) {
-                return factory.newTemplates(new StreamSource(new ByteArrayInputStream(attributeValue.get().getBytes(StandardCharsets.UTF_8))));
-            } else {
-                throw new TransformerConfigurationException("No XSLT definition is associated to " + path + " in the lookup controller service.");
-            }
-        }
-    }
-
     @OnScheduled
     public void onScheduled(final ProcessContext context) {
         final ComponentLog logger = getLogger();
@@ -290,13 +263,13 @@ public class TransformXml extends AbstractProcessor {
         if (cacheSize > 0) {
             CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
             if (cacheTTL > 0) {
-                cacheBuilder = cacheBuilder.expireAfterAccess(cacheTTL, TimeUnit.SECONDS);
+                cacheBuilder.expireAfterAccess(cacheTTL, TimeUnit.SECONDS);
             }
 
             cache = cacheBuilder.build(
                     new CacheLoader<String, Templates>() {
                         @Override
-                        public Templates load(String path) throws TransformerConfigurationException, LookupFailureException {
+                        public Templates load(final String path) throws TransformerConfigurationException, LookupFailureException {
                             return newTemplates(context, path);
                         }
                     });
@@ -313,86 +286,95 @@ public class TransformXml extends AbstractProcessor {
             return;
         }
 
-        final ComponentLog logger = getLogger();
         final StopWatch stopWatch = new StopWatch(true);
         final String path = context.getProperty(XSLT_FILE_NAME).isSet()
                 ? context.getProperty(XSLT_FILE_NAME).evaluateAttributeExpressions(original).getValue()
                         : context.getProperty(XSLT_CONTROLLER_KEY).evaluateAttributeExpressions(original).getValue();
-        final Boolean indentOutput = context.getProperty(INDENT_OUTPUT).asBoolean();
-        lookupService.set(context.getProperty(XSLT_CONTROLLER).asControllerService(LookupService.class));
 
         try {
-            FlowFile transformed = session.write(original, new StreamCallback() {
-                @Override
-                public void process(final InputStream rawIn, final OutputStream out) throws IOException {
-                    try (final InputStream in = new BufferedInputStream(rawIn)) {
-                        final Templates templates;
-                        if (cache != null) {
-                            templates = cache.get(path);
-                        } else {
-                            templates = newTemplates(context, path);
-                        }
+            final FlowFile transformed = session.write(original, (inputStream, outputStream) -> {
+                try (final InputStream bufferedInputStream = new BufferedInputStream(inputStream)) {
+                    final Templates templates;
+                    if (cache == null) {
+                        templates = newTemplates(context, path);
+                    } else {
+                        templates = cache.get(path);
+                    }
 
-                        final Transformer transformer = templates.newTransformer();
-                        transformer.setOutputProperty(OutputKeys.INDENT, (indentOutput ? "yes" : "no"));
+                    final Transformer transformer = templates.newTransformer();
+                    final String indentProperty = context.getProperty(INDENT_OUTPUT).asBoolean() ? "yes" : "no";
+                    transformer.setOutputProperty(OutputKeys.INDENT, indentProperty);
 
-                        // pass all dynamic properties to the transformer
-                        for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
-                            if (entry.getKey().isDynamic()) {
-                                String value = context.newPropertyValue(entry.getValue()).evaluateAttributeExpressions(original).getValue();
-                                transformer.setParameter(entry.getKey().getName(), value);
-                            }
+                    // pass all dynamic properties to the transformer
+                    for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
+                        if (entry.getKey().isDynamic()) {
+                            String value = context.newPropertyValue(entry.getValue()).evaluateAttributeExpressions(original).getValue();
+                            transformer.setParameter(entry.getKey().getName(), value);
                         }
-
-                        // use a StreamSource with Saxon
-                        StreamSource source = new StreamSource(in);
-                        StreamResult result = new StreamResult(out);
-                        transformer.transform(source, result);
-                    } catch (final Exception e) {
-                        throw new IOException(e);
                     }
+
+                    final Source source = new StreamSource(bufferedInputStream);
+                    final Result result = new StreamResult(outputStream);
+                    transformer.transform(source, result);
+                } catch (final Exception e) {
+                    throw new IOException(String.format("XSLT Source Path [%s] Transform Failed", path), e);
                 }
             });
             session.transfer(transformed, REL_SUCCESS);
             session.getProvenanceReporter().modifyContent(transformed, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
-            logger.info("Transformed {}", new Object[]{original});
-        } catch (ProcessException e) {
-            logger.error("Unable to transform {} due to {}", new Object[]{original, e});
+            getLogger().info("Transformation Completed {}", original);
+        } catch (final ProcessException e) {
+            getLogger().error("Transformation Failed", original, e);
             session.transfer(original, REL_FAILURE);
         }
     }
 
-    @SuppressWarnings("unused")
-    private static final class XsltValidator implements Validator {
+    @SuppressWarnings("unchecked")
+    private Templates newTemplates(final ProcessContext context, final String path) throws TransformerConfigurationException, LookupFailureException {
+        final boolean secureProcessing = context.getProperty(SECURE_PROCESSING).asBoolean();
+        final TransformerFactory transformerFactory = getTransformerFactory(secureProcessing);
+        final LookupService<String> lookupService = context.getProperty(XSLT_CONTROLLER).asControllerService(LookupService.class);
+        final boolean filePath = context.getProperty(XSLT_FILE_NAME).isSet();
+        final StreamSource templateSource = getTemplateSource(lookupService, path, filePath);
+        final Source configuredTemplateSource = secureProcessing ? getSecureSource(templateSource) : templateSource;
+        return transformerFactory.newTemplates(configuredTemplateSource);
+    }
 
-        private volatile Tuple<String, ValidationResult> cachedResult;
+    private TransformerFactory getTransformerFactory(final boolean secureProcessing) throws TransformerConfigurationException {
+        final TransformerFactory factory = TransformerFactory.newInstance();
+        if (secureProcessing) {
+            factory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
+            factory.setFeature("http://saxon.sf.net/feature/parserFeature?uri=http://xml.org/sax/features/external-parameter-entities", false);
+            factory.setFeature("http://saxon.sf.net/feature/parserFeature?uri=http://xml.org/sax/features/external-general-entities", false);
+        }
+        return factory;
+    }
 
-        @Override
-        public ValidationResult validate(final String subject, final String input, final ValidationContext validationContext) {
-            final Tuple<String, ValidationResult> lastResult = this.cachedResult;
-            if (lastResult != null && lastResult.getKey().equals(input)) {
-                return lastResult.getValue();
+    private StreamSource getTemplateSource(final LookupService<String> lookupService, final String path, final boolean filePath) throws LookupFailureException {
+        final StreamSource streamSource;
+        if (filePath) {
+            streamSource = new StreamSource(path);
+        } else {
+            final String coordinateKey = lookupService.getRequiredKeys().iterator().next();
+            final Map<String, Object> coordinates = Collections.singletonMap(coordinateKey, path);
+            final Optional<String> foundSource = lookupService.lookup(coordinates);
+            if (foundSource.isPresent() && StringUtils.isNotBlank(foundSource.get())) {
+                final String source = foundSource.get();
+                final Reader reader = new StringReader(source);
+                streamSource = new StreamSource(reader);
             } else {
-                String error = null;
-                final File stylesheet = new File(input);
-                final TransformerFactory tFactory = new net.sf.saxon.TransformerFactoryImpl();
-                final StreamSource styleSource = new StreamSource(stylesheet);
-
-                try {
-                    tFactory.newTransformer(styleSource);
-                } catch (final Exception e) {
-                    error = e.toString();
-                }
-
-                this.cachedResult = new Tuple<>(input, new ValidationResult.Builder()
-                        .input(input)
-                        .subject(subject)
-                        .valid(error == null)
-                        .explanation(error)
-                        .build());
-                return this.cachedResult.getValue();
+                throw new LookupFailureException(String.format("XSLT Source Path [%s] not found in Lookup Service", path));
             }
         }
+        return streamSource;
     }
 
+    private Source getSecureSource(final StreamSource streamSource) throws TransformerConfigurationException {
+        try {
+            final XMLStreamReader streamReader = XmlUtils.createSafeReader(streamSource);
+            return new StAXSource(streamReader);
+        } catch (final XMLStreamException e) {
+            throw new TransformerConfigurationException("XSLT Source Stream Reader creation failed", e);
+        }
+    }
 }
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTransformXml.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTransformXml.java
index 3d5f1e0..8bcef7a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTransformXml.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTransformXml.java
@@ -17,7 +17,6 @@
 package org.apache.nifi.processors.standard;
 
 import java.io.BufferedReader;
-import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
@@ -36,14 +35,14 @@ import org.junit.Test;
 public class TestTransformXml {
 
     @Test
-    public void testStylesheetNotFound() throws IOException {
+    public void testStylesheetNotFound() {
         final TestRunner controller = TestRunners.newTestRunner(TransformXml.class);
         controller.setProperty(TransformXml.XSLT_FILE_NAME, "/no/path/to/math.xsl");
         controller.assertNotValid();
     }
 
     @Test
-    public void testNonXmlContent() throws IOException {
+    public void testNonXmlContent() {
         final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
         runner.setProperty(TransformXml.XSLT_FILE_NAME, "src/test/resources/TestTransformXml/math.xsl");
 
@@ -89,10 +88,10 @@ public class TestTransformXml {
         builder.append("<data>\n");
 
         try(BufferedReader reader = new BufferedReader(new InputStreamReader(
-                new FileInputStream(new File("src/test/resources/TestTransformXml/tokens.csv"))))){
+                new FileInputStream("src/test/resources/TestTransformXml/tokens.csv")))){
 
 
-            String line = null;
+            String line;
             while ((line = reader.readLine()) != null) {
                 builder.append(line).append("\n");
             }
@@ -144,7 +143,7 @@ public class TestTransformXml {
     }
 
     @Test
-    public void testTransformBothControllerFileNotValid() throws IOException, InitializationException {
+    public void testTransformBothControllerFileNotValid() throws InitializationException {
         final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
         runner.setProperty(TransformXml.XSLT_FILE_NAME, "src/test/resources/TestTransformXml/math.xsl");
 
@@ -159,14 +158,14 @@ public class TestTransformXml {
     }
 
     @Test
-    public void testTransformNoneControllerFileNotValid() throws IOException, InitializationException {
+    public void testTransformNoneControllerFileNotValid() {
         final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
         runner.setProperty(TransformXml.CACHE_SIZE, "0");
         runner.assertNotValid();
     }
 
     @Test
-    public void testTransformControllerNoKey() throws IOException, InitializationException {
+    public void testTransformControllerNoKey() throws InitializationException {
         final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
 
         final SimpleKeyValueLookupService service = new SimpleKeyValueLookupService();
@@ -267,4 +266,35 @@ public class TestTransformXml {
         transformed.assertContentEquals(expectedContent);
     }
 
+    @Test
+    public void testTransformSecureProcessingEnabledXmlWithEntity() {
+        final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
+        runner.setProperty(TransformXml.XSLT_FILE_NAME, "src/test/resources/TestTransformXml/doc-node.xsl");
+        runner.setProperty(TransformXml.INDENT_OUTPUT, Boolean.FALSE.toString());
+
+        final String input = "<!DOCTYPE doc [<!ENTITY uri SYSTEM \"http://127.0.0.1\" >]><doc>&uri;</doc>";
+        runner.enqueue(input);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(TransformXml.REL_SUCCESS);
+        final MockFlowFile transformed = runner.getFlowFilesForRelationship(TransformXml.REL_SUCCESS).get(0);
+
+        final String expected = "<?xml version=\"1.0\" encoding=\"UTF-8\"?><doc xmlns:xs=\"http://www.w3.org/2001/XMLSchema\"/>";
+        transformed.assertContentEquals(expected);
+    }
+
+    @Test
+    public void testTransformSecureProcessingEnabledXslWithEntity() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
+        runner.setProperty(TransformXml.XSLT_FILE_NAME, "src/test/resources/TestTransformXml/doctype-entity-file-uri.xsl");
+        runner.setProperty(TransformXml.INDENT_OUTPUT, Boolean.FALSE.toString());
+
+        runner.enqueue(Paths.get("src/test/resources/TestTransformXml/doctype-entity-file-uri.xsl"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(TransformXml.REL_SUCCESS);
+        final MockFlowFile transformed = runner.getFlowFilesForRelationship(TransformXml.REL_SUCCESS).get(0);
+
+        transformed.assertContentEquals("<?xml version=\"1.0\" encoding=\"UTF-8\"?>");
+    }
 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestTransformXml/doc-node.xsl b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestTransformXml/doc-node.xsl
new file mode 100644
index 0000000..18c8e18
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestTransformXml/doc-node.xsl
@@ -0,0 +1,22 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<xsl:stylesheet version="2.0"
+                xmlns:xsl="http://www.w3.org/1999/XSL/Transform"
+                xmlns:xs="http://www.w3.org/2001/XMLSchema">
+    <xsl:template match="doc">
+        <doc><xsl:value-of select="node()"/></doc>
+    </xsl:template>
+</xsl:stylesheet>
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestTransformXml/doctype-entity-file-uri.xsl b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestTransformXml/doctype-entity-file-uri.xsl
new file mode 100644
index 0000000..12ce5c5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestTransformXml/doctype-entity-file-uri.xsl
@@ -0,0 +1,19 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<!DOCTYPE stylesheet [<!ENTITY % uri SYSTEM 'http://127.0.0.1'> %uri;]>
+<xsl:stylesheet version="2.0"
+                xmlns:xsl="http://www.w3.org/1999/XSL/Transform"
+                xmlns:xs="http://www.w3.org/2001/XMLSchema"/>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestTransformXml/math.html b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestTransformXml/math.html
index 254d6a3..129ed34 100755
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestTransformXml/math.html
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestTransformXml/math.html
@@ -1,4 +1,4 @@
-<HTML xmlns:xs="http://www.w3.org/2001/XMLSchema">
+<!DOCTYPE HTML><HTML xmlns:xs="http://www.w3.org/2001/XMLSchema">
    <H1>Test for mod</H1>
    <HR>
    <P>Should say "1": 1</P>
diff --git a/nifi-nar-bundles/nifi-standard-bundle/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/pom.xml
index 9b7030f..d1673ae 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/pom.xml
@@ -233,7 +233,7 @@
             <dependency>
                 <groupId>net.sf.saxon</groupId>
                 <artifactId>Saxon-HE</artifactId>
-                <version>9.6.0-5</version>
+                <version>10.6</version>
             </dependency>
             <dependency>
                 <groupId>joda-time</groupId>