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

2018-04-23 Thread asfgit
Github user asfgit closed the pull request at:

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


---


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

2018-04-23 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r183489279
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -84,6 +84,10 @@ public XMLRecordReader(InputStream in, RecordSchema 
schema, String rootName, Str
 
 try {
 final XMLInputFactory xmlInputFactory = 
XMLInputFactory.newInstance();
+
+// Avoid namespace replacements
+
xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
--- End diff --

@tballison Thank you for the advice! I refactored this in a way that only 
the local part is considered. 
:)


---


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

2018-04-23 Thread tballison
Github user tballison commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r183390424
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -84,6 +84,10 @@ public XMLRecordReader(InputStream in, RecordSchema 
schema, String rootName, Str
 
 try {
 final XMLInputFactory xmlInputFactory = 
XMLInputFactory.newInstance();
+
+// Avoid namespace replacements
+
xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
--- End diff --

Might want to avoid XEE vulnerability via improved configuration of 
XMLInputFactory?

https://www.owasp.org/index.php/XML_External_Entity_(XXE)_Prevention_Cheat_Sheet#XMLInputFactory_.28a_StAX_parser.29


---


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

2018-04-20 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r183152900
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
--- End diff --

Yes, exactly. I would use a property to convey this. I would be okay with 
allowing Expression Language to be used, or just allowing for a 'true'/'false' 
without Expression Language (I think in most cases, you'll want one or the 
other, not dependent upon each individual FlowFile). But if you think EL is 
important then I won't argue that point :)
One other option, which we do in a few different processors, would be to 
offer a third option that looks at a well-known attribute. So you could choose 
'true' (treat outer element as a wrapper), 'false' (treat each flowfile as a 
single record), or 'use xml.stream attribute', and when that is selected, the 
'xml.stream' attribute would be looked at to determine how to handle it - a 
value of 'true' would mean it's a stream of multiple records, 'false' would 
mean it's only 1 record, missing or any other value would throw an Exception. I 
don't have  strong preference one way or another how this should be handled, 
but wanted to present options that we typically use.


---


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

2018-04-16 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r181851731
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-16 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r181653767
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-12 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r181221789
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
+"XML data, embedded in an enclosing root tag.")
+public class XMLReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+public static final PropertyDescriptor VALIDATE_ROOT_TAG = new 
PropertyDescriptor.Builder()
+.name("validate_root_tag")
+.displayName("Validate Root Tag")
+.description("If this property is set, the name of root tags 
(e. g. ...) of incoming FlowFiles will be 
evaluated against this value. " +
+"In the case of a mismatch, an exception is thrown. 
The treatment of such FlowFiles depends on the implementation " +
+"of respective Processors.")
+.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.expressionLanguageSupported(true)
+.required(false)
+.build();
+
+public static final PropertyDescriptor VALIDATE_RECORD_TAG = new 
PropertyDescriptor.Builder()
--- End diff --

(non-record shall be skipped)


---


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

2018-04-12 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r181218609
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-12 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r181218182
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-12 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r181217494
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-12 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r181215801
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// Avoid namespace replacements
+
xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
--- End diff --

ok, I will activate namespaces and implement some tests for this.


---


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

2018-04-12 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r181215337
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
--- End diff --

when I started implementing this reader, I was wondering, how the reader 
knows whether to parse wrapped records or a single record. unfortunately we 
dont have an unambiguous indicator like we have for json: [ vs. { 
I considered to make it configurable with EL whether the reader shall 
expect a single record or an array of records. what do you think?


---


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

2018-04-12 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r181213473
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
+"XML data, embedded in an enclosing root tag.")
+public class XMLReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+public static final PropertyDescriptor VALIDATE_ROOT_TAG = new 
PropertyDescriptor.Builder()
+.name("validate_root_tag")
+.displayName("Validate Root Tag")
+.description("If this property is set, the name of root tags 
(e. g. ...) of incoming FlowFiles will be 
evaluated against this value. " +
+"In the case of a mismatch, an exception is thrown. 
The treatment of such FlowFiles depends on the implementation " +
+"of respective Processors.")
+.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.expressionLanguageSupported(true)
+.required(false)
+.build();
+
+public static final PropertyDescriptor VALIDATE_RECORD_TAG = new 
PropertyDescriptor.Builder()
+.name("validate_record_tag")
+.displayName("Validate Record Tag")
+.description("If this property is set, the name of record tags 
(e. g. ...) of incoming FlowFiles will be 
evaluated against this value. " +
+"In the case of a mismatch, the respective record will 
be skipped. If this property is not set, each level 2 starting tag will be 
treated " +
+"as the beginning of a record.")
+.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.expressionLanguageSupported(true)
+.required(false)
+.build();
+
+public static final PropertyDescriptor ATTRIBUTE_PREFIX = new 
PropertyDescriptor.Builder()
+.name("attribute_prefix")
+.displayName("Attribute Prefix")
+.description("If this property is set, the name of attributes 
will be appended by a prefix when they are added to a record.")
--- End diff --

ok


---


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

2018-04-12 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r181213403
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
+"XML data, embedded in an enclosing root tag.")
+public class XMLReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+public static final PropertyDescriptor VALIDATE_ROOT_TAG = new 
PropertyDescriptor.Builder()
+.name("validate_root_tag")
+.displayName("Validate Root Tag")
+.description("If this property is set, the name of root tags 
(e. g. ...) of incoming FlowFiles will be 
evaluated against this value. " +
+"In the case of a mismatch, an exception is thrown. 
The treatment of such FlowFiles depends on the implementation " +
+"of respective Processors.")
+.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.expressionLanguageSupported(true)
+.required(false)
+.build();
+
+public static final PropertyDescriptor VALIDATE_RECORD_TAG = new 
PropertyDescriptor.Builder()
--- End diff --

my original intention actually was to enable users to parse recordsets like 
this
```

  ...
  ...
  ...


---


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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179833060
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179834908
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179834679
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179822292
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
--- End diff --

Record Readers don't need to be thread-safe, only the RecordReaderFactory 
does.


---


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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179827156
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179830412
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179831387
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179830847
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179833900
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179831931
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179835305
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179824910
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// Avoid namespace replacements
+
xmlInputFactory.setProperty(XMLInputFactory.IS_NAMESPACE_AWARE, false);
--- End diff --

I think this approach may lead to some odd behaviors if the incoming XML is 
actually namespace aware. For example, if an element looks like:
```

```
Then the name of the element in the schema would have to be "ns:myElement" 
in order to match. But I think that's not what we want. Instead, in the schema, 
I would want to use the name 

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179820952
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
--- End diff --

I think the requirement that XML data must be wrapped in some sort of 
wrapper is going to be problematic. While this will be a fairly common case, so 
that multiple XML elements can be combined into a single FlowFile, it is also 
going to be common (probably more common) that each XML element will be its own 
standalone Record. This is especially important if this Reader is used for 
something like ListenTCPRecord or ConsumeKafkaRecord, where the data is 
received from elsewhere so no processor has a chance to wrap the content prior 
to using the XML Reader. I think we need to support both ignoring the 
outer-most element as well as incorporating the outer-most element.


---


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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179829864
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179827839
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179829981
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179840389
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
--- End diff --

More specifically, I think that if the following were the content of a 
FlowFile:
```

  John Doe
  123
  01/01/2017

```
Then I would expect to have this parse as a single Record that would match 
this schema:
```
{
  "name": "person", "namespace": "nifi",
  "type": "record",
  "fields": [
{ "name": "name", "type": "string" },
{ "name": "id", "type": "int" },
{ "name": "dob", "type": "date" }
  ]
}
```
Additionally, I would expect to be able to set a property that indicates 
that the outer-most XML element is simply a wrapper. If that property were set 
to "true", then I would expect to use that exact same schema to parse the 
following XML:
```

  
John Doe
123
01/01/2017
  
  
Jane Doe
124
01/01/2016
  
  
Jake Doe
125
01/01/2015
  

```
In this case, the 'people' element is just a wrapper and could just as 
easily be an element named 'root' or 'foo' or 'bar'.


---


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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179829341
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179820052
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
+"XML data, embedded in an enclosing root tag.")
+public class XMLReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+public static final PropertyDescriptor VALIDATE_ROOT_TAG = new 
PropertyDescriptor.Builder()
+.name("validate_root_tag")
+.displayName("Validate Root Tag")
+.description("If this property is set, the name of root tags 
(e. g. ...) of incoming FlowFiles will be 
evaluated against this value. " +
+"In the case of a mismatch, an exception is thrown. 
The treatment of such FlowFiles depends on the implementation " +
+"of respective Processors.")
+.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.expressionLanguageSupported(true)
+.required(false)
+.build();
+
+public static final PropertyDescriptor VALIDATE_RECORD_TAG = new 
PropertyDescriptor.Builder()
+.name("validate_record_tag")
+.displayName("Validate Record Tag")
+.description("If this property is set, the name of record tags 
(e. g. ...) of incoming FlowFiles will be 
evaluated against this value. " +
+"In the case of a mismatch, the respective record will 
be skipped. If this property is not set, each level 2 starting tag will be 
treated " +
+"as the beginning of a record.")
+.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.expressionLanguageSupported(true)
+.required(false)
+.build();
+
+public static final PropertyDescriptor ATTRIBUTE_PREFIX = new 
PropertyDescriptor.Builder()
+.name("attribute_prefix")
+.displayName("Attribute Prefix")
+.description("If this property is set, the name of attributes 
will be appended by a prefix when they are added to a record.")
--- End diff --

I think this is supposed to say "prepended with a prefix"


---


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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179829486
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179838657
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
 ---
@@ -0,0 +1,502 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+
+public class XMLRecordReader implements RecordReader {
+
+private final ComponentLog logger;
+private final RecordSchema schema;
+private final String recordName;
+private final String attributePrefix;
+private final String contentFieldName;
+
+// thread safety required?
+private StartElement currentRecordStartTag;
+
+private final XMLEventReader xmlEventReader;
+
+private final Supplier 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();
+
+// 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();
+

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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179819209
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
+"XML data, embedded in an enclosing root tag.")
+public class XMLReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+public static final PropertyDescriptor VALIDATE_ROOT_TAG = new 
PropertyDescriptor.Builder()
--- End diff --

I am actually in favor of removing this property all together. In order to 
properly read the records, the Record Readers will need to validate syntax of 
the data, but I don't believe that it should be validating arbitrary semantic 
meanings. I.e., I don't think that we should be checking the name of the 
outer-most element for any specific name. 


---


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

2018-04-06 Thread markap14
Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r179819928
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
+"XML data, embedded in an enclosing root tag.")
+public class XMLReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+public static final PropertyDescriptor VALIDATE_ROOT_TAG = new 
PropertyDescriptor.Builder()
+.name("validate_root_tag")
+.displayName("Validate Root Tag")
+.description("If this property is set, the name of root tags 
(e. g. ...) of incoming FlowFiles will be 
evaluated against this value. " +
+"In the case of a mismatch, an exception is thrown. 
The treatment of such FlowFiles depends on the implementation " +
+"of respective Processors.")
+.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.expressionLanguageSupported(true)
+.required(false)
+.build();
+
+public static final PropertyDescriptor VALIDATE_RECORD_TAG = new 
PropertyDescriptor.Builder()
--- End diff --

Likewise, I think we should remove this property and this sort of 
validation as well. If the user wants to validate some specific XML element 
names, the ValidateRecord processor is a great solution for that, and provides 
far more flexible validation via schema.


---


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

2018-04-01 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r178470648
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html
 ---
@@ -0,0 +1,378 @@
+
+
+
+
+
+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
+field1content/field1
+field2content/field2
+  /record
+  record
+field1content/field1
+field2content/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_fieldcontent/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_fieldcontent/array_field
+  array_fieldcontent/array_field
+  simple_fieldcontent/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:
+
+

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

2018-04-01 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r178470638
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
+"XML data, embedded in an enclosing root tag.")
+public class XMLReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+public static final PropertyDescriptor VALIDATE_ROOT_TAG = new 
PropertyDescriptor.Builder()
--- End diff --

done. additionally, I added some tests for class XMLReader


---


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

2018-04-01 Thread JohannesDaniel
Github user JohannesDaniel commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r178470625
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html
 ---
@@ -0,0 +1,378 @@
+
+
+
+
+
+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
+field1content/field1
+field2content/field2
+  /record
+  record
+field1content/field1
+field2content/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_fieldcontent/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_fieldcontent/array_field
+  array_fieldcontent/array_field
+  simple_fieldcontent/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:
+
+

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

2018-03-31 Thread pvillard31
Github user pvillard31 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r178437093
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html
 ---
@@ -0,0 +1,378 @@
+
+
+
+
+
+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
+field1content/field1
+field2content/field2
+  /record
+  record
+field1content/field1
+field2content/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_fieldcontent/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_fieldcontent/array_field
+  array_fieldcontent/array_field
+  simple_fieldcontent/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:
+
+
+   

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

2018-03-31 Thread pvillard31
Github user pvillard31 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r178437056
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html
 ---
@@ -0,0 +1,378 @@
+
+
+
+
+
+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
+field1content/field1
+field2content/field2
+  /record
+  record
+field1content/field1
+field2content/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_fieldcontent/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_fieldcontent/array_field
+  array_fieldcontent/array_field
+  simple_fieldcontent/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:
+
+
+   

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

2018-03-31 Thread pvillard31
Github user pvillard31 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r178437593
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
+"XML data, embedded in an enclosing root tag.")
+public class XMLReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+public static final PropertyDescriptor VALIDATE_ROOT_TAG = new 
PropertyDescriptor.Builder()
--- End diff --

Could this property supports expression language against incoming flow 
files? I don't think that's an easy change (and it could introduce a perf hit) 
but that would allow using the same reader for completely different XML inputs.


---


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

2018-03-31 Thread pvillard31
Github user pvillard31 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2587#discussion_r178437600
  
--- Diff: 
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
 ---
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.xml;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.DateTimeUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"xml", "record", "reader", "parser"})
+@CapabilityDescription("Reads XML content and creates Record objects. 
Records are expected in the second level of " +
+"XML data, embedded in an enclosing root tag.")
+public class XMLReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+public static final PropertyDescriptor VALIDATE_ROOT_TAG = new 
PropertyDescriptor.Builder()
+.name("validate_root_tag")
+.displayName("Validate Root Tag")
+.description("If this property is set, the name of root tags 
(e. g. ...) of incoming FlowFiles will be 
evaluated against this value. " +
+"In the case of a mismatch, an exception is thrown. 
The treatment of such FlowFiles depends on the implementation " +
+"of respective Processors.")
+.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.required(false)
+.build();
+
+public static final PropertyDescriptor VALIDATE_RECORD_TAG = new 
PropertyDescriptor.Builder()
--- End diff --

Same here (and for the next properties).


---


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

2018-03-27 Thread JohannesDaniel
GitHub user JohannesDaniel opened a pull request:

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

NIFI-4185 Add XML Record Reader

Thank you for submitting a contribution to Apache NiFi.

In order to streamline the review of the contribution we ask you
to ensure the following steps have been taken:

### For all changes:
- [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
 in the commit message?

- [ ] Does your PR title start with NIFI- where  is the JIRA number 
you are trying to resolve? Pay particular attention to the hyphen "-" character.

- [ ] Has your PR been rebased against the latest commit within the target 
branch (typically master)?

- [ ] Is your initial contribution a single, squashed commit?

### For code changes:
- [ ] Have you ensured that the full suite of tests is executed via mvn 
-Pcontrib-check clean install at the root nifi folder?
- [ ] Have you written or updated unit tests to verify your changes?
- [ ] If adding new dependencies to the code, are these dependencies 
licensed in a way that is compatible for inclusion under [ASF 
2.0](http://www.apache.org/legal/resolved.html#category-a)? 
- [ ] If applicable, have you updated the LICENSE file, including the main 
LICENSE file under nifi-assembly?
- [ ] If applicable, have you updated the NOTICE file, including the main 
NOTICE file found under nifi-assembly?
- [ ] If adding new Properties, have you added .displayName in addition to 
.name (programmatic access) for each of the new properties?

### For documentation related changes:
- [ ] Have you ensured that format looks appropriate for the output in 
which it is rendered?

### Note:
Please ensure that once the PR is submitted, you check travis-ci for build 
issues and submit an update to your PR as soon as possible.


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

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

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

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

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

This closes #2587


commit 9b4bd0dd8f1d30bfe1597d4cd069df414eb968a0
Author: JohannesDaniel 
Date:   2018-03-06T23:02:43Z

Add XML Record Reader




---