From 9b4bd0dd8f1d30bfe1597d4cd069df414eb968a0 Mon Sep 17 00:00:00 2001 From: JohannesDaniel Date: Wed, 7 Mar 2018 00:02:43 +0100 Subject: [PATCH 1/6] Add XML Record Reader --- .../pom.xml | 12 + .../java/org/apache/nifi/xml/XMLReader.java | 121 ++++ .../org/apache/nifi/xml/XMLRecordReader.java | 493 +++++++++++++ ...g.apache.nifi.controller.ControllerService | 4 +- .../additionalDetails.html | 378 ++++++++++ .../apache/nifi/xml/TestXMLRecordReader.java | 654 ++++++++++++++++++ .../src/test/resources/xml/people.xml | 22 + .../src/test/resources/xml/people2.xml | 12 + .../src/test/resources/xml/people3.xml | 17 + .../src/test/resources/xml/people4.xml | 12 + .../src/test/resources/xml/people_array.xml | 36 + .../resources/xml/people_array_simple.xml | 28 + .../test/resources/xml/people_complex1.xml | 33 + .../test/resources/xml/people_complex2.xml | 73 ++ .../src/test/resources/xml/people_empty.xml | 12 + .../src/test/resources/xml/people_invalid.xml | 21 + .../src/test/resources/xml/people_nested.xml | 38 + .../resources/xml/people_no_attributes.xml | 22 + 18 files changed, 1987 insertions(+), 1 deletion(-) mode change 100644 => 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java mode change 100644 => 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService create mode 100755 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 create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people2.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people3.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people4.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array_simple.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_complex1.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_complex2.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_empty.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_invalid.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_nested.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_no_attributes.xml diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml old mode 100644 new mode 100755 index e1d9acaeb6fe..0e02e6e827c2 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml @@ -116,6 +116,18 @@ src/test/resources/json/single-element-nested-array.json src/test/resources/json/single-element-nested.json src/test/resources/json/output/dataTypes.json + src/test/resources/xml/people.xml + src/test/resources/xml/people2.xml + src/test/resources/xml/people3.xml + src/test/resources/xml/people4.xml + src/test/resources/xml/people_array.xml + src/test/resources/xml/people_array_simple.xml + src/test/resources/xml/people_complex1.xml + src/test/resources/xml/people_complex2.xml + src/test/resources/xml/people_empty.xml + src/test/resources/xml/people_invalid.xml + src/test/resources/xml/people_nested.xml + src/test/resources/xml/people_no_attributes.xml diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java new file mode 100755 index 000000000000..e047515473ab --- /dev/null +++ b/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. ...) 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() + .name("validate_record_tag") + .displayName("Validate Record Tag") + .description("If this property is set, the name of record tags (e. g. ...) 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) + .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.") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .required(false) + .build(); + + public static final PropertyDescriptor CONTENT_FIELD_NAME = new PropertyDescriptor.Builder() + .name("content_field_name") + .displayName("Field Name for Content") + .description("If tags with content (e. g. content) are defined as nested records in the schema, " + + "the name of the tag will be used as name for the record and the value of this property will be used as name for the field. " + + "If tags with content shall be parsed together with attributes (e. g. content), " + + "they have to be defined as records. For additional information, see the section of processor usage.") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .required(false) + .build(); + + private volatile String dateFormat; + private volatile String timeFormat; + private volatile String timestampFormat; + + @OnEnabled + public void onEnabled(final ConfigurationContext context) { + this.dateFormat = context.getProperty(DateTimeUtils.DATE_FORMAT).getValue(); + this.timeFormat = context.getProperty(DateTimeUtils.TIME_FORMAT).getValue(); + this.timestampFormat = context.getProperty(DateTimeUtils.TIMESTAMP_FORMAT).getValue(); + } + + @Override + protected List getSupportedPropertyDescriptors() { + final List properties = new ArrayList<>(super.getSupportedPropertyDescriptors()); + properties.add(VALIDATE_ROOT_TAG); + properties.add(VALIDATE_RECORD_TAG); + properties.add(ATTRIBUTE_PREFIX); + properties.add(CONTENT_FIELD_NAME); + properties.add(DateTimeUtils.DATE_FORMAT); + properties.add(DateTimeUtils.TIME_FORMAT); + properties.add(DateTimeUtils.TIMESTAMP_FORMAT); + return properties; + } + + @Override + public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException, MalformedRecordException { + final ConfigurationContext context = getConfigurationContext(); + + final RecordSchema schema = getSchema(variables, in, null); + final String rootName = context.getProperty(VALIDATE_ROOT_TAG).isSet() ? context.getProperty(VALIDATE_ROOT_TAG).getValue().trim() : null; + final String recordName = context.getProperty(VALIDATE_RECORD_TAG).isSet() ? context.getProperty(VALIDATE_RECORD_TAG).getValue().trim() : null; + final String attributePrefix = context.getProperty(ATTRIBUTE_PREFIX).isSet() ? context.getProperty(ATTRIBUTE_PREFIX).getValue().trim() : null; + final String contentFieldName = context.getProperty(CONTENT_FIELD_NAME).isSet() ? context.getProperty(CONTENT_FIELD_NAME).getValue().trim() : null; + + return new XMLRecordReader(in, schema, rootName, recordName, attributePrefix, contentFieldName, dateFormat, timeFormat, timestampFormat, logger); + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java new file mode 100755 index 000000000000..cf4aa605c8cc --- /dev/null +++ b/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,493 @@ +/* + * 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 LAZY_DATE_FORMAT; + private final Supplier LAZY_TIME_FORMAT; + private final Supplier 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(); + 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 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(){{ add(oldValues); add(newValue); }}; + } + } else { + return new ArrayList(){{ 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 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 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 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 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 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 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 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 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 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(){{ 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); + } + } + return null; + } + + private void skipElement() throws XMLStreamException { + while(xmlEventReader.hasNext()){ + final XMLEvent xmlEvent = xmlEventReader.nextEvent(); + + if (xmlEvent.isStartElement()) { + skipElement(); + } + if (xmlEvent.isEndElement()) { + return; + } + } + } + + @Override + public RecordSchema getSchema() { + return schema; + } + + @Override + public void close() throws IOException { + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService old mode 100644 new mode 100755 index 628dbe5c0273..76c89c2d699b --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -25,4 +25,6 @@ org.apache.nifi.csv.CSVRecordSetWriter org.apache.nifi.grok.GrokReader -org.apache.nifi.text.FreeFormTextRecordSetWriter \ No newline at end of file +org.apache.nifi.text.FreeFormTextRecordSetWriter + +org.apache.nifi.xml.XMLReader \ No newline at end of file diff --git a/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 b/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 new file mode 100755 index 000000000000..bb696d4038d4 --- /dev/null +++ b/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 @@ + + + + + + XMLReader + + + + +

+ 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. +

+

+ Records are expected in the second level of the XML data, embedded within an enclosing root tag: +

+ +
+                <root>
+                  <record>
+                    <field1>content</field1>
+                    <field2>content</field2>
+                  </record>
+                  <record>
+                    <field1>content</field1>
+                    <field2>content</field2>
+                  </record>
+                </root>
+            
+
+ +

+ For the following examples, it is assumed that the exemplary records are enclosed by a root tag. +

+ +

Example 1: Simple Fields

+ +

+ 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, ...). +

+ + +
+                <record>
+                  <simple_field>content</simple_field>
+                </record>
+            
+
+ +

+ 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. +

+ + +
+                {
+                  "namespace": "nifi",
+                  "name": "test",
+                  "type": "record",
+                  "fields": [
+                    { "name": "simple_field", "type": "string" }
+                  ]
+                }
+            
+
+ +

Example 2: Arrays with Simple Fields

+ +

+ 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. +

+ + +
+                <record>
+                  <array_field>content</array_field>
+                  <array_field>content</array_field>
+                  <simple_field>content</simple_field>
+                </record>
+            
+
+ +

+ This record can be described by the following schema: +

+ + +
+                {
+                  "namespace": "nifi",
+                  "name": "test",
+                  "type": "record",
+                  "fields": [
+                    { "name": "array_field", "type":
+                      { "type": "array", "items": string }
+                    },
+                    { "name": "simple_field", "type": "string" }
+                  ]
+                }
+            
+
+ +

+ 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. +

+ +

Example 3: Tags with Attributes

+ +

+ XML fields frequently not only contain content, but also attributes. The following record contains a field with + an attribute "attr" and content: +

+ + +
+                <record>
+                  <field_with_attribute attr="attr_content">content of field</field_with_attribute>
+                </record>
+            
+
+ +

+ To parse the content of the field "field_with_attribute" together with the attribute "attr", two requirements have + to be fulfilled: +

+ +
    +
  • In the schema, the field has to be defined as record.
  • +
  • The property "Field Name for Content" has to be set.
  • +
  • As an option, the property "Attribute Prefix" also can be set.
  • +
+ +

+ For the example above, the following property settings are assumed: +

+ + + + + + + + + + + + + + +
Property NameProperty Value
Field Name for Contentfield_name_for_content
Attribute Prefixprefix_
+ +

+ The schema can be defined as follows: +

+ + +
+                {
+                  "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"}
+                        ]
+                    }
+                  ]
+                }
+            
+
+ +

+ 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: +

+ + +
+                Record (
+                    Record "field_with_attribute" (
+                        RecordField "prefix_attr" = "attr_content",
+                        RecordField "field_name_for_content" = "content of field"
+                    )
+                )
+            
+
+ +

+ 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. +

+ +

Example 4: Tags within tags

+ +

+ XML data is frequently nested. In this case, tags enclose other tags: +

+ +
+                <record>
+                  <field_with_embedded_fields attr="attr_content">
+                    <embedded_field>embedded content</embedded_field>
+                    <another_embedded_field>another embedded content</another_embedded_field>
+                  </field_with_embedded_fields>
+                </record>
+            
+
+ +

+ 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: +

+ + +
+                {
+                  "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"}
+                        ]
+                    }
+                  ]
+                }
+            
+
+ +

+ 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. +

+ +

Example 5: Array of records

+ +

+ 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. +

+ + +
+                <record>
+                  <array_field>
+                    <embedded_field>embedded content 1</embedded_field>
+                    <another_embedded_field>another embedded content 1</another_embedded_field>
+                  </array_field>
+                  <array_field>
+                    <embedded_field>embedded content 2</embedded_field>
+                    <another_embedded_field>another embedded content 2</another_embedded_field>
+                  </array_field>
+                </record>
+            
+
+ +

+ 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. +

+ + +
+                {
+                  "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"}
+                          ]
+                        }
+                      }
+                    }
+                  ]
+                }
+            
+
+ +

Example 6: Array in record

+ +

+ In XML data, arrays are frequently enclosed by tags: +

+ + +
+                <record>
+                  <field_enclosing_array>
+                    <element>content 1</element>
+                    <element>content 2</element>
+                  </field_enclosing_array>
+                  <field_without_array> content 3</field_without_array>
+                </record>
+            
+
+ +

+ 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: +

+ + +
+				{
+                  "namespace": "nifi",
+                  "name": "test",
+                  "type": "record",
+                  "fields": [
+                    { "name": "field_enclosing_array",
+                      "type": {
+					    "name": "EmbeddedRecord",
+                        "type": "record",
+						"fields" : [
+						  {
+                            "name": "element",
+					        "type": {
+						      "type": "array",
+							  "items": "string"
+						    }
+						  }
+						]
+					  }
+					},
+					{ "name": "field_without_array", "type": "string" }
+                  ]
+				}
+            
+
+ + diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java new file mode 100755 index 000000000000..39f914aca5a5 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java @@ -0,0 +1,654 @@ +/* + * 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.SimpleRecordSchema; +import org.apache.nifi.serialization.record.DataType; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.RecordFieldType; +import org.apache.nifi.serialization.record.RecordSchema; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +public class TestXMLRecordReader { + private final String dateFormat = RecordFieldType.DATE.getDefaultFormat(); + private final String timeFormat = RecordFieldType.TIME.getDefaultFormat(); + private final String timestampFormat = RecordFieldType.TIMESTAMP.getDefaultFormat(); + + @Test + public void testSimpleRecord() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, reader.nextRecord().getValues()); + } + + @Test + public void testSimpleRecord2() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema2(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + assertNull(reader.nextRecord(true, true).getValue("AGE")); + assertNull(reader.nextRecord(false, true).getValue("AGE")); + assertNotNull(reader.nextRecord(true, false).getValue("AGE")); + assertNotNull(reader.nextRecord(false, false).getValue("AGE")); + } + + @Test + public void testSimpleRecord3() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + assertEquals(Integer.class, reader.nextRecord(true, true).getValue("AGE").getClass()); + assertEquals(String.class, reader.nextRecord(false, true).getValue("AGE").getClass()); + } + + @Test + public void testSimpleRecord4() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + List fields = getSimpleRecordFields(); + fields.remove(2); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + assertEquals(Integer.class, reader.nextRecord(true, false).getValue("AGE").getClass()); + assertEquals(String.class, reader.nextRecord(false, false).getValue("AGE").getClass()); + } + + @Test + public void testSimpleRecordIgnoreSchema() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_no_attributes.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {"Cleve Butler", "42", "USA"}, reader.nextRecord(false, false).getValues()); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", "33", "UK"}, reader.nextRecord(false, false).getValues()); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", "74", "FR"}, reader.nextRecord(false, false).getValues()); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", "16", "USA"}, reader.nextRecord(false, false).getValues()); + } + + @Test + public void testSimpleRecordWithAttribute() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + List fields = getSimpleRecordFields(); + fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(); + assertTrue(Arrays.asList(first.getValues()).containsAll(Arrays.asList("Cleve Butler", 42, "USA", "P1"))); + assertEquals("P1", first.getAsString("ID")); + + Record second = reader.nextRecord(); + assertTrue(Arrays.asList(second.getValues()).containsAll(Arrays.asList("Ainslie Fletcher", 33, "UK", "P2"))); + assertEquals("P2", second.getAsString("ID")); + + Record third = reader.nextRecord(); + assertTrue(Arrays.asList(third.getValues()).containsAll(Arrays.asList("Amélie Bonfils", 74, "FR", "P3"))); + assertEquals("P3", third.getAsString("ID")); + + Record fourth = reader.nextRecord(); + assertTrue(Arrays.asList(fourth.getValues()).containsAll(Arrays.asList("Elenora Scrivens", 16, "USA", "P4"))); + assertEquals("P4", fourth.getAsString("ID")); + } + + @Test + public void testSimpleRecordWithAttribute2() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + List fields = getSimpleRecordFields(); + fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + "ATTR_", "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(); + assertTrue(Arrays.asList(first.getValues()).containsAll(Arrays.asList("Cleve Butler", 42, "USA"))); + assertEquals("P1", first.getAsString("ATTR_ID")); + + Record second = reader.nextRecord(); + assertTrue(Arrays.asList(second.getValues()).containsAll(Arrays.asList("Ainslie Fletcher", 33, "UK"))); + assertEquals("P2", second.getAsString("ATTR_ID")); + + Record third = reader.nextRecord(); + assertTrue(Arrays.asList(third.getValues()).containsAll(Arrays.asList("Amélie Bonfils", 74, "FR"))); + assertEquals("P3", third.getAsString("ATTR_ID")); + + Record fourth = reader.nextRecord(); + assertTrue(Arrays.asList(fourth.getValues()).containsAll(Arrays.asList("Elenora Scrivens", 16, "USA"))); + assertEquals("P4", fourth.getAsString("ATTR_ID")); + } + + @Test + public void testSimpleRecordWithAttribute3() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(Collections.emptyList()), + "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(true, true); + assertEquals(null, first.getAsString("ID")); + + Record second = reader.nextRecord(false, false); + assertEquals("P2", second.getAsString("ID")); + + Record third = reader.nextRecord(true, false); + assertEquals("P3", third.getAsString("ID")); + + Record fourth = reader.nextRecord(false, true); + assertEquals(null, fourth.getAsString("ID")); + } + + @Test + public void testSimpleRecordWithAttribute4() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people2.xml"); + List fields = getSimpleRecordFields(); + fields.add(new RecordField("ID", RecordFieldType.INT.getDataType())); + + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + assertEquals(Integer.class, reader.nextRecord(true, true).getValue("ID").getClass()); + assertEquals(String.class, reader.nextRecord(false, true).getValue("ID").getClass()); + } + + @Test + public void testSimpleRecordWithAttribute5() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people2.xml"); + List fields = getSimpleRecordFields(); + fields.add(new RecordField("ID", RecordFieldType.INT.getDataType())); + + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + assertEquals(Integer.class, reader.nextRecord(true, false).getValue("ID").getClass()); + assertEquals(String.class, reader.nextRecord(false, false).getValue("ID").getClass()); + } + + @Test + public void testSimpleRecordWithAttributeIgnoreSchema() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + List fields = getSimpleRecordFields(); + fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(false, false); + assertTrue(Arrays.asList(new Object[] {"Cleve Butler", "42", "USA", "P1"}).containsAll(Arrays.asList(first.getValues()))); + assertEquals("P1", first.getAsString("ID")); + + Record second = reader.nextRecord(false, false); + assertTrue(Arrays.asList(new Object[] {"Ainslie Fletcher", "33", "UK", "P2"}).containsAll(Arrays.asList(second.getValues()))); + assertEquals("P2", second.getAsString("ID")); + + Record third = reader.nextRecord(false, false); + assertTrue(Arrays.asList(new Object[] {"Amélie Bonfils", "74", "FR", "P3"}).containsAll(Arrays.asList(third.getValues()))); + assertEquals("P3", third.getAsString("ID")); + + Record fourth = reader.nextRecord(false, false); + assertTrue(Arrays.asList(new Object[] {"Elenora Scrivens", "16", "USA", "P4"}).containsAll(Arrays.asList(fourth.getValues()))); + assertEquals("P4", fourth.getAsString("ID")); + } + + + @Test + public void testNestedRecord() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); + RecordSchema schema = getSchemaWithNestedRecord(); + XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + Object[] valuesFirstRecord = reader.nextRecord().getValues(); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, Arrays.copyOfRange(valuesFirstRecord, 0, valuesFirstRecord.length - 1)); + Assert.assertArrayEquals(new Object[] {"292 West Street", "Jersey City"},((Record) valuesFirstRecord[valuesFirstRecord.length - 1]).getValues()); + + Object[] valuesSecondRecord = reader.nextRecord().getValues(); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, Arrays.copyOfRange(valuesSecondRecord, 0, valuesSecondRecord.length - 1)); + Assert.assertArrayEquals(new Object[] {"123 6th St.", "Seattle"},((Record) valuesSecondRecord[valuesSecondRecord.length - 1]).getValues()); + + Object[] valuesThirdRecord = reader.nextRecord().getValues(); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, Arrays.copyOfRange(valuesThirdRecord, 0, valuesThirdRecord.length - 1)); + Assert.assertArrayEquals(new Object[] {"44 Shirley Ave.", "Los Angeles"},((Record) valuesThirdRecord[valuesThirdRecord.length - 1]).getValues()); + + Object[] valuesFourthRecord = reader.nextRecord().getValues(); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, Arrays.copyOfRange(valuesFourthRecord, 0, valuesFourthRecord.length - 1)); + Assert.assertArrayEquals(new Object[] {"70 Bowman St." , "Columbus"},((Record) valuesFourthRecord[valuesFourthRecord.length - 1]).getValues()); + } + + @Test + public void testNestedRecordIgnoreSchema() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); + RecordSchema schema = getSchemaWithNestedRecord(); + XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(false, false); + Object[] valuesFirstRecord = first.getValues(); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", "42", "USA"}, Arrays.copyOfRange(valuesFirstRecord, 0, valuesFirstRecord.length - 1)); + assertEquals("P1", first.getAsString("ID")); + + Record nestedFirstRecord = (Record) first.getValue("ADDRESS"); + Assert.assertEquals("Jersey City", nestedFirstRecord.getAsString("CITY")); + Assert.assertEquals("292 West Street", nestedFirstRecord.getAsString("STREET")); + + Record second = reader.nextRecord(false, false); + Object[] valuesSecondRecord = second.getValues(); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", "33", "UK"}, Arrays.copyOfRange(valuesSecondRecord, 0, valuesSecondRecord.length - 1)); + assertEquals("P2", second.getAsString("ID")); + + Record nestedSecondRecord = (Record) second.getValue("ADDRESS"); + Assert.assertEquals("Seattle", nestedSecondRecord.getAsString("CITY")); + Assert.assertEquals("123 6th St.", nestedSecondRecord.getAsString("STREET")); + + Record third = reader.nextRecord(false, false); + Object[] valuesThirdRecord = third.getValues(); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", "74", "FR"}, Arrays.copyOfRange(valuesThirdRecord, 0, valuesThirdRecord.length - 1)); + assertEquals("P3", third.getAsString("ID")); + + Record nestedThirdRecord = (Record) third.getValue("ADDRESS"); + Assert.assertEquals("Los Angeles", nestedThirdRecord.getAsString("CITY")); + Assert.assertEquals("44 Shirley Ave.", nestedThirdRecord.getAsString("STREET")); + + Record fourth = reader.nextRecord(false, false); + Object[] valuesFourthRecord = fourth.getValues(); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", "16", "USA"}, Arrays.copyOfRange(valuesFourthRecord, 0, valuesFourthRecord.length - 1)); + assertEquals("P4", fourth.getAsString("ID")); + + Record nestedFourthRecord = (Record) fourth.getValue("ADDRESS"); + Assert.assertEquals("Columbus", nestedFourthRecord.getAsString("CITY")); + Assert.assertEquals("70 Bowman St.", nestedFourthRecord.getAsString("STREET")); + } + + @Test + public void testSimpleArray() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_array_simple.xml"); + RecordSchema schema = getSchemaWithSimpleArray(); + XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record firstRecord = reader.nextRecord(); + Object[] valuesFirstRecord = firstRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, Arrays.copyOfRange(valuesFirstRecord, 0, valuesFirstRecord.length - 1)); + Object[] nestedArrayFirstRecord = (Object[]) valuesFirstRecord[valuesFirstRecord.length - 1]; + assertEquals(2, nestedArrayFirstRecord.length); + Assert.assertArrayEquals(new Object[] {"child1", "child2"}, nestedArrayFirstRecord); + assertNotEquals(null, firstRecord.getValue("CHILD")); + + Record secondRecord = reader.nextRecord(); + Object[] valuesSecondRecord = secondRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, Arrays.copyOfRange(valuesSecondRecord, 0, valuesSecondRecord.length - 1)); + Object[] nestedArraySecondRecord = (Object[]) valuesSecondRecord[valuesSecondRecord.length - 1]; + assertEquals(1, nestedArraySecondRecord.length); + Assert.assertArrayEquals(new Object[] {"child1"}, nestedArraySecondRecord); + assertNotEquals(null, secondRecord.getValue("CHILD")); + + Record thirdRecord = reader.nextRecord(); + Object[] valuesThirdRecord = thirdRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, Arrays.copyOfRange(valuesThirdRecord, 0, valuesThirdRecord.length - 1)); + Object[] nestedArrayThirdRecord = (Object[]) valuesThirdRecord[valuesThirdRecord.length - 1]; + assertEquals(3, nestedArrayThirdRecord.length); + Assert.assertArrayEquals(new Object[] {"child1", "child2", "child3"}, nestedArrayThirdRecord); + assertNotEquals(null, thirdRecord.getValue("CHILD")); + + Record valuesFourthRecord = reader.nextRecord(); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, Arrays.copyOfRange(valuesFourthRecord.getValues(), 0, valuesFourthRecord.getValues().length - 1)); + assertEquals(null, valuesFourthRecord.getValue("CHILD")); + } + + @Test + public void testSimpleArrayIgnoreSchema() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_array_simple.xml"); + RecordSchema schema = getSchemaWithSimpleArray(); + XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(false, false); + Object[] valuesFirstRecord = first.getValues(); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", "42", "USA"}, Arrays.copyOfRange(valuesFirstRecord, 0, valuesFirstRecord.length - 1)); + Object[] nestedArrayFirstRecord = (Object[]) valuesFirstRecord[valuesFirstRecord.length - 1]; + assertEquals(2, nestedArrayFirstRecord.length); + Assert.assertArrayEquals(new Object[] {"child1", "child2"}, nestedArrayFirstRecord); + assertNotEquals(null, first.getValue("CHILD")); + + Record second = reader.nextRecord(false, false); + Object[] valuesSecondRecord = second.getValues(); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", "33", "UK"}, Arrays.copyOfRange(valuesSecondRecord, 0, valuesSecondRecord.length - 1)); + String nestedArraySecondRecord = (String) valuesSecondRecord[valuesSecondRecord.length - 1]; + Assert.assertEquals("child1", nestedArraySecondRecord); + assertNotEquals(null, second.getValue("CHILD")); + + Record third = reader.nextRecord(false, false); + Object[] valuesThirdRecord = third.getValues(); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", "74", "FR"}, Arrays.copyOfRange(valuesThirdRecord, 0, valuesThirdRecord.length - 1)); + Object[] nestedArrayThirdRecord = (Object[]) valuesThirdRecord[valuesThirdRecord.length - 1]; + assertEquals(3, nestedArrayThirdRecord.length); + Assert.assertArrayEquals(new Object[] {"child1", "child2", "child3"}, nestedArrayThirdRecord); + assertNotEquals(null, third.getValue("CHILD")); + + Record fourth = reader.nextRecord(false, false); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", "16", "USA"}, Arrays.copyOfRange(fourth.getValues(), 0, fourth.getValues().length - 1)); + assertEquals(null, fourth.getValue("CHILD")); + } + + @Test + public void testNestedArrayInNestedRecord() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_array.xml"); + RecordSchema schema = getSchemaWithNestedArray(); + XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record firstRecord = reader.nextRecord(); + Object[] valuesFirstRecord = firstRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, Arrays.copyOfRange(valuesFirstRecord, 0, valuesFirstRecord.length - 1)); + + Record nestedArrayFirstRecord = (Record) firstRecord.getValue("CHILDREN"); + assertEquals(2, ((Object[]) nestedArrayFirstRecord.getValue("CHILD")).length); + Assert.assertArrayEquals(new Object[] {"child1", "child2"}, ((Object[]) nestedArrayFirstRecord.getValue("CHILD"))); + + Record secondRecord = reader.nextRecord(); + Object[] valuesSecondRecord = secondRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, Arrays.copyOfRange(valuesSecondRecord, 0, valuesSecondRecord.length - 1)); + + Record nestedArraySecondRecord = (Record) secondRecord.getValue("CHILDREN"); + assertEquals(1, ((Object[]) nestedArraySecondRecord.getValue("CHILD")).length); + Assert.assertArrayEquals(new Object[] {"child1"}, ((Object[]) nestedArraySecondRecord.getValue("CHILD"))); + + Record thirdRecord = reader.nextRecord(); + Object[] valuesThirdRecord = thirdRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, Arrays.copyOfRange(valuesThirdRecord, 0, valuesThirdRecord.length - 1)); + + Record nestedArrayThirdRecord = (Record) thirdRecord.getValue("CHILDREN"); + assertEquals(3, ((Object[]) nestedArrayThirdRecord.getValue("CHILD")).length); + Assert.assertArrayEquals(new Object[] {"child1", "child2", "child3"}, ((Object[]) nestedArrayThirdRecord.getValue("CHILD"))); + + Record fourthRecord = reader.nextRecord(); + Object[] valuesFourthRecord = fourthRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, Arrays.copyOfRange(valuesFourthRecord, 0, valuesFourthRecord.length - 1)); + + Assert.assertEquals(null, fourthRecord.getValue("CHILDREN")); + } + + @Test + public void testDeeplyNestedArraysAndRecords() throws IOException, MalformedRecordException { + // test records in nested arrays + InputStream is = new FileInputStream("src/test/resources/xml/people_complex1.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(true, true); + Object[] grandchildren_arr = (Object[]) first.getValue("CHILDREN"); + + Record first_1_1_1 = (Record)(((Object[])((Record) grandchildren_arr[0]).getValue("CHILD"))[0]); + assertEquals("daughter", first_1_1_1.getValue("ROLE")); + assertEquals("1-1-1", first_1_1_1.getValue("ID")); + assertEquals("Selina", first_1_1_1.getValue("NAME")); + + Record first_1_1_2 = (Record)(((Object[])((Record) grandchildren_arr[0]).getValue("CHILD"))[1]); + assertEquals("son", first_1_1_2.getValue("ROLE")); + assertEquals("1-1-2", first_1_1_2.getValue("ID")); + assertEquals("Hans", first_1_1_2.getValue("NAME")); + + Record first_1_1_3 = (Record)(((Object[])((Record) grandchildren_arr[1]).getValue("CHILD"))[0]); + assertEquals("daughter", first_1_1_3.getValue("ROLE")); + assertEquals("1-2-1", first_1_1_3.getValue("ID")); + assertEquals("Selina2", first_1_1_3.getValue("NAME")); + + Record first_1_1_4 = (Record)(((Object[])((Record) grandchildren_arr[1]).getValue("CHILD"))[1]); + assertEquals("son", first_1_1_4.getValue("ROLE")); + assertEquals("1-2-2", first_1_1_4.getValue("ID")); + assertEquals("Hans2", first_1_1_4.getValue("NAME")); + + Record second = reader.nextRecord(true, true); + Object[] grandchildren_arr2 = (Object[]) second.getValue("CHILDREN"); + + Record second_2_1_1 = (Record)(((Object[])((Record) grandchildren_arr2[0]).getValue("CHILD"))[0]); + assertEquals("daughter", second_2_1_1.getValue("ROLE")); + assertEquals("2-1-1", second_2_1_1.getValue("ID")); + assertEquals("Selina3", second_2_1_1.getValue("NAME")); + } + + @Test + public void testDeeplyNestedArraysAndRecords2() throws IOException, MalformedRecordException { + // test multiply nested arrays and records (recursion) + InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(); + assertEquals("grandmother", first.getValue("ROLE")); + assertEquals("1", first.getValue("ID")); + assertEquals("Lisa", first.getValue("NAME")); + + Object[] gm_spouses = (Object[]) first.getValue("CHILDREN"); + assertEquals(2, gm_spouses.length); + + Object[] gm_spouse1_parents = (Object[]) ((Record) gm_spouses[0]).getValue("CHILD"); + assertEquals(2, gm_spouse1_parents.length); + + Record first_1_1 = (Record) gm_spouse1_parents[0]; + assertEquals("mother", first_1_1.getValue("ROLE")); + assertEquals("1-1", first_1_1.getValue("ID")); + assertEquals("Anna", first_1_1.getValue("NAME")); + + Object[] gm_spouse1_parent1_first_husband = (Object[]) first_1_1.getValue("CHILDREN"); + assertEquals(1, gm_spouse1_parent1_first_husband.length); + Object[] gm_spouse1_parent1_children = (Object[])((Record) gm_spouse1_parent1_first_husband[0]).getValue("CHILD"); + + Record first_1_1_1 = (Record) gm_spouse1_parent1_children[0]; + assertEquals("daughter", first_1_1_1.getValue("ROLE")); + assertEquals("1-1-1", first_1_1_1.getValue("ID")); + assertEquals("Selina", first_1_1_1.getValue("NAME")); + + Record first_1_1_2 = (Record) gm_spouse1_parent1_children[1]; + assertEquals("son", first_1_1_2.getValue("ROLE")); + assertEquals("1-1-2", first_1_1_2.getValue("ID")); + assertEquals("Hans", first_1_1_2.getValue("NAME")); + + Record first_1_2 = (Record) gm_spouse1_parents[1]; + assertEquals("mother", first_1_2.getValue("ROLE")); + assertEquals("1-2", first_1_2.getValue("ID")); + assertEquals("Catrina", first_1_2.getValue("NAME")); + + Object[] gm_spouse2_parents = (Object[]) ((Record) gm_spouses[1]).getValue("CHILD"); + assertEquals(1, gm_spouse2_parents.length); + + Record second = reader.nextRecord(); + Record second_2_1_1 = (Record)((Object[])((Record)((Object[])((Record)((Object[])((Record)((Object[]) second + .getValue("CHILDREN"))[0]) + .getValue("CHILD"))[0]) + .getValue("CHILDREN"))[0]) + .getValue("CHILD"))[0]; + assertEquals("daughter", second_2_1_1.getValue("ROLE")); + assertEquals("2-1-1", second_2_1_1.getValue("ID")); + assertEquals("Selina3", second_2_1_1.getValue("NAME")); + } + + @Test + public void testDeeplyNestedArraysAndRecordsIgnoreSchema() throws IOException, MalformedRecordException { + // test multiply nested arrays and records (recursion) + InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml"); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(Collections.emptyList()), + "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(false, false); + assertEquals("1", first.getValue("ID")); + assertEquals("Lisa", first.getValue("NAME")); + assertEquals("grandmother", first.getValue("ROLE")); + Object[] gm_arr = (Object[]) first.getValue("CHILDREN"); + assertEquals(2, gm_arr.length); + + Record gm_hus1_arr_rec = (Record) gm_arr[0]; + assertEquals("husband1", gm_hus1_arr_rec.getValue("SPOUSE")); + Object[] gm_hus1_arr_rec_arr = (Object[]) gm_hus1_arr_rec.getValue("CHILD"); + assertEquals(2, gm_hus1_arr_rec_arr.length); + + Record child1_1 = (Record) gm_hus1_arr_rec_arr[0]; + assertEquals("1-1", child1_1.getValue("ID")); + assertEquals("Anna", child1_1.getValue("NAME")); + assertEquals("mother", child1_1.getValue("ROLE")); + + Record child1_1_rec = (Record) child1_1.getValue("CHILDREN"); + assertEquals("first husband", child1_1_rec.getValue("ID")); + Object[] child1_1_rec_arr = (Object[]) child1_1_rec.getValue("CHILD"); + assertEquals(2, child1_1_rec_arr.length); + + Record child1_1_1 = (Record) child1_1_rec_arr[0]; + assertEquals("1-1-1", child1_1_1.getValue("ID")); + assertEquals("Selina", child1_1_1.getValue("NAME")); + assertEquals("daughter", child1_1_1.getValue("ROLE")); + + Record child1_1_2 = (Record) child1_1_rec_arr[1]; + assertEquals("1-1-2", child1_1_2.getValue("ID")); + assertEquals("Hans", child1_1_2.getValue("NAME")); + assertEquals("son", child1_1_2.getValue("ROLE")); + + Record child1_2 = (Record) gm_hus1_arr_rec_arr[1]; + assertEquals("1-2", child1_2.getValue("ID")); + assertEquals("Catrina", child1_2.getValue("NAME")); + assertEquals("mother", child1_2.getValue("ROLE")); + + Record gm_hus2_arr_rec = (Record) gm_arr[1]; + assertEquals("husband2", gm_hus2_arr_rec.getValue("SPOUSE")); + + Record child1_3 = (Record) gm_hus2_arr_rec.getValue("CHILD"); + assertEquals("1-3", child1_3.getValue("ID")); + assertEquals("Anna2", child1_3.getValue("NAME")); + assertEquals("mother", child1_3.getValue("ROLE")); + assertEquals(2, ((Object[])((Record) child1_3.getValue("CHILDREN")).getValue("CHILD")).length); + + Record second = reader.nextRecord(false, false); + assertEquals("2-1-1", ((Record)((Record)((Record)((Record) second.getValue("CHILDREN")) + .getValue("CHILD")) + .getValue("CHILDREN")) + .getValue("CHILD")) + .getValue("ID")); + } + + private List getSimpleRecordFields() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("NAME", RecordFieldType.STRING.getDataType())); + fields.add(new RecordField("AGE", RecordFieldType.INT.getDataType())); + fields.add(new RecordField("COUNTRY", RecordFieldType.STRING.getDataType())); + return fields; + } + + private List getSimpleRecordFields2() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("NAME", RecordFieldType.STRING.getDataType())); + fields.add(new RecordField("COUNTRY", RecordFieldType.STRING.getDataType())); + return fields; + } + + private List getNestedRecordFields() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("STREET", RecordFieldType.STRING.getDataType())); + fields.add(new RecordField("CITY", RecordFieldType.STRING.getDataType())); + return fields; + } + + private RecordSchema getSimpleSchema() { + return new SimpleRecordSchema(getSimpleRecordFields()); + } + + private RecordSchema getSimpleSchema2() { + return new SimpleRecordSchema(getSimpleRecordFields2()); + } + + private RecordSchema getNestedSchema() { + return new SimpleRecordSchema(getNestedRecordFields()); + } + + private RecordSchema getSchemaWithNestedRecord() { + final List fields = getSimpleRecordFields(); + final DataType recordType = RecordFieldType.RECORD.getRecordDataType(getNestedSchema()); + fields.add(new RecordField("ADDRESS", recordType)); + return new SimpleRecordSchema(fields); + } + + private RecordSchema getSchemaWithSimpleArray() { + final List fields = getSimpleRecordFields(); + final DataType arrayType = RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.STRING.getDataType()); + fields.add(new RecordField("CHILD", arrayType)); + return new SimpleRecordSchema(fields); + } + + private RecordSchema getSchemaWithNestedArray() { + final List fields = getSimpleRecordFields(); + final DataType arrayType = RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.STRING.getDataType()); + final List nestedArrayField = new ArrayList() {{ add(new RecordField("CHILD", arrayType)); }}; + + final DataType recordType = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(nestedArrayField)); + fields.add(new RecordField("CHILDREN", recordType)); + return new SimpleRecordSchema(fields); + } + + private List getSimpleFieldsForComplexData() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); + fields.add(new RecordField("NAME", RecordFieldType.STRING.getDataType())); + fields.add(new RecordField("ROLE", RecordFieldType.STRING.getDataType())); + return fields; + } + + private RecordSchema getSchemaForComplexData() { + final DataType grandchild = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(getSimpleFieldsForComplexData())); + final DataType grandchild_arr1 = RecordFieldType.ARRAY.getArrayDataType(grandchild); + final DataType grandchildren = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema( + new ArrayList() {{ add(new RecordField("CHILD", grandchild_arr1)); }})); + final DataType grandchild_arr = RecordFieldType.ARRAY.getArrayDataType(grandchildren); + return new SimpleRecordSchema( + new ArrayList() {{ add(new RecordField("CHILDREN", grandchild_arr)); }}); + } + + private RecordSchema getSchemaForComplexData2() { + final DataType grandchild = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(getSimpleFieldsForComplexData())); + final DataType grandchild_arr = RecordFieldType.ARRAY.getArrayDataType(grandchild); + + final DataType grandchildren = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema( + new ArrayList() {{ add(new RecordField("CHILD", grandchild_arr)); }})); + final DataType grandchildren_arr = RecordFieldType.ARRAY.getArrayDataType(grandchildren); + + final DataType parent = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema( + new ArrayList() {{ + add(new RecordField("CHILDREN", grandchildren_arr)); + addAll(getSimpleFieldsForComplexData()); + }})); + final DataType parent_arr = RecordFieldType.ARRAY.getArrayDataType(parent); + + final DataType parents = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema( + new ArrayList() {{ + add(new RecordField("CHILD", parent_arr)); + }})); + final DataType parents_arr = RecordFieldType.ARRAY.getArrayDataType(parents); + + final List fields = new ArrayList() {{ + add(new RecordField("CHILDREN", parents_arr)); + addAll(getSimpleFieldsForComplexData()); + }}; + return new SimpleRecordSchema(fields); + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people.xml new file mode 100755 index 000000000000..db5ba8d4446a --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people.xml @@ -0,0 +1,22 @@ + + + Cleve Butler + 42 + USA + + + Ainslie Fletcher + 33 + UK + + + Amélie Bonfils + 74 + FR + + + Elenora Scrivens + 16 + USA + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people2.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people2.xml new file mode 100755 index 000000000000..fee33895c3b8 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people2.xml @@ -0,0 +1,12 @@ + + + Cleve Butler + 42 + USA + + + Ainslie Fletcher + 33 + UK + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people3.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people3.xml new file mode 100755 index 000000000000..afff2d4be761 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people3.xml @@ -0,0 +1,17 @@ + + + Cleve Butler + 42 + USA + + + Ainslie Fletcher + 33 + UK + + + Ainslie Fletcher + 33 + UK + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people4.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people4.xml new file mode 100755 index 000000000000..fc85045a6c19 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people4.xml @@ -0,0 +1,12 @@ + + + Cleve Butler + 42 + USA + + + Ainslie Fletcher + 33 + UK + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array.xml new file mode 100755 index 000000000000..f48baebc703c --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array.xml @@ -0,0 +1,36 @@ + + + Cleve Butler + 42 + USA + + child1 + child2 + + + + Ainslie Fletcher + 33 + UK + + child1 + + + + Amélie Bonfils + 74 + FR + + child1 + child2 + child3 + + + + Elenora Scrivens + 16 + USA + + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array_simple.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array_simple.xml new file mode 100755 index 000000000000..cbd9e6336050 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array_simple.xml @@ -0,0 +1,28 @@ + + + Cleve Butler + 42 + USA + child1 + child2 + + + Ainslie Fletcher + 33 + UK + child1 + + + Amélie Bonfils + 74 + FR + child1 + child2 + child3 + + + Elenora Scrivens + 16 + USA + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_complex1.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_complex1.xml new file mode 100755 index 000000000000..474e1d332d66 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_complex1.xml @@ -0,0 +1,33 @@ + + + + + Selina + daughter + + + Hans + son + + + + + Selina2 + daughter + + + Hans2 + son + + + + + Tom + + + Selina3 + daughter + + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_complex2.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_complex2.xml new file mode 100755 index 000000000000..f25e12e4d576 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_complex2.xml @@ -0,0 +1,73 @@ + + + Lisa + grandmother + + + Anna + mother + + + Selina + daughter + + + Hans + son + + + + + Catrina + mother + + + Maria + daughter + + + + + Sarah + daughter + + + Jacob + son + + + + + + + Anna2 + mother + + + Selina2 + daughter + + + Hans2 + son + + + + + + + Tom + + + Anna3 + father + + + Selina3 + daughter + + + + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_empty.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_empty.xml new file mode 100755 index 000000000000..cba5beb8e2ac --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_empty.xml @@ -0,0 +1,12 @@ + + + + + + + + + + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_invalid.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_invalid.xml new file mode 100755 index 000000000000..e37ec77bc100 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_invalid.xml @@ -0,0 +1,21 @@ + + + Cleve Butler + 42 + USA + + Ainslie Fletcher + 33 + UK + + + Amélie Bonfils + 74 + FR + + + Elenora Scrivens + 16 + USA + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_nested.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_nested.xml new file mode 100755 index 000000000000..a3cd19a820f4 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_nested.xml @@ -0,0 +1,38 @@ + + + Cleve Butler + 42 + USA +
+ 292 West Street + Jersey City +
+
+ + Ainslie Fletcher + 33 + UK +
+ 123 6th St. + Seattle +
+
+ + Amélie Bonfils + 74 + FR +
+ 44 Shirley Ave. + Los Angeles +
+
+ + Elenora Scrivens + 16 + USA +
+ 70 Bowman St. + Columbus +
+
+
\ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_no_attributes.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_no_attributes.xml new file mode 100755 index 000000000000..fd7d5f7e29c6 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_no_attributes.xml @@ -0,0 +1,22 @@ + + + Cleve Butler + 42 + USA + + + Ainslie Fletcher + 33 + UK + + + Amélie Bonfils + 74 + FR + + + Elenora Scrivens + 16 + USA + + \ No newline at end of file From 8ea97711bedfe51db7c545d27bd731913f6b7259 Mon Sep 17 00:00:00 2001 From: JohannesDaniel Date: Sat, 31 Mar 2018 13:55:07 +0200 Subject: [PATCH 2/6] NIFI-4185 R1 --- .../java/org/apache/nifi/xml/XMLReader.java | 20 +- .../org/apache/nifi/xml/XMLRecordReader.java | 9 + .../additionalDetails.html | 32 +- .../org/apache/nifi/xml/TestXMLReader.java | 140 +++++++ .../nifi/xml/TestXMLReaderProcessor.java | 63 +++ .../apache/nifi/xml/TestXMLRecordReader.java | 390 +++++++++++++++++- .../src/test/resources/xml/people_cdata.xml | 22 + .../src/test/resources/xml/people_invalid.xml | 2 +- .../test/resources/xml/people_namespace.xml | 22 + .../xml/people_with_header_and_comments.xml | 29 ++ .../src/test/resources/xml/testschema | 11 + 11 files changed, 708 insertions(+), 32 deletions(-) create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_cdata.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_namespace.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_with_header_and_comments.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/testschema diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java index e047515473ab..faa474391dfb 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java @@ -50,6 +50,7 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact "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(); @@ -60,6 +61,7 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact "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(); @@ -68,6 +70,7 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact .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.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) .required(false) .build(); @@ -79,6 +82,7 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact "If tags with content shall be parsed together with attributes (e. g. content), " + "they have to be defined as records. For additional information, see the section of processor usage.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) .required(false) .build(); @@ -111,10 +115,18 @@ public RecordReader createRecordReader(final Map variables, fina final ConfigurationContext context = getConfigurationContext(); final RecordSchema schema = getSchema(variables, in, null); - final String rootName = context.getProperty(VALIDATE_ROOT_TAG).isSet() ? context.getProperty(VALIDATE_ROOT_TAG).getValue().trim() : null; - final String recordName = context.getProperty(VALIDATE_RECORD_TAG).isSet() ? context.getProperty(VALIDATE_RECORD_TAG).getValue().trim() : null; - final String attributePrefix = context.getProperty(ATTRIBUTE_PREFIX).isSet() ? context.getProperty(ATTRIBUTE_PREFIX).getValue().trim() : null; - final String contentFieldName = context.getProperty(CONTENT_FIELD_NAME).isSet() ? context.getProperty(CONTENT_FIELD_NAME).getValue().trim() : null; + + final String rootName = context.getProperty(VALIDATE_ROOT_TAG).isSet() ? + context.getProperty(VALIDATE_ROOT_TAG).evaluateAttributeExpressions(variables).getValue().trim() : null; + + final String recordName = context.getProperty(VALIDATE_RECORD_TAG).isSet() ? + context.getProperty(VALIDATE_RECORD_TAG).evaluateAttributeExpressions(variables).getValue().trim() : null; + + final String attributePrefix = context.getProperty(ATTRIBUTE_PREFIX).isSet() ? + context.getProperty(ATTRIBUTE_PREFIX).evaluateAttributeExpressions(variables).getValue().trim() : null; + + final String contentFieldName = context.getProperty(CONTENT_FIELD_NAME).isSet() ? + context.getProperty(CONTENT_FIELD_NAME).evaluateAttributeExpressions(variables).getValue().trim() : null; return new XMLRecordReader(in, schema, rootName, recordName, attributePrefix, contentFieldName, dateFormat, timeFormat, timestampFormat, logger); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java index cf4aa605c8cc..c99b8d412afa 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java +++ b/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); + xmlEventReader = xmlInputFactory.createXMLEventReader(in); final StartElement rootTag = getNextStartTag(); @@ -489,5 +493,10 @@ public RecordSchema getSchema() { @Override public void close() throws IOException { + try { + xmlEventReader.close(); + } catch (XMLStreamException e) { + logger.error("Unable to close XMLEventReader"); + } } } diff --git a/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 b/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 index bb696d4038d4..e02b5c67ff55 100755 --- a/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 +++ b/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 @@ -257,7 +257,7 @@

Example 4: Tags within tags

"type": "record", "fields" : [ {"name": "attr", "type": "string"}, - {"name": "embedded_field", "type": "string"} + {"name": "embedded_field", "type": "string"}, {"name": "another_embedded_field", "type": "string"} ] } @@ -275,7 +275,7 @@

Example 5: Array of records

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 + The following record contains the field "array_field", which repeatedly occurs. The field contains two embedded fields.

@@ -349,29 +349,29 @@

Example 6: Array in record

-				{
+                {
                   "namespace": "nifi",
                   "name": "test",
                   "type": "record",
                   "fields": [
                     { "name": "field_enclosing_array",
                       "type": {
-					    "name": "EmbeddedRecord",
+                        "name": "EmbeddedRecord",
                         "type": "record",
-						"fields" : [
-						  {
+                        "fields" : [
+                          {
                             "name": "element",
-					        "type": {
-						      "type": "array",
-							  "items": "string"
-						    }
-						  }
-						]
-					  }
-					},
-					{ "name": "field_without_array", "type": "string" }
+                            "type": {
+                              "type": "array",
+                              "items": "string"
+                            }
+                          }
+                        ]
+                      }
+                    },
+                    { "name": "field_without_array", "type": "string" }
                   ]
-				}
+                }
             
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java new file mode 100755 index 000000000000..65278286efe4 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java @@ -0,0 +1,140 @@ +package org.apache.nifi.xml; + +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.schema.access.SchemaAccessUtils; +import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; + +import static junit.framework.TestCase.assertEquals; + +public class TestXMLReader { + + private TestRunner runner; + private XMLReader reader; + + private final String ROOT_TAG_NAME = "root_tag"; + private final String RECORD_TAG_NAME = "record_tag"; + private final String ATTRIBUTE_PREFIX = "attribute_prefix"; + + @Before + public void setup() throws InitializationException, IOException { + + runner = TestRunners.newTestRunner(TestXMLReaderProcessor.class); + reader = new XMLReader(); + runner.addControllerService("xml_reader", reader); + runner.setProperty(TestXMLReaderProcessor.XML_READER, "xml_reader"); + + final String outputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/xml/testschema"))); + runner.setProperty(reader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY); + runner.setProperty(reader, SchemaAccessUtils.SCHEMA_TEXT, outputSchemaText); + } + + @Test + public void testNoValidation() throws FileNotFoundException { + runner.enableControllerService(reader); + + InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + + runner.enqueue(is); + runner.run(); + List flowFile = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS); + List records = Arrays.asList((new String(runner.getContentAsByteArray(flowFile.get(0)))).split("\n")); + + assertEquals(3, records.size()); + assertEquals("MapRecord[{COUNTRY=USA, ID=1, NAME=Cleve Butler, AGE=42}]", records.get(0)); + assertEquals("MapRecord[{COUNTRY=UK, ID=2, NAME=Ainslie Fletcher, AGE=33}]", records.get(1)); + assertEquals("MapRecord[{COUNTRY=UK, ID=3, NAME=Ainslie Fletcher, AGE=33}]", records.get(2)); + } + + @Test + public void testRootValidation() throws FileNotFoundException { + runner.setProperty(reader, XMLReader.VALIDATE_ROOT_TAG, "${" + ROOT_TAG_NAME + "}"); + runner.enableControllerService(reader); + + InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + runner.enqueue(is, new HashMap() {{ + put(ROOT_TAG_NAME, "PEOPLE"); + }}); + runner.run(); + + List flowFile = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS); + List records = Arrays.asList((new String(runner.getContentAsByteArray(flowFile.get(0)))).split("\n")); + + assertEquals(3, records.size()); + assertEquals("MapRecord[{COUNTRY=USA, ID=1, NAME=Cleve Butler, AGE=42}]", records.get(0)); + assertEquals("MapRecord[{COUNTRY=UK, ID=2, NAME=Ainslie Fletcher, AGE=33}]", records.get(1)); + assertEquals("MapRecord[{COUNTRY=UK, ID=3, NAME=Ainslie Fletcher, AGE=33}]", records.get(2)); + } + + @Test + public void testInvalidRoot() throws FileNotFoundException { + runner.setProperty(reader, XMLReader.VALIDATE_ROOT_TAG, "${" + ROOT_TAG_NAME + "}"); + runner.enableControllerService(reader); + + InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + runner.enqueue(is, new HashMap() {{ + put(ROOT_TAG_NAME, "WRONG_ROOT"); + }}); + try { + runner.run(); + } catch (Exception e){ + assertEquals("Name of root tag \"PEOPLE\" does not match root tag validation \"WRONG_ROOT\".", e.getMessage()); + assert(e instanceof MalformedRecordException); + } + } + + @Test + public void testRecordValidation() throws FileNotFoundException { + runner.setProperty(reader, XMLReader.VALIDATE_ROOT_TAG, "${" + ROOT_TAG_NAME + "}"); + runner.setProperty(reader, XMLReader.VALIDATE_RECORD_TAG, "${" + RECORD_TAG_NAME + "}"); + runner.enableControllerService(reader); + + InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + runner.enqueue(is, new HashMap() {{ + put(ROOT_TAG_NAME, "PEOPLE"); + put(RECORD_TAG_NAME, "PERSON"); + }}); + runner.run(); + + List flowFile = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS); + List records = Arrays.asList((new String(runner.getContentAsByteArray(flowFile.get(0)))).split("\n")); + + assertEquals(2, records.size()); + assertEquals("MapRecord[{COUNTRY=USA, ID=1, NAME=Cleve Butler, AGE=42}]", records.get(0)); + assertEquals("MapRecord[{COUNTRY=UK, ID=2, NAME=Ainslie Fletcher, AGE=33}]", records.get(1)); + } + + @Test + public void testAttributePrefix() throws FileNotFoundException { + runner.setProperty(reader, XMLReader.ATTRIBUTE_PREFIX, "${" + ATTRIBUTE_PREFIX + "}"); + runner.enableControllerService(reader); + + InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + runner.enqueue(is, new HashMap() {{ + put(ATTRIBUTE_PREFIX, "ATTR_"); + }}); + runner.run(); + + List flowFile = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS); + List records = Arrays.asList((new String(runner.getContentAsByteArray(flowFile.get(0)))).split("\n")); + + assertEquals(3, records.size()); + assertEquals("MapRecord[{COUNTRY=USA, ATTR_ID=1, NAME=Cleve Butler, AGE=42}]", records.get(0)); + assertEquals("MapRecord[{COUNTRY=UK, ATTR_ID=2, NAME=Ainslie Fletcher, AGE=33}]", records.get(1)); + assertEquals("MapRecord[{COUNTRY=UK, ATTR_ID=3, NAME=Ainslie Fletcher, AGE=33}]", records.get(2)); + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java new file mode 100755 index 000000000000..f205cfdf23d8 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java @@ -0,0 +1,63 @@ +package org.apache.nifi.xml; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.util.StringUtils; + +import java.io.InputStream; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class TestXMLReaderProcessor extends AbstractProcessor { + + static final PropertyDescriptor XML_READER = new PropertyDescriptor.Builder() + .name("xml_reader") + .description("xml_reader") + .identifiesControllerService(XMLReader.class) + .required(true) + .build(); + + public static final Relationship SUCCESS = new Relationship.Builder().name("success").description("success").build(); + + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + final RecordReaderFactory readerFactory = context.getProperty(XML_READER).asControllerService(RecordReaderFactory.class); + + final List records = new ArrayList<>(); + + try (final InputStream in = session.read(flowFile); + final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger())) { + Record record; + while ((record = reader.nextRecord()) != null) { + records.add(record.toString()); + } + } catch (Exception e) { + e.printStackTrace(); + } + + flowFile = session.write(flowFile, (out) -> out.write(StringUtils.join(records, "\n").getBytes())); + session.transfer(flowFile, SUCCESS); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return new ArrayList() {{ add(XML_READER); }}; + } + + @Override + public Set getRelationships() { + return new HashSet() {{ add(SUCCESS); }}; + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java index 39f914aca5a5..cbd43a2c2e7c 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java @@ -29,6 +29,7 @@ import org.junit.Test; import org.mockito.Mockito; +import java.io.ByteArrayInputStream; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; @@ -51,7 +52,8 @@ public class TestXMLRecordReader { @Test public void testSimpleRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, reader.nextRecord().getValues()); @@ -62,7 +64,8 @@ public void testSimpleRecord() throws IOException, MalformedRecordException { @Test public void testSimpleRecord2() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema2(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema2(), "PEOPLE", "PERSON", null, + "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertNull(reader.nextRecord(true, true).getValue("AGE")); assertNull(reader.nextRecord(false, true).getValue("AGE")); @@ -73,7 +76,8 @@ public void testSimpleRecord2() throws IOException, MalformedRecordException { @Test public void testSimpleRecord3() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", null, + "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertEquals(Integer.class, reader.nextRecord(true, true).getValue("AGE").getClass()); assertEquals(String.class, reader.nextRecord(false, true).getValue("AGE").getClass()); @@ -94,7 +98,8 @@ public void testSimpleRecord4() throws IOException, MalformedRecordException { @Test public void testSimpleRecordIgnoreSchema() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_no_attributes.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", null, + "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", "42", "USA"}, reader.nextRecord(false, false).getValues()); Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", "33", "UK"}, reader.nextRecord(false, false).getValues()); @@ -222,12 +227,357 @@ public void testSimpleRecordWithAttributeIgnoreSchema() throws IOException, Malf assertEquals("P4", fourth.getAsString("ID")); } + @Test + public void testSimpleTypeWithAttributeAsRecord() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people4.xml"); + final List fields = new ArrayList<>(); + + final List nestedFields1 = new ArrayList<>(); + nestedFields1.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); + nestedFields1.add(new RecordField("CONTENT", RecordFieldType.STRING.getDataType())); + + final DataType recordType1 = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(nestedFields1)); + fields.add(new RecordField("NAME", recordType1)); + + final List nestedFields2 = new ArrayList<>(); + nestedFields2.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); + nestedFields2.add(new RecordField("CONTENT", RecordFieldType.INT.getDataType())); + + final DataType recordType2 = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(nestedFields2)); + fields.add(new RecordField("AGE", recordType2)); + + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(true, true); + assertTrue(first.getValue("NAME") instanceof Record); + Record first_nested1 = (Record) first.getValue("NAME"); + assertTrue(first.getValue("AGE") instanceof Record); + Record first_nested2 = (Record) first.getValue("AGE"); + assertEquals("name1", first_nested1.getValue("ID")); + assertEquals("Cleve Butler", first_nested1.getValue("CONTENT")); + assertEquals("age1", first_nested2.getValue("ID")); + assertEquals(42, first_nested2.getValue("CONTENT")); + + Record second = reader.nextRecord(true, true); + assertTrue(second.getValue("NAME") instanceof Record); + Record second_nested1 = (Record) second.getValue("NAME"); + assertTrue(second.getValue("AGE") instanceof Record); + Record second_nested2 = (Record) second.getValue("AGE"); + assertEquals("name2", second_nested1.getValue("ID")); + assertEquals("Ainslie Fletcher", second_nested1.getValue("CONTENT")); + assertEquals("age2", second_nested2.getValue("ID")); + assertEquals(33, second_nested2.getValue("CONTENT")); + } + + @Test + public void testSimpleTypeWithAttributeAsRecordIgnoreSchema() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people4.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(false, false); + assertTrue(first.getValue("NAME") instanceof Record); + Record first_nested1 = (Record) first.getValue("NAME"); + assertTrue(first.getValue("AGE") instanceof Record); + Record first_nested2 = (Record) first.getValue("AGE"); + assertEquals("name1", first_nested1.getValue("ID")); + assertEquals("Cleve Butler", first_nested1.getValue("CONTENT")); + assertEquals("age1", first_nested2.getValue("ID")); + assertEquals("42", first_nested2.getValue("CONTENT")); + assertEquals("USA", first.getValue("COUNTRY")); + + Record second = reader.nextRecord(false, false); + assertTrue(second.getValue("NAME") instanceof Record); + Record second_nested1 = (Record) second.getValue("NAME"); + assertTrue(second.getValue("AGE") instanceof Record); + Record second_nested2 = (Record) second.getValue("AGE"); + assertEquals("name2", second_nested1.getValue("ID")); + assertEquals("Ainslie Fletcher", second_nested1.getValue("CONTENT")); + assertEquals("age2", second_nested2.getValue("ID")); + assertEquals("33", second_nested2.getValue("CONTENT")); + assertEquals("UK", second.getValue("COUNTRY")); + } + + @Test + public void testSimpleRecordWithHeader() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_with_header_and_comments.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + null, null, dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, reader.nextRecord().getValues()); + } + + @Test + public void testSimpleRecordWithHeaderNoValidation() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_with_header_and_comments.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), null, null, + null, null, dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, reader.nextRecord().getValues()); + } + + @Test + public void testInvalidXml() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_invalid.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), null, null, + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + int count = 0; + + /* + As there is no validation for recordName here, the reader assumes and to be records + (due to the missing starting tag ). The tag is also assumed to be a record, but the exception is thrown + before this "record" is returned. This unexpected behavior can be avoided by defining recordName and consequently + validating the tag name of records (see test testInvalidXml2). Generally, it could be considered to force users + to validate tags at root and record level. However, this only helps if the embedded tags do not have the same + tag name like the record tag. Even a tracking of the parsing depth would not help to overcome this problem. + */ + + try { + while ((reader.nextRecord()) != null) { + count++; + } + } catch (MalformedRecordException e) { + assertEquals("Could not parse XML", e.getMessage()); + assertEquals(4, count); + } + + } + + @Test + public void testInvalidXml2() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_invalid.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + int count = 0; + + try { + while ((reader.nextRecord()) != null) { + count++; + } + } catch (MalformedRecordException e) { + assertEquals("Could not parse XML", e.getMessage()); + + /* + Actually, there are two valid records in the xml file before the invalid record appears, but the exception is + thrown before the second record is returned. + */ + assertEquals(1, count); + } + } + + @Test + public void testMapField() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + List fields = getSimpleRecordFields2(); + fields.add(new RecordField("AGE", RecordFieldType.MAP.getDataType())); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record record = reader.nextRecord(); + assertEquals(null, record.getValue("AGE")); + } + + @Test + public void testChoiceForSimpleField() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + List fields = getSimpleRecordFields2(); + fields.add(new RecordField("AGE", RecordFieldType.CHOICE.getDataType())); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record record = reader.nextRecord(); + assertTrue(record.getValue("AGE") instanceof String); + assertEquals("42", record.getValue("AGE")); + } + + @Test + public void testChoiceForRecord() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); + List fields = getSimpleRecordFields(); + fields.add(new RecordField("ADDRESS", RecordFieldType.CHOICE.getDataType())); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record record = reader.nextRecord(); + assertTrue(record.getValue("ADDRESS") instanceof Record); + + Record nested = (Record) record.getValue("ADDRESS"); + assertEquals("292 West Street", nested.getValue("STREET")); + assertEquals("Jersey City", nested.getValue("CITY")); + } + + @Test + public void testCData() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_cdata.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, reader.nextRecord().getValues()); + } + + @Test + public void testNameSpaces() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_namespace.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getNameSpaceSchema(), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, reader.nextRecord().getValues()); + } + + @Test + public void testRecordExpectedSimpleFieldFoundAndNoContentFieldConfigured() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + List fields = getSimpleRecordFields2(); + final DataType recordType = RecordFieldType.RECORD.getRecordDataType(getNestedSchema()); + fields.add(new RecordField("AGE", recordType)); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {"Cleve Butler", "USA", null}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", "UK", null}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", "FR", null}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", "USA", null}, reader.nextRecord().getValues()); + } + + @Test + public void testSimpleFieldExpectedButRecordFound() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); + List fields = getSimpleRecordFields(); + fields.add(new RecordField("ADDRESS", RecordFieldType.STRING.getDataType())); + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + String message = ""; + try { + reader.nextRecord(); + } catch (Exception e) { + message = e.getMessage(); + } + assertEquals("Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.", message); + } + + @Test + public void testParseEmptyFields() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_empty.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {null, null, null}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {null, null, null}, reader.nextRecord().getValues()); + } + + @Test + public void testParseEmptyFieldsIgnoreSchema() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_empty.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {null, null, null}, reader.nextRecord(false, false).getValues()); + Assert.assertArrayEquals(new Object[] {null, null, null}, reader.nextRecord(false, false).getValues()); + } + + @Test(expected = MalformedRecordException.class) + public void testEmptyStream() throws IOException, MalformedRecordException { + InputStream is = new ByteArrayInputStream(new byte[0]); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), null, null, + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + reader.nextRecord(true, true); + } + + @Test(expected = MalformedRecordException.class) + public void testEmptyStreamWIthXmlHeader() throws IOException, MalformedRecordException { + InputStream is = new ByteArrayInputStream(("").getBytes()); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), null, null, + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + Record record = reader.nextRecord(false, false); + assertNull(record); + } + + @Test + public void testParseEmptyArray() throws IOException, MalformedRecordException { + InputStream is = new ByteArrayInputStream("".getBytes()); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "root", null, + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + assertEquals(reader.nextRecord(true, true), null); + } + + @Test(expected = MalformedRecordException.class) + public void testInvalidRootName() throws IOException, MalformedRecordException { + InputStream is = new ByteArrayInputStream("".getBytes()); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "other", null, + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + assertEquals(reader.nextRecord(true, true), null); + } + + @Test + public void testNoRootName() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), null, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, reader.nextRecord().getValues()); + } + + @Test + public void testRecordNameWithMismatch() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + int count = 0; + while (reader.nextRecord() != null) { + count++; + } + assertEquals(2, count); + } + + @Test + public void testRecordNameOnlyMismatches() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "OTHER", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + int count = 0; + while (reader.nextRecord() != null) { + count++; + } + assertEquals(0, count); + } + + @Test + public void testNoRecordName() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", null, + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + int count = 0; + while (reader.nextRecord() != null) { + count++; + } + assertEquals(3, count); + } @Test public void testNestedRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); RecordSchema schema = getSchemaWithNestedRecord(); - XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + Object[] valuesFirstRecord = reader.nextRecord().getValues(); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, Arrays.copyOfRange(valuesFirstRecord, 0, valuesFirstRecord.length - 1)); Assert.assertArrayEquals(new Object[] {"292 West Street", "Jersey City"},((Record) valuesFirstRecord[valuesFirstRecord.length - 1]).getValues()); @@ -249,7 +599,8 @@ public void testNestedRecord() throws IOException, MalformedRecordException { public void testNestedRecordIgnoreSchema() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); RecordSchema schema = getSchemaWithNestedRecord(); - XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); Object[] valuesFirstRecord = first.getValues(); @@ -292,7 +643,8 @@ public void testNestedRecordIgnoreSchema() throws IOException, MalformedRecordEx public void testSimpleArray() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_array_simple.xml"); RecordSchema schema = getSchemaWithSimpleArray(); - XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record firstRecord = reader.nextRecord(); Object[] valuesFirstRecord = firstRecord.getValues(); @@ -327,7 +679,8 @@ public void testSimpleArray() throws IOException, MalformedRecordException { public void testSimpleArrayIgnoreSchema() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_array_simple.xml"); RecordSchema schema = getSchemaWithSimpleArray(); - XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); Object[] valuesFirstRecord = first.getValues(); @@ -361,7 +714,8 @@ public void testSimpleArrayIgnoreSchema() throws IOException, MalformedRecordExc public void testNestedArrayInNestedRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_array.xml"); RecordSchema schema = getSchemaWithNestedArray(); - XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, + "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record firstRecord = reader.nextRecord(); Object[] valuesFirstRecord = firstRecord.getValues(); @@ -398,7 +752,8 @@ public void testNestedArrayInNestedRecord() throws IOException, MalformedRecordE public void testDeeplyNestedArraysAndRecords() throws IOException, MalformedRecordException { // test records in nested arrays InputStream is = new FileInputStream("src/test/resources/xml/people_complex1.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData(), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(true, true); Object[] grandchildren_arr = (Object[]) first.getValue("CHILDREN"); @@ -436,7 +791,8 @@ public void testDeeplyNestedArraysAndRecords() throws IOException, MalformedReco public void testDeeplyNestedArraysAndRecords2() throws IOException, MalformedRecordException { // test multiply nested arrays and records (recursion) InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), "PEOPLE", "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(); assertEquals("grandmother", first.getValue("ROLE")); @@ -570,6 +926,14 @@ private List getNestedRecordFields() { return fields; } + private List getNameSpaceFields() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("F:NAME", RecordFieldType.STRING.getDataType())); + fields.add(new RecordField("F:AGE", RecordFieldType.INT.getDataType())); + fields.add(new RecordField("F:COUNTRY", RecordFieldType.STRING.getDataType())); + return fields; + } + private RecordSchema getSimpleSchema() { return new SimpleRecordSchema(getSimpleRecordFields()); } @@ -582,6 +946,10 @@ private RecordSchema getNestedSchema() { return new SimpleRecordSchema(getNestedRecordFields()); } + private RecordSchema getNameSpaceSchema() { + return new SimpleRecordSchema(getNameSpaceFields()); + } + private RecordSchema getSchemaWithNestedRecord() { final List fields = getSimpleRecordFields(); final DataType recordType = RecordFieldType.RECORD.getRecordDataType(getNestedSchema()); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_cdata.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_cdata.xml new file mode 100755 index 000000000000..e957af404dbf --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_cdata.xml @@ -0,0 +1,22 @@ + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_invalid.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_invalid.xml index e37ec77bc100..da987f17dd43 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_invalid.xml +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_invalid.xml @@ -4,11 +4,11 @@ 42 USA + Ainslie Fletcher 33 UK - Amélie Bonfils 74 FR diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_namespace.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_namespace.xml new file mode 100755 index 000000000000..7963903a2663 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_namespace.xml @@ -0,0 +1,22 @@ + + + Cleve Butler + 42 + USA + + + Ainslie Fletcher + 33 + UK + + + Amélie Bonfils + 74 + FR + + + Elenora Scrivens + 16 + USA + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_with_header_and_comments.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_with_header_and_comments.xml new file mode 100755 index 000000000000..f3c83d90d60f --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_with_header_and_comments.xml @@ -0,0 +1,29 @@ + + + + + Cleve Butler + 42 + USA + + + + + + Ainslie Fletcher + 33 + UK + + + + + Amélie Bonfils + 74 + FR + + + Elenora Scrivens + 16 + USA + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/testschema b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/testschema new file mode 100755 index 000000000000..91a9f261009b --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/testschema @@ -0,0 +1,11 @@ +{ + "namespace": "nifi", + "name": "test", + "type": "record", + "fields": [ + { "name": "ID", "type": "string" }, + { "name": "NAME", "type": "string" }, + { "name": "AGE", "type": "int" }, + { "name": "COUNTRY", "type": "string" } + ] +} From 1e4010496abc436dcadc12e067cb66354ab7d426 Mon Sep 17 00:00:00 2001 From: JohannesDaniel Date: Mon, 2 Apr 2018 00:35:15 +0200 Subject: [PATCH 3/6] Fixes for publishing --- .../nifi-record-serialization-services/pom.xml | 4 ++++ .../java/org/apache/nifi/xml/XMLReader.java | 16 ++++++++-------- .../java/org/apache/nifi/xml/TestXMLReader.java | 17 +++++++++++++++++ .../apache/nifi/xml/TestXMLReaderProcessor.java | 17 +++++++++++++++++ 4 files changed, 46 insertions(+), 8 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml index 0e02e6e827c2..d996c56f2f69 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml @@ -122,12 +122,16 @@ src/test/resources/xml/people4.xml src/test/resources/xml/people_array.xml src/test/resources/xml/people_array_simple.xml + src/test/resources/xml/people_cdata.xml src/test/resources/xml/people_complex1.xml src/test/resources/xml/people_complex2.xml src/test/resources/xml/people_empty.xml src/test/resources/xml/people_invalid.xml + src/test/resources/xml/people_namespace.xml src/test/resources/xml/people_nested.xml src/test/resources/xml/people_no_attributes.xml + src/test/resources/xml/people_with_header_and_comments.xml + src/test/resources/xml/testschema diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java index faa474391dfb..2324adf9c08f 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java @@ -116,17 +116,17 @@ public RecordReader createRecordReader(final Map variables, fina final RecordSchema schema = getSchema(variables, in, null); - final String rootName = context.getProperty(VALIDATE_ROOT_TAG).isSet() ? - context.getProperty(VALIDATE_ROOT_TAG).evaluateAttributeExpressions(variables).getValue().trim() : null; + final String rootName = context.getProperty(VALIDATE_ROOT_TAG).isSet() + ? context.getProperty(VALIDATE_ROOT_TAG).evaluateAttributeExpressions(variables).getValue().trim() : null; - final String recordName = context.getProperty(VALIDATE_RECORD_TAG).isSet() ? - context.getProperty(VALIDATE_RECORD_TAG).evaluateAttributeExpressions(variables).getValue().trim() : null; + final String recordName = context.getProperty(VALIDATE_RECORD_TAG).isSet() + ? context.getProperty(VALIDATE_RECORD_TAG).evaluateAttributeExpressions(variables).getValue().trim() : null; - final String attributePrefix = context.getProperty(ATTRIBUTE_PREFIX).isSet() ? - context.getProperty(ATTRIBUTE_PREFIX).evaluateAttributeExpressions(variables).getValue().trim() : null; + final String attributePrefix = context.getProperty(ATTRIBUTE_PREFIX).isSet() + ? context.getProperty(ATTRIBUTE_PREFIX).evaluateAttributeExpressions(variables).getValue().trim() : null; - final String contentFieldName = context.getProperty(CONTENT_FIELD_NAME).isSet() ? - context.getProperty(CONTENT_FIELD_NAME).evaluateAttributeExpressions(variables).getValue().trim() : null; + final String contentFieldName = context.getProperty(CONTENT_FIELD_NAME).isSet() + ? context.getProperty(CONTENT_FIELD_NAME).evaluateAttributeExpressions(variables).getValue().trim() : null; return new XMLRecordReader(in, schema, rootName, recordName, attributePrefix, contentFieldName, dateFormat, timeFormat, timestampFormat, logger); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java index 65278286efe4..60606d13c699 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java @@ -1,3 +1,20 @@ +/* + * 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.reporting.InitializationException; diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java index f205cfdf23d8..18bfc5a87f0c 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java @@ -1,3 +1,20 @@ +/* + * 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.components.PropertyDescriptor; From 0713c66434e3b592695647dd6b3403183d78e2cb Mon Sep 17 00:00:00 2001 From: JohannesDaniel Date: Mon, 16 Apr 2018 12:16:27 +0200 Subject: [PATCH 4/6] R2 --- .../pom.xml | 4 + .../java/org/apache/nifi/xml/XMLReader.java | 54 +- .../org/apache/nifi/xml/XMLRecordReader.java | 299 +++++--- .../additionalDetails.html | 85 ++- .../org/apache/nifi/xml/TestXMLReader.java | 49 +- .../nifi/xml/TestXMLReaderProcessor.java | 1 - .../apache/nifi/xml/TestXMLRecordReader.java | 671 +++++++++++++++--- .../src/test/resources/xml/people_array.xml | 1 + .../src/test/resources/xml/people_map.xml | 18 + .../src/test/resources/xml/people_map2.xml | 32 + .../xml/people_tag_in_characters.xml | 23 + .../src/test/resources/xml/person.xml | 5 + 12 files changed, 958 insertions(+), 284 deletions(-) create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_map.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_map2.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_tag_in_characters.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/person.xml diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml index 019d45f6c32f..f25ac6da9aa9 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml @@ -133,10 +133,14 @@ src/test/resources/xml/people_complex2.xml src/test/resources/xml/people_empty.xml src/test/resources/xml/people_invalid.xml + src/test/resources/xml/people_map.xml + src/test/resources/xml/people_map2.xml src/test/resources/xml/people_namespace.xml src/test/resources/xml/people_nested.xml src/test/resources/xml/people_no_attributes.xml + src/test/resources/xml/people_tag_in_characters.xml src/test/resources/xml/people_with_header_and_comments.xml + src/test/resources/xml/person.xml src/test/resources/xml/testschema diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java index 2324adf9c08f..216592f12029 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java @@ -20,8 +20,10 @@ 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.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.schema.access.SchemaNotFoundException; @@ -43,34 +45,36 @@ "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. ...) 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.") + public static final AllowableValue RECORD_SINGLE = new AllowableValue("record_single", "Single Record"); + public static final AllowableValue RECORD_ARRAY = new AllowableValue("record_array", "Array of Records"); + + public static final PropertyDescriptor RECORD_FORMAT = new PropertyDescriptor.Builder() + .name("record_format") + .displayName("Record Format") + .description("This property defines whether the reader expects a single record an array of records") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(true) + .allowableValues(RECORD_SINGLE, RECORD_ARRAY) + .defaultValue(RECORD_SINGLE.getValue()) + .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .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. ...) 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.") + public static final PropertyDescriptor CHECK_RECORD_TAG = new PropertyDescriptor.Builder() + .name("check_record_tag") + .displayName("Check Record Tag") + .description("If this property is set, the name of record tags 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, all records will be processed.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(true) + .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .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.") + .description("If this property is set, the name of attributes will be prepended with a prefix when they are added to a record.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(true) + .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .required(false) .build(); @@ -82,7 +86,7 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact "If tags with content shall be parsed together with attributes (e. g. content), " + "they have to be defined as records. For additional information, see the section of processor usage.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(true) + .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .required(false) .build(); @@ -100,8 +104,8 @@ public void onEnabled(final ConfigurationContext context) { @Override protected List getSupportedPropertyDescriptors() { final List properties = new ArrayList<>(super.getSupportedPropertyDescriptors()); - properties.add(VALIDATE_ROOT_TAG); - properties.add(VALIDATE_RECORD_TAG); + properties.add(RECORD_FORMAT); + properties.add(CHECK_RECORD_TAG); properties.add(ATTRIBUTE_PREFIX); properties.add(CONTENT_FIELD_NAME); properties.add(DateTimeUtils.DATE_FORMAT); @@ -116,11 +120,8 @@ public RecordReader createRecordReader(final Map variables, fina final RecordSchema schema = getSchema(variables, in, null); - final String rootName = context.getProperty(VALIDATE_ROOT_TAG).isSet() - ? context.getProperty(VALIDATE_ROOT_TAG).evaluateAttributeExpressions(variables).getValue().trim() : null; - - final String recordName = context.getProperty(VALIDATE_RECORD_TAG).isSet() - ? context.getProperty(VALIDATE_RECORD_TAG).evaluateAttributeExpressions(variables).getValue().trim() : null; + final String recordName = context.getProperty(CHECK_RECORD_TAG).isSet() + ? context.getProperty(CHECK_RECORD_TAG).evaluateAttributeExpressions(variables).getValue().trim() : null; final String attributePrefix = context.getProperty(ATTRIBUTE_PREFIX).isSet() ? context.getProperty(ATTRIBUTE_PREFIX).evaluateAttributeExpressions(variables).getValue().trim() : null; @@ -128,6 +129,9 @@ public RecordReader createRecordReader(final Map variables, fina final String contentFieldName = context.getProperty(CONTENT_FIELD_NAME).isSet() ? context.getProperty(CONTENT_FIELD_NAME).evaluateAttributeExpressions(variables).getValue().trim() : null; - return new XMLRecordReader(in, schema, rootName, recordName, attributePrefix, contentFieldName, dateFormat, timeFormat, timestampFormat, logger); + final boolean isArray = context.getProperty(RECORD_FORMAT).evaluateAttributeExpressions(variables).getValue() + .equals(RECORD_ARRAY.getValue()); + + return new XMLRecordReader(in, schema, isArray, recordName, attributePrefix, contentFieldName, dateFormat, timeFormat, timestampFormat, logger); } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java index c99b8d412afa..c104103c0e16 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java @@ -27,8 +27,10 @@ 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.MapDataType; import org.apache.nifi.serialization.record.type.RecordDataType; import org.apache.nifi.serialization.record.util.DataTypeUtils; +import org.apache.nifi.util.StringUtils; import javax.xml.stream.XMLEventReader; import javax.xml.stream.XMLInputFactory; @@ -57,7 +59,6 @@ public class XMLRecordReader implements RecordReader { private final String attributePrefix; private final String contentFieldName; - // thread safety required? private StartElement currentRecordStartTag; private final XMLEventReader xmlEventReader; @@ -66,7 +67,7 @@ public class XMLRecordReader implements RecordReader { private final Supplier LAZY_TIME_FORMAT; private final Supplier LAZY_TIMESTAMP_FORMAT; - public XMLRecordReader(InputStream in, RecordSchema schema, String rootName, String recordName, String attributePrefix, String contentFieldName, + public XMLRecordReader(InputStream in, RecordSchema schema, boolean isArray, 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; @@ -85,36 +86,25 @@ 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); - 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()); + + if (isArray) { + skipNextStartTag(); } + setNextRecordStartTag(); } catch (XMLStreamException e) { throw new MalformedRecordException("Could not parse XML", e); } } - private StartElement getNextStartTag() throws XMLStreamException { + private void skipNextStartTag() throws XMLStreamException { while (xmlEventReader.hasNext()) { final XMLEvent xmlEvent = xmlEventReader.nextEvent(); if (xmlEvent.isStartElement()) { - return xmlEvent.asStartElement(); + return; } } - return null; } private void setNextRecordStartTag() throws XMLStreamException { @@ -123,12 +113,12 @@ private void setNextRecordStartTag() throws XMLStreamException { if (xmlEvent.isStartElement()) { final StartElement startElement = xmlEvent.asStartElement(); if (recordName != null) { - if (startElement.getName().toString().equals(recordName)) { + if (startElement.getName().getLocalPart().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()}); + "Record will be skipped", new Object[] {recordName, startElement.getName().getLocalPart()}); skipElement(); } } else { @@ -158,7 +148,8 @@ public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFie } } - private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map recordValues) throws XMLStreamException, MalformedRecordException { + private Object parseFieldForType(StartElement startElement, String fieldName, DataType dataType, Map recordValues, + boolean dropUnknown) throws XMLStreamException, MalformedRecordException { switch (dataType.getFieldType()) { case BOOLEAN: case BYTE: @@ -172,27 +163,36 @@ private Object parseFieldForType(StartElement startElement, String fieldName, Da 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(); + + StringBuilder content = new StringBuilder(); + + while (xmlEventReader.hasNext()) { + XMLEvent xmlEvent = xmlEventReader.nextEvent(); + if (xmlEvent.isCharacters()) { + final Characters characters = xmlEvent.asCharacters(); + if (!characters.isWhiteSpace()) { + content.append(characters.getData()); + } + } else if (xmlEvent.isEndElement()) { + final String contentToReturn = content.toString(); + + if (!StringUtils.isBlank(contentToReturn)) { + return DataTypeUtils.convertType(content.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName); + } else { + return null; + } + + } else if (xmlEvent.isStartElement()) { + this.skipElement(); } } - 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); - } + break; } + case ARRAY: { final DataType arrayDataType = ((ArrayDataType) dataType).getElementType(); - final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues); + final Object newValue = parseFieldForType(startElement, fieldName, arrayDataType, recordValues, dropUnknown); final Object oldValues = recordValues.get(fieldName); if (newValue != null) { @@ -200,15 +200,20 @@ private Object parseFieldForType(StartElement startElement, String fieldName, Da if (oldValues instanceof List) { ((List) oldValues).add(newValue); } else { - return new ArrayList(){{ add(oldValues); add(newValue); }}; + List arrayValues = new ArrayList<>(); + arrayValues.add(oldValues); + arrayValues.add(newValue); + return arrayValues; } } else { - return new ArrayList(){{ add(newValue); }}; + List arrayValues = new ArrayList<>(); + arrayValues.add(newValue); + return arrayValues; } - } else { - return null; } + return oldValues; } + case RECORD: { final RecordSchema childSchema; if (dataType instanceof RecordDataType) { @@ -216,59 +221,110 @@ private Object parseFieldForType(StartElement startElement, String fieldName, Da } else { return null; } - return parseRecord(startElement, childSchema, true, true); + + return parseRecord(startElement, childSchema, true, dropUnknown); } + case MAP: { - logger.warn("Type map is not supported by this record reader. Field will be skipped."); - skipElement(); - return null; + final DataType mapDataType = ((MapDataType) dataType).getValueType(); + final Map embeddedMap = new HashMap<>(); + + while (xmlEventReader.hasNext()) { + XMLEvent xmlEvent = xmlEventReader.nextEvent(); + + if (xmlEvent.isStartElement()) { + final StartElement subStartElement = xmlEvent.asStartElement(); + final String subFieldName = subStartElement.getName().getLocalPart(); + + final Object mapValue = parseFieldForType(subStartElement, subFieldName, mapDataType, embeddedMap, dropUnknown); + embeddedMap.put(subFieldName, mapValue); + + } else if (xmlEvent.isEndElement()) { + break; + } + } + + if (embeddedMap.size() > 0) { + return embeddedMap; + } else { + return null; + } } case CHOICE: { - return parseUnknownField(startElement); + // field choice will parse the entire tree of a field + return parseUnknownField(startElement, false, null); } } return null; } - private Object parseUnknownField(StartElement startElement) throws XMLStreamException { + private Object parseUnknownField(StartElement startElement, boolean dropUnknown, RecordSchema schema) throws XMLStreamException { // parse attributes final Map 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()); + + if (dropUnknown) { + if (schema != null) { + final Optional field = schema.getField(attributeName); + if (field.isPresent()){ + recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue()); + } + } + } else { + recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue()); + } } - boolean hasAttributes = recordValues.size() > 0; // parse fields + StringBuilder content = new StringBuilder(); + 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(); - } + content.append(characters.getData()); } } else if (xmlEvent.isStartElement()){ final StartElement subStartElement = xmlEvent.asStartElement(); - final String subFieldName = subStartElement.getName().toString(); - final Object value = parseUnknownField(subStartElement); + final String subFieldName = subStartElement.getName().getLocalPart(); + + if (dropUnknown) { + if (schema != null) { + final Optional field = schema.getField(subFieldName); + if (field.isPresent()){ + + // subElements of subStartElement can only be known if there is a corresponding field in the schema defined as record + final DataType dataType = field.get().getDataType(); + RecordSchema childSchema = null; + + if (dataType instanceof RecordDataType) { + childSchema = ((RecordDataType) dataType).getChildSchema(); + } else if (dataType instanceof ArrayDataType) { + DataType typeOfArray = ((ArrayDataType) dataType).getElementType(); + if (typeOfArray instanceof RecordDataType) { + childSchema = ((RecordDataType) typeOfArray).getChildSchema(); + } + } - if (value != null) { - putUnknownTypeInMap(recordValues, subFieldName, value); + final Object value = parseUnknownField(subStartElement, true, childSchema); + if (value != null) { + putUnknownTypeInMap(recordValues, subFieldName, value); + } + } else { + skipElement(); + } + } else { + skipElement(); + } } else { - return null; + final Object value = parseUnknownField(subStartElement, dropUnknown, schema); + if (value != null) { + putUnknownTypeInMap(recordValues, subFieldName, value); + } } } else if (xmlEvent.isEndElement()) { break; @@ -281,10 +337,28 @@ private Object parseUnknownField(StartElement startElement) throws XMLStreamExce } } - if (recordValues.size() > 0) { - return new MapRecord(new SimpleRecordSchema(Collections.emptyList()), recordValues); + final boolean hasContent = content.length() > 0; + final boolean hasFields = recordValues.size() > 0; + + if (hasContent) { + if (!hasFields) { + return content.toString(); + } else { + if (contentFieldName != null) { + recordValues.put(contentFieldName, content.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 null; + if (hasFields) { + return new MapRecord(new SimpleRecordSchema(Collections.emptyList()), recordValues); + } else { + return null; + } } } @@ -307,7 +381,7 @@ private Record parseRecord(StartElement startElement, RecordSchema schema, boole if (coerceTypes) { final Object value; final DataType dataType = field.get().getDataType(); - if ((value = parseAttributeForType(attribute, attributeName, dataType)) != null) { + if ((value = parseStringForType(attribute.getValue(), attributeName, dataType)) != null) { recordValues.put(targetFieldName, value); } @@ -323,7 +397,7 @@ private Record parseRecord(StartElement startElement, RecordSchema schema, boole final Object value; final Optional field = schema.getField(attributeName); if (field.isPresent()){ - if ((value = parseAttributeForType(attribute, attributeName, field.get().getDataType())) != null) { + if ((value = parseStringForType(attribute.getValue(), attributeName, field.get().getDataType())) != null) { recordValues.put(targetFieldName, value); } } else { @@ -338,26 +412,41 @@ private Record parseRecord(StartElement startElement, RecordSchema schema, boole } // parse fields + StringBuilder content = new StringBuilder(); while(xmlEventReader.hasNext()){ final XMLEvent xmlEvent = xmlEventReader.nextEvent(); if (xmlEvent.isStartElement()) { final StartElement subStartElement = xmlEvent.asStartElement(); - final String fieldName = subStartElement.getName().toString(); + final String fieldName = subStartElement.getName().getLocalPart(); + final Optional 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); + final Object value = parseFieldForType(subStartElement, fieldName, field.get().getDataType(), recordValues, true); if (value != null) { recordValues.put(fieldName, value); } // dropUnknown == true && coerceTypes == false + // subElements of subStartElement can only be known if there is a corresponding field in the schema defined as record } else { - final Object value = parseUnknownField(subStartElement); + final DataType dataType = field.get().getDataType(); + RecordSchema childSchema = null; + + if (dataType instanceof RecordDataType) { + childSchema = ((RecordDataType) dataType).getChildSchema(); + } else if (dataType instanceof ArrayDataType) { + DataType typeOfArray = ((ArrayDataType) dataType).getElementType(); + if (typeOfArray instanceof RecordDataType) { + childSchema = ((RecordDataType) typeOfArray).getChildSchema(); + } + } + + final Object value = parseUnknownField(subStartElement, true, childSchema); if (value != null) { putUnknownTypeInMap(recordValues, fieldName, value); } @@ -370,19 +459,20 @@ private Record parseRecord(StartElement startElement, RecordSchema schema, boole // dropUnknown == false && coerceTypes == true if (coerceTypes) { if (field.isPresent()) { - final Object value = parseFieldForType(subStartElement, fieldName, field.get().getDataType(), recordValues); + final Object value = parseFieldForType(subStartElement, fieldName, field.get().getDataType(), recordValues, false); if (value != null) { recordValues.put(fieldName, value); } } else { - final Object value = parseUnknownField(subStartElement); + final Object value = parseUnknownField(subStartElement, false, null); if (value != null) { putUnknownTypeInMap(recordValues, fieldName, value); } } + // dropUnknown == false && coerceTypes == false } else { - final Object value = parseUnknownField(subStartElement); + final Object value = parseUnknownField(subStartElement, false, null); if (value != null) { putUnknownTypeInMap(recordValues, fieldName, value); } @@ -393,19 +483,24 @@ private Record parseRecord(StartElement startElement, RecordSchema schema, boole } else if (xmlEvent.isCharacters()) { final Characters characters = xmlEvent.asCharacters(); if (!characters.isWhiteSpace()) { - if (contentFieldName != null) { - final Optional 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."); - } + content.append(characters.getData()); } } } + + if (content.length() > 0) { + if (contentFieldName != null) { + final Optional field = schema.getField(contentFieldName); + if (field.isPresent()) { + Object value = parseStringForType(content.toString(), 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 entry : recordValues.entrySet()) { if (entry.getValue() instanceof List) { recordValues.put(entry.getKey(), ((List) entry.getValue()).toArray()); @@ -426,34 +521,18 @@ private void putUnknownTypeInMap(Map values, String fieldName, O if (oldValues instanceof List) { ((List) oldValues).add(fieldValue); } else { - values.put(fieldName, new ArrayList(){{ add(oldValues); add(fieldValue); }}); + List valuesToPut = new ArrayList<>(); + valuesToPut.add(oldValues); + valuesToPut.add(fieldValue); + + values.put(fieldName, valuesToPut); } } 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) { + private Object parseStringForType(String data, String fieldName, DataType dataType) { switch (dataType.getFieldType()) { case BOOLEAN: case BYTE: @@ -467,7 +546,7 @@ private Object parseCharacterForType(Characters character, String fieldName, Dat case DATE: case TIME: case TIMESTAMP: { - return DataTypeUtils.convertType(character.toString(), dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName); + return DataTypeUtils.convertType(data, dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName); } } return null; diff --git a/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 b/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 index e02b5c67ff55..035d4abaeb63 100755 --- a/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 +++ b/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 @@ -24,11 +24,26 @@

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. + that are not defined in the schema will be skipped. Depending on how the property "Record Format" is set, + the reader either expects a single record or an array of records for each FlowFile.

- Records are expected in the second level of the XML data, embedded within an enclosing root tag: + Example: Single record

+ +
+                <record>
+                  <field1>content</field1>
+                  <field2>content</field2>
+                </record>
+            
+
+ +

+ An array of records has to be enclosed by a root tag. + Example: Array of records +

+
                 <root>
@@ -44,11 +59,7 @@
             
-

- For the following examples, it is assumed that the exemplary records are enclosed by a root tag. -

- -

Example 1: Simple Fields

+

Example: Simple Fields

The simplest kind of data within XML data are tags / fields only containing content (no attributes, no embedded tags). @@ -57,9 +68,11 @@

Example 1: Simple Fields

-                <record>
-                  <simple_field>content</simple_field>
-                </record>
+                <root>
+                  <record>
+                    <simple_field>content</simple_field>
+                  </record>
+                </root>
             
@@ -81,7 +94,7 @@

Example 1: Simple Fields

-

Example 2: Arrays with Simple Fields

+

Example: Arrays with Simple Fields

Arrays are considered as repetitive tags / fields in XML data. For the following XML data, "array_field" is considered @@ -126,7 +139,7 @@

Example 2: Arrays with Simple Fields

as an array. In this case, the reader would put it into the record object as an array with one element.

-

Example 3: Tags with Attributes

+

Example: Tags with Attributes

XML fields frequently not only contain content, but also attributes. The following record contains a field with @@ -221,11 +234,12 @@

Example 3: Tags with Attributes

the property "Field Name for Content" is set.

-

Example 4: Tags within tags

+

Example: Tags within tags

XML data is frequently nested. In this case, tags enclose other tags:

+
                 <record>
@@ -271,7 +285,7 @@ 

Example 4: Tags within tags

for tags containing attributes and content.

-

Example 5: Array of records

+

Example: Array of records

For further explanation of the logic of this reader, an example of an array of records shall be demonstrated. @@ -324,7 +338,7 @@

Example 5: Array of records

-

Example 6: Array in record

+

Example: Array in record

In XML data, arrays are frequently enclosed by tags: @@ -374,5 +388,46 @@

Example 6: Array in record

} + + +

Example: Maps

+ +

+ A map is a field embedding fields with different names: +

+ + +
+                <record>
+                  <map_field>
+                    <field1>content</field1>
+                    <field2>content</field2>
+                    ...
+                  </map_field>
+                  <simple_field>content</simple_field>
+                </record>
+            
+
+ +

+ This data can be processed using the following schema: +

+ + +
+                {
+                  "namespace": "nifi",
+                  "name": "test",
+                  "type": "record",
+                  "fields": [
+                    { "name": "map_field", "type":
+                      { "type": "map", "items": string }
+                    },
+                    { "name": "simple_field", "type": "string" }
+                  ]
+                }
+            
+
+ diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java index 60606d13c699..4e612b379e95 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java @@ -19,7 +19,6 @@ import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.schema.access.SchemaAccessUtils; -import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; @@ -61,7 +60,9 @@ public void setup() throws InitializationException, IOException { } @Test - public void testNoValidation() throws FileNotFoundException { + public void testNoCheckForRecord() throws FileNotFoundException { + runner.setProperty(reader, XMLReader.RECORD_FORMAT, XMLReader.RECORD_ARRAY); + runner.enableControllerService(reader); InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); @@ -78,46 +79,10 @@ public void testNoValidation() throws FileNotFoundException { } @Test - public void testRootValidation() throws FileNotFoundException { - runner.setProperty(reader, XMLReader.VALIDATE_ROOT_TAG, "${" + ROOT_TAG_NAME + "}"); - runner.enableControllerService(reader); - - InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); - runner.enqueue(is, new HashMap() {{ - put(ROOT_TAG_NAME, "PEOPLE"); - }}); - runner.run(); - - List flowFile = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS); - List records = Arrays.asList((new String(runner.getContentAsByteArray(flowFile.get(0)))).split("\n")); + public void testCheckForRecord() throws FileNotFoundException { + runner.setProperty(reader, XMLReader.CHECK_RECORD_TAG, "${" + RECORD_TAG_NAME + "}"); + runner.setProperty(reader, XMLReader.RECORD_FORMAT, XMLReader.RECORD_ARRAY); - assertEquals(3, records.size()); - assertEquals("MapRecord[{COUNTRY=USA, ID=1, NAME=Cleve Butler, AGE=42}]", records.get(0)); - assertEquals("MapRecord[{COUNTRY=UK, ID=2, NAME=Ainslie Fletcher, AGE=33}]", records.get(1)); - assertEquals("MapRecord[{COUNTRY=UK, ID=3, NAME=Ainslie Fletcher, AGE=33}]", records.get(2)); - } - - @Test - public void testInvalidRoot() throws FileNotFoundException { - runner.setProperty(reader, XMLReader.VALIDATE_ROOT_TAG, "${" + ROOT_TAG_NAME + "}"); - runner.enableControllerService(reader); - - InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); - runner.enqueue(is, new HashMap() {{ - put(ROOT_TAG_NAME, "WRONG_ROOT"); - }}); - try { - runner.run(); - } catch (Exception e){ - assertEquals("Name of root tag \"PEOPLE\" does not match root tag validation \"WRONG_ROOT\".", e.getMessage()); - assert(e instanceof MalformedRecordException); - } - } - - @Test - public void testRecordValidation() throws FileNotFoundException { - runner.setProperty(reader, XMLReader.VALIDATE_ROOT_TAG, "${" + ROOT_TAG_NAME + "}"); - runner.setProperty(reader, XMLReader.VALIDATE_RECORD_TAG, "${" + RECORD_TAG_NAME + "}"); runner.enableControllerService(reader); InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); @@ -138,6 +103,8 @@ public void testRecordValidation() throws FileNotFoundException { @Test public void testAttributePrefix() throws FileNotFoundException { runner.setProperty(reader, XMLReader.ATTRIBUTE_PREFIX, "${" + ATTRIBUTE_PREFIX + "}"); + runner.setProperty(reader, XMLReader.RECORD_FORMAT, XMLReader.RECORD_ARRAY); + runner.enableControllerService(reader); InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java index 18bfc5a87f0c..b4ee4ab7cf46 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java @@ -46,7 +46,6 @@ public class TestXMLReaderProcessor extends AbstractProcessor { public static final Relationship SUCCESS = new Relationship.Builder().name("success").description("success").build(); - @Override public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { FlowFile flowFile = session.get(); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java index cbd43a2c2e7c..ce9e3ea307c6 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java @@ -37,6 +37,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -49,10 +50,215 @@ public class TestXMLRecordReader { private final String timeFormat = RecordFieldType.TIME.getDefaultFormat(); private final String timestampFormat = RecordFieldType.TIMESTAMP.getDefaultFormat(); + @Test + public void testSingleRecord() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/person.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), false, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); + Assert.assertNull(reader.nextRecord()); + } + + @Test + public void testSingleRecordCheckForTag() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/person.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), false, "OTHER", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + Assert.assertNull(reader.nextRecord()); + } + + @Test + public void testMap() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_map.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForMap(), true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(); + assertEquals("P1", first.getValue("ID")); + Map firstMap = (Map) first.getValue("MAP"); + assertEquals("Cleve Butler", firstMap.get("NAME")); + assertEquals("42", firstMap.get("AGE")); + assertEquals("USA", firstMap.get("COUNTRY")); + + Record second = reader.nextRecord(); + assertEquals("P2", second.getValue("ID")); + Map secondMap = (Map) second.getValue("MAP"); + assertEquals("Ainslie Fletcher", secondMap.get("NAME")); + assertEquals("33", secondMap.get("AGE")); + assertEquals("UK", secondMap.get("COUNTRY")); + } + + @Test + public void testMapWithRecords() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_map2.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForRecordMap(), true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(); + assertEquals("P1", first.getValue("ID")); + Map firstMap = (Map) first.getValue("MAP"); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, ((Record) firstMap.get("ENTRY")).getValues()); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, ((Record) firstMap.get("ENTRY2")).getValues()); + + Record second = reader.nextRecord(); + assertEquals("P2", second.getValue("ID")); + Map secondMap = (Map) second.getValue("MAP"); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, ((Record) secondMap.get("ENTRY")).getValues()); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, ((Record) secondMap.get("ENTRY2")).getValues()); + } + + @Test + public void testTagInCharactersSimpleField() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, null}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, null}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, null}, reader.nextRecord().getValues()); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, null}, reader.nextRecord().getValues()); + } + + @Test + public void testTagInCharactersRecord() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSchemaWithNestedRecord3(), true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(true, true); + assertEquals(42, first.getValue("AGE")); + Record firstNested = (Record) first.getValue("NAME"); + assertEquals("Cleve Butler", firstNested.getValue("CONTENT")); + assertEquals("attr content", firstNested.getValue("ATTR")); + assertEquals("inner content", firstNested.getValue("INNER")); + + Record second = reader.nextRecord(true, true); + assertEquals(33, second.getValue("AGE")); + Record secondNested = (Record) second.getValue("NAME"); + assertEquals("Ainslie Fletcher", secondNested.getValue("CONTENT")); + assertEquals("attr content", secondNested.getValue("ATTR")); + assertEquals("inner content", secondNested.getValue("INNER")); + + Record third = reader.nextRecord(true, true); + assertEquals(74, third.getValue("AGE")); + Record thirdNested = (Record) third.getValue("NAME"); + assertEquals("Amélie Bonfils", thirdNested.getValue("CONTENT")); + assertEquals("attr content", thirdNested.getValue("ATTR")); + assertEquals("inner content", thirdNested.getValue("INNER")); + + Record fourth = reader.nextRecord(true, true); + assertEquals(16, fourth.getValue("AGE")); + Record fourthNested = (Record) fourth.getValue("NAME"); + assertEquals("Elenora Scrivens", fourthNested.getValue("CONTENT")); + assertEquals("attr content", fourthNested.getValue("ATTR")); + assertEquals("inner content", fourthNested.getValue("INNER")); + + Record fifth = reader.nextRecord(true, true); + assertNull(fifth.getValue("AGE")); + Record fifthNested = (Record) fifth.getValue("NAME"); + assertNull(fifthNested.getValue("CONTENT")); + assertNull(fifthNested.getValue("ATTR")); + assertEquals("inner content", fifthNested.getValue("INNER")); + } + + @Test + public void testTagInCharactersCoerceTrueDropFalse() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSchemaWithNestedRecord3(), true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(true, false); + assertEquals("P1", first.getValue("ID")); + assertEquals(42, first.getValue("AGE")); + Record firstNested = (Record) first.getValue("NAME"); + assertEquals("Cleve Butler", firstNested.getValue("CONTENT")); + assertEquals("attr content", firstNested.getValue("ATTR")); + assertEquals("inner content", firstNested.getValue("INNER")); + + Record second = reader.nextRecord(true, false); + assertEquals("P2", second.getValue("ID")); + assertEquals(33, second.getValue("AGE")); + Record secondNested = (Record) second.getValue("NAME"); + assertEquals("Ainslie Fletcher", secondNested.getValue("CONTENT")); + assertEquals("attr content", secondNested.getValue("ATTR")); + assertEquals("inner content", secondNested.getValue("INNER")); + + Record third = reader.nextRecord(true, false); + assertEquals("P3", third.getValue("ID")); + assertEquals(74, third.getValue("AGE")); + Record thirdNested = (Record) third.getValue("NAME"); + assertEquals("Amélie Bonfils", thirdNested.getValue("CONTENT")); + assertEquals("attr content", thirdNested.getValue("ATTR")); + assertEquals("inner content", thirdNested.getValue("INNER")); + + Record fourth = reader.nextRecord(true, false); + assertEquals("P4", fourth.getValue("ID")); + assertEquals(16, fourth.getValue("AGE")); + Record fourthNested = (Record) fourth.getValue("NAME"); + assertEquals("Elenora Scrivens", fourthNested.getValue("CONTENT")); + assertEquals("attr content", fourthNested.getValue("ATTR")); + assertEquals("inner content", fourthNested.getValue("INNER")); + + Record fifth = reader.nextRecord(true, false); + assertEquals("P5", fifth.getValue("ID")); + assertNull(fifth.getValue("AGE")); + Record fifthNested = (Record) fifth.getValue("NAME"); + assertNull(fifthNested.getValue("CONTENT")); + assertNull(fifthNested.getValue("ATTR")); + assertEquals("inner content", fifthNested.getValue("INNER")); + } + + @Test + public void testTagInCharactersCoerceFalseDropFalse() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(false, false); + assertEquals("P1", first.getValue("ID")); + assertEquals("42", first.getValue("AGE")); + Record firstNested = (Record) first.getValue("NAME"); + assertEquals("Cleve Butler", firstNested.getValue("CONTENT")); + assertEquals("attr content", firstNested.getValue("ATTR")); + assertEquals("inner content", firstNested.getValue("INNER")); + + Record second = reader.nextRecord(false, false); + assertEquals("P2", second.getValue("ID")); + assertEquals("33", second.getValue("AGE")); + Record secondNested = (Record) second.getValue("NAME"); + assertEquals("Ainslie Fletcher", secondNested.getValue("CONTENT")); + assertEquals("attr content", secondNested.getValue("ATTR")); + assertEquals("inner content", secondNested.getValue("INNER")); + + Record third = reader.nextRecord(false, false); + assertEquals("P3", third.getValue("ID")); + assertEquals("74", third.getValue("AGE")); + Record thirdNested = (Record) third.getValue("NAME"); + assertEquals("Amélie Bonfils", thirdNested.getValue("CONTENT")); + assertEquals("attr content", thirdNested.getValue("ATTR")); + assertEquals("inner content", thirdNested.getValue("INNER")); + + Record fourth = reader.nextRecord(false, false); + assertEquals("P4", fourth.getValue("ID")); + assertEquals("16", fourth.getValue("AGE")); + Record fourthNested = (Record) fourth.getValue("NAME"); + assertEquals("Elenora Scrivens", fourthNested.getValue("CONTENT")); + assertEquals("attr content", fourthNested.getValue("ATTR")); + assertEquals("inner content", fourthNested.getValue("INNER")); + + Record fifth = reader.nextRecord(false, false); + assertEquals("P5", fifth.getValue("ID")); + assertNull(fifth.getValue("AGE")); + Record fifthNested = (Record) fifth.getValue("NAME"); + assertNull(fifthNested.getValue("CONTENT")); + assertNull(fifthNested.getValue("ATTR")); + assertEquals("inner content", fifthNested.getValue("INNER")); + } + @Test public void testSimpleRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); @@ -64,7 +270,7 @@ public void testSimpleRecord() throws IOException, MalformedRecordException { @Test public void testSimpleRecord2() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema2(), "PEOPLE", "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema2(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertNull(reader.nextRecord(true, true).getValue("AGE")); @@ -76,7 +282,7 @@ public void testSimpleRecord2() throws IOException, MalformedRecordException { @Test public void testSimpleRecord3() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertEquals(Integer.class, reader.nextRecord(true, true).getValue("AGE").getClass()); @@ -88,7 +294,7 @@ public void testSimpleRecord4() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); List fields = getSimpleRecordFields(); fields.remove(2); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertEquals(Integer.class, reader.nextRecord(true, false).getValue("AGE").getClass()); @@ -96,9 +302,9 @@ public void testSimpleRecord4() throws IOException, MalformedRecordException { } @Test - public void testSimpleRecordIgnoreSchema() throws IOException, MalformedRecordException { + public void testSimpleRecordCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_no_attributes.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", "42", "USA"}, reader.nextRecord(false, false).getValues()); @@ -112,7 +318,7 @@ public void testSimpleRecordWithAttribute() throws IOException, MalformedRecordE InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); List fields = getSimpleRecordFields(); fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(); @@ -137,7 +343,7 @@ public void testSimpleRecordWithAttribute2() throws IOException, MalformedRecord InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); List fields = getSimpleRecordFields(); fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", "ATTR_", "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(); @@ -161,7 +367,7 @@ public void testSimpleRecordWithAttribute2() throws IOException, MalformedRecord public void testSimpleRecordWithAttribute3() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(Collections.emptyList()), - "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(true, true); assertEquals(null, first.getAsString("ID")); @@ -182,7 +388,7 @@ public void testSimpleRecordWithAttribute4() throws IOException, MalformedRecord List fields = getSimpleRecordFields(); fields.add(new RecordField("ID", RecordFieldType.INT.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertEquals(Integer.class, reader.nextRecord(true, true).getValue("ID").getClass()); @@ -195,7 +401,7 @@ public void testSimpleRecordWithAttribute5() throws IOException, MalformedRecord List fields = getSimpleRecordFields(); fields.add(new RecordField("ID", RecordFieldType.INT.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertEquals(Integer.class, reader.nextRecord(true, false).getValue("ID").getClass()); @@ -203,11 +409,11 @@ public void testSimpleRecordWithAttribute5() throws IOException, MalformedRecord } @Test - public void testSimpleRecordWithAttributeIgnoreSchema() throws IOException, MalformedRecordException { + public void testSimpleRecordWithAttributeCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); List fields = getSimpleRecordFields(); fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); @@ -246,7 +452,7 @@ public void testSimpleTypeWithAttributeAsRecord() throws IOException, MalformedR final DataType recordType2 = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(nestedFields2)); fields.add(new RecordField("AGE", recordType2)); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(true, true); @@ -271,9 +477,9 @@ public void testSimpleTypeWithAttributeAsRecord() throws IOException, MalformedR } @Test - public void testSimpleTypeWithAttributeAsRecordIgnoreSchema() throws IOException, MalformedRecordException { + public void testSimpleTypeWithAttributeAsRecordCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people4.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); @@ -302,7 +508,7 @@ public void testSimpleTypeWithAttributeAsRecordIgnoreSchema() throws IOException @Test public void testSimpleRecordWithHeader() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_with_header_and_comments.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, null, dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); @@ -314,7 +520,7 @@ public void testSimpleRecordWithHeader() throws IOException, MalformedRecordExce @Test public void testSimpleRecordWithHeaderNoValidation() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_with_header_and_comments.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), null, null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, null, null, dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); @@ -326,7 +532,7 @@ public void testSimpleRecordWithHeaderNoValidation() throws IOException, Malform @Test public void testInvalidXml() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_invalid.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), null, null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); int count = 0; @@ -353,7 +559,7 @@ tag name like the record tag. Even a tracking of the parsing depth would not hel @Test public void testInvalidXml2() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_invalid.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); int count = 0; @@ -372,24 +578,12 @@ public void testInvalidXml2() throws IOException, MalformedRecordException { } } - @Test - public void testMapField() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - List fields = getSimpleRecordFields2(); - fields.add(new RecordField("AGE", RecordFieldType.MAP.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, - "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); - - Record record = reader.nextRecord(); - assertEquals(null, record.getValue("AGE")); - } - @Test public void testChoiceForSimpleField() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); List fields = getSimpleRecordFields2(); fields.add(new RecordField("AGE", RecordFieldType.CHOICE.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record record = reader.nextRecord(); @@ -402,7 +596,7 @@ public void testChoiceForRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); List fields = getSimpleRecordFields(); fields.add(new RecordField("ADDRESS", RecordFieldType.CHOICE.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record record = reader.nextRecord(); @@ -414,9 +608,9 @@ public void testChoiceForRecord() throws IOException, MalformedRecordException { } @Test - public void testCData() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/people_cdata.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + public void testNameSpaces() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_namespace.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); @@ -426,9 +620,9 @@ public void testCData() throws IOException, MalformedRecordException { } @Test - public void testNameSpaces() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/people_namespace.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getNameSpaceSchema(), "PEOPLE", "PERSON", + public void testCData() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_cdata.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); @@ -443,7 +637,7 @@ public void testRecordExpectedSimpleFieldFoundAndNoContentFieldConfigured() thro List fields = getSimpleRecordFields2(); final DataType recordType = RecordFieldType.RECORD.getRecordDataType(getNestedSchema()); fields.add(new RecordField("AGE", recordType)); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", "USA", null}, reader.nextRecord().getValues()); @@ -457,21 +651,19 @@ public void testSimpleFieldExpectedButRecordFound() throws IOException, Malforme InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); List fields = getSimpleRecordFields(); fields.add(new RecordField("ADDRESS", RecordFieldType.STRING.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), "PEOPLE", "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); - String message = ""; - try { - reader.nextRecord(); - } catch (Exception e) { - message = e.getMessage(); - } - assertEquals("Error parsing XML. Either the XML is invalid or there is a mismatch between schema type definitions and XML structure.", message); + + assertNull(reader.nextRecord().getValue("ADDRESS")); + assertNull(reader.nextRecord().getValue("ADDRESS")); + assertNull(reader.nextRecord().getValue("ADDRESS")); + assertNull(reader.nextRecord().getValue("ADDRESS")); } @Test public void testParseEmptyFields() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_empty.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {null, null, null}, reader.nextRecord().getValues()); @@ -479,9 +671,9 @@ public void testParseEmptyFields() throws IOException, MalformedRecordException } @Test - public void testParseEmptyFieldsIgnoreSchema() throws IOException, MalformedRecordException { + public void testParseEmptyFieldsCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_empty.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {null, null, null}, reader.nextRecord(false, false).getValues()); @@ -489,9 +681,17 @@ public void testParseEmptyFieldsIgnoreSchema() throws IOException, MalformedReco } @Test(expected = MalformedRecordException.class) - public void testEmptyStream() throws IOException, MalformedRecordException { + public void testEmptyStreamAsSingleRecord() throws IOException, MalformedRecordException { + InputStream is = new ByteArrayInputStream(new byte[0]); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), false, null, + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + reader.nextRecord(true, true); + } + + @Test(expected = MalformedRecordException.class) + public void testEmptyStreamAsArray() throws IOException, MalformedRecordException { InputStream is = new ByteArrayInputStream(new byte[0]); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), null, null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); reader.nextRecord(true, true); } @@ -499,7 +699,7 @@ public void testEmptyStream() throws IOException, MalformedRecordException { @Test(expected = MalformedRecordException.class) public void testEmptyStreamWIthXmlHeader() throws IOException, MalformedRecordException { InputStream is = new ByteArrayInputStream(("").getBytes()); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), null, null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record record = reader.nextRecord(false, false); assertNull(record); @@ -508,37 +708,16 @@ public void testEmptyStreamWIthXmlHeader() throws IOException, MalformedRecordEx @Test public void testParseEmptyArray() throws IOException, MalformedRecordException { InputStream is = new ByteArrayInputStream("".getBytes()); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "root", null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); - assertEquals(reader.nextRecord(true, true), null); - } - - @Test(expected = MalformedRecordException.class) - public void testInvalidRootName() throws IOException, MalformedRecordException { - InputStream is = new ByteArrayInputStream("".getBytes()); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "other", null, - null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); - - assertEquals(reader.nextRecord(true, true), null); - } - - @Test - public void testNoRootName() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), null, "PERSON", - null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); - - Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); - Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, reader.nextRecord().getValues()); - Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", 74, "FR"}, reader.nextRecord().getValues()); - Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", 16, "USA"}, reader.nextRecord().getValues()); + Assert.assertNull(reader.nextRecord()); } @Test public void testRecordNameWithMismatch() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); int count = 0; while (reader.nextRecord() != null) { @@ -550,7 +729,7 @@ public void testRecordNameWithMismatch() throws IOException, MalformedRecordExce @Test public void testRecordNameOnlyMismatches() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", "OTHER", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "OTHER", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); int count = 0; while (reader.nextRecord() != null) { @@ -562,7 +741,7 @@ public void testRecordNameOnlyMismatches() throws IOException, MalformedRecordEx @Test public void testNoRecordName() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), "PEOPLE", null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); int count = 0; while (reader.nextRecord() != null) { @@ -575,7 +754,7 @@ public void testNoRecordName() throws IOException, MalformedRecordException { public void testNestedRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); RecordSchema schema = getSchemaWithNestedRecord(); - XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Object[] valuesFirstRecord = reader.nextRecord().getValues(); @@ -596,10 +775,10 @@ public void testNestedRecord() throws IOException, MalformedRecordException { } @Test - public void testNestedRecordIgnoreSchema() throws IOException, MalformedRecordException { + public void testNestedRecordCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); RecordSchema schema = getSchemaWithNestedRecord(); - XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); @@ -639,11 +818,196 @@ public void testNestedRecordIgnoreSchema() throws IOException, MalformedRecordEx Assert.assertEquals("70 Bowman St.", nestedFourthRecord.getAsString("STREET")); } + @Test + public void testNestedRecordFieldsToIgnoreCoerceTrueDropTrue() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); + + // Fields "AGE" and "ADDRESS/CITY" are not defined here + RecordSchema schema = getSchemaWithNestedRecord2(); + XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record firstRecord = reader.nextRecord(true, true); + Object[] valuesFirstRecord = firstRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", "USA"}, Arrays.copyOfRange(valuesFirstRecord, 0, valuesFirstRecord.length - 1)); + Record firstRecordNested = (Record) firstRecord.getValue("ADDRESS"); + Assert.assertEquals("292 West Street", firstRecordNested.getValue("STREET")); + Assert.assertNull(firstRecord.getValue("AGE")); + Assert.assertNull(firstRecordNested.getValue("CITY")); + + Record secondRecord = reader.nextRecord(true, true); + Object[] valuesSecondRecord = secondRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", "UK"}, Arrays.copyOfRange(valuesSecondRecord, 0, valuesSecondRecord.length - 1)); + Record secondRecordNested = (Record) secondRecord.getValue("ADDRESS"); + Assert.assertEquals("123 6th St.", secondRecordNested.getValue("STREET")); + Assert.assertNull(secondRecord.getValue("AGE")); + Assert.assertNull(secondRecordNested.getValue("CITY")); + + Record thirdRecord = reader.nextRecord(true, true); + Object[] valuesThirdRecord = thirdRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", "FR"}, Arrays.copyOfRange(valuesThirdRecord, 0, valuesThirdRecord.length - 1)); + Record thirdRecordNested = (Record) thirdRecord.getValue("ADDRESS"); + Assert.assertEquals("44 Shirley Ave.", thirdRecordNested.getValue("STREET")); + Assert.assertNull(thirdRecord.getValue("AGE")); + Assert.assertNull(thirdRecordNested.getValue("CITY")); + + Record fourthRecord = reader.nextRecord(true, true); + Object[] valuesFourthRecord = fourthRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", "USA"}, Arrays.copyOfRange(valuesFourthRecord, 0, valuesFourthRecord.length - 1)); + Record fourthRecordNested = (Record) fourthRecord.getValue("ADDRESS"); + Assert.assertEquals("70 Bowman St.", fourthRecordNested.getValue("STREET")); + Assert.assertNull(fourthRecord.getValue("AGE")); + Assert.assertNull(fourthRecordNested.getValue("CITY")); + } + + @Test + public void testNestedRecordFieldsToIgnoreCoerceFalseDropTrue() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); + + // Fields "AGE" and "ADDRESS/CITY" are not defined here + RecordSchema schema = getSchemaWithNestedRecord2(); + XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record firstRecord = reader.nextRecord(false, true); + Object[] valuesFirstRecord = firstRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", "USA"}, Arrays.copyOfRange(valuesFirstRecord, 0, valuesFirstRecord.length - 1)); + Record firstRecordNested = (Record) firstRecord.getValue("ADDRESS"); + Assert.assertEquals("292 West Street", firstRecordNested.getValue("STREET")); + Assert.assertNull(firstRecord.getValue("AGE")); + Assert.assertNull(firstRecordNested.getValue("CITY")); + + Record secondRecord = reader.nextRecord(false, true); + Object[] valuesSecondRecord = secondRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", "UK"}, Arrays.copyOfRange(valuesSecondRecord, 0, valuesSecondRecord.length - 1)); + Record secondRecordNested = (Record) secondRecord.getValue("ADDRESS"); + Assert.assertEquals("123 6th St.", secondRecordNested.getValue("STREET")); + Assert.assertNull(secondRecord.getValue("AGE")); + Assert.assertNull(secondRecordNested.getValue("CITY")); + + Record thirdRecord = reader.nextRecord(false, true); + Object[] valuesThirdRecord = thirdRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", "FR"}, Arrays.copyOfRange(valuesThirdRecord, 0, valuesThirdRecord.length - 1)); + Record thirdRecordNested = (Record) thirdRecord.getValue("ADDRESS"); + Assert.assertEquals("44 Shirley Ave.", thirdRecordNested.getValue("STREET")); + Assert.assertNull(thirdRecord.getValue("AGE")); + Assert.assertNull(thirdRecordNested.getValue("CITY")); + + Record fourthRecord = reader.nextRecord(false, true); + Object[] valuesFourthRecord = fourthRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", "USA"}, Arrays.copyOfRange(valuesFourthRecord, 0, valuesFourthRecord.length - 1)); + Record fourthRecordNested = (Record) fourthRecord.getValue("ADDRESS"); + Assert.assertEquals("70 Bowman St.", fourthRecordNested.getValue("STREET")); + Assert.assertNull(fourthRecord.getValue("AGE")); + Assert.assertNull(fourthRecordNested.getValue("CITY")); + } + + @Test + public void testNestedRecordFieldsToIgnoreCoerceTrueDropFalse() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); + + // Fields "AGE" and "ADDRESS/CITY" are not defined here + RecordSchema schema = getSchemaWithNestedRecord2(); + XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record firstRecord = reader.nextRecord(true, false); + Object[] valuesFirstRecord = firstRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", "USA"}, Arrays.copyOfRange(valuesFirstRecord, 0, valuesFirstRecord.length - 1)); + Record firstRecordNested = (Record) firstRecord.getValue("ADDRESS"); + Assert.assertEquals("292 West Street", firstRecordNested.getValue("STREET")); + Assert.assertNotNull(firstRecord.getValue("AGE")); + Assert.assertEquals("42", firstRecord.getValue("AGE")); + Assert.assertNotNull(firstRecordNested.getValue("CITY")); + Assert.assertEquals("Jersey City", firstRecordNested.getValue("CITY")); + + Record secondRecord = reader.nextRecord(true, false); + Object[] valuesSecondRecord = secondRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", "UK"}, Arrays.copyOfRange(valuesSecondRecord, 0, valuesSecondRecord.length - 1)); + Record secondRecordNested = (Record) secondRecord.getValue("ADDRESS"); + Assert.assertEquals("123 6th St.", secondRecordNested.getValue("STREET")); + Assert.assertNotNull(secondRecord.getValue("AGE")); + Assert.assertEquals("33", secondRecord.getValue("AGE")); + Assert.assertNotNull(secondRecordNested.getValue("CITY")); + Assert.assertEquals("Seattle", secondRecordNested.getValue("CITY")); + + Record thirdRecord = reader.nextRecord(true, false); + Object[] valuesThirdRecord = thirdRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", "FR"}, Arrays.copyOfRange(valuesThirdRecord, 0, valuesThirdRecord.length - 1)); + Record thirdRecordNested = (Record) thirdRecord.getValue("ADDRESS"); + Assert.assertEquals("44 Shirley Ave.", thirdRecordNested.getValue("STREET")); + Assert.assertNotNull(thirdRecord.getValue("AGE")); + Assert.assertEquals("74", thirdRecord.getValue("AGE")); + Assert.assertNotNull(thirdRecordNested.getValue("CITY")); + Assert.assertEquals("Los Angeles", thirdRecordNested.getValue("CITY")); + + Record fourthRecord = reader.nextRecord(true, false); + Object[] valuesFourthRecord = fourthRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", "USA"}, Arrays.copyOfRange(valuesFourthRecord, 0, valuesFourthRecord.length - 1)); + Record fourthRecordNested = (Record) fourthRecord.getValue("ADDRESS"); + Assert.assertEquals("70 Bowman St.", fourthRecordNested.getValue("STREET")); + Assert.assertNotNull(fourthRecord.getValue("AGE")); + Assert.assertEquals("16", fourthRecord.getValue("AGE")); + Assert.assertNotNull(fourthRecordNested.getValue("CITY")); + Assert.assertEquals("Columbus", fourthRecordNested.getValue("CITY")); + } + + @Test + public void testNestedRecordFieldsToIgnoreCoerceFalseDropFalse() throws IOException, MalformedRecordException { + InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); + + // Fields "AGE" and "ADDRESS/CITY" are not defined here + RecordSchema schema = getSchemaWithNestedRecord2(); + XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record firstRecord = reader.nextRecord(false, false); + Object[] valuesFirstRecord = firstRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Cleve Butler", "USA"}, Arrays.copyOfRange(valuesFirstRecord, 0, valuesFirstRecord.length - 1)); + Record firstRecordNested = (Record) firstRecord.getValue("ADDRESS"); + Assert.assertEquals("292 West Street", firstRecordNested.getValue("STREET")); + Assert.assertNotNull(firstRecord.getValue("AGE")); + Assert.assertEquals("42", firstRecord.getValue("AGE")); + Assert.assertNotNull(firstRecordNested.getValue("CITY")); + Assert.assertEquals("Jersey City", firstRecordNested.getValue("CITY")); + + Record secondRecord = reader.nextRecord(false, false); + Object[] valuesSecondRecord = secondRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", "UK"}, Arrays.copyOfRange(valuesSecondRecord, 0, valuesSecondRecord.length - 1)); + Record secondRecordNested = (Record) secondRecord.getValue("ADDRESS"); + Assert.assertEquals("123 6th St.", secondRecordNested.getValue("STREET")); + Assert.assertNotNull(secondRecord.getValue("AGE")); + Assert.assertEquals("33", secondRecord.getValue("AGE")); + Assert.assertNotNull(secondRecordNested.getValue("CITY")); + Assert.assertEquals("Seattle", secondRecordNested.getValue("CITY")); + + Record thirdRecord = reader.nextRecord(false, false); + Object[] valuesThirdRecord = thirdRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Amélie Bonfils", "FR"}, Arrays.copyOfRange(valuesThirdRecord, 0, valuesThirdRecord.length - 1)); + Record thirdRecordNested = (Record) thirdRecord.getValue("ADDRESS"); + Assert.assertEquals("44 Shirley Ave.", thirdRecordNested.getValue("STREET")); + Assert.assertNotNull(thirdRecord.getValue("AGE")); + Assert.assertEquals("74", thirdRecord.getValue("AGE")); + Assert.assertNotNull(thirdRecordNested.getValue("CITY")); + Assert.assertEquals("Los Angeles", thirdRecordNested.getValue("CITY")); + + Record fourthRecord = reader.nextRecord(false, false); + Object[] valuesFourthRecord = fourthRecord.getValues(); + Assert.assertArrayEquals(new Object[] {"Elenora Scrivens", "USA"}, Arrays.copyOfRange(valuesFourthRecord, 0, valuesFourthRecord.length - 1)); + Record fourthRecordNested = (Record) fourthRecord.getValue("ADDRESS"); + Assert.assertEquals("70 Bowman St.", fourthRecordNested.getValue("STREET")); + Assert.assertNotNull(fourthRecord.getValue("AGE")); + Assert.assertEquals("16", fourthRecord.getValue("AGE")); + Assert.assertNotNull(fourthRecordNested.getValue("CITY")); + Assert.assertEquals("Columbus", fourthRecordNested.getValue("CITY")); + } + + @Test public void testSimpleArray() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_array_simple.xml"); RecordSchema schema = getSchemaWithSimpleArray(); - XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record firstRecord = reader.nextRecord(); @@ -676,10 +1040,10 @@ public void testSimpleArray() throws IOException, MalformedRecordException { } @Test - public void testSimpleArrayIgnoreSchema() throws IOException, MalformedRecordException { + public void testSimpleArrayCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_array_simple.xml"); RecordSchema schema = getSchemaWithSimpleArray(); - XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); @@ -714,7 +1078,7 @@ public void testSimpleArrayIgnoreSchema() throws IOException, MalformedRecordExc public void testNestedArrayInNestedRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_array.xml"); RecordSchema schema = getSchemaWithNestedArray(); - XMLRecordReader reader = new XMLRecordReader(is, schema, "PEOPLE", "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record firstRecord = reader.nextRecord(); @@ -752,7 +1116,7 @@ public void testNestedArrayInNestedRecord() throws IOException, MalformedRecordE public void testDeeplyNestedArraysAndRecords() throws IOException, MalformedRecordException { // test records in nested arrays InputStream is = new FileInputStream("src/test/resources/xml/people_complex1.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData(), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(true, true); @@ -791,7 +1155,7 @@ public void testDeeplyNestedArraysAndRecords() throws IOException, MalformedReco public void testDeeplyNestedArraysAndRecords2() throws IOException, MalformedRecordException { // test multiply nested arrays and records (recursion) InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), "PEOPLE", "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(); @@ -844,11 +1208,67 @@ public void testDeeplyNestedArraysAndRecords2() throws IOException, MalformedRec } @Test - public void testDeeplyNestedArraysAndRecordsIgnoreSchema() throws IOException, MalformedRecordException { + public void testDeeplyNestedArraysAndRecordsCoerceFalseDropTrue() throws IOException, MalformedRecordException { + // test multiply nested arrays and records (recursion) + InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), true, "PERSON", + null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + + Record first = reader.nextRecord(false, true); + + assertEquals("grandmother", first.getValue("ROLE")); + assertEquals("1", first.getValue("ID")); + assertEquals("Lisa", first.getValue("NAME")); + + Object[] gm_spouses = (Object[]) first.getValue("CHILDREN"); + assertEquals(2, gm_spouses.length); + + Object[] gm_spouse1_parents = (Object[]) ((Record) gm_spouses[0]).getValue("CHILD"); + assertEquals(2, gm_spouse1_parents.length); + + Record first_1_1 = (Record) gm_spouse1_parents[0]; + assertEquals("mother", first_1_1.getValue("ROLE")); + assertEquals("1-1", first_1_1.getValue("ID")); + assertEquals("Anna", first_1_1.getValue("NAME")); + + Record gm_spouse1_parent1_first_husband = (Record) first_1_1.getValue("CHILDREN"); + Object[] gm_spouse1_parent1_children = (Object[])gm_spouse1_parent1_first_husband.getValue("CHILD"); + + Record first_1_1_1 = (Record) gm_spouse1_parent1_children[0]; + assertEquals("daughter", first_1_1_1.getValue("ROLE")); + assertEquals("1-1-1", first_1_1_1.getValue("ID")); + assertEquals("Selina", first_1_1_1.getValue("NAME")); + + Record first_1_1_2 = (Record) gm_spouse1_parent1_children[1]; + assertEquals("son", first_1_1_2.getValue("ROLE")); + assertEquals("1-1-2", first_1_1_2.getValue("ID")); + assertEquals("Hans", first_1_1_2.getValue("NAME")); + + Record first_1_2 = (Record) gm_spouse1_parents[1]; + assertEquals("mother", first_1_2.getValue("ROLE")); + assertEquals("1-2", first_1_2.getValue("ID")); + assertEquals("Catrina", first_1_2.getValue("NAME")); + + Record gm_spouse2_parents = (Record) ((Record) gm_spouses[1]).getValue("CHILD"); + assertEquals("1-3", gm_spouse2_parents.getValue("ID")); + + Record second = reader.nextRecord(false, true); + Record second_2_1_1 = (Record)((Record)((Record)((Record) second + .getValue("CHILDREN")) + .getValue("CHILD")) + .getValue("CHILDREN")) + .getValue("CHILD"); + assertEquals("daughter", second_2_1_1.getValue("ROLE")); + assertEquals("2-1-1", second_2_1_1.getValue("ID")); + assertEquals("Selina3", second_2_1_1.getValue("NAME")); + } + + @Test + public void testDeeplyNestedArraysAndRecordsCoerceFalseDropFalse() throws IOException, MalformedRecordException { // test multiply nested arrays and records (recursion) InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml"); XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(Collections.emptyList()), - "PEOPLE", "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); assertEquals("1", first.getValue("ID")); @@ -919,6 +1339,13 @@ private List getSimpleRecordFields2() { return fields; } + private List getSimpleRecordFields3() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("AGE", RecordFieldType.INT.getDataType())); + fields.add(new RecordField("COUNTRY", RecordFieldType.STRING.getDataType())); + return fields; + } + private List getNestedRecordFields() { final List fields = new ArrayList<>(); fields.add(new RecordField("STREET", RecordFieldType.STRING.getDataType())); @@ -926,6 +1353,20 @@ private List getNestedRecordFields() { return fields; } + private List getNestedRecordFields2() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("STREET", RecordFieldType.STRING.getDataType())); + return fields; + } + + private List getNestedRecordFields3() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("CONTENT", RecordFieldType.STRING.getDataType())); + fields.add(new RecordField("ATTR", RecordFieldType.STRING.getDataType())); + fields.add(new RecordField("INNER", RecordFieldType.STRING.getDataType())); + return fields; + } + private List getNameSpaceFields() { final List fields = new ArrayList<>(); fields.add(new RecordField("F:NAME", RecordFieldType.STRING.getDataType())); @@ -946,6 +1387,14 @@ private RecordSchema getNestedSchema() { return new SimpleRecordSchema(getNestedRecordFields()); } + private RecordSchema getNestedSchema2() { + return new SimpleRecordSchema(getNestedRecordFields2()); + } + + private RecordSchema getNestedSchema3() { + return new SimpleRecordSchema(getNestedRecordFields3()); + } + private RecordSchema getNameSpaceSchema() { return new SimpleRecordSchema(getNameSpaceFields()); } @@ -957,6 +1406,20 @@ private RecordSchema getSchemaWithNestedRecord() { return new SimpleRecordSchema(fields); } + private RecordSchema getSchemaWithNestedRecord2() { + final List fields = getSimpleRecordFields2(); + final DataType recordType = RecordFieldType.RECORD.getRecordDataType(getNestedSchema2()); + fields.add(new RecordField("ADDRESS", recordType)); + return new SimpleRecordSchema(fields); + } + + private RecordSchema getSchemaWithNestedRecord3() { + final List fields = getSimpleRecordFields3(); + final DataType recordType = RecordFieldType.RECORD.getRecordDataType(getNestedSchema3()); + fields.add(new RecordField("NAME", recordType)); + return new SimpleRecordSchema(fields); + } + private RecordSchema getSchemaWithSimpleArray() { final List fields = getSimpleRecordFields(); final DataType arrayType = RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.STRING.getDataType()); @@ -982,12 +1445,36 @@ private List getSimpleFieldsForComplexData() { return fields; } + private RecordSchema getSchemaForMap() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); + + final DataType map = RecordFieldType.MAP.getMapDataType(RecordFieldType.STRING.getDataType()); + fields.add(new RecordField("MAP", map)); + + return new SimpleRecordSchema(fields); + } + + private RecordSchema getSchemaForRecordMap() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); + + final DataType record = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(getSimpleRecordFields())); + + final DataType map = RecordFieldType.MAP.getMapDataType(record); + fields.add(new RecordField("MAP", map)); + + return new SimpleRecordSchema(fields); + } + private RecordSchema getSchemaForComplexData() { final DataType grandchild = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(getSimpleFieldsForComplexData())); final DataType grandchild_arr1 = RecordFieldType.ARRAY.getArrayDataType(grandchild); + final DataType grandchildren = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema( new ArrayList() {{ add(new RecordField("CHILD", grandchild_arr1)); }})); final DataType grandchild_arr = RecordFieldType.ARRAY.getArrayDataType(grandchildren); + return new SimpleRecordSchema( new ArrayList() {{ add(new RecordField("CHILDREN", grandchild_arr)); }}); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array.xml index f48baebc703c..ff304c3ecb41 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array.xml +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_array.xml @@ -24,6 +24,7 @@ child1 child2 child3 + diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_map.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_map.xml new file mode 100755 index 000000000000..a115a0f2de6d --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_map.xml @@ -0,0 +1,18 @@ + + + P1 + + Cleve Butler + 42 + USA + + + + P2 + + Ainslie Fletcher + 33 + UK + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_map2.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_map2.xml new file mode 100755 index 000000000000..a348c76cb78b --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_map2.xml @@ -0,0 +1,32 @@ + + + P1 + + + Cleve Butler + 42 + USA + + + Ainslie Fletcher + 33 + UK + + + + + P2 + + + Amélie Bonfils + 74 + FR + + + Elenora Scrivens + 16 + USA + + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_tag_in_characters.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_tag_in_characters.xml new file mode 100755 index 000000000000..a2101dc80cfd --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people_tag_in_characters.xml @@ -0,0 +1,23 @@ + + + Cleve inner contentButler + 42 + + + Ainslie inner contentFletcher + 33 + + + Amélie inner contentBonfils + 74 + + + Elenora inner contentScrivens + 16 + + + inner content + + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/person.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/person.xml new file mode 100755 index 000000000000..f2cd20bee7e9 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/person.xml @@ -0,0 +1,5 @@ + + Cleve Butler + 42 + USA + \ No newline at end of file From 174c86c8037e47c750230b8ab2de6f14266c24d6 Mon Sep 17 00:00:00 2001 From: JohannesDaniel Date: Sun, 22 Apr 2018 16:18:41 +0200 Subject: [PATCH 5/6] NIFI-4185 R3 --- .../pom.xml | 2 +- .../java/org/apache/nifi/xml/XMLReader.java | 38 ++-- .../org/apache/nifi/xml/XMLRecordReader.java | 19 +- .../org/apache/nifi/xml/TestXMLReader.java | 110 +++++++---- .../apache/nifi/xml/TestXMLRecordReader.java | 179 ++++++------------ .../src/test/resources/xml/people3.xml | 19 +- .../src/test/resources/xml/people4.xml | 12 -- .../src/test/resources/xml/testschema2 | 19 ++ 8 files changed, 176 insertions(+), 222 deletions(-) delete mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people4.xml create mode 100755 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/testschema2 diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml index f25ac6da9aa9..4d47701ff601 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml @@ -125,7 +125,6 @@ src/test/resources/xml/people.xml src/test/resources/xml/people2.xml src/test/resources/xml/people3.xml - src/test/resources/xml/people4.xml src/test/resources/xml/people_array.xml src/test/resources/xml/people_array_simple.xml src/test/resources/xml/people_cdata.xml @@ -142,6 +141,7 @@ src/test/resources/xml/people_with_header_and_comments.xml src/test/resources/xml/person.xml src/test/resources/xml/testschema + src/test/resources/xml/testschema2 diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java index 216592f12029..73ee75dfa8b7 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java @@ -45,30 +45,26 @@ "XML data, embedded in an enclosing root tag.") public class XMLReader extends SchemaRegistryService implements RecordReaderFactory { - public static final AllowableValue RECORD_SINGLE = new AllowableValue("record_single", "Single Record"); - public static final AllowableValue RECORD_ARRAY = new AllowableValue("record_array", "Array of Records"); + public static final AllowableValue RECORD_SINGLE = new AllowableValue("false"); + public static final AllowableValue RECORD_ARRAY = new AllowableValue("true"); + public static final AllowableValue RECORD_EVALUATE = new AllowableValue("${xml.stream.is.array}","Use attribute xml.stream.is.array"); public static final PropertyDescriptor RECORD_FORMAT = new PropertyDescriptor.Builder() .name("record_format") - .displayName("Record Format") - .description("This property defines whether the reader expects a single record an array of records") + .displayName("Expect Records as Array") + .description("This property defines whether the reader expects a single record an array of records. If the property is " + + "set to \"true\", the reader expects an array of records and the outer element of the XML will be treated as a " + + "wrapper for the records. If the property is set to \"false\", the reader expects a single record for each FlowFile " + + "(without wrapper-element). If the property is set to \"Use attribute xml.stream.is.array\", the attribute " + + "\"xml.stream.is.array\" will be evaluated for each FlowFile whether to treat its content as array " + + "of records (in the case of \"true\") or as single record (in the case of \"false\".") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .allowableValues(RECORD_SINGLE, RECORD_ARRAY) + .allowableValues(RECORD_SINGLE, RECORD_ARRAY, RECORD_EVALUATE) .defaultValue(RECORD_SINGLE.getValue()) .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .required(false) .build(); - public static final PropertyDescriptor CHECK_RECORD_TAG = new PropertyDescriptor.Builder() - .name("check_record_tag") - .displayName("Check Record Tag") - .description("If this property is set, the name of record tags 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, all records will be processed.") - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) - .required(false) - .build(); - public static final PropertyDescriptor ATTRIBUTE_PREFIX = new PropertyDescriptor.Builder() .name("attribute_prefix") .displayName("Attribute Prefix") @@ -105,7 +101,6 @@ public void onEnabled(final ConfigurationContext context) { protected List getSupportedPropertyDescriptors() { final List properties = new ArrayList<>(super.getSupportedPropertyDescriptors()); properties.add(RECORD_FORMAT); - properties.add(CHECK_RECORD_TAG); properties.add(ATTRIBUTE_PREFIX); properties.add(CONTENT_FIELD_NAME); properties.add(DateTimeUtils.DATE_FORMAT); @@ -115,23 +110,20 @@ protected List getSupportedPropertyDescriptors() { } @Override - public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException, MalformedRecordException { + public RecordReader createRecordReader(final Map variables, final InputStream in, final ComponentLog logger) + throws IOException, SchemaNotFoundException, MalformedRecordException { final ConfigurationContext context = getConfigurationContext(); final RecordSchema schema = getSchema(variables, in, null); - final String recordName = context.getProperty(CHECK_RECORD_TAG).isSet() - ? context.getProperty(CHECK_RECORD_TAG).evaluateAttributeExpressions(variables).getValue().trim() : null; - final String attributePrefix = context.getProperty(ATTRIBUTE_PREFIX).isSet() ? context.getProperty(ATTRIBUTE_PREFIX).evaluateAttributeExpressions(variables).getValue().trim() : null; final String contentFieldName = context.getProperty(CONTENT_FIELD_NAME).isSet() ? context.getProperty(CONTENT_FIELD_NAME).evaluateAttributeExpressions(variables).getValue().trim() : null; - final boolean isArray = context.getProperty(RECORD_FORMAT).evaluateAttributeExpressions(variables).getValue() - .equals(RECORD_ARRAY.getValue()); + final boolean isArray = Boolean.parseBoolean(context.getProperty(RECORD_FORMAT).evaluateAttributeExpressions(variables).getValue()); - return new XMLRecordReader(in, schema, isArray, recordName, attributePrefix, contentFieldName, dateFormat, timeFormat, timestampFormat, logger); + return new XMLRecordReader(in, schema, isArray, attributePrefix, contentFieldName, dateFormat, timeFormat, timestampFormat, logger); } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java index c104103c0e16..e819b92deaeb 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java @@ -55,7 +55,6 @@ 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; @@ -67,10 +66,9 @@ public class XMLRecordReader implements RecordReader { private final Supplier LAZY_TIME_FORMAT; private final Supplier LAZY_TIMESTAMP_FORMAT; - public XMLRecordReader(InputStream in, RecordSchema schema, boolean isArray, String recordName, String attributePrefix, String contentFieldName, + public XMLRecordReader(InputStream in, RecordSchema schema, boolean isArray, 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; @@ -112,19 +110,8 @@ private void setNextRecordStartTag() throws XMLStreamException { final XMLEvent xmlEvent = xmlEventReader.nextEvent(); if (xmlEvent.isStartElement()) { final StartElement startElement = xmlEvent.asStartElement(); - if (recordName != null) { - if (startElement.getName().getLocalPart().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().getLocalPart()}); - skipElement(); - } - } else { - currentRecordStartTag = startElement; - return; - } + currentRecordStartTag = startElement; + return; } } currentRecordStartTag = null; diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java index 4e612b379e95..e1c767ed3900 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java @@ -22,11 +22,9 @@ import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; -import org.junit.Before; import org.junit.Test; import java.io.FileInputStream; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.nio.file.Files; @@ -39,75 +37,96 @@ public class TestXMLReader { - private TestRunner runner; private XMLReader reader; - private final String ROOT_TAG_NAME = "root_tag"; - private final String RECORD_TAG_NAME = "record_tag"; private final String ATTRIBUTE_PREFIX = "attribute_prefix"; + private final String CONTENT_NAME = "content_field"; + private final String EVALUATE_IS_ARRAY = "xml.stream.is.array"; - @Before - public void setup() throws InitializationException, IOException { + public TestRunner setup(String filePath) throws InitializationException, IOException { - runner = TestRunners.newTestRunner(TestXMLReaderProcessor.class); + TestRunner runner = TestRunners.newTestRunner(TestXMLReaderProcessor.class); reader = new XMLReader(); runner.addControllerService("xml_reader", reader); runner.setProperty(TestXMLReaderProcessor.XML_READER, "xml_reader"); - final String outputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/xml/testschema"))); + final String outputSchemaText = new String(Files.readAllBytes(Paths.get(filePath))); runner.setProperty(reader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY); runner.setProperty(reader, SchemaAccessUtils.SCHEMA_TEXT, outputSchemaText); + + return runner; } @Test - public void testNoCheckForRecord() throws FileNotFoundException { - runner.setProperty(reader, XMLReader.RECORD_FORMAT, XMLReader.RECORD_ARRAY); + public void testRecordFormat() throws IOException, InitializationException { + TestRunner runner = setup("src/test/resources/xml/testschema"); - runner.enableControllerService(reader); + runner.setProperty(reader, XMLReader.RECORD_FORMAT, XMLReader.RECORD_EVALUATE); - InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + runner.enableControllerService(reader); - runner.enqueue(is); + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + runner.enqueue(is, new HashMap() {{ + put(EVALUATE_IS_ARRAY, "true"); + }}); runner.run(); + List flowFile = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS); List records = Arrays.asList((new String(runner.getContentAsByteArray(flowFile.get(0)))).split("\n")); - assertEquals(3, records.size()); - assertEquals("MapRecord[{COUNTRY=USA, ID=1, NAME=Cleve Butler, AGE=42}]", records.get(0)); - assertEquals("MapRecord[{COUNTRY=UK, ID=2, NAME=Ainslie Fletcher, AGE=33}]", records.get(1)); - assertEquals("MapRecord[{COUNTRY=UK, ID=3, NAME=Ainslie Fletcher, AGE=33}]", records.get(2)); + assertEquals(4, records.size()); } @Test - public void testCheckForRecord() throws FileNotFoundException { - runner.setProperty(reader, XMLReader.CHECK_RECORD_TAG, "${" + RECORD_TAG_NAME + "}"); + public void testRecordFormat2() throws IOException, InitializationException { + TestRunner runner = setup("src/test/resources/xml/testschema"); + runner.setProperty(reader, XMLReader.RECORD_FORMAT, XMLReader.RECORD_ARRAY); runner.enableControllerService(reader); - InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); + runner.enqueue(is, new HashMap() {{ + put(EVALUATE_IS_ARRAY, "true"); + }}); + runner.run(); + + List flowFile = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS); + List records = Arrays.asList((new String(runner.getContentAsByteArray(flowFile.get(0)))).split("\n")); + + assertEquals(4, records.size()); + } + + @Test + public void testRecordFormat3() throws IOException, InitializationException { + TestRunner runner = setup("src/test/resources/xml/testschema"); + + runner.setProperty(reader, XMLReader.RECORD_FORMAT, XMLReader.RECORD_SINGLE); + + runner.enableControllerService(reader); + + InputStream is = new FileInputStream("src/test/resources/xml/person.xml"); runner.enqueue(is, new HashMap() {{ - put(ROOT_TAG_NAME, "PEOPLE"); - put(RECORD_TAG_NAME, "PERSON"); + put(EVALUATE_IS_ARRAY, "true"); }}); runner.run(); List flowFile = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS); List records = Arrays.asList((new String(runner.getContentAsByteArray(flowFile.get(0)))).split("\n")); - assertEquals(2, records.size()); - assertEquals("MapRecord[{COUNTRY=USA, ID=1, NAME=Cleve Butler, AGE=42}]", records.get(0)); - assertEquals("MapRecord[{COUNTRY=UK, ID=2, NAME=Ainslie Fletcher, AGE=33}]", records.get(1)); + assertEquals(1, records.size()); } @Test - public void testAttributePrefix() throws FileNotFoundException { + public void testAttributePrefix() throws IOException, InitializationException { + TestRunner runner = setup("src/test/resources/xml/testschema"); + runner.setProperty(reader, XMLReader.ATTRIBUTE_PREFIX, "${" + ATTRIBUTE_PREFIX + "}"); runner.setProperty(reader, XMLReader.RECORD_FORMAT, XMLReader.RECORD_ARRAY); runner.enableControllerService(reader); - InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); runner.enqueue(is, new HashMap() {{ put(ATTRIBUTE_PREFIX, "ATTR_"); }}); @@ -116,9 +135,36 @@ public void testAttributePrefix() throws FileNotFoundException { List flowFile = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS); List records = Arrays.asList((new String(runner.getContentAsByteArray(flowFile.get(0)))).split("\n")); - assertEquals(3, records.size()); - assertEquals("MapRecord[{COUNTRY=USA, ATTR_ID=1, NAME=Cleve Butler, AGE=42}]", records.get(0)); - assertEquals("MapRecord[{COUNTRY=UK, ATTR_ID=2, NAME=Ainslie Fletcher, AGE=33}]", records.get(1)); - assertEquals("MapRecord[{COUNTRY=UK, ATTR_ID=3, NAME=Ainslie Fletcher, AGE=33}]", records.get(2)); + assertEquals(4, records.size()); + assertEquals("MapRecord[{COUNTRY=USA, ATTR_ID=P1, NAME=Cleve Butler, AGE=42}]", records.get(0)); + assertEquals("MapRecord[{COUNTRY=UK, ATTR_ID=P2, NAME=Ainslie Fletcher, AGE=33}]", records.get(1)); + assertEquals("MapRecord[{COUNTRY=FR, ATTR_ID=P3, NAME=Amélie Bonfils, AGE=74}]", records.get(2)); + assertEquals("MapRecord[{COUNTRY=USA, ATTR_ID=P4, NAME=Elenora Scrivens, AGE=16}]", records.get(3)); + } + + @Test + public void testContentField() throws IOException, InitializationException { + TestRunner runner = setup("src/test/resources/xml/testschema2"); + + runner.setProperty(reader, XMLReader.CONTENT_FIELD_NAME, "${" + CONTENT_NAME + "}"); + runner.setProperty(reader, XMLReader.RECORD_FORMAT, XMLReader.RECORD_ARRAY); + + runner.enableControllerService(reader); + + InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml"); + runner.enqueue(is, new HashMap() {{ + put(CONTENT_NAME, "CONTENT"); + }}); + runner.run(); + + List flowFile = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS); + List records = Arrays.asList((new String(runner.getContentAsByteArray(flowFile.get(0)))).split("\n")); + + assertEquals(5, records.size()); + assertEquals("MapRecord[{ID=P1, NAME=MapRecord[{CONTENT=Cleve Butler, ATTR=attr content, INNER=inner content}], AGE=42}]", records.get(0)); + assertEquals("MapRecord[{ID=P2, NAME=MapRecord[{CONTENT=Ainslie Fletcher, ATTR=attr content, INNER=inner content}], AGE=33}]", records.get(1)); + assertEquals("MapRecord[{ID=P3, NAME=MapRecord[{CONTENT=Amélie Bonfils, ATTR=attr content, INNER=inner content}], AGE=74}]", records.get(2)); + assertEquals("MapRecord[{ID=P4, NAME=MapRecord[{CONTENT=Elenora Scrivens, ATTR=attr content, INNER=inner content}], AGE=16}]", records.get(3)); + assertEquals("MapRecord[{ID=P5, NAME=MapRecord[{INNER=inner content}]}]", records.get(4)); } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java index ce9e3ea307c6..ef3d692634aa 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java @@ -53,24 +53,16 @@ public class TestXMLRecordReader { @Test public void testSingleRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/person.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), false, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), false, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); Assert.assertNull(reader.nextRecord()); } - @Test - public void testSingleRecordCheckForTag() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/person.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), false, "OTHER", - null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); - Assert.assertNull(reader.nextRecord()); - } - @Test public void testMap() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_map.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaForMap(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForMap(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(); @@ -91,7 +83,7 @@ public void testMap() throws IOException, MalformedRecordException { @Test public void testMapWithRecords() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_map2.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaForRecordMap(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForRecordMap(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(); @@ -110,7 +102,7 @@ public void testMapWithRecords() throws IOException, MalformedRecordException { @Test public void testTagInCharactersSimpleField() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, null}, reader.nextRecord().getValues()); @@ -122,7 +114,7 @@ public void testTagInCharactersSimpleField() throws IOException, MalformedRecord @Test public void testTagInCharactersRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaWithNestedRecord3(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSchemaWithNestedRecord3(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(true, true); @@ -164,7 +156,7 @@ public void testTagInCharactersRecord() throws IOException, MalformedRecordExcep @Test public void testTagInCharactersCoerceTrueDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaWithNestedRecord3(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSchemaWithNestedRecord3(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(true, false); @@ -211,7 +203,7 @@ public void testTagInCharactersCoerceTrueDropFalse() throws IOException, Malform @Test public void testTagInCharactersCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); @@ -258,7 +250,7 @@ public void testTagInCharactersCoerceFalseDropFalse() throws IOException, Malfor @Test public void testSimpleRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); @@ -270,7 +262,7 @@ public void testSimpleRecord() throws IOException, MalformedRecordException { @Test public void testSimpleRecord2() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema2(), true, "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema2(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertNull(reader.nextRecord(true, true).getValue("AGE")); @@ -282,7 +274,7 @@ public void testSimpleRecord2() throws IOException, MalformedRecordException { @Test public void testSimpleRecord3() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertEquals(Integer.class, reader.nextRecord(true, true).getValue("AGE").getClass()); @@ -294,7 +286,7 @@ public void testSimpleRecord4() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); List fields = getSimpleRecordFields(); fields.remove(2); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertEquals(Integer.class, reader.nextRecord(true, false).getValue("AGE").getClass()); @@ -304,7 +296,7 @@ public void testSimpleRecord4() throws IOException, MalformedRecordException { @Test public void testSimpleRecordCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_no_attributes.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", "42", "USA"}, reader.nextRecord(false, false).getValues()); @@ -318,7 +310,7 @@ public void testSimpleRecordWithAttribute() throws IOException, MalformedRecordE InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); List fields = getSimpleRecordFields(); fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(); @@ -343,7 +335,7 @@ public void testSimpleRecordWithAttribute2() throws IOException, MalformedRecord InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); List fields = getSimpleRecordFields(); fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "ATTR_", "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(); @@ -367,7 +359,7 @@ public void testSimpleRecordWithAttribute2() throws IOException, MalformedRecord public void testSimpleRecordWithAttribute3() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(Collections.emptyList()), - true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(true, true); assertEquals(null, first.getAsString("ID")); @@ -388,7 +380,7 @@ public void testSimpleRecordWithAttribute4() throws IOException, MalformedRecord List fields = getSimpleRecordFields(); fields.add(new RecordField("ID", RecordFieldType.INT.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertEquals(Integer.class, reader.nextRecord(true, true).getValue("ID").getClass()); @@ -401,7 +393,7 @@ public void testSimpleRecordWithAttribute5() throws IOException, MalformedRecord List fields = getSimpleRecordFields(); fields.add(new RecordField("ID", RecordFieldType.INT.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertEquals(Integer.class, reader.nextRecord(true, false).getValue("ID").getClass()); @@ -413,7 +405,7 @@ public void testSimpleRecordWithAttributeCoerceFalseDropFalse() throws IOExcepti InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); List fields = getSimpleRecordFields(); fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); @@ -435,7 +427,7 @@ public void testSimpleRecordWithAttributeCoerceFalseDropFalse() throws IOExcepti @Test public void testSimpleTypeWithAttributeAsRecord() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/people4.xml"); + InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); final List fields = new ArrayList<>(); final List nestedFields1 = new ArrayList<>(); @@ -452,7 +444,7 @@ public void testSimpleTypeWithAttributeAsRecord() throws IOException, MalformedR final DataType recordType2 = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(nestedFields2)); fields.add(new RecordField("AGE", recordType2)); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(true, true); @@ -478,8 +470,8 @@ public void testSimpleTypeWithAttributeAsRecord() throws IOException, MalformedR @Test public void testSimpleTypeWithAttributeAsRecordCoerceFalseDropFalse() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/people4.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); @@ -508,7 +500,7 @@ public void testSimpleTypeWithAttributeAsRecordCoerceFalseDropFalse() throws IOE @Test public void testSimpleRecordWithHeader() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_with_header_and_comments.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, null, dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); @@ -520,8 +512,7 @@ public void testSimpleRecordWithHeader() throws IOException, MalformedRecordExce @Test public void testSimpleRecordWithHeaderNoValidation() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_with_header_and_comments.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, - null, null, dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, null, dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); Assert.assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, reader.nextRecord().getValues()); @@ -532,17 +523,13 @@ public void testSimpleRecordWithHeaderNoValidation() throws IOException, Malform @Test public void testInvalidXml() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_invalid.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, - null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); int count = 0; /* - As there is no validation for recordName here, the reader assumes and to be records - (due to the missing starting tag ). The tag is also assumed to be a record, but the exception is thrown - before this "record" is returned. This unexpected behavior can be avoided by defining recordName and consequently - validating the tag name of records (see test testInvalidXml2). Generally, it could be considered to force users - to validate tags at root and record level. However, this only helps if the embedded tags do not have the same - tag name like the record tag. Even a tracking of the parsing depth would not help to overcome this problem. + Due to the missing starting tag for the third entry in people_invalid.xml, the reader assumes and + to be records. The tag is also assumed to be a record, but the exception is thrown + before the "record" for is returned. Even a tracking of the parsing depth would not help to overcome this problem. */ try { @@ -556,34 +543,12 @@ tag name like the record tag. Even a tracking of the parsing depth would not hel } - @Test - public void testInvalidXml2() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/people_invalid.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", - null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); - int count = 0; - - try { - while ((reader.nextRecord()) != null) { - count++; - } - } catch (MalformedRecordException e) { - assertEquals("Could not parse XML", e.getMessage()); - - /* - Actually, there are two valid records in the xml file before the invalid record appears, but the exception is - thrown before the second record is returned. - */ - assertEquals(1, count); - } - } - @Test public void testChoiceForSimpleField() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people.xml"); List fields = getSimpleRecordFields2(); fields.add(new RecordField("AGE", RecordFieldType.CHOICE.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record record = reader.nextRecord(); @@ -596,7 +561,7 @@ public void testChoiceForRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); List fields = getSimpleRecordFields(); fields.add(new RecordField("ADDRESS", RecordFieldType.CHOICE.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record record = reader.nextRecord(); @@ -610,7 +575,7 @@ public void testChoiceForRecord() throws IOException, MalformedRecordException { @Test public void testNameSpaces() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_namespace.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); @@ -622,7 +587,7 @@ public void testNameSpaces() throws IOException, MalformedRecordException { @Test public void testCData() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_cdata.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues()); @@ -637,7 +602,7 @@ public void testRecordExpectedSimpleFieldFoundAndNoContentFieldConfigured() thro List fields = getSimpleRecordFields2(); final DataType recordType = RecordFieldType.RECORD.getRecordDataType(getNestedSchema()); fields.add(new RecordField("AGE", recordType)); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {"Cleve Butler", "USA", null}, reader.nextRecord().getValues()); @@ -651,7 +616,7 @@ public void testSimpleFieldExpectedButRecordFound() throws IOException, Malforme InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); List fields = getSimpleRecordFields(); fields.add(new RecordField("ADDRESS", RecordFieldType.STRING.getDataType())); - XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); assertNull(reader.nextRecord().getValue("ADDRESS")); @@ -663,7 +628,7 @@ public void testSimpleFieldExpectedButRecordFound() throws IOException, Malforme @Test public void testParseEmptyFields() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_empty.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {null, null, null}, reader.nextRecord().getValues()); @@ -673,7 +638,7 @@ public void testParseEmptyFields() throws IOException, MalformedRecordException @Test public void testParseEmptyFieldsCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_empty.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertArrayEquals(new Object[] {null, null, null}, reader.nextRecord(false, false).getValues()); @@ -683,23 +648,21 @@ public void testParseEmptyFieldsCoerceFalseDropFalse() throws IOException, Malfo @Test(expected = MalformedRecordException.class) public void testEmptyStreamAsSingleRecord() throws IOException, MalformedRecordException { InputStream is = new ByteArrayInputStream(new byte[0]); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), false, null, - null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), false, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); reader.nextRecord(true, true); } @Test(expected = MalformedRecordException.class) public void testEmptyStreamAsArray() throws IOException, MalformedRecordException { InputStream is = new ByteArrayInputStream(new byte[0]); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, - null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); reader.nextRecord(true, true); } @Test(expected = MalformedRecordException.class) public void testEmptyStreamWIthXmlHeader() throws IOException, MalformedRecordException { InputStream is = new ByteArrayInputStream(("").getBytes()); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record record = reader.nextRecord(false, false); assertNull(record); @@ -708,53 +671,17 @@ public void testEmptyStreamWIthXmlHeader() throws IOException, MalformedRecordEx @Test public void testParseEmptyArray() throws IOException, MalformedRecordException { InputStream is = new ByteArrayInputStream("".getBytes()); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, + XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Assert.assertNull(reader.nextRecord()); } - @Test - public void testRecordNameWithMismatch() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "PERSON", - null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); - int count = 0; - while (reader.nextRecord() != null) { - count++; - } - assertEquals(2, count); - } - - @Test - public void testRecordNameOnlyMismatches() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, "OTHER", - null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); - int count = 0; - while (reader.nextRecord() != null) { - count++; - } - assertEquals(0, count); - } - - @Test - public void testNoRecordName() throws IOException, MalformedRecordException { - InputStream is = new FileInputStream("src/test/resources/xml/people3.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, - null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); - int count = 0; - while (reader.nextRecord() != null) { - count++; - } - assertEquals(3, count); - } - @Test public void testNestedRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); RecordSchema schema = getSchemaWithNestedRecord(); - XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Object[] valuesFirstRecord = reader.nextRecord().getValues(); @@ -778,7 +705,7 @@ public void testNestedRecord() throws IOException, MalformedRecordException { public void testNestedRecordCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml"); RecordSchema schema = getSchemaWithNestedRecord(); - XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); @@ -824,7 +751,7 @@ public void testNestedRecordFieldsToIgnoreCoerceTrueDropTrue() throws IOExceptio // Fields "AGE" and "ADDRESS/CITY" are not defined here RecordSchema schema = getSchemaWithNestedRecord2(); - XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record firstRecord = reader.nextRecord(true, true); @@ -866,7 +793,7 @@ public void testNestedRecordFieldsToIgnoreCoerceFalseDropTrue() throws IOExcepti // Fields "AGE" and "ADDRESS/CITY" are not defined here RecordSchema schema = getSchemaWithNestedRecord2(); - XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record firstRecord = reader.nextRecord(false, true); @@ -908,7 +835,7 @@ public void testNestedRecordFieldsToIgnoreCoerceTrueDropFalse() throws IOExcepti // Fields "AGE" and "ADDRESS/CITY" are not defined here RecordSchema schema = getSchemaWithNestedRecord2(); - XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record firstRecord = reader.nextRecord(true, false); @@ -958,7 +885,7 @@ public void testNestedRecordFieldsToIgnoreCoerceFalseDropFalse() throws IOExcept // Fields "AGE" and "ADDRESS/CITY" are not defined here RecordSchema schema = getSchemaWithNestedRecord2(); - XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record firstRecord = reader.nextRecord(false, false); @@ -1007,7 +934,7 @@ public void testNestedRecordFieldsToIgnoreCoerceFalseDropFalse() throws IOExcept public void testSimpleArray() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_array_simple.xml"); RecordSchema schema = getSchemaWithSimpleArray(); - XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record firstRecord = reader.nextRecord(); @@ -1043,7 +970,7 @@ public void testSimpleArray() throws IOException, MalformedRecordException { public void testSimpleArrayCoerceFalseDropFalse() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_array_simple.xml"); RecordSchema schema = getSchemaWithSimpleArray(); - XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, schema, true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); @@ -1078,7 +1005,7 @@ public void testSimpleArrayCoerceFalseDropFalse() throws IOException, MalformedR public void testNestedArrayInNestedRecord() throws IOException, MalformedRecordException { InputStream is = new FileInputStream("src/test/resources/xml/people_array.xml"); RecordSchema schema = getSchemaWithNestedArray(); - XMLRecordReader reader = new XMLRecordReader(is, schema, true, "PERSON", null, + XMLRecordReader reader = new XMLRecordReader(is, schema, true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record firstRecord = reader.nextRecord(); @@ -1116,7 +1043,7 @@ public void testNestedArrayInNestedRecord() throws IOException, MalformedRecordE public void testDeeplyNestedArraysAndRecords() throws IOException, MalformedRecordException { // test records in nested arrays InputStream is = new FileInputStream("src/test/resources/xml/people_complex1.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(true, true); @@ -1155,7 +1082,7 @@ public void testDeeplyNestedArraysAndRecords() throws IOException, MalformedReco public void testDeeplyNestedArraysAndRecords2() throws IOException, MalformedRecordException { // test multiply nested arrays and records (recursion) InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(); @@ -1211,7 +1138,7 @@ public void testDeeplyNestedArraysAndRecords2() throws IOException, MalformedRec public void testDeeplyNestedArraysAndRecordsCoerceFalseDropTrue() throws IOException, MalformedRecordException { // test multiply nested arrays and records (recursion) InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml"); - XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), true, "PERSON", + XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, true); @@ -1268,7 +1195,7 @@ public void testDeeplyNestedArraysAndRecordsCoerceFalseDropFalse() throws IOExce // test multiply nested arrays and records (recursion) InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml"); XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(Collections.emptyList()), - true, "PERSON", null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); + true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)); Record first = reader.nextRecord(false, false); assertEquals("1", first.getValue("ID")); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people3.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people3.xml index afff2d4be761..fc85045a6c19 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people3.xml +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people3.xml @@ -1,17 +1,12 @@ - - - Cleve Butler - 42 + + + Cleve Butler + 42 USA - - Ainslie Fletcher - 33 + + Ainslie Fletcher + 33 UK - - Ainslie Fletcher - 33 - UK - \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people4.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people4.xml deleted file mode 100755 index fc85045a6c19..000000000000 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/people4.xml +++ /dev/null @@ -1,12 +0,0 @@ - - - Cleve Butler - 42 - USA - - - Ainslie Fletcher - 33 - UK - - \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/testschema2 b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/testschema2 new file mode 100755 index 000000000000..416cf0719ba5 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/xml/testschema2 @@ -0,0 +1,19 @@ +{ + "namespace": "nifi", + "name": "test", + "type": "record", + "fields": [ + { "name": "ID", "type": "string" }, + { "name": "NAME", "type": { + "type": "record", + "name": "nested", + "fields": [ + { "name": "ATTR", "type": "string" }, + { "name": "INNER", "type": "string" }, + { "name": "CONTENT", "type": "string" } + ] + } + }, + { "name": "AGE", "type": "int" } + ] +} From d8d1a0eadccc2444a46e32568fa39b7380c641b0 Mon Sep 17 00:00:00 2001 From: JohannesDaniel Date: Sun, 22 Apr 2018 16:25:00 +0200 Subject: [PATCH 6/6] Correction in additionalDetails --- .../docs/org.apache.nifi.xml.XMLReader/additionalDetails.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/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 b/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 index 035d4abaeb63..46cd25978b2b 100755 --- a/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 +++ b/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 @@ -24,8 +24,8 @@

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. Depending on how the property "Record Format" is set, - the reader either expects a single record or an array of records for each FlowFile. + that are not defined in the schema will be skipped. Depending on whether the property "Expect Records as Array" + is set to "false" or "true", the reader either expects a single record or an array of records for each FlowFile.

Example: Single record