You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2017/04/11 23:33:37 UTC

[14/19] nifi git commit: NIFI-1280 added support for RecordSchema in SchemaRegistry

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java
deleted file mode 100644
index de207f4..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * 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.serialization;
-
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.nifi.serialization.record.DataType;
-import org.apache.nifi.serialization.record.RecordField;
-import org.apache.nifi.serialization.record.RecordFieldType;
-import org.apache.nifi.serialization.record.RecordSchema;
-
-public class DataTypeUtils {
-
-    public static Double toDouble(final Object value, final Double defaultValue) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).doubleValue();
-        }
-
-        if (value instanceof String) {
-            return Double.parseDouble((String) value);
-        }
-
-        return defaultValue;
-    }
-
-    public static Float toFloat(final Object value, final Float defaultValue) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).floatValue();
-        }
-
-        if (value instanceof String) {
-            return Float.parseFloat((String) value);
-        }
-
-        return defaultValue;
-    }
-
-    public static Long toLong(final Object value, final Long defaultValue) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).longValue();
-        }
-
-        if (value instanceof String) {
-            return Long.parseLong((String) value);
-        }
-
-        return defaultValue;
-    }
-
-
-
-    public static Integer toInteger(final Object value, final Integer defaultValue) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).intValue();
-        }
-
-        if (value instanceof String) {
-            return Integer.parseInt((String) value);
-        }
-
-        return defaultValue;
-    }
-
-
-    /**
-     * Deduces the type of RecordFieldType that should be used for a value of the given type,
-     * or returns <code>null</code> if the value is null
-     *
-     * @param value the value whose type should be deduced
-     * @return the type of RecordFieldType that should be used for a value of the given type,
-     *         or <code>null</code> if the value is null
-     */
-    public static DataType inferDataType(final Object value) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof String) {
-            return RecordFieldType.STRING.getDataType();
-        }
-        if (value instanceof Long) {
-            return RecordFieldType.LONG.getDataType();
-        }
-        if (value instanceof Integer) {
-            return RecordFieldType.INT.getDataType();
-        }
-        if (value instanceof Double) {
-            return RecordFieldType.DOUBLE.getDataType();
-        }
-        if (value instanceof Float) {
-            return RecordFieldType.FLOAT.getDataType();
-        }
-        if (value instanceof Boolean) {
-            return RecordFieldType.BOOLEAN.getDataType();
-        }
-        if (value instanceof Byte) {
-            return RecordFieldType.BYTE.getDataType();
-        }
-        if (value instanceof Character) {
-            return RecordFieldType.CHAR.getDataType();
-        }
-        if (value instanceof Short) {
-            return RecordFieldType.SHORT.getDataType();
-        }
-        if (value instanceof Date) {
-            return RecordFieldType.DATE.getDataType();
-        }
-        if (value instanceof Object[] || value instanceof List) {
-            return RecordFieldType.ARRAY.getDataType();
-        }
-        if (value instanceof Map) {
-            @SuppressWarnings("unchecked")
-            final Map<String, Object> map = (Map<String, Object>) value;
-            final RecordSchema childSchema = determineSchema(map);
-            return RecordFieldType.RECORD.getDataType(childSchema);
-        }
-
-        return RecordFieldType.RECORD.getDataType();
-    }
-
-    public static RecordSchema determineSchema(final Map<String, Object> valueMap) {
-        final List<RecordField> fields = new ArrayList<>(valueMap.size());
-        for (final Map.Entry<String, Object> entry : valueMap.entrySet()) {
-            final DataType valueType = inferDataType(entry.getValue());
-            final String fieldName = entry.getKey();
-            final RecordField field = new RecordField(fieldName, valueType);
-            fields.add(field);
-        }
-        return new SimpleRecordSchema(fields);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java
new file mode 100644
index 0000000..5545090
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java
@@ -0,0 +1,57 @@
+/*
+ * 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.serialization;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+
+public abstract class DateTimeTextRecordSetWriter extends AbstractControllerService {
+
+    private volatile String dateFormat;
+    private volatile String timeFormat;
+    private volatile String timestampFormat;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(DateTimeUtils.DATE_FORMAT, DateTimeUtils.TIME_FORMAT, DateTimeUtils.TIMESTAMP_FORMAT);
+    }
+
+    @OnEnabled
+    public void captureValues(final ConfigurationContext context) {
+        this.dateFormat = context.getProperty(DateTimeUtils.DATE_FORMAT).getValue();
+        this.timeFormat = context.getProperty(DateTimeUtils.TIME_FORMAT).getValue();
+        this.timestampFormat = context.getProperty(DateTimeUtils.TIMESTAMP_FORMAT).getValue();
+    }
+
+    protected String getDateFormat() {
+        return dateFormat;
+    }
+
+    protected String getTimeFormat() {
+        return timeFormat;
+    }
+
+    protected String getTimestampFormat() {
+        return timestampFormat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java
new file mode 100644
index 0000000..d5ab8c5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java
@@ -0,0 +1,50 @@
+/*
+ * 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.serialization;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+public class DateTimeUtils {
+    public static final PropertyDescriptor DATE_FORMAT = new PropertyDescriptor.Builder()
+        .name("Date Format")
+        .description("Specifies the format to use when reading/writing Date fields")
+        .expressionLanguageSupported(false)
+        .defaultValue(RecordFieldType.DATE.getDefaultFormat())
+        .addValidator(new SimpleDateFormatValidator())
+        .required(true)
+        .build();
+
+    public static final PropertyDescriptor TIME_FORMAT = new PropertyDescriptor.Builder()
+        .name("Time Format")
+        .description("Specifies the format to use when reading/writing Time fields")
+        .expressionLanguageSupported(false)
+        .defaultValue(RecordFieldType.TIME.getDefaultFormat())
+        .addValidator(new SimpleDateFormatValidator())
+        .required(true)
+        .build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+        .name("Timestamp Format")
+        .description("Specifies the format to use when reading/writing Timestamp fields")
+        .expressionLanguageSupported(false)
+        .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+        .addValidator(new SimpleDateFormatValidator())
+        .required(true)
+        .build();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java
new file mode 100644
index 0000000..ee25e64
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java
@@ -0,0 +1,110 @@
+/*
+ * 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.serialization;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public abstract class SchemaRegistryRecordReader extends AbstractControllerService {
+
+    protected static final PropertyDescriptor REQUIRED_SCHEMA_REGISTRY = new PropertyDescriptor.Builder()
+        .name("Schema Registry")
+        .description("Specifies the Controller Service to use for the Schema Registry")
+        .identifiesControllerService(SchemaRegistry.class)
+        .required(true)
+        .build();
+
+    protected static final PropertyDescriptor OPTIONAL_SCHEMA_REGISTRY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(REQUIRED_SCHEMA_REGISTRY)
+        .required(false)
+        .build();
+
+    protected static final PropertyDescriptor REQUIRED_SCHEMA_NAME = new PropertyDescriptor.Builder()
+        .name("Schema Name")
+        .description("Name of the Schema that is stored in the Schema Registry")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .required(true)
+        .build();
+
+    protected static final PropertyDescriptor OPTIONAL_SCHEMA_NAME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(REQUIRED_SCHEMA_NAME)
+        .required(false)
+        .build();
+
+
+    private volatile SchemaRegistry schemaRegistry;
+    private volatile PropertyValue schemaName;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>(2);
+        if (isSchemaRequired()) {
+            properties.add(REQUIRED_SCHEMA_REGISTRY);
+            properties.add(REQUIRED_SCHEMA_NAME);
+        } else {
+            properties.add(OPTIONAL_SCHEMA_REGISTRY);
+            properties.add(OPTIONAL_SCHEMA_NAME);
+        }
+
+        return properties;
+    }
+
+    @OnEnabled
+    public void storeRegistryValues(final ConfigurationContext context) {
+        schemaRegistry = context.getProperty(REQUIRED_SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class);
+        schemaName = context.getProperty(REQUIRED_SCHEMA_NAME);
+    }
+
+    public RecordSchema getSchema(final FlowFile flowFile) {
+        final String evaluatedSchemaName = schemaName.evaluateAttributeExpressions(flowFile).getValue();
+        final RecordSchema schema = schemaRegistry.retrieveSchema(evaluatedSchemaName);
+        return schema;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        if (validationContext.getProperty(OPTIONAL_SCHEMA_REGISTRY).isSet() && !validationContext.getProperty(OPTIONAL_SCHEMA_NAME).isSet()) {
+            return Collections.singleton(new ValidationResult.Builder()
+                .subject("Schema Registry")
+                .explanation("If the Schema Registry is configured, the Schema name must also be configured")
+                .valid(false)
+                .build());
+        }
+
+        return Collections.emptyList();
+    }
+
+    protected boolean isSchemaRequired() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/UserTypeOverrideRowReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/UserTypeOverrideRowReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/UserTypeOverrideRowReader.java
deleted file mode 100644
index be0b8ad..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/UserTypeOverrideRowReader.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * 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.serialization;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.nifi.annotation.lifecycle.OnEnabled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.AbstractControllerService;
-import org.apache.nifi.controller.ConfigurationContext;
-import org.apache.nifi.serialization.DataTypeValidator;
-import org.apache.nifi.serialization.record.DataType;
-import org.apache.nifi.serialization.record.RecordFieldType;
-
-public abstract class UserTypeOverrideRowReader extends AbstractControllerService {
-    private volatile Map<String, DataType> fieldTypeOverrides;
-
-    @Override
-    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder()
-            .name(propertyDescriptorName)
-            .dynamic(true)
-            .addValidator(new DataTypeValidator())
-            .build();
-    }
-
-    @OnEnabled
-    public void createFieldTypeOverrides(final ConfigurationContext context) {
-        final Map<String, DataType> overrides = new HashMap<>(context.getProperties().size());
-        for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
-            if (!entry.getKey().isDynamic()) {
-                continue;
-            }
-
-            final String fieldName = entry.getKey().getName();
-            final String dataTypeName = entry.getValue();
-            if (dataTypeName == null) {
-                continue;
-            }
-
-            final DataType dataType;
-            final String[] splits = dataTypeName.split("\\:");
-            if (splits.length == 2) {
-                final RecordFieldType fieldType = RecordFieldType.of(splits[0]);
-                final String format = splits[1];
-                dataType = fieldType.getDataType(format);
-            } else {
-                final RecordFieldType fieldType = RecordFieldType.of(dataTypeName);
-                dataType = fieldType.getDataType();
-            }
-
-            overrides.put(fieldName, dataType);
-        }
-
-        this.fieldTypeOverrides = Collections.unmodifiableMap(overrides);
-    }
-
-    protected Map<String, DataType> getFieldTypeOverrides() {
-        return fieldTypeOverrides;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java
index 07da00e..598a8c4 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java
@@ -33,10 +33,10 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
 
-@Tags({"text", "freeform", "expression", "language", "el", "resultset", "writer", "serialize"})
-@CapabilityDescription("Writes the contents of a Database ResultSet as free-form text. The configured "
+@Tags({"text", "freeform", "expression", "language", "el", "record", "recordset", "resultset", "writer", "serialize"})
+@CapabilityDescription("Writes the contents of a RecordSet as free-form text. The configured "
     + "text is able to make use of the Expression Language to reference each of the columns that are available "
-    + "in the ResultSet. Each record in the ResultSet will be separated by a single newline character.")
+    + "in a Record. Each record in the RecordSet will be separated by a single newline character.")
 public class FreeFormTextRecordSetWriter extends AbstractControllerService implements RecordSetWriterFactory {
     static final PropertyDescriptor TEXT = new PropertyDescriptor.Builder()
         .name("Text")

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.csv.CSVReader/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.csv.CSVReader/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.csv.CSVReader/additionalDetails.html
index e6dfd0c..fe748a7 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.csv.CSVReader/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.csv.CSVReader/additionalDetails.html
@@ -17,83 +17,103 @@
     <head>
         <meta charset="utf-8"/>
         <title>CSVReader</title>
-        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
+        <link rel="stylesheet" href="/nifi-docs/css/component-usage.css" type="text/css"/>
     </head>
 
     <body>
         <p>
         	The CSVReader Controller Service, expects input in such a way that the first line of a FlowFile specifies the name of
         	each column in the data. Following the first line, the rest of the FlowFile is expected to be valid CSV data from which
-        	to form appropriate Records. By default, the schema for a FlowFile is inferred by extracting the name of each column from
-        	the first line of the CSV and assumes that all columns are of type <code>string</code>. Of course, we may want to treat some
-        	columns as a data type other than <code>string</code>. This can be accomplished by adding a user-defined property where the
-        	name of the property is the same as the name of a CSV column and the value of the property is the data type to use.
+        	to form appropriate Records. The reader allows for customization of the CSV Format, such as which character should be used
+        	to separate CSV fields, which character should be used for quoting and when to quote fields, which character should denote
+        	a comment, etc.
         </p>
-        
-        <p>
-        	When specifying a data type for a field, the following values are valid:
-        </p>
-        
-        <ul>
-        	<li><b>string</b></li>
-        	<li><b>boolean</b></li>
-        	<li><b>byte</b></li>
-        	<li><b>char</b></li>
-        	<li><b>short</b></li>
-        	<li><b>int</b></li>
-        	<li><b>bigint</b></li>
-        	<li><b>long</b></li>
-        	<li><b>float</b></li>
-        	<li><b>double</b></li>
-        	<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
-        		by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).
-        	</li>
-        	<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
-        		by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).
-        	</li>
-        	<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
-        		<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
-        		<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).</li>
-        	<li><b>object</b> - <i>This data type does not apply to CSV data.</i></li>
-        	<li><b>array</b> - <i>This data type does not apply to CSV data.</i></li>
-        </ul>
-        
+
+
+		<h2>Schemas and Type Coercion</h2>
+		
+		<p>
+			When a record is parsed from incoming data, it is separated into fields. Each of these fields is then looked up against the
+			configured schema (by field name) in order to determine what the type of the data should be. If the field is not present in
+			the schema, that field is omitted from the Record. If the field is found in the schema, the data type of the received data
+			is compared against the data type specified in the schema. If the types match, the value of that field is used as-is. If the
+			schema indicates that the field should be of a different type, then the Controller Service will attempt to coerce the data
+			into the type specified by the schema. If the field cannot be coerced into the specified type, an Exception will be thrown.
+		</p>
+		
+		<p>
+			The following rules apply when attempting to coerce a field value from one data type to another:
+		</p>
+			
+		<ul>
+			<li>Any data type can be coerced into a String type.</li>
+			<li>Any numeric data type (Byte, Short, Int, Long, Float, Double) can be coerced into any other numeric data type.</li>
+			<li>Any numeric value can be coerced into a Date, Time, or Timestamp type, by assuming that the Long value is the number of
+			milliseconds since epoch (Midnight GMT, January 1, 1970).</li>
+			<li>A String value can be coerced into a Date, Time, or Timestamp type, if its format matches the configured "Date Format," "Time Format,"
+				or "Timestamp Format."</li>
+			<li>A String value can be coerced into a numeric value if the value is of the appropriate type. For example, the String value
+				<code>8</code> can be coerced into any numeric type. However, the String value <code>8.2</code> can be coerced into a Double or Float
+				type but not an Integer.</li>
+			<li>A String value of "true" or "false" (regardless of case) can be coerced into a Boolean value.</li>
+			<li>A String value that is not empty can be coerced into a Char type. If the String contains more than 1 character, the first character is used
+				and the rest of the characters are ignored.</li>
+			<li>Any "date/time" type (Date, Time, Timestamp) can be coerced into any other "date/time" type.</li>
+			<li>Any "date/time" type can be coerced into a Long type, representing the number of milliseconds since epoch (Midnight GMT, January 1, 1970).</li>
+			<li>Any "date/time" type can be coerced into a String. The format of the String is whatever DateFormat is configured for the corresponding
+				property (Date Format, Time Format, Timestamp Format property).</li>
+		</ul>
+		
+		<p>
+			If none of the above rules apply when attempting to coerce a value from one data type to another, the coercion will fail and an Exception
+			will be thrown.
+		</p>
+			
+			
+
+		<h2>Examples</h2>
+		
         <p>
         	As an example, consider a FlowFile whose contents consists of the following:
         </p>
-        
+
         <code>
-        	id, name, balance, notes<br />
-        	1, John, 48.23, "Our very<br />
+        	id, name, balance, join_date, notes<br />
+        	1, John, 48.23, 04/03/2007 "Our very<br />
 first customer!"<br />
-        	2, Jane, 1245.89,<br />
-        	3, Frank Franklin, "48481.29",<br />
+        	2, Jane, 1245.89, 08/22/2009,<br />
+        	3, Frank Franklin, "48481.29", 04/04/2016,<br />
         </code>
         
         <p>
-        	Additionally, let's consider that this Controller Service is configured with the following user-defined properties:
+        	Additionally, let's consider that this Controller Service is configured with the Schema Registry pointing to an AvroSchemaRegistry and the schema is
+        	configured as the following:
         </p>
         
-		<table>
-    		<head>
-    			<th>Property Name</th>
-    			<th>Property Value</th>
-    		</head>
-    		<body>
-    			<tr>
-    				<td>balance</td>
-    				<td><code>float</code></td>
-    			</tr>
-    		</body>
-    	</table>
-    	
+		<code>
+		<pre>
+		{
+		  "namespace": "nifi",
+		  "name": "balances",
+		  "type": "record",
+		  "fields": [
+		  		{ "name": "id", "type": "int" },
+		  		{ "name": "name": "type": "string" },
+		  		{ "name": "balance": "type": "double" },
+		  		{ "name": "join_date", "type": {
+		  			"type": "int",
+		  			"logicalType": "date"
+		  		},
+		  		{ "name": "notes": "type": "string" }
+		  ]
+		}
+		</pre>
+		</code>
+
     	<p>
+    		In the example above, we see that the 'join_date' column is a Date type. In order for the CSV Reader to be able to properly parse a value as a date,
+    		we need to provide the reader with the date format to use. In this example, we would configure the Date Format property to be <code>MM/dd/yyyy</code>
+    		to indicate that it is a two-digit month, followed by a two-digit day, followed by a four-digit year - each separated by a slash.
     		In this case, the result will be that this FlowFile consists of 3 different records. The first record will contain the following values:
     	</p>
 
@@ -116,6 +136,10 @@ first customer!"<br />
     				<td>48.23</td>
     			</tr>
     			<tr>
+    				<td>join_date</td>
+    				<td>04/03/2007</td>
+    			</tr>
+    			<tr>
     				<td>notes</td>
     				<td>Our very<br />first customer!</td>
     			</tr>
@@ -145,6 +169,10 @@ first customer!"<br />
     				<td>1245.89</td>
     			</tr>
     			<tr>
+    				<td>join_date</td>
+    				<td>08/22/2009</td>
+    			</tr>
+    			<tr>
     				<td>notes</td>
     				<td></td>
     			</tr>
@@ -174,6 +202,10 @@ first customer!"<br />
     				<td>48481.29</td>
     			</tr>
     			<tr>
+    				<td>join_date</td>
+    				<td>04/04/2016</td>
+    			</tr>
+    			<tr>
     				<td>notes</td>
     				<td></td>
     			</tr>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.grok.GrokReader/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.grok.GrokReader/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.grok.GrokReader/additionalDetails.html
index 3a41f47..97423ec 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.grok.GrokReader/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.grok.GrokReader/additionalDetails.html
@@ -17,18 +17,23 @@
     <head>
         <meta charset="utf-8"/>
         <title>GrokReader</title>
-        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
+        <link rel="stylesheet" href="/nifi-docs/css/component-usage.css" type="text/css"/>
     </head>
 
     <body>
         <p>
-        	The GrokReader Controller Service, provides a means for parsing and structuring input that is
+        	The GrokReader Controller Service provides a means for parsing and structuring input that is
         	made up of unstructured text, such as log files. Grok allows users to add a naming construct to
         	Regular Expressions such that they can be composed in order to create expressions that are easier
-        	to manage and work with. This Controller Service consists of one Required Property and one Optional
-        	Property. The Optional Property is named <code>Grok Pattern File</code> and specifies the filename of
+        	to manage and work with. This Controller Service consists of one Required Property and a few Optional
+        	Properties. The is named <code>Grok Pattern File</code> property specifies the filename of
         	a file that contains Grok Patterns that can be used for parsing log data. If not specified, a default
-        	patterns file will be used. Its contains are provided below.
+        	patterns file will be used. Its contents are provided below. There are also properties for specifying
+        	the schema to use when parsing data. The schema is not required. However, when data is parsed
+        	a Record is created that contains all of the fields present in the Grok Expression (explained below),
+        	and all fields are of type String. If a schema is chosen, the field can be declared to be a different,
+        	compatible type, such as number. Additionally, if the schema does not contain one of the fields in the
+        	parsed data, that field will be ignored. This can be used to filter out fields that are not of interest.
 		</p>
 		
 		<p>
@@ -48,45 +53,49 @@
         	to a field named <code>STACK_TRACE</code>. Otherwise, the line is appended to the last field defined in the Grok Expression. This
         	is done because typically the last field is a 'message' type of field, which can consist of new-lines.
         </p>
-        
-        <p>
-        	By default, all fields that are extracted are considered to be of type <code>string</code>. This can be overridden
-        	by adding a user-defined property where the name of the property matches the name of the field that is present in the
-        	configured Grok Expression. The value of the user-defined property is the data type to use.
-        	When specifying a data type for a field, the following values are valid:
-        </p>
-        
-        <ul>
-        	<li><b>string</b></li>
-        	<li><b>boolean</b></li>
-        	<li><b>byte</b></li>
-        	<li><b>char</b></li>
-        	<li><b>short</b></li>
-        	<li><b>int</b></li>
-        	<li><b>bigint</b></li>
-        	<li><b>long</b></li>
-        	<li><b>float</b></li>
-        	<li><b>double</b></li>
-        	<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
-        		by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).
-        	</li>
-        	<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
-        		by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).
-        	</li>
-        	<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
-        		<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
-        		<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).</li>
-        	<li><b>object</b> - <i>This data type does not apply to CSV data.</i></li>
-        	<li><b>array</b> - <i>This data type does not apply to CSV data.</i></li>
-        </ul>
-        
-        
+
+
+		<h2>Schemas and Type Coercion</h2>
+		
+		<p>
+			When a record is parsed from incoming data, it is separated into fields. Each of these fields is then looked up against the
+			configured schema (by field name) in order to determine what the type of the data should be. If the field is not present in
+			the schema, that field is omitted from the Record. If the field is found in the schema, the data type of the received data
+			is compared against the data type specified in the schema. If the types match, the value of that field is used as-is. If the
+			schema indicates that the field should be of a different type, then the Controller Service will attempt to coerce the data
+			into the type specified by the schema. If the field cannot be coerced into the specified type, an Exception will be thrown.
+		</p>
+		
+		<p>
+			The following rules apply when attempting to coerce a field value from one data type to another:
+		</p>
+			
+		<ul>
+			<li>Any data type can be coerced into a String type.</li>
+			<li>Any numeric data type (Byte, Short, Int, Long, Float, Double) can be coerced into any other numeric data type.</li>
+			<li>Any numeric value can be coerced into a Date, Time, or Timestamp type, by assuming that the Long value is the number of
+			milliseconds since epoch (Midnight GMT, January 1, 1970).</li>
+			<li>A String value can be coerced into a Date, Time, or Timestamp type, if its format matches the configured "Date Format," "Time Format,"
+				or "Timestamp Format."</li>
+			<li>A String value can be coerced into a numeric value if the value is of the appropriate type. For example, the String value
+				<code>8</code> can be coerced into any numeric type. However, the String value <code>8.2</code> can be coerced into a Double or Float
+				type but not an Integer.</li>
+			<li>A String value of "true" or "false" (regardless of case) can be coerced into a Boolean value.</li>
+			<li>A String value that is not empty can be coerced into a Char type. If the String contains more than 1 character, the first character is used
+				and the rest of the characters are ignored.</li>
+			<li>Any "date/time" type (Date, Time, Timestamp) can be coerced into any other "date/time" type.</li>
+			<li>Any "date/time" type can be coerced into a Long type, representing the number of milliseconds since epoch (Midnight GMT, January 1, 1970).</li>
+			<li>Any "date/time" type can be coerced into a String. The format of the String is whatever DateFormat is configured for the corresponding
+				property (Date Format, Time Format, Timestamp Format property).</li>
+		</ul>
+		
+		<p>
+			If none of the above rules apply when attempting to coerce a value from one data type to another, the coercion will fail and an Exception
+			will be thrown.
+		</p>
+		
+		
+
         <h2>
         	Examples
 		</h2>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonPathReader/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonPathReader/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonPathReader/additionalDetails.html
index 2b69f7e..aceb54d 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonPathReader/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonPathReader/additionalDetails.html
@@ -17,15 +17,15 @@
     <head>
         <meta charset="utf-8"/>
         <title>JsonPathReader</title>
-        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
+        <link rel="stylesheet" href="/nifi-docs/css/component-usage.css" type="text/css"/>
     </head>
 
     <body>
         <p>
         	The JsonPathReader Controller Service, parses FlowFiles that are in the JSON format. User-defined properties
-        	specify how to extract all relevant fields from the JSON in order to create a row-oriented record. The Controller
+        	specify how to extract all relevant fields from the JSON in order to create a Record. The Controller
         	Service will not be valid unless at least one JSON Path is provided. Unlike the
-        	<a href="../org.apache.nifi.json.FlatJsonReader/additionalDetails.html">FlatJsonReader</a> Controller Service, this
+        	<a href="../org.apache.nifi.json.JsonTreeReader/additionalDetails.html">JsonTreeReader</a> Controller Service, this
         	service will return a record that contains only those fields that have been configured via JSON Path.
         </p>
         
@@ -39,129 +39,143 @@
         	Supplying a JSON Path is accomplished by adding a user-defined property where the name of the property becomes the name
         	of the field in the Record that is returned. The value of the property must be a valid JSON Path expression. This JSON Path
         	will be evaluated against each top-level JSON Object in the FlowFile, and the result will be the value of the field whose
-        	name is specified by the property name. By default, the type of each field is inferred automatically based on the values of
-        	the first JSON Object encountered for the FlowFile. This can be overridden by changing the name of the user-defined property
-        	by adding a colon (:) and specifying the data type. For example: <code>balance:double</code> or <code>dob:date:MM/dd/yyyy</code>.
-        	In this case, the data type and option format are not included in the field name. So for the aforementioned examples, we would
-        	end up with field names <code>balance</code> and <code>dob</code>.
+        	name is specified by the property name. If any JSON Path is given but no field is present in the Schema with the proper name,
+        	then the field will be skipped.
         </p>
         
 		<p>
-        	When specifying a data type for a field, the following values are valid:
-        </p>
-        
-        <ul>
-        	<li><b>string</b></li>
-        	<li><b>boolean</b></li>
-        	<li><b>byte</b></li>
-        	<li><b>char</b></li>
-        	<li><b>short</b></li>
-        	<li><b>int</b></li>
-        	<li><b>bigint</b></li>
-        	<li><b>long</b></li>
-        	<li><b>float</b></li>
-        	<li><b>double</b></li>
-        	<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
-        		by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).
-        	</li>
-        	<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
-        		by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).
-        	</li>
-        	<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
-        		<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
-        		<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).</li>
-        	<li><b>object</b> - The value will be returned as a <code>Map&lt;String, Object&gt;</code>. The types of the values in the Map
-        		are always inferred. The type used for the values may not be the same for each record. For example, consider the following
-        		JSON array:
-        			<br /><br />
-        			<code>
-        			[{
-        			    id: 17,
-        			  	name: "John",
-        			    child: {
-        			    	id: "1"
-        			    },
-        			    siblingIds: [4, "8"]
-        			  },
-        			  <br />{
-        			  	id: 98,
-        			    name: "Jane",
-        			    child: {
-        			        id: 2
-        			    },
-        			    siblingIds: []
-        			  }]
-        			</code>
-        			<br /><br />
-        		In this case, the <code>child</code> element would be inferred to be of type <code>object</code>. Since nested types
-        		are inferred on a per-record basis, for the first record, the <code>child</code> field would return a <code>Map</code>
-        		where the value of the <code>id</code> entry is a <code>string</code>. However, for the second record, the <code>child</code>
-        		field would return a <code>Map</code> where the value of the <code>id</code> entry is an <code>int</code>.
-        		<br />
-        		Moreover, the <code>siblingIds</code> of the John will be an <code>array</code> where the first element is an <code>int</code>
-        		and the second element is a <code>string</code>. The <code>siblingIds</code> of Jane will be an empty array.
-        		</li>
-        	<li><b>array</b> - An array of values. The types of the values are always inferred and may not be the same for each element
-        		in the array, or for two arrays from different JSON objects.</li>
-        </ul>
+			This Controller Service must be configured with a schema. Each JSON Path that is evaluated and is found in the "root level"
+			of the schema will produce a Field in the Record. I.e., the schema should match the Record that is created by evaluating all
+			of the JSON Paths. It should not match the "incoming JSON" that is read from the FlowFile.
+		</p>
+
+
+		<h2>Schemas and Type Coercion</h2>
+		
+		<p>
+			When a record is parsed from incoming data, it is separated into fields. Each of these fields is then looked up against the
+			configured schema (by field name) in order to determine what the type of the data should be. If the field is not present in
+			the schema, that field is omitted from the Record. If the field is found in the schema, the data type of the received data
+			is compared against the data type specified in the schema. If the types match, the value of that field is used as-is. If the
+			schema indicates that the field should be of a different type, then the Controller Service will attempt to coerce the data
+			into the type specified by the schema. If the field cannot be coerced into the specified type, an Exception will be thrown.
+		</p>
+		
+		<p>
+			The following rules apply when attempting to coerce a field value from one data type to another:
+		</p>
+			
+		<ul>
+			<li>Any data type can be coerced into a String type.</li>
+			<li>Any numeric data type (Byte, Short, Int, Long, Float, Double) can be coerced into any other numeric data type.</li>
+			<li>Any numeric value can be coerced into a Date, Time, or Timestamp type, by assuming that the Long value is the number of
+			milliseconds since epoch (Midnight GMT, January 1, 1970).</li>
+			<li>A String value can be coerced into a Date, Time, or Timestamp type, if its format matches the configured "Date Format," "Time Format,"
+				or "Timestamp Format."</li>
+			<li>A String value can be coerced into a numeric value if the value is of the appropriate type. For example, the String value
+				<code>8</code> can be coerced into any numeric type. However, the String value <code>8.2</code> can be coerced into a Double or Float
+				type but not an Integer.</li>
+			<li>A String value of "true" or "false" (regardless of case) can be coerced into a Boolean value.</li>
+			<li>A String value that is not empty can be coerced into a Char type. If the String contains more than 1 character, the first character is used
+				and the rest of the characters are ignored.</li>
+			<li>Any "date/time" type (Date, Time, Timestamp) can be coerced into any other "date/time" type.</li>
+			<li>Any "date/time" type can be coerced into a Long type, representing the number of milliseconds since epoch (Midnight GMT, January 1, 1970).</li>
+			<li>Any "date/time" type can be coerced into a String. The format of the String is whatever DateFormat is configured for the corresponding
+				property (Date Format, Time Format, Timestamp Format property).</li>
+		</ul>
+		
+		<p>
+			If none of the above rules apply when attempting to coerce a value from one data type to another, the coercion will fail and an Exception
+			will be thrown.
+		</p>
+		
+		
 
+		<h2>Examples</h2>
 
         <p>
         	As an example, consider a FlowFile whose content contains the following JSON:
         </p>
         
         <code>
-			[{
-			    id: 17,
-			  	name: "John",
-			    child: {
-			    	id: "1"
-			    },
-			    siblingIds: [4, "8"]
-			  },
-			  <br />{
-			  	id: 98,
-			    name: "Jane",
-			    child: {
-			        id: 2
-			    },
-			    gender: "F",
-			    siblingIds: []
-			  }]
+        <pre>
+[{
+    "id": 17,
+    "name": "John",
+    "child": {
+        "id": "1"
+    },
+    "siblingIds": [4, 8],
+    "siblings": [
+        { "name": "Jeremy", "id": 4 },
+        { "name": "Julia", "id": 8}
+    ]
+  },
+  {
+    "id": 98,
+    "name": "Jane",
+    "child": {
+        "id": 2
+    },
+    "gender": "F",
+    "siblingIds": [],
+    "siblings": []
+  }]
+		</pre>
+        </code>
+        
+        <p>
+    		And the following schema has been configured:
+        </p>
+        
+        <code>
+        <pre>
+{
+	"namespace": "nifi",
+	"name": "person",
+	"type": "record",
+	"fields": [
+		{ "name": "id", "type": "int" },
+		{ "name": "name", "type": "string" },
+		{ "name": "childId", "type": "long" },
+		{ "name": "gender", "type": "string" },
+		{ "name": "siblingNames", "type": {
+			"type": "array",
+			"items": "string"
+		}}
+	]
+}
+        </pre>
         </code>
         
         <p>
         	If we configure this Controller Service with the following user-defined properties:
         	
         	<table>
-        		<head>
+        		<tr>
         			<th>Property Name</th>
         			<th>Property Value</th>
-        		</head>
-        		<body>
-        			<tr>
-        				<td>id</td>
-        				<td><code>$.id</code></td>
-        			</tr>
-        			<tr>
-        				<td>name</td>
-        				<td><code>$.name</code></td>
-        			</tr>
-        			<tr>
-        				<td>childId:long</td>
-        				<td><code>$.child.id</code></td>
-        			</tr>
-        			<tr>
-        				<td>gender:string</td>
-        				<td><code>$.gender</code></td>
-        			</tr>
-        		</body>
+        		</tr>
+    			<tr>
+    				<td>id</td>
+    				<td><code>$.id</code></td>
+    			</tr>
+    			<tr>
+    				<td>name</td>
+    				<td><code>$.name</code></td>
+    			</tr>
+    			<tr>
+    				<td>childId</td>
+    				<td><code>$.child.id</code></td>
+    			</tr>
+    			<tr>
+    				<td>gender</td>
+    				<td><code>$.gender</code></td>
+    			</tr>
+    			<tr>
+    				<td>siblingNames</td>
+    				<td><code>$.siblings[*].name</code></td>
+    			</tr>
         	</table>
         </p>
         
@@ -169,28 +183,30 @@
 			In this case, the FlowFile will generate two Records. The first record will consist of the following key/value pairs:
 
         	<table>
-        		<head>
-        			<th>Field Name</th>
-        			<th>Field Value</th>
-        		</head>
-        		<body>
-        			<tr>
-        				<td>id</td>
-        				<td>17</td>
-        			</tr>
-        			<tr>
-        				<td>name</td>
-        				<td>John</td>
-        			</tr>
-        			<tr>
-        				<td>childId</td>
-        				<td>1</td>
-        			</tr>
-        			<tr>
-        				<td>gender</td>
-        				<td><i>null</i></td>
-        			</tr>
-				</body>
+        		<tr>
+	    			<th>Field Name</th>
+	    			<th>Field Value</th>
+				</tr>
+    			<tr>
+    				<td>id</td>
+    				<td>17</td>
+    			</tr>
+    			<tr>
+    				<td>name</td>
+    				<td>John</td>
+    			</tr>
+    			<tr>
+    				<td>childId</td>
+    				<td>1</td>
+    			</tr>
+    			<tr>
+    				<td>gender</td>
+    				<td><i>null</i></td>
+    			</tr>
+    			<tr>
+    				<td>siblingNames</td>
+    				<td><i>array of two elements: </i><code>Jeremy</code><i> and </i><code>Julia</code></td>
+    			</tr>
 			</table>
 		</p>
 		
@@ -198,28 +214,30 @@
 			The second record will consist of the following key/value pairs:
 
         	<table>
-        		<head>
+        		<tr>
         			<th>Field Name</th>
         			<th>Field Value</th>
-        		</head>
-        		<body>
-        			<tr>
-        				<td>id</td>
-        				<td>98</td>
-        			</tr>
-        			<tr>
-        				<td>name</td>
-        				<td>Jane</td>
-        			</tr>
-        			<tr>
-        				<td>childId</td>
-        				<td>2</td>
-        			</tr>
-        			<tr>
-        				<td>gender</td>
-        				<td>F</td>
-        			</tr>
-				</body>
+        		</tr>
+    			<tr>
+    				<td>id</td>
+    				<td>98</td>
+    			</tr>
+    			<tr>
+    				<td>name</td>
+    				<td>Jane</td>
+    			</tr>
+    			<tr>
+    				<td>childId</td>
+    				<td>2</td>
+    			</tr>
+    			<tr>
+    				<td>gender</td>
+    				<td>F</td>
+    			</tr>
+    			<tr>
+    				<td>siblingNames</td>
+    				<td><i>empty array</i></td>
+    			</tr>
 			</table>
 		</p>
 		

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonTreeReader/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonTreeReader/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonTreeReader/additionalDetails.html
index 7d6be7a..90980d1 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonTreeReader/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonTreeReader/additionalDetails.html
@@ -17,86 +17,223 @@
     <head>
         <meta charset="utf-8"/>
         <title>JsonTreeReader</title>
-        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
+        <link rel="stylesheet" href="/nifi-docs/css/component-usage.css" type="text/css"/>
     </head>
 
     <body>
         <p>
-        	The JsonTreeReader Controller Service, by default, derives the schema for a FlowFile
-        	based on the first JSON Object in the FlowFile. For each field found, the data type
-        	is inferred. However, the type of a field can be overridden by adding a user-defined property to
-        	the Controller Service. The name of the property should be the same as the name of the
-        	JSON field. The value of the property denotes the data type of the corresponding field.
-        	If no JSON field is found with a matching name, then a field will be added to the schema,
-        	and a <code>null</code> value will be used for any record for which the JSON field
-        	is not present. If a field is found with a matching name, but the type is different,
-        	the Controller Service will attempt to coerce the value into the user-defined type. If unable
-        	to do so, an Exception will be thrown.
+        	The JsonTreeReader Controller Service reads a JSON Object and creates a Record object for the entire
+        	JSON Object tree. The Controller Service must be configured with a Schema that describes the structure
+        	of the JSON data. If any field exists in the JSON that is not in the schema, that field will be skipped.
+        	If the schema contains a field for which no JSON field exists, a null value will be used in the Record.
         </p>
         
         <p>
-        	When specifying a data type for a field, the following values are valid:
+        	If the root element of the JSON is a JSON Array, each JSON Object within that array will be treated as
+        	its own separate Record. If the root element is a JSON Object, the JSON will all be treated as a single
+        	Record.
         </p>
+
+
+		<h2>Schemas and Type Coercion</h2>
+		
+		<p>
+			When a record is parsed from incoming data, it is separated into fields. Each of these fields is then looked up against the
+			configured schema (by field name) in order to determine what the type of the data should be. If the field is not present in
+			the schema, that field is omitted from the Record. If the field is found in the schema, the data type of the received data
+			is compared against the data type specified in the schema. If the types match, the value of that field is used as-is. If the
+			schema indicates that the field should be of a different type, then the Controller Service will attempt to coerce the data
+			into the type specified by the schema. If the field cannot be coerced into the specified type, an Exception will be thrown.
+		</p>
+		
+		<p>
+			The following rules apply when attempting to coerce a field value from one data type to another:
+		</p>
+			
+		<ul>
+			<li>Any data type can be coerced into a String type.</li>
+			<li>Any numeric data type (Byte, Short, Int, Long, Float, Double) can be coerced into any other numeric data type.</li>
+			<li>Any numeric value can be coerced into a Date, Time, or Timestamp type, by assuming that the Long value is the number of
+			milliseconds since epoch (Midnight GMT, January 1, 1970).</li>
+			<li>A String value can be coerced into a Date, Time, or Timestamp type, if its format matches the configured "Date Format," "Time Format,"
+				or "Timestamp Format."</li>
+			<li>A String value can be coerced into a numeric value if the value is of the appropriate type. For example, the String value
+				<code>8</code> can be coerced into any numeric type. However, the String value <code>8.2</code> can be coerced into a Double or Float
+				type but not an Integer.</li>
+			<li>A String value of "true" or "false" (regardless of case) can be coerced into a Boolean value.</li>
+			<li>A String value that is not empty can be coerced into a Char type. If the String contains more than 1 character, the first character is used
+				and the rest of the characters are ignored.</li>
+			<li>Any "date/time" type (Date, Time, Timestamp) can be coerced into any other "date/time" type.</li>
+			<li>Any "date/time" type can be coerced into a Long type, representing the number of milliseconds since epoch (Midnight GMT, January 1, 1970).</li>
+			<li>Any "date/time" type can be coerced into a String. The format of the String is whatever DateFormat is configured for the corresponding
+				property (Date Format, Time Format, Timestamp Format property).</li>
+		</ul>
+		
+		<p>
+			If none of the above rules apply when attempting to coerce a value from one data type to another, the coercion will fail and an Exception
+			will be thrown.
+		</p>
+		
+		
+        <h2>Examples</h2>
+        
+        <p>
+        	As an example, consider the following JSON is read:
+        </p>
+<code>
+<pre>
+[{
+    "id": 17,
+    "name": "John",
+    "child": {
+        "id": "1"
+    },
+    "dob": "10-29-1982"
+    "siblings": [
+        { "name": "Jeremy", "id": 4 },
+        { "name": "Julia", "id": 8}
+    ]
+  },
+  {
+    "id": 98,
+    "name": "Jane",
+    "child": {
+        "id": 2
+    },
+    "dob": "08-30-1984"
+    "gender": "F",
+    "siblingIds": [],
+    "siblings": []
+  }]
+</pre>
+</code>
+        
+        <p>
+        	Also, consider that the schema that is configured for this JSON is as follows (assuming that the AvroSchemaRegistry
+        	Controller Service is chosen to denote the Schema:
+        </p>
+
+<code>
+<pre>
+{
+	"namespace": "nifi",
+	"name": "person",
+	"type": "record",
+	"fields": [
+		{ "name": "id", "type": "int" },
+		{ "name": "name", "type": "string" },
+		{ "name": "gender", "type": "string" },
+		{ "name": "dob", "type": {
+			"type": "int",
+			"logicalType": "date"
+		}},
+		{ "name": "siblings", "type": {
+			"type": "array",
+			"items": {
+				"type": "record",
+				"fields": [
+					{ "name": "name", "type": "string" }
+				]
+			}
+		}}
+	]
+}
+</pre>
+</code>
+        
+        <p>
+        	Let us also assume that this Controller Service is configured with the "Date Format" property set to "MM-dd-yyyy", as this
+        	matches the date format used for our JSON data. This will result in the JSON creating two separate records, because the root
+        	element is a JSON array with two elements.
+        </p>
+        
+        <p>
+        	The first Record will consist of the following values:
+        </p>
+        
+        <table>
+        	<tr>
+    			<th>Field Name</th>
+    			<th>Field Value</th>
+        	</tr>
+    		<tr>
+    			<td>id</td>
+    			<td>17</td>
+    		</tr>
+    		<tr>
+    			<td>name</td>
+    			<td>John</td>
+    		</tr>
+    		<tr>
+    			<td>gender</td>
+    			<td><i>null</i></td>
+    		</tr>
+    		<tr>
+    			<td>dob</td>
+    			<td>11-30-1983</td>
+    		</tr>
+    		<tr>
+    			<td>siblings</td>
+    			<td>
+    				<i>array with two elements, each of which is itself a Record:</i>
+    				<br />
+    				<table>
+    					<tr>
+							<th>Field Name</th>
+							<th>Field Value</th>
+						</tr>
+						<tr>
+							<td>name</td>
+							<td>Jeremy</td>
+						</tr>
+    				</table>
+    				<br />
+    				<i>and:</i>
+    				<br />
+    				<table>
+						<tr>
+							<th>Field Name</th>
+							<th>Field Value</th>
+						</tr>
+						<tr>
+							<td>name</td>
+							<td>Julia</td>
+						</tr>
+    				</table>
+    			</td>
+    		</tr>
+        </table>
+        
+        <p>
+        	The second Record will consist of the following values:
+        </p>
+        
+		<table>
+			<tr>
+    			<th>Field Name</th>
+    			<th>Field Value</th>
+        	</tr>
+    		<tr>
+    			<td>id</td>
+    			<td>98</td>
+    		</tr>
+    		<tr>
+    			<td>name</td>
+    			<td>Jane</td>
+    		</tr>
+    		<tr>
+    			<td>gender</td>
+    			<td>F</td>
+    		</tr>
+    		<tr>
+    			<td>dob</td>
+    			<td>08-30-1984</td>
+    		</tr>
+    		<tr>
+    			<td>siblings</td>
+    			<td><i>empty array</i></td>
+    		</tr>
+        </table>
         
-        <ul>
-        	<li><b>string</b></li>
-        	<li><b>boolean</b></li>
-        	<li><b>byte</b></li>
-        	<li><b>char</b></li>
-        	<li><b>short</b></li>
-        	<li><b>int</b></li>
-        	<li><b>bigint</b></li>
-        	<li><b>long</b></li>
-        	<li><b>float</b></li>
-        	<li><b>double</b></li>
-        	<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
-        		by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).
-        	</li>
-        	<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
-        		by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).
-        	</li>
-        	<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
-        		<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
-        		<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
-        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
-        		SimpleDateFormat Patterns</a> for more information).</li>
-        	<li><b>object</b> - The value will be returned as a <code>Map&lt;String, Object&gt;</code>. The types of the values in the Map
-        		are always inferred. The type used for the values may not be the same for each record. For example, consider the following
-        		JSON array:
-        			<br /><br />
-        			<code>
-        			[{
-        			    id: 17,
-        			  	name: "John",
-        			    child: {
-        			    	id: "1"
-        			    },
-        			    siblingIds: [4, "8"]
-        			  },
-        			  <br />{
-        			  	id: 98,
-        			    name: "Jane",
-        			    child: {
-        			        id: 2
-        			    },
-        			    siblingIds: []
-        			  }]
-        			</code>
-        			<br /><br />
-        		In this case, the <code>child</code> element would be inferred to be of type <code>object</code>. Since nested types
-        		are inferred on a per-record basis, for the first record, the <code>child</code> field would return a <code>Map</code>
-        		where the value of the <code>id</code> entry is a <code>string</code>. However, for the second record, the <code>child</code>
-        		field would return a <code>Map</code> where the value of the <code>id</code> entry is an <code>int</code>.
-        		<br />
-        		Moreover, the <code>siblingIds</code> of the John will be an <code>array</code> where the first element is an <code>int</code>
-        		and the second element is a <code>string</code>. The <code>siblingIds</code> of Jane will be an empty array.
-        		</li>
-        	<li><b>array</b> - An array of values. The types of the values are always inferred and may not be the same for each element
-        		in the array, or for two arrays from different JSON objects.</li>
-        </ul>
     </body>
 </html>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java
index 2ec3441..dfbd721 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java
@@ -23,15 +23,20 @@ import static org.junit.Assert.assertNull;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.TimeZone;
 
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
@@ -52,46 +57,100 @@ import org.junit.Test;
 
 public class TestAvroRecordReader {
 
+
+    @Test
+    public void testLogicalTypes() throws IOException, ParseException, MalformedRecordException {
+        final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/logical-types.avsc"));
+
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        final String expectedTime = "2017-04-04 14:20:33.000";
+        final DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
+        df.setTimeZone(TimeZone.getTimeZone("gmt"));
+        final long timeLong = df.parse(expectedTime).getTime();
+
+        final long secondsSinceMidnight = 33 + (20 * 60) + (14 * 60 * 60);
+        final long millisSinceMidnight = secondsSinceMidnight * 1000L;
+
+
+        final byte[] serialized;
+        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter);
+            final DataFileWriter<GenericRecord> writer = dataFileWriter.create(schema, baos)) {
+
+            final GenericRecord record = new GenericData.Record(schema);
+            record.put("timeMillis", millisSinceMidnight);
+            record.put("timeMicros", millisSinceMidnight * 1000L);
+            record.put("timestampMillis", timeLong);
+            record.put("timestampMicros", timeLong * 1000L);
+            record.put("date", 17261);
+
+            writer.append(record);
+            writer.flush();
+
+            serialized = baos.toByteArray();
+        }
+
+        try (final InputStream in = new ByteArrayInputStream(serialized)) {
+            final AvroRecordReader reader = new AvroRecordReader(in);
+            final RecordSchema recordSchema = reader.getSchema();
+
+            assertEquals(RecordFieldType.TIME, recordSchema.getDataType("timeMillis").get().getFieldType());
+            assertEquals(RecordFieldType.TIME, recordSchema.getDataType("timeMicros").get().getFieldType());
+            assertEquals(RecordFieldType.TIMESTAMP, recordSchema.getDataType("timestampMillis").get().getFieldType());
+            assertEquals(RecordFieldType.TIMESTAMP, recordSchema.getDataType("timestampMicros").get().getFieldType());
+            assertEquals(RecordFieldType.DATE, recordSchema.getDataType("date").get().getFieldType());
+
+            final Record record = reader.nextRecord();
+            assertEquals(new java.sql.Time(millisSinceMidnight), record.getValue("timeMillis"));
+            assertEquals(new java.sql.Time(millisSinceMidnight), record.getValue("timeMicros"));
+            assertEquals(new java.sql.Timestamp(timeLong), record.getValue("timestampMillis"));
+            assertEquals(new java.sql.Timestamp(timeLong), record.getValue("timestampMicros"));
+            assertEquals(new java.sql.Date(timeLong).toString(), record.getValue("date").toString());
+        }
+    }
+
     @Test
+    @SuppressWarnings({"unchecked", "rawtypes"})
     public void testDataTypes() throws IOException, MalformedRecordException {
         final List<Field> accountFields = new ArrayList<>();
-        accountFields.add(new Field("accountId", Schema.create(Type.LONG), null, null));
-        accountFields.add(new Field("accountName", Schema.create(Type.STRING), null, null));
+        accountFields.add(new Field("accountId", Schema.create(Type.LONG), null, (Object) null));
+        accountFields.add(new Field("accountName", Schema.create(Type.STRING), null, (Object) null));
         final Schema accountSchema = Schema.createRecord("account", null, null, false);
         accountSchema.setFields(accountFields);
 
         final List<Field> catFields = new ArrayList<>();
-        catFields.add(new Field("catTailLength", Schema.create(Type.INT), null, null));
-        catFields.add(new Field("catName", Schema.create(Type.STRING), null, null));
+        catFields.add(new Field("catTailLength", Schema.create(Type.INT), null, (Object) null));
+        catFields.add(new Field("catName", Schema.create(Type.STRING), null, (Object) null));
         final Schema catSchema = Schema.createRecord("cat", null, null, false);
         catSchema.setFields(catFields);
 
         final List<Field> dogFields = new ArrayList<>();
-        dogFields.add(new Field("dogTailLength", Schema.create(Type.INT), null, null));
-        dogFields.add(new Field("dogName", Schema.create(Type.STRING), null, null));
+        dogFields.add(new Field("dogTailLength", Schema.create(Type.INT), null, (Object) null));
+        dogFields.add(new Field("dogName", Schema.create(Type.STRING), null, (Object) null));
         final Schema dogSchema = Schema.createRecord("dog", null, null, false);
         dogSchema.setFields(dogFields);
 
         final List<Field> fields = new ArrayList<>();
-        fields.add(new Field("name", Schema.create(Type.STRING), null, null));
-        fields.add(new Field("age", Schema.create(Type.INT), null, null));
-        fields.add(new Field("balance", Schema.create(Type.DOUBLE), null, null));
-        fields.add(new Field("rate", Schema.create(Type.FLOAT), null, null));
-        fields.add(new Field("debt", Schema.create(Type.BOOLEAN), null, null));
-        fields.add(new Field("nickname", Schema.create(Type.NULL), null, null));
-        fields.add(new Field("binary", Schema.create(Type.BYTES), null, null));
-        fields.add(new Field("fixed", Schema.createFixed("fixed", null, null, 5), null, null));
-        fields.add(new Field("map", Schema.createMap(Schema.create(Type.STRING)), null, null));
-        fields.add(new Field("array", Schema.createArray(Schema.create(Type.LONG)), null, null));
-        fields.add(new Field("account", accountSchema, null, null));
+        fields.add(new Field("name", Schema.create(Type.STRING), null, (Object) null));
+        fields.add(new Field("age", Schema.create(Type.INT), null, (Object) null));
+        fields.add(new Field("balance", Schema.create(Type.DOUBLE), null, (Object) null));
+        fields.add(new Field("rate", Schema.create(Type.FLOAT), null, (Object) null));
+        fields.add(new Field("debt", Schema.create(Type.BOOLEAN), null, (Object) null));
+        fields.add(new Field("nickname", Schema.create(Type.NULL), null, (Object) null));
+        fields.add(new Field("binary", Schema.create(Type.BYTES), null, (Object) null));
+        fields.add(new Field("fixed", Schema.createFixed("fixed", null, null, 5), null, (Object) null));
+        fields.add(new Field("map", Schema.createMap(Schema.create(Type.STRING)), null, (Object) null));
+        fields.add(new Field("array", Schema.createArray(Schema.create(Type.LONG)), null, (Object) null));
+        fields.add(new Field("account", accountSchema, null, (Object) null));
         fields.add(new Field("desiredbalance", Schema.createUnion( // test union of NULL and other type with no value
             Arrays.asList(Schema.create(Type.NULL), Schema.create(Type.DOUBLE))),
-            null, null));
+            null, (Object) null));
         fields.add(new Field("dreambalance", Schema.createUnion( // test union of NULL and other type with a value
             Arrays.asList(Schema.create(Type.NULL), Schema.create(Type.DOUBLE))),
-            null, null));
-        fields.add(new Field("favAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, null));
-        fields.add(new Field("otherFavAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, null));
+            null, (Object) null));
+        fields.add(new Field("favAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, (Object) null));
+        fields.add(new Field("otherFavAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, (Object) null));
 
         final Schema schema = Schema.createRecord("record", null, null, false);
         schema.setFields(fields);
@@ -103,6 +162,12 @@ public class TestAvroRecordReader {
         map.put("greeting", "hello");
         map.put("salutation", "good-bye");
 
+        final List<RecordField> mapFields = new ArrayList<>();
+        mapFields.add(new RecordField("greeting", RecordFieldType.STRING.getDataType()));
+        mapFields.add(new RecordField("salutation", RecordFieldType.STRING.getDataType()));
+        final RecordSchema mapSchema = new SimpleRecordSchema(mapFields);
+        final Record expectedRecord = new MapRecord(mapSchema, (Map) map);
+
         final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
         try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter);
             final DataFileWriter<GenericRecord> writer = dataFileWriter.create(schema, baos)) {
@@ -168,9 +233,9 @@ public class TestAvroRecordReader {
             assertEquals(0.045F, values[3]);
             assertEquals(false, values[4]);
             assertEquals(null, values[5]);
-            assertArrayEquals("binary".getBytes(StandardCharsets.UTF_8), (byte[]) values[6]);
-            assertArrayEquals("fixed".getBytes(StandardCharsets.UTF_8), (byte[]) values[7]);
-            assertEquals(map, values[8]);
+            assertArrayEquals(toObjectArray("binary".getBytes(StandardCharsets.UTF_8)), (Object[]) values[6]);
+            assertArrayEquals(toObjectArray("fixed".getBytes(StandardCharsets.UTF_8)), (Object[]) values[7]);
+            assertEquals(expectedRecord, values[8]);
             assertArrayEquals(new Object[] {1L, 2L}, (Object[]) values[9]);
 
             final Map<String, Object> accountValues = new HashMap<>();
@@ -215,6 +280,14 @@ public class TestAvroRecordReader {
         }
     }
 
+    private Object[] toObjectArray(final byte[] bytes) {
+        final Object[] array = new Object[bytes.length];
+        for (int i = 0; i < bytes.length; i++) {
+            array[i] = Byte.valueOf(bytes[i]);
+        }
+        return array;
+    }
+
     public static enum Status {
         GOOD, BAD;
     }