You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/04/13 03:51:28 UTC

[GitHub] [nifi] exceptionfactory opened a new pull request, #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons

exceptionfactory opened a new pull request, #5962:
URL: https://github.com/apache/nifi/pull/5962

   #### Description of PR
   
   NIFI-9901 Adds the `nifi-xml-processing` module to `nifi-commons` and refactors XML handling to use common components.
   
   Components in `nifi-xml-processing` replace the usage of `XmlUtils` in `nifi-security-utils` with interfaces and implementation classes specific to each type of XML processing. The module includes the following interfaces derived from standard Java XML components:
   
   - `DocumentProvider` for DOM Documents
   - `InputSourceParser` for SAX Parsing
   - `XMLEventReaderProvider` and `XMLStreamReaderProvider` for StAX Parsing
   - `SchemaValidator` for XML Schema Validation
   
   The `nifi-xml-processing` Maven configuration includes `spotbugs-maven-plugin` with `findsecbugs-plugin` to analyze components for XML processing vulnerabilities during the build.
   
   General changes include refactoring references to `javax.xml.parsers.DocumentBuilderFactory` and `javax.xml.stream.XMLInputFactory` throughout the system to use `nifi-xml-processing` components.
   
   Specific changes include updates to the `EvaluateXPath` and `EvaluateXQuery` Processors to disable Document Type Declaration Validation in the default configuration. Other adjustments include relocating Apache Commons Configuration 2 classes to `nifi-lookup-services`, which is the only reference to those components. This relocation allows `nifi-xml-processing` to avoid any external dependencies.
   
   As a result of this refactoring, `nifi-security-utils` no longer contains any XML processing components, which allows some referencing modules to avoid unnecessary transitive dependencies.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [X] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [X] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [X] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [X] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [X] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [X] Have you written or updated unit tests to verify your changes?
   - [X] Have you verified that the full build is successful on JDK 8?
   - [X] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #5962:
URL: https://github.com/apache/nifi/pull/5962#discussion_r849843776


##########
nifi-framework-api/src/main/java/org/apache/nifi/authorization/AbstractPolicyBasedAuthorizer.java:
##########
@@ -427,8 +429,7 @@ private PoliciesUsersAndGroups parsePoliciesUsersAndGroups(final String fingerpr
 
         final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
         try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
-            final DocumentBuilder docBuilder = createSafeDocumentBuilder();
-            final Document document = docBuilder.parse(in);
+            final Document document = parseFingerprint(in);

Review Comment:
   Good question! This is only remaining location with a local reference to the `DocumentBuilderFactory`. PR #5514 for NIFI-9069 included the direct implementation, as opposed to `XmlUtils` due to issues with class loading between `nifi-framework-api` and dependent modules. It might be possible to revisit the problem, but this change maintains a limited scope for `nifi-framework-api` module dependencies.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #5962:
URL: https://github.com/apache/nifi/pull/5962#discussion_r849840041


##########
nifi-commons/nifi-xml-processing/src/main/java/org/apache/nifi/xml/processing/parsers/StandardDocumentProvider.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.nifi.xml.processing.parsers;
+
+import org.apache.nifi.xml.processing.ProcessingException;
+import org.apache.nifi.xml.processing.ProcessingFeature;
+import org.w3c.dom.Document;
+import org.xml.sax.ErrorHandler;
+import org.xml.sax.SAXException;
+
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.validation.Schema;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+
+/**
+ * Standard implementation of Document Provider with secure processing enabled
+ */
+public class StandardDocumentProvider implements DocumentProvider {
+    private boolean namespaceAware;
+
+    private Schema schema;
+
+    private ErrorHandler errorHandler;
+
+    /**
+     * Set Error Handler
+     *
+     * @param errorHandler Error Handler
+     */
+    public void setErrorHandler(final ErrorHandler errorHandler) {
+        this.errorHandler = errorHandler;
+    }
+
+    /**
+     * Set Namespace Aware status on DocumentBuilderFactory
+     *
+     * @param namespaceAware Namespace Awareness
+     */
+    public void setNamespaceAware(final boolean namespaceAware) {
+        this.namespaceAware = namespaceAware;
+    }
+
+    /**
+     * Set Namespace Aware status on DocumentBuilderFactory
+     *
+     * @param schema Schema for validation or null to disable validation
+     */
+    public void setSchema(final Schema schema) {
+        this.schema = schema;
+    }
+
+    @Override
+    public Document newDocument() {
+        final DocumentBuilderFactory documentBuilderFactory = getDocumentBuilderFactory();
+
+        try {
+            documentBuilderFactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, ProcessingFeature.SECURE_PROCESSING.isEnabled());

Review Comment:
   The purpose of the `ProcessingFeature.SECURE_PROCESSING` reference was to define the property value in a central location that could be reused in multiple classes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] greyp9 commented on a diff in pull request #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #5962:
URL: https://github.com/apache/nifi/pull/5962#discussion_r849740132


##########
nifi-commons/nifi-xml-processing/src/main/java/org/apache/nifi/xml/processing/sax/StandardInputSourceParser.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.nifi.xml.processing.sax;
+
+import org.apache.nifi.xml.processing.ProcessingException;
+import org.apache.nifi.xml.processing.ProcessingFeature;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+import org.xml.sax.XMLReader;
+
+import javax.xml.XMLConstants;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * Standard implementation of Input Source Parser with secure processing enabled
+ */
+public class StandardInputSourceParser implements InputSourceParser {
+    private boolean namespaceAware;
+
+    /**
+     * Set Namespace Aware status on SAXParserFactory
+     *
+     * @param namespaceAware Namespace Aware status
+     */
+    public void setNamespaceAware(final boolean namespaceAware) {
+        this.namespaceAware = namespaceAware;
+    }
+
+    /**
+     * Parse Input Source using Content Handler
+     *
+     * @param inputSource Input Source to be parsed
+     * @param contentHandler Content Handler used during parsing
+     */
+    @Override
+    public void parse(final InputSource inputSource, final ContentHandler contentHandler) {
+        Objects.requireNonNull(inputSource, "InputSource required");
+        Objects.requireNonNull(contentHandler, "ContentHandler required");
+
+        try {
+            parseInputSource(inputSource, contentHandler);
+        } catch (final ParserConfigurationException|SAXException e) {
+            throw new ProcessingException("Parser Configuration failed", e);

Review Comment:
   This would encompass both the parser configuration and the parse operation.
   
   `Parser Configuration / Parse Operation failed`



##########
nifi-commons/nifi-xml-processing/src/main/java/org/apache/nifi/xml/processing/parsers/StandardDocumentProvider.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.nifi.xml.processing.parsers;
+
+import org.apache.nifi.xml.processing.ProcessingException;
+import org.apache.nifi.xml.processing.ProcessingFeature;
+import org.w3c.dom.Document;
+import org.xml.sax.ErrorHandler;
+import org.xml.sax.SAXException;
+
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.validation.Schema;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Objects;
+
+/**
+ * Standard implementation of Document Provider with secure processing enabled
+ */
+public class StandardDocumentProvider implements DocumentProvider {
+    private boolean namespaceAware;
+
+    private Schema schema;
+
+    private ErrorHandler errorHandler;
+
+    /**
+     * Set Error Handler
+     *
+     * @param errorHandler Error Handler
+     */
+    public void setErrorHandler(final ErrorHandler errorHandler) {
+        this.errorHandler = errorHandler;
+    }
+
+    /**
+     * Set Namespace Aware status on DocumentBuilderFactory
+     *
+     * @param namespaceAware Namespace Awareness
+     */
+    public void setNamespaceAware(final boolean namespaceAware) {
+        this.namespaceAware = namespaceAware;
+    }
+
+    /**
+     * Set Namespace Aware status on DocumentBuilderFactory
+     *
+     * @param schema Schema for validation or null to disable validation
+     */
+    public void setSchema(final Schema schema) {
+        this.schema = schema;
+    }
+
+    @Override
+    public Document newDocument() {
+        final DocumentBuilderFactory documentBuilderFactory = getDocumentBuilderFactory();
+
+        try {
+            documentBuilderFactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, ProcessingFeature.SECURE_PROCESSING.isEnabled());

Review Comment:
   What is the benefit of defining the value here to be a lookup?



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java:
##########
@@ -137,7 +138,7 @@ public class EvaluateXPath extends AbstractProcessor {
             .description("Specifies whether or not the XML content should be validated against the DTD.")
             .required(true)
             .allowableValues("true", "false")
-            .defaultValue("true")

Review Comment:
   Is there a little context for this change?



##########
nifi-framework-api/src/main/java/org/apache/nifi/authorization/AbstractPolicyBasedAuthorizer.java:
##########
@@ -427,8 +429,7 @@ private PoliciesUsersAndGroups parsePoliciesUsersAndGroups(final String fingerpr
 
         final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
         try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
-            final DocumentBuilder docBuilder = createSafeDocumentBuilder();
-            final Document document = docBuilder.parse(in);
+            final Document document = parseFingerprint(in);

Review Comment:
   Why doesn't this usage use the new `StandardDocumentProvider`?



##########
nifi-commons/nifi-xml-processing/src/main/java/org/apache/nifi/xml/processing/validation/StandardSchemaValidator.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.nifi.xml.processing.validation;
+
+import org.apache.nifi.xml.processing.ProcessingException;
+import org.apache.nifi.xml.processing.ProcessingFeature;
+import org.xml.sax.SAXException;
+
+import javax.xml.XMLConstants;
+import javax.xml.transform.Source;
+import javax.xml.validation.Schema;
+import javax.xml.validation.Validator;
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * Standard implementation of XML Schema Validator with secure processing enabled
+ */
+public class StandardSchemaValidator implements SchemaValidator {
+    /**
+     * Validate Source using Schema
+     *
+     * @param schema Schema source for Validator
+     * @param source Source to be validated
+     */
+    @Override
+    public void validate(final Schema schema, final Source source) {

Review Comment:
   Do we only support the specification of a single schema when validating a document?



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java:
##########
@@ -156,7 +152,7 @@ public class EvaluateXQuery extends AbstractProcessor {
             .description("Specifies whether or not the XML content should be validated against the DTD.")
             .required(true)
             .allowableValues("true", "false")
-            .defaultValue("true")
+            .defaultValue("false")

Review Comment:
   Why this change?



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java:
##########
@@ -162,10 +163,6 @@ public class EvaluateXPath extends AbstractProcessor {
 
     private final AtomicReference<XPathFactory> factoryRef = new AtomicReference<>();
 
-    static {

Review Comment:
   !



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] greyp9 closed pull request #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons

Posted by GitBox <gi...@apache.org>.
greyp9 closed pull request #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons
URL: https://github.com/apache/nifi/pull/5962


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #5962:
URL: https://github.com/apache/nifi/pull/5962#discussion_r849845697


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java:
##########
@@ -156,7 +152,7 @@ public class EvaluateXQuery extends AbstractProcessor {
             .description("Specifies whether or not the XML content should be validated against the DTD.")
             .required(true)
             .allowableValues("true", "false")
-            .defaultValue("true")
+            .defaultValue("false")

Review Comment:
   Disabling Document Type Validation in the default configuration provides a more secure starting point for new instances of the Processor. The implementation in `StandardDocumentProvider` provides standard security restrictions on Document Type Validation, so enabling the `Validate DTD` property is not the optimal configuration. Changing the default value to `false` retains the property for deployments where embedded DTD validation is desired.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #5962:
URL: https://github.com/apache/nifi/pull/5962#discussion_r849846738


##########
nifi-commons/nifi-xml-processing/src/main/java/org/apache/nifi/xml/processing/validation/StandardSchemaValidator.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.nifi.xml.processing.validation;
+
+import org.apache.nifi.xml.processing.ProcessingException;
+import org.apache.nifi.xml.processing.ProcessingFeature;
+import org.xml.sax.SAXException;
+
+import javax.xml.XMLConstants;
+import javax.xml.transform.Source;
+import javax.xml.validation.Schema;
+import javax.xml.validation.Validator;
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * Standard implementation of XML Schema Validator with secure processing enabled
+ */
+public class StandardSchemaValidator implements SchemaValidator {
+    /**
+     * Validate Source using Schema
+     *
+     * @param schema Schema source for Validator
+     * @param source Source to be validated
+     */
+    @Override
+    public void validate(final Schema schema, final Source source) {

Review Comment:
   Yes, the `Schema` object provides the `newValidator()` method, so this interface encapsulates that operation and sets standard properties on the `Validator`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on pull request #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on PR #5962:
URL: https://github.com/apache/nifi/pull/5962#issuecomment-1098442359

   Thanks for the initial feedback @greyp9! Will plan on updating the one exception message noted, as well as anything else, following additional feedback from testing.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #5962:
URL: https://github.com/apache/nifi/pull/5962#discussion_r849840721


##########
nifi-commons/nifi-xml-processing/src/main/java/org/apache/nifi/xml/processing/sax/StandardInputSourceParser.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.nifi.xml.processing.sax;
+
+import org.apache.nifi.xml.processing.ProcessingException;
+import org.apache.nifi.xml.processing.ProcessingFeature;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+import org.xml.sax.XMLReader;
+
+import javax.xml.XMLConstants;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * Standard implementation of Input Source Parser with secure processing enabled
+ */
+public class StandardInputSourceParser implements InputSourceParser {
+    private boolean namespaceAware;
+
+    /**
+     * Set Namespace Aware status on SAXParserFactory
+     *
+     * @param namespaceAware Namespace Aware status
+     */
+    public void setNamespaceAware(final boolean namespaceAware) {
+        this.namespaceAware = namespaceAware;
+    }
+
+    /**
+     * Parse Input Source using Content Handler
+     *
+     * @param inputSource Input Source to be parsed
+     * @param contentHandler Content Handler used during parsing
+     */
+    @Override
+    public void parse(final InputSource inputSource, final ContentHandler contentHandler) {
+        Objects.requireNonNull(inputSource, "InputSource required");
+        Objects.requireNonNull(contentHandler, "ContentHandler required");
+
+        try {
+            parseInputSource(inputSource, contentHandler);
+        } catch (final ParserConfigurationException|SAXException e) {
+            throw new ProcessingException("Parser Configuration failed", e);

Review Comment:
   That's a good point, will adjust to a more generalized `Parsing failed` message for the exception.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #5962: NIFI-9901 Add nifi-xml-processing to nifi-commons

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #5962:
URL: https://github.com/apache/nifi/pull/5962#discussion_r849849414


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java:
##########
@@ -137,7 +138,7 @@ public class EvaluateXPath extends AbstractProcessor {
             .description("Specifies whether or not the XML content should be validated against the DTD.")
             .required(true)
             .allowableValues("true", "false")
-            .defaultValue("true")

Review Comment:
   As mentioned for `EvaluateXQuery`, disabling validation of the Document Type Declaration in the default configuration provides a more secure starting point for new instances of the Processor.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org