You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2022/02/25 19:57:17 UTC

[nifi] branch main updated: NIFI-9072: improvements to ValidateXML including validate XML in attr… (#5324)

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

markap14 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 4e3871f  NIFI-9072: improvements to ValidateXML including validate XML in attr… (#5324)
4e3871f is described below

commit 4e3871fec751cfcccb3a34a724d338321e1e8067
Author: markobean <ma...@gmail.com>
AuthorDate: Fri Feb 25 14:57:04 2022 -0500

    NIFI-9072: improvements to ValidateXML including validate XML in attr… (#5324)
    
    NIFI-9072: Improvements to ValidateXML including validate XML in attributes and non-schema, well-formed XML only validation, and additional documentation
    - Updated description and added System Resource Consideration to discourage using large XML documents in attributes
    - Migrate from XML DocumentBuilderFactory to SafeXMLConfiguration
---
 .../nifi/processors/standard/ValidateXml.java      | 110 +++++++++++++++----
 .../additionalDetails.html                         | 104 ++++++++++++++++++
 .../nifi/processors/standard/TestValidateXml.java  | 122 +++++++++++++++++++--
 3 files changed, 305 insertions(+), 31 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java
index ca2f30f..b7184d4 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java
@@ -21,6 +21,8 @@ import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -38,15 +40,21 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.xml.SafeXMLConfiguration;
 import org.xml.sax.SAXException;
 
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.transform.stream.StreamSource;
 import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 import javax.xml.validation.Validator;
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URL;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
@@ -64,26 +72,40 @@ import java.util.concurrent.atomic.AtomicReference;
     @WritesAttribute(attribute = "validatexml.invalid.error", description = "If the flow file is routed to the invalid relationship "
             + "the attribute will contain the error message resulting from the validation failure.")
 })
-@CapabilityDescription("Validates the contents of FlowFiles against a user-specified XML Schema file")
+@CapabilityDescription("Validates XML contained in a FlowFile. By default, the XML is contained in the FlowFile content. If the 'XML Source Attribute' property is set, the XML to be validated "
+        + "is contained in the specified attribute. It is not recommended to use attributes to hold large XML documents; doing so could adversely affect system performance. "
+        + "Full schema validation is performed if the processor is configured with the XSD schema details. Otherwise, the only validation performed is "
+        + "to ensure the XML syntax is correct and well-formed, e.g. all opening tags are properly closed.")
+@SystemResourceConsideration(resource = SystemResource.MEMORY, description = "While this processor supports processing XML within attributes, it is strongly discouraged to hold "
+        + "large amounts of data in attributes. In general, attribute values should be as small as possible and hold no more than a couple hundred characters.")
 public class ValidateXml extends AbstractProcessor {
 
     public static final String ERROR_ATTRIBUTE_KEY = "validatexml.invalid.error";
 
     public static final PropertyDescriptor SCHEMA_FILE = new PropertyDescriptor.Builder()
             .name("Schema File")
-            .description("The path to the Schema file that is to be used for validation")
-            .required(true)
+            .displayName("Schema File")
+            .description("The file path or URL to the XSD Schema file that is to be used for validation. If this property is blank, only XML syntax/structure will be validated.")
+            .required(false)
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE, ResourceType.URL)
             .build();
+    public static final PropertyDescriptor XML_SOURCE_ATTRIBUTE = new PropertyDescriptor.Builder()
+            .name("XML Source Attribute")
+            .displayName("XML Source Attribute")
+            .description("The name of the attribute containing XML to be validated. If this property is blank, the FlowFile content will be validated.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
+            .build();
 
     public static final Relationship REL_VALID = new Relationship.Builder()
             .name("valid")
-            .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+            .description("FlowFiles that are successfully validated against the schema, if provided, or verified to be well-formed XML are routed to this relationship")
             .build();
     public static final Relationship REL_INVALID = new Relationship.Builder()
             .name("invalid")
-            .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+            .description("FlowFiles that are not valid according to the specified schema or contain invalid XML are routed to this relationship")
             .build();
 
     private static final String SCHEMA_LANGUAGE = "http://www.w3.org/2001/XMLSchema";
@@ -96,6 +118,7 @@ public class ValidateXml extends AbstractProcessor {
     protected void init(final ProcessorInitializationContext context) {
         final List<PropertyDescriptor> properties = new ArrayList<>();
         properties.add(SCHEMA_FILE);
+        properties.add(XML_SOURCE_ATTRIBUTE);
         this.properties = Collections.unmodifiableList(properties);
 
         final Set<Relationship> relationships = new HashSet<>();
@@ -116,13 +139,13 @@ public class ValidateXml extends AbstractProcessor {
 
     @OnScheduled
     public void parseSchema(final ProcessContext context) throws SAXException {
-        try {
+        if (context.getProperty(SCHEMA_FILE).isSet()) {
             final URL url = context.getProperty(SCHEMA_FILE).evaluateAttributeExpressions().asResource().asURL();
             final SchemaFactory schemaFactory = SchemaFactory.newInstance(SCHEMA_LANGUAGE);
             final Schema schema = schemaFactory.newSchema(url);
-            this.schemaRef.set(schema);
-        } catch (final SAXException e) {
-            throw e;
+            schemaRef.set(schema);
+        } else {
+            schemaRef.set(null);
         }
     }
 
@@ -134,35 +157,74 @@ public class ValidateXml extends AbstractProcessor {
         }
 
         final Schema schema = schemaRef.get();
-        final Validator validator = schema.newValidator();
+        final Validator validator = schema == null ? null : schema.newValidator();
         final ComponentLog logger = getLogger();
+        final boolean attributeContainsXML = context.getProperty(XML_SOURCE_ATTRIBUTE).isSet();
 
         for (FlowFile flowFile : flowFiles) {
             final AtomicBoolean valid = new AtomicBoolean(true);
-            final AtomicReference<Exception> exception = new AtomicReference<Exception>(null);
-
-            session.read(flowFile, new InputStreamCallback() {
-                @Override
-                public void process(final InputStream in) throws IOException {
-                    try {
-                        validator.validate(new StreamSource(in));
-                    } catch (final IllegalArgumentException | SAXException e) {
-                        valid.set(false);
-                        exception.set(e);
-                    }
+            final AtomicReference<Exception> exception = new AtomicReference<>(null);
+            SafeXMLConfiguration safeXMLConfiguration = new SafeXMLConfiguration();
+            safeXMLConfiguration.setValidating(false);
+
+            try {
+                DocumentBuilder docBuilder = safeXMLConfiguration.createDocumentBuilder();
+
+                if (attributeContainsXML) {
+                    // If XML source attribute is set, validate attribute value
+                    String xml = flowFile.getAttribute(context.getProperty(XML_SOURCE_ATTRIBUTE).evaluateAttributeExpressions().getValue());
+                    ByteArrayInputStream bais = new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+
+                    validate(validator, docBuilder, bais);
+                } else {
+                    // If XML source attribute is not set, validate flowfile content
+                    session.read(flowFile, new InputStreamCallback() {
+                        @Override
+                        public void process(final InputStream in) throws IOException {
+                            try {
+                                validate(validator, docBuilder, in);
+                            } catch (final IllegalArgumentException | SAXException e) {
+                                valid.set(false);
+                                exception.set(e);
+                            }
+                        }
+                    });
                 }
-            });
+            } catch (final IllegalArgumentException | SAXException | ParserConfigurationException | IOException e) {
+                valid.set(false);
+                exception.set(e);
+            }
 
+            // determine source location of XML for logging purposes
+            String xmlSource = attributeContainsXML ? "attribute '" + context.getProperty(XML_SOURCE_ATTRIBUTE).evaluateAttributeExpressions().getValue() + "'" : "content";
             if (valid.get()) {
-                logger.debug("Successfully validated {} against schema; routing to 'valid'", new Object[]{flowFile});
+                if (context.getProperty(SCHEMA_FILE).isSet()) {
+                    logger.debug("Successfully validated XML in {} of {} against schema; routing to 'valid'", xmlSource, flowFile);
+                } else {
+                    logger.debug("Successfully validated XML is well-formed in {} of {}; routing to 'valid'", xmlSource, flowFile);
+                }
                 session.getProvenanceReporter().route(flowFile, REL_VALID);
                 session.transfer(flowFile, REL_VALID);
             } else {
                 flowFile = session.putAttribute(flowFile, ERROR_ATTRIBUTE_KEY, exception.get().getLocalizedMessage());
-                logger.info("Failed to validate {} against schema due to {}; routing to 'invalid'", new Object[]{flowFile, exception.get().getLocalizedMessage()});
+                if (context.getProperty(SCHEMA_FILE).isSet()) {
+                    logger.info("Failed to validate XML in {} of {} against schema due to {}; routing to 'invalid'", xmlSource, flowFile, exception.get().getLocalizedMessage());
+                } else {
+                    logger.info("Failed to validate XML is well-formed in {} of {} due to {}; routing to 'invalid'", xmlSource, flowFile, exception.get().getLocalizedMessage());
+                }
                 session.getProvenanceReporter().route(flowFile, REL_INVALID);
                 session.transfer(flowFile, REL_INVALID);
             }
         }
     }
+
+    private void validate(final Validator validator, final DocumentBuilder docBuilder, final InputStream in) throws IllegalArgumentException, SAXException, IOException {
+        if (validator != null) {
+            // If schema is provided, validator will be non-null
+            validator.validate(new StreamSource(in));
+        } else {
+            // Only verify that the XML is well-formed; no schema check
+            docBuilder.parse(in);
+        }
+    }
 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ValidateXml/additionalDetails.html b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ValidateXml/additionalDetails.html
new file mode 100644
index 0000000..2b5df57
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ValidateXml/additionalDetails.html
@@ -0,0 +1,104 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>ValidateCsv</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+
+<body>
+
+<h2>Usage Information</h2>
+
+<p>
+    In order to fully validate XML, a schema must be provided. The ValidateXML processor allows the schema to be specified in the
+    property 'Schema File'. The following example illustrates how an XSD schema and XML data work together.
+</p>
+
+<p>Example XSD specification</p>
+<p>
+    <pre>
+        &ltxs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" targetNamespace="http://namespace/1" xmlns:tns="http://namespace/1" elementFormDefault="unqualified"&gt
+            &ltxs:element name="bundle" type="tns:BundleType"&gt&lt/xs:element&gt
+
+            &ltxs:complexType name="BundleType"&gt
+                &ltxs:sequence&gt
+                    &ltxs:element name="node" type="tns:NodeType" maxOccurs="unbounded" minOccurs="0"&gt&lt/xs:element&gt
+                &lt/xs:sequence&gt
+            &lt/xs:complexType&gt
+            &ltxs:complexType name="NodeType"&gt
+                &ltxs:sequence&gt
+                    &ltxs:element name="subNode" type="tns:SubNodeType" maxOccurs="unbounded" minOccurs="0"&gt&lt/xs:element&gt
+                &lt/xs:sequence&gt
+            &lt/xs:complexType&gt
+            &ltxs:complexType name="SubNodeType"&gt
+                &ltxs:sequence&gt
+                    &ltxs:element name="value" type="xs:string"&gt&lt/xs:element&gt
+                &lt/xs:sequence&gt
+            &lt/xs:complexType&gt
+        &lt/xs:schema&gt
+    </pre>
+</p>
+
+<p>Given the schema defined in the above XSD, the following are valid XML data.</p>
+
+<p>
+    <pre>
+        &ltns:bundle xmlns:ns="http://namespace/1"&gt
+            &ltnode&gt
+                &ltsubNode&gt
+                    &ltvalue&gtHello&lt/value&gt
+                &lt/subNode&gt
+                &ltsubNode&gt
+                    &ltvalue&gtWorld!&lt/value&gt
+                &lt/subNode&gt
+            &lt/node&gt
+        &lt/ns:bundle&gt
+    </pre>
+</p>
+<p>
+    <pre>
+        &ltns:bundle xmlns:ns="http://namespace/1"&gt
+            &ltnode&gt
+                &ltsubNode&gt
+                    &ltvalue&gtHello World!&lt/value&gt
+                &lt/subNode&gt
+            &lt/node&gt
+        &lt/ns:bundle&gt
+    </pre>
+</p>
+<p>The following are invalid XML data. The resulting <span style="font-family: monospace">validatexml.invalid.error</span> attribute is shown.
+    <pre>
+        &ltns:bundle xmlns:ns="http://namespace/1"&gt
+            &ltnode&gtHello World!&lt/node&gt
+        &lt/ns:bundle&gt
+<br>    validatexml.invalid.error: cvc-complex-type.2.3: Element 'node' cannot have character [children], because the type's content type is element-only.
+    </pre>
+</p>
+<p>
+    <pre>
+        &ltns:bundle xmlns:ns="http://namespace/1"&gt
+            &ltnode&gt
+                &ltvalue&gtHello World!&lt/value&gt
+            &lt/node&gt
+        &lt/ns:bundle&gt
+    <br>    validatexml.invalid.error: cvc-complex-type.2.4.a: Invalid content was found starting with element 'value'. One of '{subNode}' is expected.
+    </pre>
+
+</p>
+</body>
+</html>
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateXml.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateXml.java
index b41f869..3a03b22 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateXml.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateXml.java
@@ -18,16 +18,23 @@ package org.apache.nifi.processors.standard;
 
 import java.io.IOException;
 import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Test;
-import org.xml.sax.SAXException;
 
 public class TestValidateXml {
 
+    private static final String VALID_XML = "<ns:bundle xmlns:ns=\"http://namespace/1\"><node><subNode><value>Hello</value></subNode>" +
+            "<subNode><value>World!</value></subNode></node></ns:bundle>";
+    private static final String INVALID_XML = "<this>is an invalid</xml>";
+    private static final String NONCOMPLIANT_XML = "<ns:bundle xmlns:ns=\"http://namespace/1\"><this>is good XML, but violates schema</this></ns:bundle>";
+
     @Test
-    public void testValid() throws IOException, SAXException {
+    public void testValid() throws IOException {
+        // Valid XML in FF content, XSD provided
         final TestRunner runner = TestRunners.newTestRunner(new ValidateXml());
         runner.setProperty(ValidateXml.SCHEMA_FILE, "src/test/resources/TestXml/XmlBundle.xsd");
 
@@ -38,11 +45,20 @@ public class TestValidateXml {
     }
 
     @Test
-    public void testInvalid() throws IOException, SAXException {
+    public void testInvalid() {
+        // Invalid XML in FF content, XSD provided
         final TestRunner runner = TestRunners.newTestRunner(new ValidateXml());
         runner.setProperty(ValidateXml.SCHEMA_FILE, "src/test/resources/TestXml/XmlBundle.xsd");
 
-        runner.enqueue("<this>is an invalid</xml>");
+        runner.enqueue(INVALID_XML);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ValidateXml.REL_INVALID, 1);
+        runner.assertAllFlowFilesContainAttribute(ValidateXml.REL_INVALID, ValidateXml.ERROR_ATTRIBUTE_KEY);
+
+        runner.clearTransferState();
+        runner.enqueue(NONCOMPLIANT_XML);
+
         runner.run();
 
         runner.assertAllFlowFilesTransferred(ValidateXml.REL_INVALID, 1);
@@ -50,7 +66,7 @@ public class TestValidateXml {
     }
 
     @Test
-    public void testValidEL() throws IOException, SAXException {
+    public void testValidEL() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ValidateXml());
         runner.setProperty(ValidateXml.SCHEMA_FILE, "${my.schema}");
         runner.setVariable("my.schema", "src/test/resources/TestXml/XmlBundle.xsd");
@@ -62,15 +78,107 @@ public class TestValidateXml {
     }
 
     @Test(expected = AssertionError.class)
-    public void testInvalidEL() throws IOException, SAXException {
+    public void testInvalidEL() {
         final TestRunner runner = TestRunners.newTestRunner(new ValidateXml());
         runner.setProperty(ValidateXml.SCHEMA_FILE, "${my.schema}");
 
-        runner.enqueue("<this>is an invalid</xml>");
+        runner.enqueue(INVALID_XML);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ValidateXml.REL_INVALID, 1);
+        runner.assertAllFlowFilesContainAttribute(ValidateXml.REL_INVALID, ValidateXml.ERROR_ATTRIBUTE_KEY);
+    }
+
+    @Test
+    public void testValidXMLAttributeWithSchema()  {
+        // Valid XML in FF attribute, XSD provided
+        final TestRunner runner = TestRunners.newTestRunner(new ValidateXml());
+        runner.setProperty(ValidateXml.SCHEMA_FILE, "src/test/resources/TestXml/XmlBundle.xsd");
+        runner.setProperty(ValidateXml.XML_SOURCE_ATTRIBUTE, "xml.attribute");
+        Map<String, String> attributes = new HashMap<>();
+        attributes.put("xml.attribute", VALID_XML);
+
+        runner.enqueue("XML is in attribute, not content", attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ValidateXml.REL_VALID, 1);
+    }
+
+    @Test
+    public void testInvalidXMLAttributeWithSchema() {
+        // Invalid XML in FF attribute, XSD provided
+        final TestRunner runner = TestRunners.newTestRunner(new ValidateXml());
+        runner.setProperty(ValidateXml.SCHEMA_FILE, "src/test/resources/TestXml/XmlBundle.xsd");
+        runner.setProperty(ValidateXml.XML_SOURCE_ATTRIBUTE, "xml.attribute");
+        Map<String, String> attributes = new HashMap<>();
+        attributes.put("xml.attribute", INVALID_XML);
+
+        runner.enqueue("flowfile content is irrelevant", attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ValidateXml.REL_INVALID, 1);
+        runner.assertAllFlowFilesContainAttribute(ValidateXml.REL_INVALID, ValidateXml.ERROR_ATTRIBUTE_KEY);
+
+        runner.clearTransferState();
+        attributes.clear();
+        attributes.put("xml.attribute", NONCOMPLIANT_XML);
+
+        runner.enqueue("flowfile content is irrelevant", attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ValidateXml.REL_INVALID, 1);
+        runner.assertAllFlowFilesContainAttribute(ValidateXml.REL_INVALID, ValidateXml.ERROR_ATTRIBUTE_KEY);
+    }
+
+    @Test
+    public void testValidXMLAttributeStructure() {
+        // Valid XML in FF attribute, no XSD provided
+        final TestRunner runner = TestRunners.newTestRunner(new ValidateXml());
+        runner.setProperty(ValidateXml.XML_SOURCE_ATTRIBUTE, "xml.attribute");
+        Map<String, String> attributes = new HashMap<>();
+        attributes.put("xml.attribute", VALID_XML);
+
+        runner.enqueue("XML is in attribute, not content", attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ValidateXml.REL_VALID, 1);
+    }
+
+    @Test
+    public void testInvalidXMLAttributeStructure() {
+        // Invalid XML in FF attribute, no XSD provided
+        final TestRunner runner = TestRunners.newTestRunner(new ValidateXml());
+        runner.setProperty(ValidateXml.XML_SOURCE_ATTRIBUTE, "xml.attribute");
+        Map<String, String> attributes = new HashMap<>();
+        attributes.put("xml.attribute", INVALID_XML);
+
+        runner.enqueue("XML is in attribute, not content", attributes);
         runner.run();
 
         runner.assertAllFlowFilesTransferred(ValidateXml.REL_INVALID, 1);
         runner.assertAllFlowFilesContainAttribute(ValidateXml.REL_INVALID, ValidateXml.ERROR_ATTRIBUTE_KEY);
     }
 
+    @Test
+    public void testValidXMLContentStructure() throws IOException {
+        // Valid XML in FF content, no XSD provided
+        final TestRunner runner = TestRunners.newTestRunner(new ValidateXml());
+
+        runner.enqueue(Paths.get("src/test/resources/TestXml/xml-snippet.xml"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ValidateXml.REL_VALID, 1);
+    }
+
+    @Test
+    public void testInvalidXMLContentStructure() {
+        // Invalid XML in FF content, no XSD provided
+        final TestRunner runner = TestRunners.newTestRunner(new ValidateXml());
+
+        runner.enqueue(INVALID_XML);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ValidateXml.REL_INVALID, 1);
+        runner.assertAllFlowFilesContainAttribute(ValidateXml.REL_INVALID, ValidateXml.ERROR_ATTRIBUTE_KEY);
+    }
 }