You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by JohannesDaniel <gi...@git.apache.org> on 2018/03/27 20:53:45 UTC

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

GitHub user JohannesDaniel opened a pull request:

    https://github.com/apache/nifi/pull/2587

    NIFI-4185 Add XML Record Reader

    Thank you for submitting a contribution to Apache NiFi.
    
    In order to streamline the review of the contribution we ask you
    to ensure the following steps have been taken:
    
    ### For all changes:
    - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
         in the commit message?
    
    - [ ] 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.
    
    - [ ] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    - [ ] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [ ] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
    - [ ] Have you written or updated unit tests to verify your changes?
    - [ ] 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 travis-ci for build issues and submit an update to your PR as soon as possible.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/JohannesDaniel/nifi NIFI-4185

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/2587.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2587
    
----
commit 9b4bd0dd8f1d30bfe1597d4cd069df414eb968a0
Author: JohannesDaniel <jo...@...>
Date:   2018-03-06T23:02:43Z

    Add XML Record Reader

----


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r178470625
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html ---
    @@ -0,0 +1,378 @@
    +<!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>XMLReader</title>
    +        <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
    +    </head>
    +
    +    <body>
    +    <p>
    +        The XMLReader Controller Service reads XML content and creates Record objects. The Controller Service
    +        must be configured with a schema that describes the structure of the XML data. Fields in the XML data
    +        that are not defined in the schema will be skipped.
    +    </p>
    +    <p>
    +        Records are expected in the second level of the XML data, embedded within an enclosing root tag:
    +    </p>
    +    <code>
    +            <pre>
    +                &lt;root&gt;
    +                  &lt;record&gt;
    +                    &lt;field1&gt;content&lt;/field1&gt;
    +                    &lt;field2&gt;content&lt;/field2&gt;
    +                  &lt;/record&gt;
    +                  &lt;record&gt;
    +                    &lt;field1&gt;content&lt;/field1&gt;
    +                    &lt;field2&gt;content&lt;/field2&gt;
    +                  &lt;/record&gt;
    +                &lt;/root&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        For the following examples, it is assumed that the exemplary records are enclosed by a root tag.
    +    </p>
    +
    +    <h2>Example 1: Simple Fields</h2>
    +
    +    <p>
    +        The simplest kind of data within XML data are tags / fields only containing content (no attributes, no embedded tags).
    +        They can be described in the schema by simple types (e. g. INT, STRING, ...).
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;simple_field&gt;content&lt;/simple_field&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        This record can be described by a schema containing one field (e. g. of type string). By providing this schema,
    +        the reader expects zero or one occurrences of "simple_field" in the record.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "namespace": "nifi",
    +                  "name": "test",
    +                  "type": "record",
    +                  "fields": [
    +                    { "name": "simple_field", "type": "string" }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <h2>Example 2: Arrays with Simple Fields</h2>
    +
    +    <p>
    +        Arrays are considered as repetitive tags / fields in XML data. For the following XML data, "array_field" is considered
    +        to be an array enclosing simple fields, whereas "simple_field" is considered to be a simple field not enclosed in
    +        an array.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;array_field&gt;content&lt;/array_field&gt;
    +                  &lt;array_field&gt;content&lt;/array_field&gt;
    +                  &lt;simple_field&gt;content&lt;/simple_field&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        This record can be described by the following schema:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "namespace": "nifi",
    +                  "name": "test",
    +                  "type": "record",
    +                  "fields": [
    +                    { "name": "array_field", "type":
    +                      { "type": "array", "items": string }
    +                    },
    +                    { "name": "simple_field", "type": "string" }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        If a field in a schema is embedded in an array, the reader expects zero, one or more occurrences of the field
    +        in a record. The field "array_field" principally also could be defined as a simple field, but then the second occurrence
    +        of this field would replace the first in the record object. Moreover, the field "simple_field" could also be defined
    +        as an array. In this case, the reader would put it into the record object as an array with one element.
    +    </p>
    +
    +    <h2>Example 3: Tags with Attributes</h2>
    +
    +    <p>
    +        XML fields frequently not only contain content, but also attributes. The following record contains a field with
    +        an attribute "attr" and content:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;field_with_attribute attr="attr_content"&gt;content of field&lt;/field_with_attribute&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        To parse the content of the field "field_with_attribute" together with the attribute "attr", two requirements have
    +        to be fulfilled:
    +    </p>
    +
    +    <ul>
    +        <li>In the schema, the field has to be defined as record.</li>
    +        <li>The property "Field Name for Content" has to be set.</li>
    +        <li>As an option, the property "Attribute Prefix" also can be set.</li>
    +    </ul>
    +
    +    <p>
    +        For the example above, the following property settings are assumed:
    +    </p>
    +
    +    <table>
    +        <tr>
    +            <th>Property Name</th>
    +            <th>Property Value</th>
    +        </tr>
    +        <tr>
    +            <td>Field Name for Content</td>
    +            <td><code>field_name_for_content</code></td>
    +        </tr>
    +        <tr>
    +            <td>Attribute Prefix</td>
    +            <td><code>prefix_</code></td>
    +        </tr>
    +    </table>
    +
    +    <p>
    +        The schema can be defined as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "name": "test",
    +                  "namespace": "nifi",
    +                  "type": "record",
    +                  "fields": [
    +                    {
    +                      "name": "field_with_attribute",
    +                      "type": {
    +                        "name": "RecordForTag",
    +                        "type": "record",
    +                        "fields" : [
    +                          {"name": "attr", "type": "string"},
    +                          {"name": "field_name_for_content", "type": "string"}
    +                        ]
    +                    }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Note that the field "field_name_for_content" not only has to be defined in the property section, but also in the
    +        schema, whereas the prefix for attributes is not part of the schema. It will be appended when an attribute named
    +        "attr" is found at the respective position in the XML data and added to the record. The record object of the above
    +        example will be structured as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                Record (
    +                    Record "field_with_attribute" (
    +                        RecordField "prefix_attr" = "attr_content",
    +                        RecordField "field_name_for_content" = "content of field"
    +                    )
    +                )
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Principally, the field "field_with_attribute" could also be defined as a simple field. In this case, the attributes
    +        simply would be ignored. Vice versa, the simple field in example 1 above could also be defined as a record (assuming that
    +        the property "Field Name for Content" is set.
    +    </p>
    +
    +    <h2>Example 4: Tags within tags</h2>
    +
    +    <p>
    +        XML data is frequently nested. In this case, tags enclose other tags:
    +    </p>
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;field_with_embedded_fields attr=&quot;attr_content&quot;&gt;
    +                    &lt;embedded_field&gt;embedded content&lt;/embedded_field&gt;
    +                    &lt;another_embedded_field&gt;another embedded content&lt;/another_embedded_field&gt;
    +                  &lt;/field_with_embedded_fields&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        The enclosing fields always have to be defined as records, irrespective whether they include attributes to be
    +        parsed or not. In this example, the tag "field_with_embedded_fields" encloses the fields "embedded_field" and
    +        "another_embedded_field", which are both simple fields. The schema can be defined as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "name": "test",
    +                  "namespace": "nifi",
    +                  "type": "record",
    +                  "fields": [
    +                    {
    +                      "name": "field_with_embedded_fields",
    +                      "type": {
    +                        "name": "RecordForEmbedded",
    +                        "type": "record",
    +                        "fields" : [
    +                          {"name": "attr", "type": "string"},
    +                          {"name": "embedded_field", "type": "string"}
    +                          {"name": "another_embedded_field", "type": "string"}
    +                        ]
    +                    }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Notice that this case does not require the property "Field Name for Content" to be set as this is only required
    +        for tags containing attributes and content.
    +    </p>
    +
    +    <h2>Example 5: Array of records</h2>
    +
    +    <p>
    +        For further explanation of the logic of this reader, an example of an array of records shall be demonstrated.
    +        The following record contains the field "array_element", which repeatedly occurs. The field contains two
    +        embedded fields.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;array_field&gt;
    +                    &lt;embedded_field&gt;embedded content 1&lt;/embedded_field&gt;
    +                    &lt;another_embedded_field&gt;another embedded content 1&lt;/another_embedded_field&gt;
    +                  &lt;/array_field&gt;
    +                  &lt;array_field&gt;
    +                    &lt;embedded_field&gt;embedded content 2&lt;/embedded_field&gt;
    +                    &lt;another_embedded_field&gt;another embedded content 2&lt;/another_embedded_field&gt;
    +                  &lt;/array_field&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        This XML data can be parsed similarly to the data in example 4. However, the record defined in the schema of
    +        example 4 has to be embedded in an array.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "namespace": "nifi",
    +                  "name": "test",
    +                  "type": "record",
    +                  "fields": [
    +                    { "name": "array_field",
    +                      "type": {
    +                        "type": "array",
    +                        "items": {
    +                          "name": "RecordInArray",
    +                          "type": "record",
    +                          "fields" : [
    +                            {"name": "embedded_field", "type": "string"},
    +                            {"name": "another_embedded_field", "type": "string"}
    +                          ]
    +                        }
    +                      }
    +                    }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <h2>Example 6: Array in record</h2>
    +
    +    <p>
    +        In XML data, arrays are frequently enclosed by tags:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;field_enclosing_array&gt;
    +                    &lt;element&gt;content 1&lt;/element&gt;
    +                    &lt;element&gt;content 2&lt;/element&gt;
    +                  &lt;/field_enclosing_array&gt;
    +                  &lt;field_without_array&gt; content 3&lt;/field_without_array&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        For the schema, embedded tags have to be described by records. Therefore, the field "field_enclosing_array"
    +        is a record that embeds an array with elements of type string:
    +    </p>
    +
    +    <code>
    +            <pre>
    --- End diff --
    
    done


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179824910
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    --- End diff --
    
    I think this approach may lead to some odd behaviors if the incoming XML is actually namespace aware. For example, if an element looks like:
    ```
    <ns:myElement xmlns:ns="urn:nifi"><child /></ns:myElement>
    ```
    Then the name of the element in the schema would have to be "ns:myElement" in order to match. But I think that's not what we want. Instead, in the schema, I would want to use the name "myElement".
    
    So I think the approach that I would use here instead would be to set IS_NAMESPACE_AWARE to true. Then, when we have a StartElement object, instead of calling StartElement.getName().toString() I would call StartElement.getName().getLocalPart()
    
    In that case, we would get "myElement" as the name of the outer element and "child" as the name of the inner element. I.e., we would basically just ignore the namespaces. In either case, it will be very important that we explain clearly how this behaves in the documentation.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179829981
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    --- End diff --
    
    I'd avoid the anonymous inner class creation here as well and just create the ArrayList and add the value, then return that arraylist.


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    @markap14 thank you for the comprehensive review. I will start refactoring the implementations with respect to the improvements that are clear.


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    This is now merged to master. Thanks again for the contribution!


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r178437600
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    +        "XML data, embedded in an enclosing root tag.")
    +public class XMLReader extends SchemaRegistryService implements RecordReaderFactory {
    +
    +    public static final PropertyDescriptor VALIDATE_ROOT_TAG = new PropertyDescriptor.Builder()
    +            .name("validate_root_tag")
    +            .displayName("Validate Root Tag")
    +            .description("If this property is set, the name of root tags (e. g. <root><record>...</record></root>) of incoming FlowFiles will be evaluated against this value. " +
    +                    "In the case of a mismatch, an exception is thrown. The treatment of such FlowFiles depends on the implementation " +
    +                    "of respective Processors.")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor VALIDATE_RECORD_TAG = new PropertyDescriptor.Builder()
    --- End diff --
    
    Same here (and for the next properties).


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r181217494
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    --- End diff --
    
    why?


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r181218182
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    +            }
    +            case MAP: {
    +                logger.warn("Type map is not supported by this record reader. Field will be skipped.");
    --- End diff --
    
    ok, I ignored this so far as I had no clear idea how a map could look like in xml


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179822292
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    --- End diff --
    
    Record Readers don't need to be thread-safe, only the RecordReaderFactory does.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r181221789
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    +        "XML data, embedded in an enclosing root tag.")
    +public class XMLReader extends SchemaRegistryService implements RecordReaderFactory {
    +
    +    public static final PropertyDescriptor VALIDATE_ROOT_TAG = new PropertyDescriptor.Builder()
    +            .name("validate_root_tag")
    +            .displayName("Validate Root Tag")
    +            .description("If this property is set, the name of root tags (e. g. <root><record>...</record></root>) of incoming FlowFiles will be evaluated against this value. " +
    +                    "In the case of a mismatch, an exception is thrown. The treatment of such FlowFiles depends on the implementation " +
    +                    "of respective Processors.")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor VALIDATE_RECORD_TAG = new PropertyDescriptor.Builder()
    --- End diff --
    
    (non-record shall be skipped)


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r178437056
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html ---
    @@ -0,0 +1,378 @@
    +<!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>XMLReader</title>
    +        <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
    +    </head>
    +
    +    <body>
    +    <p>
    +        The XMLReader Controller Service reads XML content and creates Record objects. The Controller Service
    +        must be configured with a schema that describes the structure of the XML data. Fields in the XML data
    +        that are not defined in the schema will be skipped.
    +    </p>
    +    <p>
    +        Records are expected in the second level of the XML data, embedded within an enclosing root tag:
    +    </p>
    +    <code>
    +            <pre>
    +                &lt;root&gt;
    +                  &lt;record&gt;
    +                    &lt;field1&gt;content&lt;/field1&gt;
    +                    &lt;field2&gt;content&lt;/field2&gt;
    +                  &lt;/record&gt;
    +                  &lt;record&gt;
    +                    &lt;field1&gt;content&lt;/field1&gt;
    +                    &lt;field2&gt;content&lt;/field2&gt;
    +                  &lt;/record&gt;
    +                &lt;/root&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        For the following examples, it is assumed that the exemplary records are enclosed by a root tag.
    +    </p>
    +
    +    <h2>Example 1: Simple Fields</h2>
    +
    +    <p>
    +        The simplest kind of data within XML data are tags / fields only containing content (no attributes, no embedded tags).
    +        They can be described in the schema by simple types (e. g. INT, STRING, ...).
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;simple_field&gt;content&lt;/simple_field&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        This record can be described by a schema containing one field (e. g. of type string). By providing this schema,
    +        the reader expects zero or one occurrences of "simple_field" in the record.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "namespace": "nifi",
    +                  "name": "test",
    +                  "type": "record",
    +                  "fields": [
    +                    { "name": "simple_field", "type": "string" }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <h2>Example 2: Arrays with Simple Fields</h2>
    +
    +    <p>
    +        Arrays are considered as repetitive tags / fields in XML data. For the following XML data, "array_field" is considered
    +        to be an array enclosing simple fields, whereas "simple_field" is considered to be a simple field not enclosed in
    +        an array.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;array_field&gt;content&lt;/array_field&gt;
    +                  &lt;array_field&gt;content&lt;/array_field&gt;
    +                  &lt;simple_field&gt;content&lt;/simple_field&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        This record can be described by the following schema:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "namespace": "nifi",
    +                  "name": "test",
    +                  "type": "record",
    +                  "fields": [
    +                    { "name": "array_field", "type":
    +                      { "type": "array", "items": string }
    +                    },
    +                    { "name": "simple_field", "type": "string" }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        If a field in a schema is embedded in an array, the reader expects zero, one or more occurrences of the field
    +        in a record. The field "array_field" principally also could be defined as a simple field, but then the second occurrence
    +        of this field would replace the first in the record object. Moreover, the field "simple_field" could also be defined
    +        as an array. In this case, the reader would put it into the record object as an array with one element.
    +    </p>
    +
    +    <h2>Example 3: Tags with Attributes</h2>
    +
    +    <p>
    +        XML fields frequently not only contain content, but also attributes. The following record contains a field with
    +        an attribute "attr" and content:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;field_with_attribute attr="attr_content"&gt;content of field&lt;/field_with_attribute&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        To parse the content of the field "field_with_attribute" together with the attribute "attr", two requirements have
    +        to be fulfilled:
    +    </p>
    +
    +    <ul>
    +        <li>In the schema, the field has to be defined as record.</li>
    +        <li>The property "Field Name for Content" has to be set.</li>
    +        <li>As an option, the property "Attribute Prefix" also can be set.</li>
    +    </ul>
    +
    +    <p>
    +        For the example above, the following property settings are assumed:
    +    </p>
    +
    +    <table>
    +        <tr>
    +            <th>Property Name</th>
    +            <th>Property Value</th>
    +        </tr>
    +        <tr>
    +            <td>Field Name for Content</td>
    +            <td><code>field_name_for_content</code></td>
    +        </tr>
    +        <tr>
    +            <td>Attribute Prefix</td>
    +            <td><code>prefix_</code></td>
    +        </tr>
    +    </table>
    +
    +    <p>
    +        The schema can be defined as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "name": "test",
    +                  "namespace": "nifi",
    +                  "type": "record",
    +                  "fields": [
    +                    {
    +                      "name": "field_with_attribute",
    +                      "type": {
    +                        "name": "RecordForTag",
    +                        "type": "record",
    +                        "fields" : [
    +                          {"name": "attr", "type": "string"},
    +                          {"name": "field_name_for_content", "type": "string"}
    +                        ]
    +                    }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Note that the field "field_name_for_content" not only has to be defined in the property section, but also in the
    +        schema, whereas the prefix for attributes is not part of the schema. It will be appended when an attribute named
    +        "attr" is found at the respective position in the XML data and added to the record. The record object of the above
    +        example will be structured as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                Record (
    +                    Record "field_with_attribute" (
    +                        RecordField "prefix_attr" = "attr_content",
    +                        RecordField "field_name_for_content" = "content of field"
    +                    )
    +                )
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Principally, the field "field_with_attribute" could also be defined as a simple field. In this case, the attributes
    +        simply would be ignored. Vice versa, the simple field in example 1 above could also be defined as a record (assuming that
    +        the property "Field Name for Content" is set.
    +    </p>
    +
    +    <h2>Example 4: Tags within tags</h2>
    +
    +    <p>
    +        XML data is frequently nested. In this case, tags enclose other tags:
    +    </p>
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;field_with_embedded_fields attr=&quot;attr_content&quot;&gt;
    +                    &lt;embedded_field&gt;embedded content&lt;/embedded_field&gt;
    +                    &lt;another_embedded_field&gt;another embedded content&lt;/another_embedded_field&gt;
    +                  &lt;/field_with_embedded_fields&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        The enclosing fields always have to be defined as records, irrespective whether they include attributes to be
    +        parsed or not. In this example, the tag "field_with_embedded_fields" encloses the fields "embedded_field" and
    +        "another_embedded_field", which are both simple fields. The schema can be defined as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "name": "test",
    +                  "namespace": "nifi",
    +                  "type": "record",
    +                  "fields": [
    +                    {
    +                      "name": "field_with_embedded_fields",
    +                      "type": {
    +                        "name": "RecordForEmbedded",
    +                        "type": "record",
    +                        "fields" : [
    +                          {"name": "attr", "type": "string"},
    +                          {"name": "embedded_field", "type": "string"}
    +                          {"name": "another_embedded_field", "type": "string"}
    +                        ]
    +                    }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Notice that this case does not require the property "Field Name for Content" to be set as this is only required
    +        for tags containing attributes and content.
    +    </p>
    +
    +    <h2>Example 5: Array of records</h2>
    +
    +    <p>
    +        For further explanation of the logic of this reader, an example of an array of records shall be demonstrated.
    +        The following record contains the field "array_element", which repeatedly occurs. The field contains two
    --- End diff --
    
    typo: ``contains the field "array_field"``


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r181215801
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    --- End diff --
    
    ok, I will activate namespaces and implement some tests for this.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179835305
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    +            }
    +            case MAP: {
    +                logger.warn("Type map is not supported by this record reader. Field will be skipped.");
    +                skipElement();
    +                return null;
    +            }
    +            case CHOICE: {
    +                return parseUnknownField(startElement);
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private Object parseUnknownField(StartElement startElement) throws XMLStreamException {
    +        // parse attributes
    +        final Map<String, Object> recordValues = new HashMap<>();
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +            recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue());
    +        }
    +        boolean hasAttributes = recordValues.size() > 0;
    +
    +        // parse fields
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    xmlEventReader.nextEvent();
    +                    if (hasAttributes) {
    +                        if (contentFieldName != null) {
    +                            recordValues.put(contentFieldName, characters.toString());
    +                        } else {
    +                            logger.debug("Found content for field that has to be parsed as record but property \"Field Name for Content\" is not set. " +
    +                                    "The content will not be added to the record.");
    +                        }
    +                        return new MapRecord(new SimpleRecordSchema(Collections.emptyList()), recordValues);
    +                    } else {
    +                        return characters.toString();
    +                    }
    +                }
    +            } else if (xmlEvent.isStartElement()){
    +                final StartElement subStartElement = xmlEvent.asStartElement();
    +                final String subFieldName = subStartElement.getName().toString();
    +                final Object value = parseUnknownField(subStartElement);
    +
    +                if (value != null) {
    +                    putUnknownTypeInMap(recordValues, subFieldName, value);
    +                } else {
    +                    return null;
    +                }
    +            } else if (xmlEvent.isEndElement()) {
    +                break;
    +            }
    +        }
    +
    +        for (final Map.Entry<String,Object> entry : recordValues.entrySet()) {
    +            if (entry.getValue() instanceof List) {
    +                recordValues.put(entry.getKey(), ((List) entry.getValue()).toArray());
    +            }
    +        }
    +
    +        if (recordValues.size() > 0) {
    +            return new MapRecord(new SimpleRecordSchema(Collections.emptyList()), recordValues);
    +        } else {
    +            return null;
    +        }
    +    }
    +
    +    private Record parseRecord(StartElement startElement, RecordSchema schema, boolean coerceTypes, boolean dropUnknown) throws XMLStreamException, MalformedRecordException {
    +        final Map<String, Object> recordValues = new HashMap<>();
    +
    +        // parse attributes
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +
    +            final String targetFieldName = attributePrefix == null ? attributeName : attributePrefix + attributeName;
    +
    +            if (dropUnknown) {
    +                final Optional<RecordField> field = schema.getField(attributeName);
    +                if (field.isPresent()){
    +
    +                    // dropUnknown == true && coerceTypes == true
    +                    if (coerceTypes) {
    +                        final Object value;
    +                        final DataType dataType = field.get().getDataType();
    +                        if ((value = parseAttributeForType(attribute, attributeName, dataType)) != null) {
    +                            recordValues.put(targetFieldName, value);
    +                        }
    +
    +                    // dropUnknown == true && coerceTypes == false
    +                    } else {
    +                        recordValues.put(targetFieldName, attribute.getValue());
    +                    }
    +                }
    +            } else {
    +
    +                // dropUnknown == false && coerceTypes == true
    +                if (coerceTypes) {
    +                    final Object value;
    +                    final Optional<RecordField> field = schema.getField(attributeName);
    +                    if (field.isPresent()){
    +                        if ((value = parseAttributeForType(attribute, attributeName, field.get().getDataType())) != null) {
    +                            recordValues.put(targetFieldName, value);
    +                        }
    +                    } else {
    +                        recordValues.put(targetFieldName, attribute.getValue());
    +                    }
    +
    +                    // dropUnknown == false && coerceTypes == false
    +                } else {
    +                    recordValues.put(targetFieldName, attribute.getValue());
    +                }
    +            }
    +        }
    +
    +        // parse fields
    +        while(xmlEventReader.hasNext()){
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement subStartElement = xmlEvent.asStartElement();
    +                final String fieldName = subStartElement.getName().toString();
    +                final Optional<RecordField> field = schema.getField(fieldName);
    +
    +                if (dropUnknown) {
    +                    if (field.isPresent()) {
    +                        // dropUnknown == true && coerceTypes == true
    +                        if (coerceTypes) {
    +                            final Object value = parseFieldForType(subStartElement, fieldName, field.get().getDataType(), recordValues);
    +                            if (value != null) {
    +                                recordValues.put(fieldName, value);
    +                            }
    +
    +                        // dropUnknown == true && coerceTypes == false
    +                        } else {
    +                            final Object value = parseUnknownField(subStartElement);
    +                            if (value != null) {
    +                                putUnknownTypeInMap(recordValues, fieldName, value);
    +                            }
    +                        }
    +
    +                    } else {
    +                        skipElement();
    +                    }
    +                } else {
    +                    // dropUnknown == false && coerceTypes == true
    +                    if (coerceTypes) {
    +                        if (field.isPresent()) {
    +                            final Object value = parseFieldForType(subStartElement, fieldName, field.get().getDataType(), recordValues);
    +                            if (value != null) {
    +                                recordValues.put(fieldName, value);
    +                            }
    +                        } else {
    +                            final Object value = parseUnknownField(subStartElement);
    +                            if (value != null) {
    +                                putUnknownTypeInMap(recordValues, fieldName, value);
    +                            }
    +                        }
    +
    +                    } else {
    +                        final Object value = parseUnknownField(subStartElement);
    +                        if (value != null) {
    +                            putUnknownTypeInMap(recordValues, fieldName, value);
    +                        }
    +                    }
    +                }
    +            } else if (xmlEvent.isEndElement()) {
    +                break;
    +            } else if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    if (contentFieldName != null) {
    +                        final Optional<RecordField> field = schema.getField(contentFieldName);
    +                        if (field.isPresent()) {
    +                            Object value = parseCharacterForType(characters, contentFieldName, field.get().getDataType());
    +                            recordValues.put(contentFieldName, value);
    +                        }
    +                    } else {
    +                        logger.debug("Found content for field that is defined as record but property \"Field Name for Content\" is not set. " +
    +                                "The content will not be added to record.");
    +                    }
    +                }
    +            }
    +        }
    +        for (final Map.Entry<String,Object> entry : recordValues.entrySet()) {
    +            if (entry.getValue() instanceof List) {
    +                recordValues.put(entry.getKey(), ((List) entry.getValue()).toArray());
    +            }
    +        }
    +
    +        if (recordValues.size() > 0) {
    +            return new MapRecord(schema, recordValues);
    +        } else {
    +            return null;
    +        }
    +    }
    +
    +    private void putUnknownTypeInMap(Map<String, Object> values, String fieldName, Object fieldValue) {
    +        final Object oldValues = values.get(fieldName);
    +
    +        if (oldValues != null) {
    +            if (oldValues instanceof List) {
    +                ((List) oldValues).add(fieldValue);
    +            } else {
    +                values.put(fieldName, new ArrayList<Object>(){{ add(oldValues); add(fieldValue); }});
    +            }
    +        } else {
    +            values.put(fieldName, fieldValue);
    +        }
    +    }
    +
    +    private Object parseAttributeForType(Attribute attribute, String fieldName, DataType dataType) {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                return DataTypeUtils.convertType(attribute.getValue(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private Object parseCharacterForType(Characters character, String fieldName, DataType dataType) {
    --- End diff --
    
    Probably can create a single parseStringDataForType(String data, String fieldName, DataType dataType) method and call that from both parseAttributeForType and parseCharacterForType, rather than having to repeat the large case statement.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/nifi/pull/2587


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179840389
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    --- End diff --
    
    More specifically, I think that if the following were the content of a FlowFile:
    ```
    <person>
      <name>John Doe</name>
      <id>123</id>
      <dob>01/01/2017</dob>
    </person>
    ```
    Then I would expect to have this parse as a single Record that would match this schema:
    ```
    {
      "name": "person", "namespace": "nifi",
      "type": "record",
      "fields": [
        { "name": "name", "type": "string" },
        { "name": "id", "type": "int" },
        { "name": "dob", "type": "date" }
      ]
    }
    ```
    Additionally, I would expect to be able to set a property that indicates that the outer-most XML element is simply a wrapper. If that property were set to "true", then I would expect to use that exact same schema to parse the following XML:
    ```
    <people>
      <person>
        <name>John Doe</name>
        <id>123</id>
        <dob>01/01/2017</dob>
      </person>
      <person>
        <name>Jane Doe</name>
        <id>124</id>
        <dob>01/01/2016</dob>
      </person>
      <person>
        <name>Jake Doe</name>
        <id>125</id>
        <dob>01/01/2015</dob>
      </person>
    </people>
    ```
    In this case, the 'people' element is just a wrapper and could just as easily be an element named 'root' or 'foo' or 'bar'.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179827839
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    --- End diff --
    
    In this case, we are moving on to the next event. We then check if this next event is either endElement or startElement. If it's neither of those, then we fall through to the ARRAY case statement. Need to be sure to address the 'else' case here.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179834679
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    +            }
    +            case MAP: {
    +                logger.warn("Type map is not supported by this record reader. Field will be skipped.");
    +                skipElement();
    +                return null;
    +            }
    +            case CHOICE: {
    +                return parseUnknownField(startElement);
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private Object parseUnknownField(StartElement startElement) throws XMLStreamException {
    +        // parse attributes
    +        final Map<String, Object> recordValues = new HashMap<>();
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +            recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue());
    +        }
    +        boolean hasAttributes = recordValues.size() > 0;
    +
    +        // parse fields
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    xmlEventReader.nextEvent();
    +                    if (hasAttributes) {
    +                        if (contentFieldName != null) {
    +                            recordValues.put(contentFieldName, characters.toString());
    +                        } else {
    +                            logger.debug("Found content for field that has to be parsed as record but property \"Field Name for Content\" is not set. " +
    +                                    "The content will not be added to the record.");
    +                        }
    +                        return new MapRecord(new SimpleRecordSchema(Collections.emptyList()), recordValues);
    +                    } else {
    +                        return characters.toString();
    +                    }
    +                }
    +            } else if (xmlEvent.isStartElement()){
    +                final StartElement subStartElement = xmlEvent.asStartElement();
    +                final String subFieldName = subStartElement.getName().toString();
    +                final Object value = parseUnknownField(subStartElement);
    +
    +                if (value != null) {
    +                    putUnknownTypeInMap(recordValues, subFieldName, value);
    +                } else {
    +                    return null;
    +                }
    +            } else if (xmlEvent.isEndElement()) {
    +                break;
    +            }
    +        }
    +
    +        for (final Map.Entry<String,Object> entry : recordValues.entrySet()) {
    +            if (entry.getValue() instanceof List) {
    +                recordValues.put(entry.getKey(), ((List) entry.getValue()).toArray());
    +            }
    +        }
    +
    +        if (recordValues.size() > 0) {
    +            return new MapRecord(new SimpleRecordSchema(Collections.emptyList()), recordValues);
    +        } else {
    +            return null;
    +        }
    +    }
    +
    +    private Record parseRecord(StartElement startElement, RecordSchema schema, boolean coerceTypes, boolean dropUnknown) throws XMLStreamException, MalformedRecordException {
    +        final Map<String, Object> recordValues = new HashMap<>();
    +
    +        // parse attributes
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +
    +            final String targetFieldName = attributePrefix == null ? attributeName : attributePrefix + attributeName;
    +
    +            if (dropUnknown) {
    +                final Optional<RecordField> field = schema.getField(attributeName);
    +                if (field.isPresent()){
    +
    +                    // dropUnknown == true && coerceTypes == true
    +                    if (coerceTypes) {
    +                        final Object value;
    +                        final DataType dataType = field.get().getDataType();
    +                        if ((value = parseAttributeForType(attribute, attributeName, dataType)) != null) {
    +                            recordValues.put(targetFieldName, value);
    +                        }
    +
    +                    // dropUnknown == true && coerceTypes == false
    +                    } else {
    +                        recordValues.put(targetFieldName, attribute.getValue());
    +                    }
    +                }
    +            } else {
    +
    +                // dropUnknown == false && coerceTypes == true
    +                if (coerceTypes) {
    +                    final Object value;
    +                    final Optional<RecordField> field = schema.getField(attributeName);
    +                    if (field.isPresent()){
    +                        if ((value = parseAttributeForType(attribute, attributeName, field.get().getDataType())) != null) {
    +                            recordValues.put(targetFieldName, value);
    +                        }
    +                    } else {
    +                        recordValues.put(targetFieldName, attribute.getValue());
    +                    }
    +
    +                    // dropUnknown == false && coerceTypes == false
    +                } else {
    +                    recordValues.put(targetFieldName, attribute.getValue());
    +                }
    +            }
    +        }
    +
    +        // parse fields
    +        while(xmlEventReader.hasNext()){
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement subStartElement = xmlEvent.asStartElement();
    +                final String fieldName = subStartElement.getName().toString();
    +                final Optional<RecordField> field = schema.getField(fieldName);
    +
    +                if (dropUnknown) {
    +                    if (field.isPresent()) {
    +                        // dropUnknown == true && coerceTypes == true
    +                        if (coerceTypes) {
    +                            final Object value = parseFieldForType(subStartElement, fieldName, field.get().getDataType(), recordValues);
    +                            if (value != null) {
    +                                recordValues.put(fieldName, value);
    +                            }
    +
    +                        // dropUnknown == true && coerceTypes == false
    +                        } else {
    +                            final Object value = parseUnknownField(subStartElement);
    +                            if (value != null) {
    +                                putUnknownTypeInMap(recordValues, fieldName, value);
    +                            }
    +                        }
    +
    +                    } else {
    +                        skipElement();
    +                    }
    +                } else {
    +                    // dropUnknown == false && coerceTypes == true
    +                    if (coerceTypes) {
    +                        if (field.isPresent()) {
    +                            final Object value = parseFieldForType(subStartElement, fieldName, field.get().getDataType(), recordValues);
    +                            if (value != null) {
    +                                recordValues.put(fieldName, value);
    +                            }
    +                        } else {
    +                            final Object value = parseUnknownField(subStartElement);
    +                            if (value != null) {
    +                                putUnknownTypeInMap(recordValues, fieldName, value);
    +                            }
    +                        }
    +
    +                    } else {
    +                        final Object value = parseUnknownField(subStartElement);
    +                        if (value != null) {
    +                            putUnknownTypeInMap(recordValues, fieldName, value);
    +                        }
    +                    }
    +                }
    +            } else if (xmlEvent.isEndElement()) {
    +                break;
    +            } else if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    if (contentFieldName != null) {
    +                        final Optional<RecordField> field = schema.getField(contentFieldName);
    +                        if (field.isPresent()) {
    +                            Object value = parseCharacterForType(characters, contentFieldName, field.get().getDataType());
    +                            recordValues.put(contentFieldName, value);
    +                        }
    +                    } else {
    +                        logger.debug("Found content for field that is defined as record but property \"Field Name for Content\" is not set. " +
    +                                "The content will not be added to record.");
    +                    }
    +                }
    +            }
    +        }
    +        for (final Map.Entry<String,Object> entry : recordValues.entrySet()) {
    +            if (entry.getValue() instanceof List) {
    +                recordValues.put(entry.getKey(), ((List) entry.getValue()).toArray());
    +            }
    +        }
    +
    +        if (recordValues.size() > 0) {
    +            return new MapRecord(schema, recordValues);
    +        } else {
    +            return null;
    +        }
    +    }
    +
    +    private void putUnknownTypeInMap(Map<String, Object> values, String fieldName, Object fieldValue) {
    +        final Object oldValues = values.get(fieldName);
    +
    +        if (oldValues != null) {
    +            if (oldValues instanceof List) {
    +                ((List) oldValues).add(fieldValue);
    +            } else {
    +                values.put(fieldName, new ArrayList<Object>(){{ add(oldValues); add(fieldValue); }});
    --- End diff --
    
    Would avoid the anonymous inner class here too.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179829864
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    --- End diff --
    
    I would want to avoid creating anonymous inner classes here for the sake of adding and instead just create a new ArrayList and add both elements.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179820052
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    +        "XML data, embedded in an enclosing root tag.")
    +public class XMLReader extends SchemaRegistryService implements RecordReaderFactory {
    +
    +    public static final PropertyDescriptor VALIDATE_ROOT_TAG = new PropertyDescriptor.Builder()
    +            .name("validate_root_tag")
    +            .displayName("Validate Root Tag")
    +            .description("If this property is set, the name of root tags (e. g. <root><record>...</record></root>) of incoming FlowFiles will be evaluated against this value. " +
    +                    "In the case of a mismatch, an exception is thrown. The treatment of such FlowFiles depends on the implementation " +
    +                    "of respective Processors.")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor VALIDATE_RECORD_TAG = new PropertyDescriptor.Builder()
    +            .name("validate_record_tag")
    +            .displayName("Validate Record Tag")
    +            .description("If this property is set, the name of record tags (e. g. <root><record>...</record></root>) of incoming FlowFiles will be evaluated against this value. " +
    +                    "In the case of a mismatch, the respective record will be skipped. If this property is not set, each level 2 starting tag will be treated " +
    +                    "as the beginning of a record.")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor ATTRIBUTE_PREFIX = new PropertyDescriptor.Builder()
    +            .name("attribute_prefix")
    +            .displayName("Attribute Prefix")
    +            .description("If this property is set, the name of attributes will be appended by a prefix when they are added to a record.")
    --- End diff --
    
    I think this is supposed to say "prepended with a prefix"


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r178470648
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html ---
    @@ -0,0 +1,378 @@
    +<!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>XMLReader</title>
    +        <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
    +    </head>
    +
    +    <body>
    +    <p>
    +        The XMLReader Controller Service reads XML content and creates Record objects. The Controller Service
    +        must be configured with a schema that describes the structure of the XML data. Fields in the XML data
    +        that are not defined in the schema will be skipped.
    +    </p>
    +    <p>
    +        Records are expected in the second level of the XML data, embedded within an enclosing root tag:
    +    </p>
    +    <code>
    +            <pre>
    +                &lt;root&gt;
    +                  &lt;record&gt;
    +                    &lt;field1&gt;content&lt;/field1&gt;
    +                    &lt;field2&gt;content&lt;/field2&gt;
    +                  &lt;/record&gt;
    +                  &lt;record&gt;
    +                    &lt;field1&gt;content&lt;/field1&gt;
    +                    &lt;field2&gt;content&lt;/field2&gt;
    +                  &lt;/record&gt;
    +                &lt;/root&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        For the following examples, it is assumed that the exemplary records are enclosed by a root tag.
    +    </p>
    +
    +    <h2>Example 1: Simple Fields</h2>
    +
    +    <p>
    +        The simplest kind of data within XML data are tags / fields only containing content (no attributes, no embedded tags).
    +        They can be described in the schema by simple types (e. g. INT, STRING, ...).
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;simple_field&gt;content&lt;/simple_field&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        This record can be described by a schema containing one field (e. g. of type string). By providing this schema,
    +        the reader expects zero or one occurrences of "simple_field" in the record.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "namespace": "nifi",
    +                  "name": "test",
    +                  "type": "record",
    +                  "fields": [
    +                    { "name": "simple_field", "type": "string" }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <h2>Example 2: Arrays with Simple Fields</h2>
    +
    +    <p>
    +        Arrays are considered as repetitive tags / fields in XML data. For the following XML data, "array_field" is considered
    +        to be an array enclosing simple fields, whereas "simple_field" is considered to be a simple field not enclosed in
    +        an array.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;array_field&gt;content&lt;/array_field&gt;
    +                  &lt;array_field&gt;content&lt;/array_field&gt;
    +                  &lt;simple_field&gt;content&lt;/simple_field&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        This record can be described by the following schema:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "namespace": "nifi",
    +                  "name": "test",
    +                  "type": "record",
    +                  "fields": [
    +                    { "name": "array_field", "type":
    +                      { "type": "array", "items": string }
    +                    },
    +                    { "name": "simple_field", "type": "string" }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        If a field in a schema is embedded in an array, the reader expects zero, one or more occurrences of the field
    +        in a record. The field "array_field" principally also could be defined as a simple field, but then the second occurrence
    +        of this field would replace the first in the record object. Moreover, the field "simple_field" could also be defined
    +        as an array. In this case, the reader would put it into the record object as an array with one element.
    +    </p>
    +
    +    <h2>Example 3: Tags with Attributes</h2>
    +
    +    <p>
    +        XML fields frequently not only contain content, but also attributes. The following record contains a field with
    +        an attribute "attr" and content:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;field_with_attribute attr="attr_content"&gt;content of field&lt;/field_with_attribute&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        To parse the content of the field "field_with_attribute" together with the attribute "attr", two requirements have
    +        to be fulfilled:
    +    </p>
    +
    +    <ul>
    +        <li>In the schema, the field has to be defined as record.</li>
    +        <li>The property "Field Name for Content" has to be set.</li>
    +        <li>As an option, the property "Attribute Prefix" also can be set.</li>
    +    </ul>
    +
    +    <p>
    +        For the example above, the following property settings are assumed:
    +    </p>
    +
    +    <table>
    +        <tr>
    +            <th>Property Name</th>
    +            <th>Property Value</th>
    +        </tr>
    +        <tr>
    +            <td>Field Name for Content</td>
    +            <td><code>field_name_for_content</code></td>
    +        </tr>
    +        <tr>
    +            <td>Attribute Prefix</td>
    +            <td><code>prefix_</code></td>
    +        </tr>
    +    </table>
    +
    +    <p>
    +        The schema can be defined as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "name": "test",
    +                  "namespace": "nifi",
    +                  "type": "record",
    +                  "fields": [
    +                    {
    +                      "name": "field_with_attribute",
    +                      "type": {
    +                        "name": "RecordForTag",
    +                        "type": "record",
    +                        "fields" : [
    +                          {"name": "attr", "type": "string"},
    +                          {"name": "field_name_for_content", "type": "string"}
    +                        ]
    +                    }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Note that the field "field_name_for_content" not only has to be defined in the property section, but also in the
    +        schema, whereas the prefix for attributes is not part of the schema. It will be appended when an attribute named
    +        "attr" is found at the respective position in the XML data and added to the record. The record object of the above
    +        example will be structured as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                Record (
    +                    Record "field_with_attribute" (
    +                        RecordField "prefix_attr" = "attr_content",
    +                        RecordField "field_name_for_content" = "content of field"
    +                    )
    +                )
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Principally, the field "field_with_attribute" could also be defined as a simple field. In this case, the attributes
    +        simply would be ignored. Vice versa, the simple field in example 1 above could also be defined as a record (assuming that
    +        the property "Field Name for Content" is set.
    +    </p>
    +
    +    <h2>Example 4: Tags within tags</h2>
    +
    +    <p>
    +        XML data is frequently nested. In this case, tags enclose other tags:
    +    </p>
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;field_with_embedded_fields attr=&quot;attr_content&quot;&gt;
    +                    &lt;embedded_field&gt;embedded content&lt;/embedded_field&gt;
    +                    &lt;another_embedded_field&gt;another embedded content&lt;/another_embedded_field&gt;
    +                  &lt;/field_with_embedded_fields&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        The enclosing fields always have to be defined as records, irrespective whether they include attributes to be
    +        parsed or not. In this example, the tag "field_with_embedded_fields" encloses the fields "embedded_field" and
    +        "another_embedded_field", which are both simple fields. The schema can be defined as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "name": "test",
    +                  "namespace": "nifi",
    +                  "type": "record",
    +                  "fields": [
    +                    {
    +                      "name": "field_with_embedded_fields",
    +                      "type": {
    +                        "name": "RecordForEmbedded",
    +                        "type": "record",
    +                        "fields" : [
    +                          {"name": "attr", "type": "string"},
    +                          {"name": "embedded_field", "type": "string"}
    +                          {"name": "another_embedded_field", "type": "string"}
    +                        ]
    +                    }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Notice that this case does not require the property "Field Name for Content" to be set as this is only required
    +        for tags containing attributes and content.
    +    </p>
    +
    +    <h2>Example 5: Array of records</h2>
    +
    +    <p>
    +        For further explanation of the logic of this reader, an example of an array of records shall be demonstrated.
    +        The following record contains the field "array_element", which repeatedly occurs. The field contains two
    --- End diff --
    
    done


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179833900
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    +            }
    +            case MAP: {
    +                logger.warn("Type map is not supported by this record reader. Field will be skipped.");
    +                skipElement();
    +                return null;
    +            }
    +            case CHOICE: {
    +                return parseUnknownField(startElement);
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private Object parseUnknownField(StartElement startElement) throws XMLStreamException {
    +        // parse attributes
    +        final Map<String, Object> recordValues = new HashMap<>();
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +            recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue());
    +        }
    +        boolean hasAttributes = recordValues.size() > 0;
    +
    +        // parse fields
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    xmlEventReader.nextEvent();
    +                    if (hasAttributes) {
    +                        if (contentFieldName != null) {
    +                            recordValues.put(contentFieldName, characters.toString());
    +                        } else {
    +                            logger.debug("Found content for field that has to be parsed as record but property \"Field Name for Content\" is not set. " +
    +                                    "The content will not be added to the record.");
    +                        }
    +                        return new MapRecord(new SimpleRecordSchema(Collections.emptyList()), recordValues);
    +                    } else {
    +                        return characters.toString();
    +                    }
    +                }
    +            } else if (xmlEvent.isStartElement()){
    +                final StartElement subStartElement = xmlEvent.asStartElement();
    +                final String subFieldName = subStartElement.getName().toString();
    +                final Object value = parseUnknownField(subStartElement);
    +
    +                if (value != null) {
    +                    putUnknownTypeInMap(recordValues, subFieldName, value);
    +                } else {
    +                    return null;
    +                }
    +            } else if (xmlEvent.isEndElement()) {
    +                break;
    +            }
    +        }
    +
    +        for (final Map.Entry<String,Object> entry : recordValues.entrySet()) {
    +            if (entry.getValue() instanceof List) {
    +                recordValues.put(entry.getKey(), ((List) entry.getValue()).toArray());
    +            }
    +        }
    +
    +        if (recordValues.size() > 0) {
    +            return new MapRecord(new SimpleRecordSchema(Collections.emptyList()), recordValues);
    +        } else {
    +            return null;
    +        }
    +    }
    +
    +    private Record parseRecord(StartElement startElement, RecordSchema schema, boolean coerceTypes, boolean dropUnknown) throws XMLStreamException, MalformedRecordException {
    +        final Map<String, Object> recordValues = new HashMap<>();
    +
    +        // parse attributes
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    --- End diff --
    
    Should probably use getName().getLocalPart() here as well.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179829341
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    --- End diff --
    
    I'm not certain about this logic. The following is valid XML:
    ```
    <root>
      <child>My <great /> family</child>
    </root>
    ```
    If the XML Reader were to encounter this, it would throw a MalformedRecordException because it would encounter "characters" for "My " and then encounter a startElement. One valid option would be to skip the "great" element. Another valid option is probably to go ahead and throw the Exception as you. But I think this is an important case to add to the documentation in additionalDetails.


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    @JohannesDaniel thanks for the contribution! This is definitely something that I've been wanting to implement for a while but haven't had a chance yet. Will be happy to review.


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    Can you maybe post the XML that led to the empty record?


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179827156
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    --- End diff --
    
    Probably should use characters.getData() here, instead of characters.toString() as there is no guarantee of what toString() will provide


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    @pvillard31 here we go :)


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    @markap14 @pvillard31 
    - I refactored some code as the cases (coerce==true && drop==false) and (coerce==false && drop==true) in some cases showed an unexpected behavior
    - Data like <tag>content<subtag>content</subtag>content</tag> now can be parsed
    - Maps (e. g. <map_field><key1>value1</key1><key2>value2</key2></map_field>) are now supported
    - The reader is now able to parse single records (e. g. <record><field1/><field2/></record>) as well as arrays of records (e. g. <root><record/><record/></root>). I added a property to make it configurable whether the reader shall expect a single record or an array. One question: As there are only two options for this, I defined AllowableValues for this property. Despite that, I think it would be reasonable to enable EL for this property. But how can this be realized?
    - I removed the root validation, but remained the check for record tag names in order to support processing data like this <root><record/><record/><other/></root> (tag "other" will be ignored if check for record tag name is activated)


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179829486
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    --- End diff --
    
    Of note, one valid way that this could occur is if there were a CDATA tag followed by regular characters, or vice versa. I think in such a case we could get a Characters event followed by a Characters event.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179833060
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    +            }
    +            case MAP: {
    +                logger.warn("Type map is not supported by this record reader. Field will be skipped.");
    +                skipElement();
    +                return null;
    +            }
    +            case CHOICE: {
    +                return parseUnknownField(startElement);
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private Object parseUnknownField(StartElement startElement) throws XMLStreamException {
    +        // parse attributes
    +        final Map<String, Object> recordValues = new HashMap<>();
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +            recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue());
    +        }
    +        boolean hasAttributes = recordValues.size() > 0;
    +
    +        // parse fields
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    xmlEventReader.nextEvent();
    +                    if (hasAttributes) {
    +                        if (contentFieldName != null) {
    +                            recordValues.put(contentFieldName, characters.toString());
    --- End diff --
    
    I think we need to avoid using characters.toString() and use getData() instead


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    @markap14 thanks for the help with the patch file. I am fine with it :)
    
    If you have not planned to do that yourself, I would start implementing a XMLWriter as soon as this has been merged. Or should we first discuss your plan with the Record attributes you mentioned above?


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179819209
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    +        "XML data, embedded in an enclosing root tag.")
    +public class XMLReader extends SchemaRegistryService implements RecordReaderFactory {
    +
    +    public static final PropertyDescriptor VALIDATE_ROOT_TAG = new PropertyDescriptor.Builder()
    --- End diff --
    
    I am actually in favor of removing this property all together. In order to properly read the records, the Record Readers will need to validate syntax of the data, but I don't believe that it should be validating arbitrary semantic meanings. I.e., I don't think that we should be checking the name of the outer-most element for any specific name. 


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179820952
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    --- End diff --
    
    I think the requirement that XML data must be wrapped in some sort of wrapper is going to be problematic. While this will be a fairly common case, so that multiple XML elements can be combined into a single FlowFile, it is also going to be common (probably more common) that each XML element will be its own standalone Record. This is especially important if this Reader is used for something like ListenTCPRecord or ConsumeKafkaRecord, where the data is received from elsewhere so no processor has a chance to wrap the content prior to using the XML Reader. I think we need to support both ignoring the outer-most element as well as incorporating the outer-most element.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by tballison <gi...@git.apache.org>.
Github user tballison commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r183390424
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -84,6 +84,10 @@ public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, Str
     
             try {
                 final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    --- End diff --
    
    Might want to avoid XEE vulnerability via improved configuration of XMLInputFactory?
    https://www.owasp.org/index.php/XML_External_Entity_(XXE)_Prevention_Cheat_Sheet#XMLInputFactory_.28a_StAX_parser.29


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r183489279
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -84,6 +84,10 @@ public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, Str
     
             try {
                 final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    --- End diff --
    
    @tballison Thank you for the advice! I refactored this in a way that only the local part is considered. 
    :)


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179819928
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    +        "XML data, embedded in an enclosing root tag.")
    +public class XMLReader extends SchemaRegistryService implements RecordReaderFactory {
    +
    +    public static final PropertyDescriptor VALIDATE_ROOT_TAG = new PropertyDescriptor.Builder()
    +            .name("validate_root_tag")
    +            .displayName("Validate Root Tag")
    +            .description("If this property is set, the name of root tags (e. g. <root><record>...</record></root>) of incoming FlowFiles will be evaluated against this value. " +
    +                    "In the case of a mismatch, an exception is thrown. The treatment of such FlowFiles depends on the implementation " +
    +                    "of respective Processors.")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor VALIDATE_RECORD_TAG = new PropertyDescriptor.Builder()
    --- End diff --
    
    Likewise, I think we should remove this property and this sort of validation as well. If the user wants to validate some specific XML element names, the ValidateRecord processor is a great solution for that, and provides far more flexible validation via schema.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179838657
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    +            }
    +            case MAP: {
    +                logger.warn("Type map is not supported by this record reader. Field will be skipped.");
    +                skipElement();
    +                return null;
    +            }
    +            case CHOICE: {
    +                return parseUnknownField(startElement);
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private Object parseUnknownField(StartElement startElement) throws XMLStreamException {
    +        // parse attributes
    +        final Map<String, Object> recordValues = new HashMap<>();
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +            recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue());
    +        }
    +        boolean hasAttributes = recordValues.size() > 0;
    +
    +        // parse fields
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    xmlEventReader.nextEvent();
    +                    if (hasAttributes) {
    +                        if (contentFieldName != null) {
    +                            recordValues.put(contentFieldName, characters.toString());
    +                        } else {
    +                            logger.debug("Found content for field that has to be parsed as record but property \"Field Name for Content\" is not set. " +
    --- End diff --
    
    I wonder if it makes sense in this case to use the name of the field  as the 'contentFieldName'. For example, if we had the following XML snippet:
    ```
    <person>
      <location gurantee="NONE">1.469, -3.875</location>
    </person>
    ```
    Then if the Content Field Name were null, we would just use the name of the field (in this case 'location') so that a flattened representation of the record would look like:
    ```
    person.location.guarantee = NONE
    person.location.location = 1.469, -3.875
    ```
    Thoughts?


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    @JohannesDaniel this is great! I've been doing a good bit of testing to ensure that everything works as expected. I had just a few more comments, mostly around the descriptions in the property descriptor and the handling of invalid values for the "Expect Records as Array" property. To make it a little easier to understand, I'm just going to attach a PATCH file to the JIRA. Can you please look over the patch file? If you're okay with the patch file, then I'm ready to merge. Thanks again, you've put in a lot of work here to make this consistent with the current approaches and to provide a huge new capability for many users!


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r181213403
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    +        "XML data, embedded in an enclosing root tag.")
    +public class XMLReader extends SchemaRegistryService implements RecordReaderFactory {
    +
    +    public static final PropertyDescriptor VALIDATE_ROOT_TAG = new PropertyDescriptor.Builder()
    +            .name("validate_root_tag")
    +            .displayName("Validate Root Tag")
    +            .description("If this property is set, the name of root tags (e. g. <root><record>...</record></root>) of incoming FlowFiles will be evaluated against this value. " +
    +                    "In the case of a mismatch, an exception is thrown. The treatment of such FlowFiles depends on the implementation " +
    +                    "of respective Processors.")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor VALIDATE_RECORD_TAG = new PropertyDescriptor.Builder()
    --- End diff --
    
    my original intention actually was to enable users to parse recordsets like this
    ```
    <root>
      <record>...
      <record>...
      <non-record>...


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179830412
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    --- End diff --
    
    In this case, I think we want to return oldValues, not null, as oldValues may already have a value set.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r183152900
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    --- End diff --
    
    Yes, exactly. I would use a property to convey this. I would be okay with allowing Expression Language to be used, or just allowing for a 'true'/'false' without Expression Language (I think in most cases, you'll want one or the other, not dependent upon each individual FlowFile). But if you think EL is important then I won't argue that point :)
    One other option, which we do in a few different processors, would be to offer a third option that looks at a well-known attribute. So you could choose 'true' (treat outer element as a wrapper), 'false' (treat each flowfile as a single record), or 'use xml.stream attribute', and when that is selected, the 'xml.stream' attribute would be looked at to determine how to handle it - a value of 'true' would mean it's a stream of multiple records, 'false' would mean it's only 1 record, missing or any other value would throw an Exception. I don't have  strong preference one way or another how this should be handled, but wanted to present options that we typically use.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r181215337
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    --- End diff --
    
    when I started implementing this reader, I was wondering, how the reader knows whether to parse wrapped records or a single record. unfortunately we dont have an unambiguous indicator like we have for json: [ vs. { 
    I considered to make it configurable with EL whether the reader shall expect a single record or an array of records. what do you think?


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r178437093
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html ---
    @@ -0,0 +1,378 @@
    +<!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>XMLReader</title>
    +        <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
    +    </head>
    +
    +    <body>
    +    <p>
    +        The XMLReader Controller Service reads XML content and creates Record objects. The Controller Service
    +        must be configured with a schema that describes the structure of the XML data. Fields in the XML data
    +        that are not defined in the schema will be skipped.
    +    </p>
    +    <p>
    +        Records are expected in the second level of the XML data, embedded within an enclosing root tag:
    +    </p>
    +    <code>
    +            <pre>
    +                &lt;root&gt;
    +                  &lt;record&gt;
    +                    &lt;field1&gt;content&lt;/field1&gt;
    +                    &lt;field2&gt;content&lt;/field2&gt;
    +                  &lt;/record&gt;
    +                  &lt;record&gt;
    +                    &lt;field1&gt;content&lt;/field1&gt;
    +                    &lt;field2&gt;content&lt;/field2&gt;
    +                  &lt;/record&gt;
    +                &lt;/root&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        For the following examples, it is assumed that the exemplary records are enclosed by a root tag.
    +    </p>
    +
    +    <h2>Example 1: Simple Fields</h2>
    +
    +    <p>
    +        The simplest kind of data within XML data are tags / fields only containing content (no attributes, no embedded tags).
    +        They can be described in the schema by simple types (e. g. INT, STRING, ...).
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;simple_field&gt;content&lt;/simple_field&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        This record can be described by a schema containing one field (e. g. of type string). By providing this schema,
    +        the reader expects zero or one occurrences of "simple_field" in the record.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "namespace": "nifi",
    +                  "name": "test",
    +                  "type": "record",
    +                  "fields": [
    +                    { "name": "simple_field", "type": "string" }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <h2>Example 2: Arrays with Simple Fields</h2>
    +
    +    <p>
    +        Arrays are considered as repetitive tags / fields in XML data. For the following XML data, "array_field" is considered
    +        to be an array enclosing simple fields, whereas "simple_field" is considered to be a simple field not enclosed in
    +        an array.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;array_field&gt;content&lt;/array_field&gt;
    +                  &lt;array_field&gt;content&lt;/array_field&gt;
    +                  &lt;simple_field&gt;content&lt;/simple_field&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        This record can be described by the following schema:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "namespace": "nifi",
    +                  "name": "test",
    +                  "type": "record",
    +                  "fields": [
    +                    { "name": "array_field", "type":
    +                      { "type": "array", "items": string }
    +                    },
    +                    { "name": "simple_field", "type": "string" }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        If a field in a schema is embedded in an array, the reader expects zero, one or more occurrences of the field
    +        in a record. The field "array_field" principally also could be defined as a simple field, but then the second occurrence
    +        of this field would replace the first in the record object. Moreover, the field "simple_field" could also be defined
    +        as an array. In this case, the reader would put it into the record object as an array with one element.
    +    </p>
    +
    +    <h2>Example 3: Tags with Attributes</h2>
    +
    +    <p>
    +        XML fields frequently not only contain content, but also attributes. The following record contains a field with
    +        an attribute "attr" and content:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;field_with_attribute attr="attr_content"&gt;content of field&lt;/field_with_attribute&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        To parse the content of the field "field_with_attribute" together with the attribute "attr", two requirements have
    +        to be fulfilled:
    +    </p>
    +
    +    <ul>
    +        <li>In the schema, the field has to be defined as record.</li>
    +        <li>The property "Field Name for Content" has to be set.</li>
    +        <li>As an option, the property "Attribute Prefix" also can be set.</li>
    +    </ul>
    +
    +    <p>
    +        For the example above, the following property settings are assumed:
    +    </p>
    +
    +    <table>
    +        <tr>
    +            <th>Property Name</th>
    +            <th>Property Value</th>
    +        </tr>
    +        <tr>
    +            <td>Field Name for Content</td>
    +            <td><code>field_name_for_content</code></td>
    +        </tr>
    +        <tr>
    +            <td>Attribute Prefix</td>
    +            <td><code>prefix_</code></td>
    +        </tr>
    +    </table>
    +
    +    <p>
    +        The schema can be defined as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "name": "test",
    +                  "namespace": "nifi",
    +                  "type": "record",
    +                  "fields": [
    +                    {
    +                      "name": "field_with_attribute",
    +                      "type": {
    +                        "name": "RecordForTag",
    +                        "type": "record",
    +                        "fields" : [
    +                          {"name": "attr", "type": "string"},
    +                          {"name": "field_name_for_content", "type": "string"}
    +                        ]
    +                    }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Note that the field "field_name_for_content" not only has to be defined in the property section, but also in the
    +        schema, whereas the prefix for attributes is not part of the schema. It will be appended when an attribute named
    +        "attr" is found at the respective position in the XML data and added to the record. The record object of the above
    +        example will be structured as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                Record (
    +                    Record "field_with_attribute" (
    +                        RecordField "prefix_attr" = "attr_content",
    +                        RecordField "field_name_for_content" = "content of field"
    +                    )
    +                )
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Principally, the field "field_with_attribute" could also be defined as a simple field. In this case, the attributes
    +        simply would be ignored. Vice versa, the simple field in example 1 above could also be defined as a record (assuming that
    +        the property "Field Name for Content" is set.
    +    </p>
    +
    +    <h2>Example 4: Tags within tags</h2>
    +
    +    <p>
    +        XML data is frequently nested. In this case, tags enclose other tags:
    +    </p>
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;field_with_embedded_fields attr=&quot;attr_content&quot;&gt;
    +                    &lt;embedded_field&gt;embedded content&lt;/embedded_field&gt;
    +                    &lt;another_embedded_field&gt;another embedded content&lt;/another_embedded_field&gt;
    +                  &lt;/field_with_embedded_fields&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        The enclosing fields always have to be defined as records, irrespective whether they include attributes to be
    +        parsed or not. In this example, the tag "field_with_embedded_fields" encloses the fields "embedded_field" and
    +        "another_embedded_field", which are both simple fields. The schema can be defined as follows:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "name": "test",
    +                  "namespace": "nifi",
    +                  "type": "record",
    +                  "fields": [
    +                    {
    +                      "name": "field_with_embedded_fields",
    +                      "type": {
    +                        "name": "RecordForEmbedded",
    +                        "type": "record",
    +                        "fields" : [
    +                          {"name": "attr", "type": "string"},
    +                          {"name": "embedded_field", "type": "string"}
    +                          {"name": "another_embedded_field", "type": "string"}
    +                        ]
    +                    }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <p>
    +        Notice that this case does not require the property "Field Name for Content" to be set as this is only required
    +        for tags containing attributes and content.
    +    </p>
    +
    +    <h2>Example 5: Array of records</h2>
    +
    +    <p>
    +        For further explanation of the logic of this reader, an example of an array of records shall be demonstrated.
    +        The following record contains the field "array_element", which repeatedly occurs. The field contains two
    +        embedded fields.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;array_field&gt;
    +                    &lt;embedded_field&gt;embedded content 1&lt;/embedded_field&gt;
    +                    &lt;another_embedded_field&gt;another embedded content 1&lt;/another_embedded_field&gt;
    +                  &lt;/array_field&gt;
    +                  &lt;array_field&gt;
    +                    &lt;embedded_field&gt;embedded content 2&lt;/embedded_field&gt;
    +                    &lt;another_embedded_field&gt;another embedded content 2&lt;/another_embedded_field&gt;
    +                  &lt;/array_field&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        This XML data can be parsed similarly to the data in example 4. However, the record defined in the schema of
    +        example 4 has to be embedded in an array.
    +    </p>
    +
    +    <code>
    +            <pre>
    +                {
    +                  "namespace": "nifi",
    +                  "name": "test",
    +                  "type": "record",
    +                  "fields": [
    +                    { "name": "array_field",
    +                      "type": {
    +                        "type": "array",
    +                        "items": {
    +                          "name": "RecordInArray",
    +                          "type": "record",
    +                          "fields" : [
    +                            {"name": "embedded_field", "type": "string"},
    +                            {"name": "another_embedded_field", "type": "string"}
    +                          ]
    +                        }
    +                      }
    +                    }
    +                  ]
    +                }
    +            </pre>
    +    </code>
    +
    +    <h2>Example 6: Array in record</h2>
    +
    +    <p>
    +        In XML data, arrays are frequently enclosed by tags:
    +    </p>
    +
    +    <code>
    +            <pre>
    +                &lt;record&gt;
    +                  &lt;field_enclosing_array&gt;
    +                    &lt;element&gt;content 1&lt;/element&gt;
    +                    &lt;element&gt;content 2&lt;/element&gt;
    +                  &lt;/field_enclosing_array&gt;
    +                  &lt;field_without_array&gt; content 3&lt;/field_without_array&gt;
    +                &lt;/record&gt;
    +            </pre>
    +    </code>
    +
    +    <p>
    +        For the schema, embedded tags have to be described by records. Therefore, the field "field_enclosing_array"
    +        is a record that embeds an array with elements of type string:
    +    </p>
    +
    +    <code>
    +            <pre>
    --- End diff --
    
    Rendering of the below schema is not great, can you fix the indentation?


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    Hey @JohannesDaniel - just made new tests and can't reproduce what I saw previously... so let's forget about it :) (thanks for the unit tests though!)
    
    I tested the EL support for tag validation, working well, thanks for the addition. I'd just make one suggestion: right now for the record tag, "in the case of a mismatch, the respective record will be skipped". I'd suggest to make this behavior configurable through a parameter in the CS: skip the record (current behavior) or throw an exception for the flow file (as you're doing for the root tag validation).
    
    Regarding the performance test, it was a GFF => ConvertRecord (XML to JSON) => UpdateAttribute. Don't remember the exact numbers but IIRC it was around 1300 flowfile per second with an XML payload of 6KB (single record in my case).
    
    Overall, it's really cool!
    I'll try to have a look over the code during the week.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r178437593
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    +        "XML data, embedded in an enclosing root tag.")
    +public class XMLReader extends SchemaRegistryService implements RecordReaderFactory {
    +
    +    public static final PropertyDescriptor VALIDATE_ROOT_TAG = new PropertyDescriptor.Builder()
    --- End diff --
    
    Could this property supports expression language against incoming flow files? I don't think that's an easy change (and it could introduce a perf hit) but that would allow using the same reader for completely different XML inputs.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r181653767
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    +            }
    +            case MAP: {
    +                logger.warn("Type map is not supported by this record reader. Field will be skipped.");
    +                skipElement();
    +                return null;
    +            }
    +            case CHOICE: {
    +                return parseUnknownField(startElement);
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private Object parseUnknownField(StartElement startElement) throws XMLStreamException {
    +        // parse attributes
    +        final Map<String, Object> recordValues = new HashMap<>();
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +            recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue());
    +        }
    +        boolean hasAttributes = recordValues.size() > 0;
    +
    +        // parse fields
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    xmlEventReader.nextEvent();
    +                    if (hasAttributes) {
    +                        if (contentFieldName != null) {
    +                            recordValues.put(contentFieldName, characters.toString());
    +                        } else {
    +                            logger.debug("Found content for field that has to be parsed as record but property \"Field Name for Content\" is not set. " +
    --- End diff --
    
    But what if the xml has the following structure?
    
    ```
    <person>
      <location location="NONE">1.469, -3.875</location>
    </person>
    ```
    
    then the will replace the attribute as long as location ist not defined as an array...


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    Hi @pvillard31 
    
    thank you for your comments! I realized all your suggestions. I like your news regarding the performance :-) Which kind of transformation did you test? XML => Record or XML => JSON (e. g. with ConvertRecord)?
    
    For any reason some tests disappeared for a certain commit at my local git (probably, I wanted to reorder the tests,  but deleted them, omg ...). However, I inserted them again (this is why there are many more tests now). 
    
    In addition, I adjusted the definition about how namespaces shall be treated. 
    
    I implemented several tests for XMLReader to verify that the usage of expression language works as expected.
    
    However, I was not able to reproduce your observation regarding the empty record for the header 
    ```
    <?xml version="1.0" encoding="utf-8"?>
    ```
    
    I implemented the following tests:
    ```
    testSimpleRecordWithHeader()
    testSimpleRecordWithHeaderNoValidation()
    ```
    
    Actually, they work as expected. 


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r181218609
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    +            }
    +            case MAP: {
    +                logger.warn("Type map is not supported by this record reader. Field will be skipped.");
    +                skipElement();
    +                return null;
    +            }
    +            case CHOICE: {
    +                return parseUnknownField(startElement);
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private Object parseUnknownField(StartElement startElement) throws XMLStreamException {
    +        // parse attributes
    +        final Map<String, Object> recordValues = new HashMap<>();
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +            recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue());
    +        }
    +        boolean hasAttributes = recordValues.size() > 0;
    +
    +        // parse fields
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    xmlEventReader.nextEvent();
    +                    if (hasAttributes) {
    +                        if (contentFieldName != null) {
    +                            recordValues.put(contentFieldName, characters.toString());
    +                        } else {
    +                            logger.debug("Found content for field that has to be parsed as record but property \"Field Name for Content\" is not set. " +
    --- End diff --
    
    nice idea


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179834908
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    +            }
    +            case MAP: {
    +                logger.warn("Type map is not supported by this record reader. Field will be skipped.");
    +                skipElement();
    +                return null;
    +            }
    +            case CHOICE: {
    +                return parseUnknownField(startElement);
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private Object parseUnknownField(StartElement startElement) throws XMLStreamException {
    +        // parse attributes
    +        final Map<String, Object> recordValues = new HashMap<>();
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +            recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue());
    +        }
    +        boolean hasAttributes = recordValues.size() > 0;
    +
    +        // parse fields
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    xmlEventReader.nextEvent();
    +                    if (hasAttributes) {
    +                        if (contentFieldName != null) {
    +                            recordValues.put(contentFieldName, characters.toString());
    +                        } else {
    +                            logger.debug("Found content for field that has to be parsed as record but property \"Field Name for Content\" is not set. " +
    +                                    "The content will not be added to the record.");
    +                        }
    +                        return new MapRecord(new SimpleRecordSchema(Collections.emptyList()), recordValues);
    +                    } else {
    +                        return characters.toString();
    +                    }
    +                }
    +            } else if (xmlEvent.isStartElement()){
    +                final StartElement subStartElement = xmlEvent.asStartElement();
    +                final String subFieldName = subStartElement.getName().toString();
    +                final Object value = parseUnknownField(subStartElement);
    +
    +                if (value != null) {
    +                    putUnknownTypeInMap(recordValues, subFieldName, value);
    +                } else {
    +                    return null;
    +                }
    +            } else if (xmlEvent.isEndElement()) {
    +                break;
    +            }
    +        }
    +
    +        for (final Map.Entry<String,Object> entry : recordValues.entrySet()) {
    +            if (entry.getValue() instanceof List) {
    +                recordValues.put(entry.getKey(), ((List) entry.getValue()).toArray());
    +            }
    +        }
    +
    +        if (recordValues.size() > 0) {
    +            return new MapRecord(new SimpleRecordSchema(Collections.emptyList()), recordValues);
    +        } else {
    +            return null;
    +        }
    +    }
    +
    +    private Record parseRecord(StartElement startElement, RecordSchema schema, boolean coerceTypes, boolean dropUnknown) throws XMLStreamException, MalformedRecordException {
    +        final Map<String, Object> recordValues = new HashMap<>();
    +
    +        // parse attributes
    +        final Iterator iterator = startElement.getAttributes();
    +        while (iterator.hasNext()) {
    +            final Attribute attribute = (Attribute) iterator.next();
    +            final String attributeName = attribute.getName().toString();
    +
    +            final String targetFieldName = attributePrefix == null ? attributeName : attributePrefix + attributeName;
    +
    +            if (dropUnknown) {
    +                final Optional<RecordField> field = schema.getField(attributeName);
    +                if (field.isPresent()){
    +
    +                    // dropUnknown == true && coerceTypes == true
    +                    if (coerceTypes) {
    +                        final Object value;
    +                        final DataType dataType = field.get().getDataType();
    +                        if ((value = parseAttributeForType(attribute, attributeName, dataType)) != null) {
    +                            recordValues.put(targetFieldName, value);
    +                        }
    +
    +                    // dropUnknown == true && coerceTypes == false
    +                    } else {
    +                        recordValues.put(targetFieldName, attribute.getValue());
    +                    }
    +                }
    +            } else {
    +
    +                // dropUnknown == false && coerceTypes == true
    +                if (coerceTypes) {
    +                    final Object value;
    +                    final Optional<RecordField> field = schema.getField(attributeName);
    +                    if (field.isPresent()){
    +                        if ((value = parseAttributeForType(attribute, attributeName, field.get().getDataType())) != null) {
    +                            recordValues.put(targetFieldName, value);
    +                        }
    +                    } else {
    +                        recordValues.put(targetFieldName, attribute.getValue());
    +                    }
    +
    +                    // dropUnknown == false && coerceTypes == false
    +                } else {
    +                    recordValues.put(targetFieldName, attribute.getValue());
    +                }
    +            }
    +        }
    +
    +        // parse fields
    +        while(xmlEventReader.hasNext()){
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement subStartElement = xmlEvent.asStartElement();
    +                final String fieldName = subStartElement.getName().toString();
    +                final Optional<RecordField> field = schema.getField(fieldName);
    +
    +                if (dropUnknown) {
    +                    if (field.isPresent()) {
    +                        // dropUnknown == true && coerceTypes == true
    +                        if (coerceTypes) {
    +                            final Object value = parseFieldForType(subStartElement, fieldName, field.get().getDataType(), recordValues);
    +                            if (value != null) {
    +                                recordValues.put(fieldName, value);
    +                            }
    +
    +                        // dropUnknown == true && coerceTypes == false
    +                        } else {
    +                            final Object value = parseUnknownField(subStartElement);
    +                            if (value != null) {
    +                                putUnknownTypeInMap(recordValues, fieldName, value);
    +                            }
    +                        }
    +
    +                    } else {
    +                        skipElement();
    +                    }
    +                } else {
    +                    // dropUnknown == false && coerceTypes == true
    +                    if (coerceTypes) {
    +                        if (field.isPresent()) {
    +                            final Object value = parseFieldForType(subStartElement, fieldName, field.get().getDataType(), recordValues);
    +                            if (value != null) {
    +                                recordValues.put(fieldName, value);
    +                            }
    +                        } else {
    +                            final Object value = parseUnknownField(subStartElement);
    +                            if (value != null) {
    +                                putUnknownTypeInMap(recordValues, fieldName, value);
    +                            }
    +                        }
    +
    +                    } else {
    +                        final Object value = parseUnknownField(subStartElement);
    +                        if (value != null) {
    +                            putUnknownTypeInMap(recordValues, fieldName, value);
    +                        }
    +                    }
    +                }
    +            } else if (xmlEvent.isEndElement()) {
    +                break;
    +            } else if (xmlEvent.isCharacters()) {
    +                final Characters characters = xmlEvent.asCharacters();
    +                if (!characters.isWhiteSpace()) {
    +                    if (contentFieldName != null) {
    +                        final Optional<RecordField> field = schema.getField(contentFieldName);
    +                        if (field.isPresent()) {
    +                            Object value = parseCharacterForType(characters, contentFieldName, field.get().getDataType());
    +                            recordValues.put(contentFieldName, value);
    +                        }
    +                    } else {
    +                        logger.debug("Found content for field that is defined as record but property \"Field Name for Content\" is not set. " +
    +                                "The content will not be added to record.");
    +                    }
    +                }
    +            }
    +        }
    +        for (final Map.Entry<String,Object> entry : recordValues.entrySet()) {
    +            if (entry.getValue() instanceof List) {
    +                recordValues.put(entry.getKey(), ((List) entry.getValue()).toArray());
    +            }
    +        }
    +
    +        if (recordValues.size() > 0) {
    +            return new MapRecord(schema, recordValues);
    +        } else {
    +            return null;
    +        }
    +    }
    +
    +    private void putUnknownTypeInMap(Map<String, Object> values, String fieldName, Object fieldValue) {
    +        final Object oldValues = values.get(fieldName);
    +
    +        if (oldValues != null) {
    +            if (oldValues instanceof List) {
    +                ((List) oldValues).add(fieldValue);
    +            } else {
    +                values.put(fieldName, new ArrayList<Object>(){{ add(oldValues); add(fieldValue); }});
    +            }
    +        } else {
    +            values.put(fieldName, fieldValue);
    +        }
    +    }
    +
    +    private Object parseAttributeForType(Attribute attribute, String fieldName, DataType dataType) {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                return DataTypeUtils.convertType(attribute.getValue(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private Object parseCharacterForType(Characters character, String fieldName, DataType dataType) {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                return DataTypeUtils.convertType(character.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    --- End diff --
    
    Should use getData() here too


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179831387
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    --- End diff --
    
    We can't pass 'true' here for the last two arguments - if `nextRecord(false, false)` were to be called, we need to honor those values and pass them along here.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r181213473
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    +        "XML data, embedded in an enclosing root tag.")
    +public class XMLReader extends SchemaRegistryService implements RecordReaderFactory {
    +
    +    public static final PropertyDescriptor VALIDATE_ROOT_TAG = new PropertyDescriptor.Builder()
    +            .name("validate_root_tag")
    +            .displayName("Validate Root Tag")
    +            .description("If this property is set, the name of root tags (e. g. <root><record>...</record></root>) of incoming FlowFiles will be evaluated against this value. " +
    +                    "In the case of a mismatch, an exception is thrown. The treatment of such FlowFiles depends on the implementation " +
    +                    "of respective Processors.")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor VALIDATE_RECORD_TAG = new PropertyDescriptor.Builder()
    +            .name("validate_record_tag")
    +            .displayName("Validate Record Tag")
    +            .description("If this property is set, the name of record tags (e. g. <root><record>...</record></root>) of incoming FlowFiles will be evaluated against this value. " +
    +                    "In the case of a mismatch, the respective record will be skipped. If this property is not set, each level 2 starting tag will be treated " +
    +                    "as the beginning of a record.")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor ATTRIBUTE_PREFIX = new PropertyDescriptor.Builder()
    +            .name("attribute_prefix")
    +            .displayName("Attribute Prefix")
    +            .description("If this property is set, the name of attributes will be appended by a prefix when they are added to a record.")
    --- End diff --
    
    ok


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    Thanks for pinging me @JohannesDaniel - I'll also try to have a look over the WE / next week. It'd be a great addition!


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    @markap14 thank you for the response. 
    
    I will simply remove that record tag validation as there are indeed many ways to do that before the data is processed by this reader. 
    
    There is one little corner case, I need to discuss: 
    Assuming we have the following data:
    ```
    <record><map_field><key1>value1</key1><key2>...
    ```
    If the reader is used with (coerce==true), the field "map_field" can be parsed by defining a map in the schema. The embedded key fields do not have to be defined, its values only have to be of the defined type for the map. 
    
    If the reader is used with (coerce==false && dropUnknown==true), the reader will parse all fields that exist in the schema ignoring its type. However, the data above will not be parsable even if the map exists in the schema. In this case, the reader identifies "map_field" as a field that exists in the schema, but the reader is not aware that it is of type map. Therefore, the reader will not parse the embedded key fields, as they don't exist in the schema. The field "map_field" will be classified as an empty field and not added to the record.
    
    Furthermore, even if the reader is used with (coerce==false && dropUnknown==true), it will be type-aware to some extent. The reader first checks, whether fields exist in the schema. If that is the case, the reader additionally will check whether they are of type record (or of type array embedding records, respectively). If that is also the case, the reader will retrieve the subschema in order to be enabled to check whether subtags of the current tag are known. 



---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179831931
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    +            case RECORD: {
    +                final RecordSchema childSchema;
    +                if (dataType instanceof RecordDataType) {
    +                    childSchema = ((RecordDataType) dataType).getChildSchema();
    +                } else {
    +                    return null;
    +                }
    +                return parseRecord(startElement, childSchema, true, true);
    +            }
    +            case MAP: {
    +                logger.warn("Type map is not supported by this record reader. Field will be skipped.");
    --- End diff --
    
    Is there a reason that we don't want to support parsing MAP type? I think it's a viable option, such something like:
    ```
    <myElement>
      <myMap>
        <key1>Hello</key1>
        <key2>Adios</key2>
        <another-key>74</another-key>
      </myMap>
    </myElement>
    ```


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r178470638
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.DateTimeUtils;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SchemaRegistryService;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +@Tags({"xml", "record", "reader", "parser"})
    +@CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
    +        "XML data, embedded in an enclosing root tag.")
    +public class XMLReader extends SchemaRegistryService implements RecordReaderFactory {
    +
    +    public static final PropertyDescriptor VALIDATE_ROOT_TAG = new PropertyDescriptor.Builder()
    --- End diff --
    
    done. additionally, I added some tests for class XMLReader


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    @markap14 
    - Added EL for record format property
    - Removed record tag validation


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r179830847
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    +                }
    +            }
    --- End diff --
    
    Need to be sure to add a break statement here to avoid falling through to the RECORD case. I believe this would happen, actually, if newValues != null and oldValues != null and oldValues instanceof List. In that case, we would call add() on the oldValues but not return, so we'd fall through to the RECORD case.


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    @JohannesDaniel thanks for the update! I commented above re: the use of Expression Language in the property descriptor.
    I do still feel like the check for 'record tag names' is unnecessary, as the reader should not be responsible for filtering the data but rather just for reading it. There already exist mechanisms for filtering the data (You could use PartitionRecord + RouteOnAttribute, ValidateRecord, or QueryRecord just off the top of my head to achieve this). Additionally, we have the Schema for the Record Reader. So if the element name matches the top-level Schema name (or one of them, if the top-level field is a UNION/CHOICE element), then we could use that. So, with your example above, if you only want to read the `<record/><record/>` part, your schema should indicate that the top-level field name is `record`. In that case, it should filter out the `other` record. Does that make sense? 


---

[GitHub] nifi issue #2587: NIFI-4185 Add XML Record Reader

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/2587
  
    @JohannesDaniel that would be great! I think the Record attribute stuff will significantly improve how we are able to handle XML-based records. But I think the approach that you've taken here is a great first step and will be very beneficial to the community. I'd recommend going ahead with the XML Writer and then we can work in the attribute-related stuff later.


---

[GitHub] nifi pull request #2587: NIFI-4185 Add XML Record Reader

Posted by JohannesDaniel <gi...@git.apache.org>.
Github user JohannesDaniel commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2587#discussion_r181851731
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java ---
    @@ -0,0 +1,502 @@
    +/*
    + * 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;
    +
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.DataType;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordField;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.serialization.record.type.ArrayDataType;
    +import org.apache.nifi.serialization.record.type.RecordDataType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import javax.xml.stream.XMLEventReader;
    +import javax.xml.stream.XMLInputFactory;
    +import javax.xml.stream.XMLStreamException;
    +import javax.xml.stream.events.Attribute;
    +import javax.xml.stream.events.Characters;
    +import javax.xml.stream.events.StartElement;
    +import javax.xml.stream.events.XMLEvent;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.text.DateFormat;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Supplier;
    +
    +public class XMLRecordReader implements RecordReader {
    +
    +    private final ComponentLog logger;
    +    private final RecordSchema schema;
    +    private final String recordName;
    +    private final String attributePrefix;
    +    private final String contentFieldName;
    +
    +    // thread safety required?
    +    private StartElement currentRecordStartTag;
    +
    +    private final XMLEventReader xmlEventReader;
    +
    +    private final Supplier<DateFormat> LAZY_DATE_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIME_FORMAT;
    +    private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
    +
    +    public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName,
    +                           final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
    +        this.schema = schema;
    +        this.recordName = recordName;
    +        this.attributePrefix = attributePrefix;
    +        this.contentFieldName = contentFieldName;
    +        this.logger = logger;
    +
    +        final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
    +        final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
    +        final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
    +
    +        LAZY_DATE_FORMAT = () -> df;
    +        LAZY_TIME_FORMAT = () -> tf;
    +        LAZY_TIMESTAMP_FORMAT = () -> tsf;
    +
    +        try {
    +            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
    +
    +            // Avoid namespace replacements
    +            xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
    +
    +            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
    +            final StartElement rootTag = getNextStartTag();
    +
    +            // root tag validation
    +            if (rootName != null && !rootName.equals(rootTag.getName().toString())) {
    +                final StringBuffer message = new StringBuffer();
    +                message.append("Name of root tag \"")
    +                        .append(rootTag.getName().toString())
    +                        .append("\" does not match root tag validation \"")
    +                        .append(rootName)
    +                        .append("\".");
    +                throw new MalformedRecordException(message.toString());
    +            }
    +            setNextRecordStartTag();
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private StartElement getNextStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                return xmlEvent.asStartElement();
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void setNextRecordStartTag() throws XMLStreamException {
    +        while (xmlEventReader.hasNext()) {
    +            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +            if (xmlEvent.isStartElement()) {
    +                final StartElement startElement = xmlEvent.asStartElement();
    +                if (recordName != null) {
    +                    if (startElement.getName().toString().equals(recordName)) {
    +                        currentRecordStartTag = startElement;
    +                        return;
    +                    } else {
    +                        logger.debug("Mismatch between expected record tag name {} and actual tag name in XML {}. " +
    +                                "Record will be skipped", new Object[] {recordName, startElement.getName().toString()});
    +                        skipElement();
    +                    }
    +                } else {
    +                    currentRecordStartTag = startElement;
    +                    return;
    +                }
    +            }
    +        }
    +        currentRecordStartTag = null;
    +    }
    +
    +    @Override
    +    public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
    +        if (currentRecordStartTag == null) {
    +            return null;
    +        }
    +        try {
    +            final Record record = parseRecord(currentRecordStartTag, this.schema, coerceTypes, dropUnknownFields);
    +            setNextRecordStartTag();
    +            if (record != null) {
    +                return record;
    +            } else {
    +                return new MapRecord(this.schema, Collections.EMPTY_MAP);
    +            }
    +        } catch (XMLStreamException e) {
    +            throw new MalformedRecordException("Could not parse XML", e);
    +        }
    +    }
    +
    +    private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map<String, Object> recordValues) throws XMLStreamException, MalformedRecordException {
    +        switch (dataType.getFieldType()) {
    +            case BOOLEAN:
    +            case BYTE:
    +            case CHAR:
    +            case DOUBLE:
    +            case FLOAT:
    +            case INT:
    +            case LONG:
    +            case SHORT:
    +            case STRING:
    +            case DATE:
    +            case TIME:
    +            case TIMESTAMP: {
    +                XMLEvent xmlEvent = xmlEventReader.nextEvent();
    +                if (xmlEvent.isCharacters()) {
    +                    final Characters characters = xmlEvent.asCharacters();
    +                    if (!characters.isWhiteSpace()) {
    +                        xmlEventReader.nextEvent();
    +                        return DataTypeUtils.convertType(characters.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
    +                    } else {
    +                        xmlEvent = xmlEventReader.nextEvent();
    +                    }
    +                }
    +                if (xmlEvent.isEndElement()) {
    +                    return null;
    +                } else if (xmlEvent.isStartElement()) {
    +                    final String message = "Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.";
    +                    throw new MalformedRecordException(message);
    +                }
    +            }
    +            case ARRAY: {
    +                final DataType arrayDataType = ((ArrayDataType) dataType).getElementType();
    +
    +                final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues);
    +                final Object oldValues = recordValues.get(fieldName);
    +
    +                if (newValue != null) {
    +                    if (oldValues != null) {
    +                        if (oldValues instanceof List) {
    +                            ((List) oldValues).add(newValue);
    +                        } else {
    +                            return new ArrayList<Object>(){{ add(oldValues); add(newValue); }};
    +                        }
    +                    } else {
    +                        return new ArrayList<Object>(){{ add(newValue); }};
    +                    }
    +                } else {
    +                    return null;
    --- End diff --
    
    This already works. If null is returned, the outer loop doesn't do anything, so the field containing the list remains. However, I changed this to make the logic more obvious.


---