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 2019/02/11 18:00:44 UTC

[nifi] branch master updated: NIFI-5938: Added ability to infer record schema on read from JsonTreeReader, JsonPathReader, XML Reader, and CSV Reader. - Updates to make UpdateRecord and RecordPath automatically update Record schema when performing update and perform the updates on the first record in UpdateRecord before obtaining Writer Schema. This allows the Writer to to inherit the Schema of the updated Record instead of the Schema of the Record as it was when it was read. - Updated JoltTransformRecord so that sc [...]

This is an automated email from the ASF dual-hosted git repository.

mattyb149 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/master by this push:
     new 36c0a99  NIFI-5938: Added ability to infer record schema on read from JsonTreeReader, JsonPathReader, XML Reader, and CSV Reader.  - Updates to make UpdateRecord and RecordPath automatically update Record schema when performing update and perform the updates on the first record in UpdateRecord before obtaining Writer Schema. This allows the Writer to  to inherit the Schema of the updated Record instead of the Schema of the Record as it was when it was read.  - Updated JoltTransfo [...]
36c0a99 is described below

commit 36c0a99e91c492329c0df69ee4ae961e57295f84
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Wed Dec 19 18:20:52 2018 -0500

    NIFI-5938: Added ability to infer record schema on read from JsonTreeReader, JsonPathReader, XML Reader, and CSV Reader.
     - Updates to make UpdateRecord and RecordPath automatically update Record schema when performing update and perform the updates on the first record in UpdateRecord before obtaining Writer Schema. This allows the Writer to  to inherit the Schema of the updated Record instead of the Schema of the Record as it was when it was read.
     - Updated JoltTransformRecord so that schema is inferred on the first transformed object before passing the schema to the Record Writer, so that if writer inherits schema from record, the schema that is inherited is the trans transformed schema
     - Updated LookupRecord to allow for Record fields to be arbitrarily added
     - Implemented ContentClaimInputStream
     - Added controller service for caching schemas
     - UpdatedQueryRecord to cache schemas automatically up to some number of schemas, which will significantly inprove throughput in many cases, especially with inferred schemas.
    
    NIFI-5938: Updated AvroTypeUtil so that if creating an Avro Schema using a field name that is not valid for Avro, it creates a Schema that uses a different, valid field name and adds an alias for the given field name so that the fields still are looked up appropriately. Fixed a bug in finding the appropriate Avro field when aliases are used. Updated ContentClaimInputStream so that if mark() is called followed by multiple calls to reset(), that each reset() call is successful instead o [...]
    
    NIFI-5938: Added another unit test for AvroTypeUtil
    
    NIFI-5938: If using inferred schema in CSV Reader, do not consider first record as a header line. Also addressed a bug in StandardConfigurationContext that was exposed by CSVReader, in which calling getProperty(PropertyDescriptor) did not properly lookup the canonical representation of the Property Descriptor from the component before attempting to get a default value
    
    Signed-off-by: Matthew Burgess <ma...@apache.org>
    
    This closes #3253
---
 .../nifi/record/path/ArrayIndexFieldValue.java     |  10 +-
 .../org/apache/nifi/record/path/FieldValue.java    |  15 +-
 .../nifi/record/path/MapEntryFieldValue.java       |  10 +-
 .../nifi/record/path/StandardFieldValue.java       |  25 +-
 .../apache/nifi/serialization/RecordReader.java    |   8 +-
 .../nifi/serialization/SimpleRecordSchema.java     |   4 +-
 .../org/apache/nifi/serialization/WriteResult.java |   4 +-
 .../nifi/serialization/record/MapRecord.java       |  65 ++-
 .../apache/nifi/serialization/record/Record.java   |  37 +-
 .../nifi/serialization/record/RecordField.java     |   2 +-
 .../nifi/serialization/record/RecordFieldType.java |  66 ++-
 .../serialization/record/type/ArrayDataType.java   |   2 +-
 .../serialization/record/type/MapDataType.java     |   4 +-
 .../serialization/record/type/RecordDataType.java  |   2 +-
 .../serialization/record/util/DataTypeUtils.java   | 350 ++++++++++++--
 .../remote/io/socket/ssl/SSLSocketChannel.java     |  32 +-
 .../org/apache/nifi/util/text/DateTimeMatcher.java |  68 +++
 .../nifi/util/text/DateTimeMatcherCompiler.java    |  56 +++
 .../apache/nifi/util/text/ListDateTimeMatcher.java |  44 ++
 .../nifi/util/text/RegexDateTimeMatcher.java       | 538 +++++++++++++++++++++
 .../nifi/util/text/SimpleDateFormatMatcher.java    |  38 ++
 .../util/text/StartsWithDigitsDateTimeMatcher.java |  43 ++
 .../nifi/util/text/TestRegexDateTimeMatcher.java   |  69 +++
 .../org/apache/nifi/util/MockProcessSession.java   |  42 +-
 .../java/org/apache/nifi/avro/AvroTypeUtil.java    |  88 +++-
 .../JsonInferenceSchemaRegistryService.java        |   5 +-
 .../nifi/serialization/SchemaRegistryService.java  |  17 +-
 .../org/apache/nifi/avro/TestAvroTypeUtil.java     |  74 ++-
 .../processors/hadoop/AbstractPutHDFSRecord.java   |  34 +-
 .../listen/SSLSocketChannelRecordReader.java       |   4 +-
 .../repository/StandardProcessSession.java         |  21 +-
 .../repository/io/ContentClaimInputStream.java     | 158 ++++++
 .../service/StandardConfigurationContext.java      |  24 +-
 .../repository/TestStandardProcessSession.java     |   4 +
 .../repository/io/TestContentClaimInputStream.java | 192 ++++++++
 .../nifi/processors/hive/PutHive3Streaming.java    |  10 +-
 .../jolt/record/JoltTransformRecord.java           |  91 +++-
 .../jolt/record/TestJoltTransformRecord.java       |  18 +-
 .../kafka/pubsub/PublishKafkaRecord_0_10.java      |  29 +-
 .../kafka/pubsub/PublishKafkaRecord_0_11.java      |  35 +-
 .../kafka/pubsub/PublishKafkaRecord_1_0.java       |  35 +-
 .../kafka/pubsub/PublishKafkaRecord_2_0.java       |   5 +-
 .../standard/AbstractRecordProcessor.java          |  29 +-
 .../processors/standard/AbstractRouteRecord.java   |  75 +--
 .../nifi/processors/standard/ConvertRecord.java    |   3 +-
 .../nifi/processors/standard/LookupRecord.java     |  29 +-
 .../nifi/processors/standard/QueryRecord.java      | 259 ++++------
 .../nifi/processors/standard/UpdateRecord.java     |  59 ++-
 .../nifi/queryrecord/FlowFileEnumerator.java       |   2 +-
 .../org/apache/nifi/queryrecord/FlowFileTable.java |  50 +-
 .../apache/nifi/queryrecord/FlowFileTableScan.java |   8 +-
 .../nifi/processors/standard/TestLookupRecord.java |  49 +-
 .../org/apache/nifi/lookup/RestLookupService.java  |   8 +-
 .../nifi/serialization/RecordReaderFactory.java    |  28 +-
 .../serialization/RecordSchemaCacheService.java    |  51 ++
 .../nifi-record-serialization-services/pom.xml     |  21 +-
 .../main/java/org/apache/nifi/avro/AvroReader.java |  13 +-
 .../apache/nifi/csv/CSVHeaderSchemaStrategy.java   |  29 +-
 .../main/java/org/apache/nifi/csv/CSVReader.java   |  52 +-
 .../java/org/apache/nifi/csv/CSVRecordSource.java  |  61 +++
 .../org/apache/nifi/csv/CSVSchemaInference.java    | 130 +++++
 .../main/java/org/apache/nifi/grok/GrokReader.java |  12 +-
 .../nifi/json/AbstractJsonRowRecordReader.java     |  44 +-
 .../java/org/apache/nifi/json/JsonPathReader.java  |  57 ++-
 .../apache/nifi/json/JsonPathRowRecordReader.java  |  45 +-
 .../org/apache/nifi/json/JsonRecordSource.java     |  55 +++
 .../org/apache/nifi/json/JsonSchemaInference.java  | 112 +++++
 .../java/org/apache/nifi/json/JsonTreeReader.java  |  51 +-
 .../apache/nifi/json/JsonTreeRowRecordReader.java  |  24 +-
 .../java/org/apache/nifi/json/WriteJsonResult.java |  28 +-
 .../inference/CachedSchemaAccessStrategy.java      |  66 +++
 .../nifi/schema/inference/FieldTypeInference.java  | 109 +++++
 .../inference/HierarchicalSchemaInference.java     | 137 ++++++
 .../inference/InferSchemaAccessStrategy.java       |  62 +++
 .../apache/nifi/schema/inference/RecordSource.java |  24 +
 .../nifi/schema/inference/RecordSourceFactory.java |  27 ++
 .../schema/inference/SchemaInferenceEngine.java    |  27 ++
 .../nifi/schema/inference/SchemaInferenceUtil.java |  59 +++
 .../nifi/schema/inference/TimeValueInference.java  |  69 +++
 .../nifi/schema/inference/VolatileSchemaCache.java | 154 ++++++
 .../serialization/CacheIdSchemaAccessWriter.java   |  62 +++
 .../SchemaRegistryRecordSetWriter.java             |  65 ++-
 .../org/apache/nifi/syslog/Syslog5424Reader.java   |  11 +-
 .../java/org/apache/nifi/syslog/SyslogReader.java  |  11 +-
 .../nifi/text/FreeFormTextRecordSetWriter.java     |  13 +-
 .../java/org/apache/nifi/xml/WriteXMLResult.java   |  35 +-
 .../main/java/org/apache/nifi/xml/XMLReader.java   |  64 ++-
 .../java/org/apache/nifi/xml/XMLRecordReader.java  |   4 +
 .../org/apache/nifi/xml/XMLRecordSetWriter.java    |   3 +-
 .../apache/nifi/xml/inference/XmlArrayNode.java    |  54 +++
 .../nifi/xml/inference/XmlContainerNode.java       |  50 ++
 .../org/apache/nifi/xml/inference/XmlNode.java     |  25 +
 .../org/apache/nifi/xml/inference/XmlNodeType.java |  23 +
 .../apache/nifi/xml/inference/XmlRecordSource.java | 133 +++++
 .../nifi/xml/inference/XmlSchemaInference.java     | 117 +++++
 .../org/apache/nifi/xml/inference/XmlTextNode.java |  41 ++
 .../org.apache.nifi.controller.ControllerService   |   4 +-
 .../additionalDetails.html                         | 134 ++++-
 .../additionalDetails.html                         | 124 ++++-
 .../additionalDetails.html                         | 124 ++++-
 .../additionalDetails.html                         |  98 ++++
 .../org/apache/nifi/csv/TestCSVRecordReader.java   |   2 +-
 .../apache/nifi/csv/TestCSVSchemaInference.java    |  78 +++
 .../nifi/csv/TestJacksonCSVRecordReader.java       |   2 +-
 .../org/apache/nifi/csv/TestWriteCSVResult.java    |  38 +-
 .../json/TestInferJsonSchemaAccessStrategy.java    | 196 ++++++++
 .../nifi/json/TestJsonTreeRowRecordReader.java     |  43 +-
 .../schema/inference/TestVolatileSchemaCache.java  | 103 ++++
 .../org/apache/nifi/xml/TestInferXmlSchema.java    |  99 ++++
 .../org/apache/nifi/xml/TestWriteXMLResult.java    | 130 +++--
 .../src/test/resources/csv/prov-events.csv         |   3 +
 .../src/test/resources/json/docs-example.json      |  10 +
 .../src/test/resources/json/output/dataTypes.json  |   6 +-
 .../src/test/resources/json/prov-events.json       |  79 +++
 .../src/test/resources/xml/people_nested.xml       |   8 +
 .../src/test/resources/xml/person.xml              |   3 +
 116 files changed, 5485 insertions(+), 1006 deletions(-)

diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/ArrayIndexFieldValue.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/ArrayIndexFieldValue.java
index a753579..b69fbce 100644
--- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/ArrayIndexFieldValue.java
+++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/ArrayIndexFieldValue.java
@@ -17,11 +17,12 @@
 
 package org.apache.nifi.record.path;
 
-import java.util.Objects;
-
+import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 
+import java.util.Objects;
+
 public class ArrayIndexFieldValue extends StandardFieldValue {
     private final int index;
 
@@ -54,6 +55,11 @@ public class ArrayIndexFieldValue extends StandardFieldValue {
     }
 
     @Override
+    public void updateValue(final Object newValue, final DataType dataType) {
+        getParentRecord().get().setArrayValue(getField().getFieldName(), getArrayIndex(), newValue);
+    }
+
+    @Override
     public int hashCode() {
         return Objects.hash(getValue(), getField(), getParent(), index);
     }
diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/FieldValue.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/FieldValue.java
index 88828a5..084ce73 100644
--- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/FieldValue.java
+++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/FieldValue.java
@@ -17,11 +17,12 @@
 
 package org.apache.nifi.record.path;
 
-import java.util.Optional;
-
+import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordField;
 
+import java.util.Optional;
+
 public interface FieldValue {
     /**
      * @return the value of the field
@@ -51,4 +52,14 @@ public interface FieldValue {
      * @param newValue the new value to set on the record field
      */
     void updateValue(Object newValue);
+
+    /**
+     * Updates the record to which the field belongs, so that it now has the given value. If the FieldValue
+     * points to a Field that does not currently exist in the Record, the field will be created in the Record Schema
+     * as an 'inactive field', which can then be incoporated into the Record's schema by calling {@link Record#incorporateInactiveFields()}.
+     *
+     * @param newValue the value to set for the field
+     * @param dataType the data type to use if the Record's schema does not already include this field
+     */
+    void updateValue(Object newValue, DataType dataType);
 }
diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/MapEntryFieldValue.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/MapEntryFieldValue.java
index f52af3d1..f28064d 100644
--- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/MapEntryFieldValue.java
+++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/MapEntryFieldValue.java
@@ -17,10 +17,11 @@
 
 package org.apache.nifi.record.path;
 
-import java.util.Objects;
-
+import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.RecordField;
 
+import java.util.Objects;
+
 public class MapEntryFieldValue extends StandardFieldValue {
     private final String mapKey;
 
@@ -39,6 +40,11 @@ public class MapEntryFieldValue extends StandardFieldValue {
     }
 
     @Override
+    public void updateValue(final Object newValue, final DataType dataType) {
+        getParentRecord().get().setMapValue(getField().getFieldName(), getMapKey(), newValue);
+    }
+
+    @Override
     public int hashCode() {
         return Objects.hash(getValue(), getField(), getParent(), mapKey);
     }
diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/StandardFieldValue.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/StandardFieldValue.java
index 7526c0c..8ab03ca 100644
--- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/StandardFieldValue.java
+++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/StandardFieldValue.java
@@ -17,14 +17,15 @@
 
 package org.apache.nifi.record.path;
 
-import java.util.Arrays;
-import java.util.Objects;
-import java.util.Optional;
-
 import org.apache.nifi.record.path.util.Filters;
+import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordField;
 
+import java.util.Arrays;
+import java.util.Objects;
+import java.util.Optional;
+
 public class StandardFieldValue implements FieldValue {
     private final Object value;
     private final RecordField field;
@@ -117,10 +118,21 @@ public class StandardFieldValue implements FieldValue {
 
     @Override
     public void updateValue(final Object newValue) {
+        updateValue(newValue, getField());
+    }
+
+    @Override
+    public void updateValue(final Object newValue, final DataType dataType) {
+        final RecordField currentField = getField();
+        final RecordField recordField = new RecordField(currentField.getFieldName(), dataType, currentField.getDefaultValue(), currentField.getAliases(), currentField.isNullable());
+        updateValue(newValue, recordField);
+    }
+
+    private void updateValue(final Object newValue, final RecordField field) {
         final Optional<Record> parentRecord = getParentRecord();
         if (!parentRecord.isPresent()) {
             if (value instanceof Record) {
-                ((Record) value).setValue(getField().getFieldName(), newValue);
+                ((Record) value).setValue(field, newValue);
                 return;
             } else if (value == null) {
                 return; // value is null, nothing to update
@@ -129,6 +141,7 @@ public class StandardFieldValue implements FieldValue {
             }
         }
 
-        parentRecord.get().setValue(getField().getFieldName(), newValue);
+        parentRecord.get().setValue(field, newValue);
+
     }
 }
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/RecordReader.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/RecordReader.java
index 188f032..1346e34 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/RecordReader.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/RecordReader.java
@@ -17,13 +17,13 @@
 
 package org.apache.nifi.serialization;
 
-import java.io.Closeable;
-import java.io.IOException;
-
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 
+import java.io.Closeable;
+import java.io.IOException;
+
 /**
  * <p>
  * A RowRecordReader is responsible for parsing data and returning a record at a time
@@ -47,7 +47,7 @@ public interface RecordReader extends Closeable {
      * @throws SchemaValidationException if a Record contains a field that violates the schema and cannot be coerced into the appropriate field type.
      */
     default Record nextRecord() throws IOException, MalformedRecordException {
-        return nextRecord(true, true);
+        return nextRecord(true, false);
     }
 
     /**
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
index 7f78448..453d88a 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
@@ -43,11 +43,11 @@ public class SimpleRecordSchema implements RecordSchema {
     private volatile int hashCode;
 
     public SimpleRecordSchema(final List<RecordField> fields) {
-        this(fields, createText(fields), null, false, SchemaIdentifier.EMPTY);
+        this(fields, null, null, false, SchemaIdentifier.EMPTY);
     }
 
     public SimpleRecordSchema(final List<RecordField> fields, final SchemaIdentifier id) {
-        this(fields, createText(fields), null, false, id);
+        this(fields, null, null, false, id);
     }
 
     public SimpleRecordSchema(final String text, final String schemaFormat, final SchemaIdentifier id) {
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/WriteResult.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/WriteResult.java
index 3fb2741..ac77493 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/WriteResult.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/WriteResult.java
@@ -51,7 +51,7 @@ public interface WriteResult {
      * @param attributes the attributes to add to the FlowFile
      * @return A {@link WriteResult} representing the given parameters
      */
-    public static WriteResult of(final int recordCount, final Map<String, String> attributes) {
+    static WriteResult of(final int recordCount, final Map<String, String> attributes) {
         return new WriteResult() {
             @Override
             public int getRecordCount() {
@@ -65,5 +65,5 @@ public interface WriteResult {
         };
     }
 
-    public static final WriteResult EMPTY = of(0, Collections.emptyMap());
+    WriteResult EMPTY = of(0, Collections.emptyMap());
 }
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/MapRecord.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
index 57b7ac3..2f8a766 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
@@ -17,29 +17,34 @@
 
 package org.apache.nifi.serialization.record;
 
+import org.apache.nifi.serialization.SchemaValidationException;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.MapDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
+
 import java.nio.charset.StandardCharsets;
 import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Set;
 import java.util.function.Supplier;
 
-import org.apache.nifi.serialization.SchemaValidationException;
-import org.apache.nifi.serialization.record.type.ArrayDataType;
-import org.apache.nifi.serialization.record.type.MapDataType;
-import org.apache.nifi.serialization.record.util.DataTypeUtils;
-import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
-
 public class MapRecord implements Record {
     private RecordSchema schema;
     private final Map<String, Object> values;
     private Optional<SerializedForm> serializedForm;
     private final boolean checkTypes;
     private final boolean dropUnknownFields;
-
+    private Set<RecordField> inactiveFields = null;
 
     public MapRecord(final RecordSchema schema, final Map<String, Object> values) {
         this(schema, values, false, false);
@@ -304,11 +309,33 @@ public class MapRecord implements Record {
     }
 
     @Override
+    public Map<String, Object> toMap() {
+        return Collections.unmodifiableMap(values);
+    }
+
+    @Override
+    public void setValue(final RecordField field, final Object value) {
+        final Optional<RecordField> existingField = setValueAndGetField(field.getFieldName(), value);
+
+        if (!existingField.isPresent()) {
+            if (inactiveFields == null) {
+                inactiveFields = new LinkedHashSet<>();
+            }
+
+            inactiveFields.add(field);
+        }
+    }
+
+    @Override
     public void setValue(final String fieldName, final Object value) {
+        setValueAndGetField(fieldName, value);
+    }
+
+    private Optional<RecordField> setValueAndGetField(final String fieldName, final Object value) {
         final Optional<RecordField> field = getSchema().getField(fieldName);
         if (!field.isPresent()) {
             if (dropUnknownFields) {
-                return;
+                return field;
             }
 
             final Object previousValue = values.put(fieldName, value);
@@ -316,7 +343,7 @@ public class MapRecord implements Record {
                 serializedForm = Optional.empty();
             }
 
-            return;
+            return field;
         }
 
         final RecordField recordField = field.get();
@@ -325,6 +352,8 @@ public class MapRecord implements Record {
         if (!Objects.equals(coerced, previousValue)) {
             serializedForm = Optional.empty();
         }
+
+        return field;
     }
 
     @Override
@@ -406,6 +435,24 @@ public class MapRecord implements Record {
     }
 
     @Override
+    public void incorporateInactiveFields() {
+        if (inactiveFields == null) {
+            return;
+        }
+
+        final List<RecordField> allFields = new ArrayList<>(schema.getFieldCount() + inactiveFields.size());
+        allFields.addAll(schema.getFields());
+
+        for (final RecordField field : inactiveFields) {
+            if (!allFields.contains(field)) {
+                allFields.add(field);
+            }
+        }
+
+        this.schema = new SimpleRecordSchema(allFields);
+    }
+
+    @Override
     public Set<String> getRawFieldNames() {
         return values.keySet();
     }
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/Record.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/Record.java
index 1a89225..a916d29 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/Record.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/Record.java
@@ -17,12 +17,13 @@
 
 package org.apache.nifi.serialization.record;
 
+import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
+
 import java.util.Date;
+import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
-
 public interface Record {
 
     RecordSchema getSchema();
@@ -53,8 +54,8 @@ public interface Record {
      * default value is <code>null</code>. Note that all values for this Record will still be valid according
      * to this Record's Schema after this operation completes, as no type will be changed except to become more
      * lenient. However, if incorporating the other schema does modify this schema, then the schema text
-     * returned by {@link #getSchemaText()}, the schema format returned by {@link #getSchemaFormat()}, and
-     * the SchemaIdentifier returned by {@link #getIdentifier()} for this record's schema may all become Empty.
+     * returned by {@link RecordSchema#getSchemaText() getSchemaText()}, the schema format returned by {@link RecordSchema#getSchemaFormat() getSchemaFormat()}, and
+     * the SchemaIdentifier returned by {@link RecordSchema#getIdentifier() getIdentifier()} for this record's schema may all become Empty.
      *
      * @param other the other schema to incorporate into this Record's schema
      *
@@ -63,6 +64,14 @@ public interface Record {
     void incorporateSchema(RecordSchema other);
 
     /**
+     * Updates the Record's schema to incorporate all of the fields that were added via the {@link #setValue(RecordField, Object)}
+     * method that did not exist in the schema.
+     *
+     * @throws UnsupportedOperationException if this record does not support incorporating other fields
+     */
+    void incorporateInactiveFields();
+
+    /**
      * <p>
      * Returns a view of the the values of the fields in this Record. Note that this method returns values only for
      * those entries in the Record's schema. This allows the Record to guarantee that it will return the values in
@@ -120,6 +129,20 @@ public interface Record {
     void setValue(String fieldName, Object value);
 
     /**
+     * Updates the value of the given field to the given value. If the field specified is not present in this Record's schema,
+     * this method will track of the field as an 'inactive field', which can then be added into the Record's schema via the
+     * {@link #incorporateInactiveFields} method. This method should not be called after each invocation of {@link #setValue(RecordField, Object)}
+     * but rather should be called only once all updates to the Record have completed, in order to optimize performance.
+     *
+     * If this method changes any value in the Record, any {@link SerializedForm} that was provided will be removed (i.e., any
+     * subsequent call to {@link #getSerializedForm()}} will return an empty Optional).
+     *
+     * @param field the field to update
+     * @param value the value to set
+     */
+    void setValue(RecordField field, Object value);
+
+    /**
      * Updates the value of a the specified index of a field. If the field specified
      * is not present in this Record's schema, this method will do nothing. If the field specified
      * is not an Array, an IllegalArgumentException will be thrown. If the field specified is an array
@@ -164,4 +187,10 @@ public interface Record {
      * @return a Set that contains the names of all of the fields that are present in the Record
      */
     Set<String> getRawFieldNames();
+
+    /**
+     * Converts the Record into a Map whose keys are the same as the Record's field names and the values are the field values
+     * @return a Map that represents the values in the Record.
+     */
+    Map<String, Object> toMap();
 }
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordField.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordField.java
index 2c4954c..125a44d 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordField.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordField.java
@@ -71,7 +71,7 @@ public class RecordField {
 
         // If aliases is the empty set, don't bother with the expense of wrapping in an unmodifiableSet.
         Objects.requireNonNull(aliases);
-        if ((Set<?>) aliases == Collections.EMPTY_SET) {
+        if (aliases == Collections.EMPTY_SET) {
             this.aliases = aliases;
         } else {
             this.aliases = Collections.unmodifiableSet(aliases);
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
index 2e6b5d7..de9aa58 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
@@ -23,18 +23,16 @@ import org.apache.nifi.serialization.record.type.MapDataType;
 import org.apache.nifi.serialization.record.type.RecordDataType;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 public enum RecordFieldType {
     /**
-     * A String field type. Fields of this type use a {@code java.lang.String} value.
-     */
-    STRING("string"),
-
-    /**
      * A boolean field type. Fields of this type use a {@code boolean} value.
      */
     BOOLEAN("boolean"),
@@ -45,29 +43,24 @@ public enum RecordFieldType {
     BYTE("byte"),
 
     /**
-     * A char field type. Fields of this type use a {@code char} value.
-     */
-    CHAR("char"),
-
-    /**
      * A short field type. Fields of this type use a {@code short} value.
      */
-    SHORT("short"),
+    SHORT("short", BYTE),
 
     /**
      * An int field type. Fields of this type use an {@code int} value.
      */
-    INT("int"),
+    INT("int", SHORT, BYTE),
 
     /**
-     * A bigint field type. Fields of this type use a {@code java.math.BigInteger} value.
+     * A long field type. Fields of this type use a {@code long} value.
      */
-    BIGINT("bigint"),
+    LONG("long", SHORT, BYTE, INT),
 
     /**
-     * A long field type. Fields of this type use a {@code long} value.
+     * A bigint field type. Fields of this type use a {@code java.math.BigInteger} value.
      */
-    LONG("long"),
+    BIGINT("bigint", SHORT, BYTE, INT, LONG),
 
     /**
      * A float field type. Fields of this type use a {@code float} value.
@@ -77,7 +70,7 @@ public enum RecordFieldType {
     /**
      * A double field type. Fields of this type use a {@code double} value.
      */
-    DOUBLE("double"),
+    DOUBLE("double", FLOAT),
 
     /**
      * A date field type. Fields of this type use a {@code java.sql.Date} value.
@@ -95,6 +88,16 @@ public enum RecordFieldType {
     TIMESTAMP("timestamp", "yyyy-MM-dd HH:mm:ss"),
 
     /**
+     * A char field type. Fields of this type use a {@code char} value.
+     */
+    CHAR("char"),
+
+    /**
+     * A String field type. Fields of this type use a {@code java.lang.String} value.
+     */
+    STRING("string", BOOLEAN, BYTE, CHAR, SHORT, INT, BIGINT, LONG, FLOAT, DOUBLE, DATE, TIME, TIMESTAMP),
+
+    /**
      * <p>
      * A record field type. Fields of this type use a {@code org.apache.nifi.serialization.record.Record} value. A Record DataType should be
      * created by providing the {@link RecordSchema} for the record:
@@ -215,23 +218,38 @@ public enum RecordFieldType {
     private final String simpleName;
     private final String defaultFormat;
     private final DataType defaultDataType;
+    private final Set<RecordFieldType> narrowDataTypes;
 
     private RecordFieldType(final String simpleName) {
-        this(simpleName, null);
+        this.simpleName = simpleName;
+        this.defaultFormat = null;
+        this.defaultDataType = new DataType(this, defaultFormat);
+        this.narrowDataTypes = Collections.emptySet();
+    }
+
+    private RecordFieldType(final String simpleName, final RecordFieldType... narrowDataTypes) {
+        this.simpleName = simpleName;
+        this.defaultFormat = null;
+        this.defaultDataType = new DataType(this, defaultFormat);
+
+        this.narrowDataTypes = new HashSet<>(Arrays.asList(narrowDataTypes));
     }
 
     private RecordFieldType(final String simpleName, final String defaultFormat) {
         this.simpleName = simpleName;
         this.defaultFormat = defaultFormat;
         this.defaultDataType = new DataType(this, defaultFormat);
+        this.narrowDataTypes = Collections.emptySet();
     }
 
     private RecordFieldType(final String simpleName, final String defaultFormat, final DataType defaultDataType) {
         this.simpleName = simpleName;
         this.defaultFormat = defaultFormat;
         this.defaultDataType = defaultDataType;
+        this.narrowDataTypes = Collections.emptySet();
     }
 
+
     public String getDefaultFormat() {
         return defaultFormat;
     }
@@ -330,6 +348,18 @@ public enum RecordFieldType {
         return new MapDataType(valueDataType);
     }
 
+    /**
+     * Determines whether or this this RecordFieldType is "wider" than the provided type. A type "A" is said to be wider
+     * than another type "B" iff A encompasses all values of B and more. For example, the LONG type is wider than INT, and INT
+     * is wider than SHORT. "Complex" types (MAP, RECORD, ARRAY, CHOICE) are not wider than any other type, and no other type is
+     * wider than a complex type. The STRING type is wider than all types with the exception of complex types.
+     *
+     * @param fieldType the type to compare against
+     * @return <code>true</code> if <code>this</code> is wider than the provided type, <code>false</code> otherwise.
+     */
+    public boolean isWiderThan(final RecordFieldType fieldType) {
+        return narrowDataTypes.contains(fieldType);
+    }
 
     public static RecordFieldType of(final String typeString) {
       return SIMPLE_NAME_MAP.get(typeString);
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
index 46dc447..f9f2569 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
@@ -57,7 +57,7 @@ public class ArrayDataType extends DataType {
         }
 
         final ArrayDataType other = (ArrayDataType) obj;
-        return getFieldType().equals(other.getFieldType()) && Objects.equals(elementType, other.elementType);
+        return Objects.equals(elementType, other.elementType);
     }
 
     @Override
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
index 5ed1c39..6435195 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
@@ -57,11 +57,11 @@ public class MapDataType extends DataType {
         }
 
         final MapDataType other = (MapDataType) obj;
-        return getValueType().equals(other.getValueType()) && Objects.equals(valueType, other.valueType);
+        return Objects.equals(valueType, other.valueType);
     }
 
     @Override
     public String toString() {
-        return "MAP[" + valueType + "]";
+        return "MAP<" + valueType + ">";
     }
 }
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java
index f7e9631..af2044a 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java
@@ -58,7 +58,7 @@ public class RecordDataType extends DataType {
         }
 
         final RecordDataType other = (RecordDataType) obj;
-        return getFieldType().equals(other.getFieldType()) && Objects.equals(childSchema, other.childSchema);
+        return Objects.equals(childSchema, other.childSchema);
     }
 
     @Override
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
index f206a64..d6e4878 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
@@ -17,6 +17,20 @@
 
 package org.apache.nifi.serialization.record.util;
 
+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.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.ChoiceDataType;
+import org.apache.nifi.serialization.record.type.MapDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.math.BigInteger;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
@@ -31,6 +45,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -41,20 +56,6 @@ import java.util.function.Predicate;
 import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
-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.RecordFieldType;
-import org.apache.nifi.serialization.record.RecordSchema;
-import org.apache.nifi.serialization.record.type.ArrayDataType;
-import org.apache.nifi.serialization.record.type.ChoiceDataType;
-import org.apache.nifi.serialization.record.type.MapDataType;
-import org.apache.nifi.serialization.record.type.RecordDataType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 public class DataTypeUtils {
     private static final Logger logger = LoggerFactory.getLogger(DataTypeUtils.class);
 
@@ -279,7 +280,7 @@ public class DataTypeUtils {
             }
 
             final Map<?, ?> map = (Map<?, ?>) value;
-            final Map<String, Object> coercedValues = new HashMap<>();
+            final Map<String, Object> coercedValues = new LinkedHashMap<>();
 
             for (final Map.Entry<?, ?> entry : map.entrySet()) {
                 final Object keyValue = entry.getKey();
@@ -304,8 +305,214 @@ public class DataTypeUtils {
         throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Record for field " + fieldName);
     }
 
+    public static Record toRecord(final Object value, final String fieldName) {
+        return toRecord(value, fieldName, StandardCharsets.UTF_8);
+    }
+
+    public static RecordSchema inferSchema(final Map<String, Object> values, final String fieldName, final Charset charset) {
+        if (values == null) {
+            return null;
+        }
+
+        final List<RecordField> inferredFieldTypes = new ArrayList<>();
+        final Map<String, Object> coercedValues = new LinkedHashMap<>();
+
+        for (final Map.Entry<?, ?> entry : values.entrySet()) {
+            final Object keyValue = entry.getKey();
+            if (keyValue == null) {
+                continue;
+            }
+
+            final String key = keyValue.toString();
+            final Object rawValue = entry.getValue();
+            final DataType inferredDataType = inferDataType(rawValue, RecordFieldType.STRING.getDataType());
+
+            final RecordField recordField = new RecordField(key, inferredDataType, true);
+            inferredFieldTypes.add(recordField);
+
+            final Object coercedValue = convertType(rawValue, inferredDataType, fieldName, charset);
+            coercedValues.put(key, coercedValue);
+        }
+
+        final RecordSchema inferredSchema = new SimpleRecordSchema(inferredFieldTypes);
+        return inferredSchema;
+    }
+
+    public static Record toRecord(final Object value, final String fieldName, final Charset charset) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Record) {
+            return ((Record) value);
+        }
+
+        final List<RecordField> inferredFieldTypes = new ArrayList<>();
+        if (value instanceof Map) {
+            final Map<?, ?> map = (Map<?, ?>) value;
+            final Map<String, Object> coercedValues = new LinkedHashMap<>();
+
+            for (final Map.Entry<?, ?> entry : map.entrySet()) {
+                final Object keyValue = entry.getKey();
+                if (keyValue == null) {
+                    continue;
+                }
+
+                final String key = keyValue.toString();
+                final Object rawValue = entry.getValue();
+                final DataType inferredDataType = inferDataType(rawValue, RecordFieldType.STRING.getDataType());
+
+                final RecordField recordField = new RecordField(key, inferredDataType, true);
+                inferredFieldTypes.add(recordField);
+
+                final Object coercedValue = convertType(rawValue, inferredDataType, fieldName, charset);
+                coercedValues.put(key, coercedValue);
+            }
+
+            final RecordSchema inferredSchema = new SimpleRecordSchema(inferredFieldTypes);
+            return new MapRecord(inferredSchema, coercedValues);
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Record for field " + fieldName);
+    }
+
+    public static DataType inferDataType(final Object value, final DataType defaultType) {
+        if (value == null) {
+            return defaultType;
+        }
+
+        if (value instanceof String) {
+            return RecordFieldType.STRING.getDataType();
+        }
+
+        if (value instanceof Record) {
+            final RecordSchema schema = ((Record) value).getSchema();
+            return RecordFieldType.RECORD.getRecordDataType(schema);
+        }
+
+        if (value instanceof Number) {
+            if (value instanceof Long) {
+                return RecordFieldType.LONG.getDataType();
+            }
+            if (value instanceof Integer) {
+                return RecordFieldType.INT.getDataType();
+            }
+            if (value instanceof Short) {
+                return RecordFieldType.SHORT.getDataType();
+            }
+            if (value instanceof Byte) {
+                return RecordFieldType.BYTE.getDataType();
+            }
+            if (value instanceof Float) {
+                return RecordFieldType.FLOAT.getDataType();
+            }
+            if (value instanceof Double) {
+                return RecordFieldType.DOUBLE.getDataType();
+            }
+            if (value instanceof BigInteger) {
+                return RecordFieldType.BIGINT.getDataType();
+            }
+        }
+
+        if (value instanceof Boolean) {
+            return RecordFieldType.BOOLEAN.getDataType();
+        }
+        if (value instanceof java.sql.Time) {
+            return RecordFieldType.TIME.getDataType();
+        }
+        if (value instanceof java.sql.Timestamp) {
+            return RecordFieldType.TIMESTAMP.getDataType();
+        }
+        if (value instanceof java.util.Date) {
+            return RecordFieldType.DATE.getDataType();
+        }
+        if (value instanceof Character) {
+            return RecordFieldType.CHAR.getDataType();
+        }
+
+        // A value of a Map could be either a Record or a Map type. In either case, it must have Strings as keys.
+        if (value instanceof Map) {
+            final Map<String, ?> map = (Map<String, ?>) value;
+            return inferRecordDataType(map);
+//            // Check if all types are the same.
+//            if (map.isEmpty()) {
+//                return RecordFieldType.MAP.getMapDataType(RecordFieldType.STRING.getDataType());
+//            }
+//
+//            Object valueFromMap = null;
+//            Class<?> valueClass = null;
+//            for (final Object val : map.values()) {
+//                if (val == null) {
+//                    continue;
+//                }
+//
+//                valueFromMap = val;
+//                final Class<?> currentValClass = val.getClass();
+//                if (valueClass == null) {
+//                    valueClass = currentValClass;
+//                } else {
+//                    // If we have two elements that are of different types, then we cannot have a Map. Must be a Record.
+//                    if (valueClass != currentValClass) {
+//                        return inferRecordDataType(map);
+//                    }
+//                }
+//            }
+//
+//            // All values appear to be of the same type, so assume that it's a map.
+//            final DataType elementDataType = inferDataType(valueFromMap, RecordFieldType.STRING.getDataType());
+//            return RecordFieldType.MAP.getMapDataType(elementDataType);
+        }
+        if (value instanceof Object[]) {
+            final Object[] array = (Object[]) value;
+
+            DataType mergedDataType = null;
+            for (final Object arrayValue : array) {
+                final DataType inferredDataType = inferDataType(arrayValue, RecordFieldType.STRING.getDataType());
+                mergedDataType = mergeDataTypes(mergedDataType, inferredDataType);
+            }
+
+            if (mergedDataType == null) {
+                mergedDataType = RecordFieldType.STRING.getDataType();
+            }
+
+            return RecordFieldType.ARRAY.getArrayDataType(mergedDataType);
+        }
+        if (value instanceof Iterable) {
+            final Iterable iterable = (Iterable<?>) value;
+
+            DataType mergedDataType = null;
+            for (final Object arrayValue : iterable) {
+                final DataType inferredDataType = inferDataType(arrayValue, RecordFieldType.STRING.getDataType());
+                mergedDataType = mergeDataTypes(mergedDataType, inferredDataType);
+            }
+
+            if (mergedDataType == null) {
+                mergedDataType = RecordFieldType.STRING.getDataType();
+            }
+
+            return RecordFieldType.ARRAY.getArrayDataType(mergedDataType);
+        }
+
+        return defaultType;
+    }
+
+    private static DataType inferRecordDataType(final Map<String, ?> map) {
+        final List<RecordField> fields = new ArrayList<>(map.size());
+        for (final Map.Entry<String, ?> entry : map.entrySet()) {
+            final String key = entry.getKey();
+            final Object value = entry.getValue();
+
+            final DataType dataType = inferDataType(value, RecordFieldType.STRING.getDataType());
+            final RecordField field = new RecordField(key, dataType, true);
+            fields.add(field);
+        }
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+        return RecordFieldType.RECORD.getRecordDataType(schema);
+    }
+
     public static boolean isRecordTypeCompatible(final Object value) {
-        return value != null && value instanceof Record;
+        return value instanceof Record;
     }
 
     public static Object[] toArray(final Object value, final String fieldName, final DataType elementDataType) {
@@ -374,7 +581,7 @@ public class DataTypeUtils {
                 return (Map<String, Object>) value;
             }
 
-            final Map<String, Object> transformed = new HashMap<>();
+            final Map<String, Object> transformed = new LinkedHashMap<>();
             for (final Map.Entry<?, ?> entry : original.entrySet()) {
                 final Object key = entry.getKey();
                 if (key == null) {
@@ -395,7 +602,7 @@ public class DataTypeUtils {
                     + " because Record does not have an associated Schema");
             }
 
-            final Map<String, Object> map = new HashMap<>();
+            final Map<String, Object> map = new LinkedHashMap<>();
             for (final String recordFieldName : recordSchema.getFieldNames()) {
                 map.put(recordFieldName, record.getValue(recordFieldName));
             }
@@ -426,20 +633,20 @@ public class DataTypeUtils {
             if (recordSchema == null) {
                 throw new IllegalTypeConversionException("Cannot convert value of type Record to Map because Record does not have an associated Schema");
             }
-            final Map<String, Object> recordMap = new HashMap<>();
+
+            final Map<String, Object> recordMap = new LinkedHashMap<>();
             for (RecordField field : recordSchema.getFields()) {
                 final DataType fieldDataType = field.getDataType();
                 final String fieldName = field.getFieldName();
                 Object fieldValue = record.getValue(fieldName);
+
                 if (fieldValue == null) {
                     recordMap.put(fieldName, null);
                 } else if (isScalarValue(fieldDataType, fieldValue)) {
                     recordMap.put(fieldName, fieldValue);
-
                 } else if (fieldDataType instanceof RecordDataType) {
                     Record nestedRecord = (Record) fieldValue;
                     recordMap.put(fieldName, convertRecordFieldtoObject(nestedRecord, fieldDataType));
-
                 } else if (fieldDataType instanceof MapDataType) {
                     recordMap.put(fieldName, convertRecordMapToJavaMap((Map) fieldValue, ((MapDataType)fieldDataType).getValueType()));
 
@@ -452,7 +659,7 @@ public class DataTypeUtils {
             }
             return recordMap;
         } else if (value instanceof Map) {
-            return convertRecordMapToJavaMap((Map)value, ((MapDataType)dataType).getValueType());
+            return convertRecordMapToJavaMap((Map) value, ((MapDataType) dataType).getValueType());
         } else if (dataType != null && isScalarValue(dataType, value)) {
             return value;
         }
@@ -1186,14 +1393,103 @@ public class DataTypeUtils {
             defaultValue = thisField.getDefaultValue();
         }
 
-        final DataType dataType;
-        if (thisField.getDataType().equals(otherField.getDataType())) {
-            dataType = thisField.getDataType();
+        final DataType dataType = mergeDataTypes(thisField.getDataType(), otherField.getDataType());
+        return new RecordField(fieldName, dataType, defaultValue, aliases, thisField.isNullable() || otherField.isNullable());
+    }
+
+    public static DataType mergeDataTypes(final DataType thisDataType, final DataType otherDataType) {
+        if (thisDataType == null) {
+            return otherDataType;
+        }
+
+        if (otherDataType == null) {
+            return thisDataType;
+        }
+
+        if (thisDataType.equals(otherDataType)) {
+            return thisDataType;
         } else {
-            dataType = RecordFieldType.CHOICE.getChoiceDataType(thisField.getDataType(), otherField.getDataType());
+            // If one type is 'wider' than the other (such as an INT and a LONG), just use the wider type (LONG, in this case),
+            // rather than using a CHOICE of the two.
+            final Optional<DataType> widerType = getWiderType(thisDataType, otherDataType);
+            if (widerType.isPresent()) {
+                return widerType.get();
+            }
+
+            final Set<DataType> possibleTypes = new LinkedHashSet<>();
+            if (thisDataType.getFieldType() == RecordFieldType.CHOICE) {
+                possibleTypes.addAll(((ChoiceDataType) thisDataType).getPossibleSubTypes());
+            } else {
+                possibleTypes.add(thisDataType);
+            }
+
+            if (otherDataType.getFieldType() == RecordFieldType.CHOICE) {
+                possibleTypes.addAll(((ChoiceDataType) otherDataType).getPossibleSubTypes());
+            } else {
+                possibleTypes.add(otherDataType);
+            }
+
+            return RecordFieldType.CHOICE.getChoiceDataType(new ArrayList<>(possibleTypes));
         }
+    }
 
-        return new RecordField(fieldName, dataType, defaultValue, aliases, thisField.isNullable() || otherField.isNullable());
+    public static Optional<DataType> getWiderType(final DataType thisDataType, final DataType otherDataType) {
+        final RecordFieldType thisFieldType = thisDataType.getFieldType();
+        final RecordFieldType otherFieldType = otherDataType.getFieldType();
+
+        final int thisIntTypeValue = getIntegerTypeValue(thisFieldType);
+        final int otherIntTypeValue = getIntegerTypeValue(otherFieldType);
+        if (thisIntTypeValue > -1 && otherIntTypeValue > -1) {
+            if (thisIntTypeValue > otherIntTypeValue) {
+                return Optional.of(thisDataType);
+            }
+
+            return Optional.of(otherDataType);
+        }
+
+        switch (thisFieldType) {
+            case FLOAT:
+                if (otherFieldType == RecordFieldType.DOUBLE) {
+                    return Optional.of(otherDataType);
+                }
+                break;
+            case DOUBLE:
+                if (otherFieldType == RecordFieldType.FLOAT) {
+                    return Optional.of(thisDataType);
+                }
+                break;
+
+
+            case CHAR:
+                if (otherFieldType == RecordFieldType.STRING) {
+                    return Optional.of(otherDataType);
+                }
+                break;
+            case STRING:
+                if (otherFieldType == RecordFieldType.CHAR) {
+                    return Optional.of(thisDataType);
+                }
+                break;
+        }
+
+        return Optional.empty();
+    }
+
+    private static int getIntegerTypeValue(final RecordFieldType fieldType) {
+        switch (fieldType) {
+            case BIGINT:
+                return 4;
+            case LONG:
+                return 3;
+            case INT:
+                return 2;
+            case SHORT:
+                return 1;
+            case BYTE:
+                return 0;
+            default:
+                return -1;
+        }
     }
 
     public static boolean isScalarValue(final DataType dataType, final Object value) {
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
index f225bb4..dc36926 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
@@ -16,6 +16,19 @@
  */
 package org.apache.nifi.remote.io.socket.ssl;
 
+import org.apache.nifi.remote.exception.TransmissionDisabledException;
+import org.apache.nifi.remote.io.socket.BufferStateManager;
+import org.apache.nifi.remote.io.socket.BufferStateManager.Direction;
+import org.apache.nifi.security.util.CertificateUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLEngineResult;
+import javax.net.ssl.SSLEngineResult.Status;
+import javax.net.ssl.SSLHandshakeException;
+import javax.net.ssl.SSLPeerUnverifiedException;
 import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetAddress;
@@ -30,18 +43,6 @@ import java.security.cert.Certificate;
 import java.security.cert.CertificateException;
 import java.security.cert.X509Certificate;
 import java.util.concurrent.TimeUnit;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLEngine;
-import javax.net.ssl.SSLEngineResult;
-import javax.net.ssl.SSLEngineResult.Status;
-import javax.net.ssl.SSLHandshakeException;
-import javax.net.ssl.SSLPeerUnverifiedException;
-import org.apache.nifi.remote.exception.TransmissionDisabledException;
-import org.apache.nifi.remote.io.socket.BufferStateManager;
-import org.apache.nifi.remote.io.socket.BufferStateManager.Direction;
-import org.apache.nifi.security.util.CertificateUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class SSLSocketChannel implements Closeable {
 
@@ -582,7 +583,12 @@ public class SSLSocketChannel implements Closeable {
                     continue;
                 }
                 case CLOSED:
-                    throw new IOException("Channel is closed");
+                    copied = copyFromAppDataBuffer(buffer, offset, len);
+                    if (copied == 0) {
+                        return -1;
+                    }
+                    streamInManager.compact();
+                    return copied;
                 case OK: {
                     copied = copyFromAppDataBuffer(buffer, offset, len);
                     if (copied == 0) {
diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/DateTimeMatcher.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/DateTimeMatcher.java
new file mode 100644
index 0000000..2e35ec2
--- /dev/null
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/DateTimeMatcher.java
@@ -0,0 +1,68 @@
+/*
+ * 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.util.text;
+
+/**
+ * <p>
+ * A utility class that can be used to determine whether or not a String matches a given date/time format, as specified
+ * by the Time Format used in {@link java.text.SimpleDateFormat}. It is not uncommon to see code written along the lines of:
+ * </p>
+ *
+ * <code><pre>
+ * final String format = "yyyy/MM/dd HH:mm:ss.SSS";
+ * try {
+ *     new SimpleDateFormat(format).parse(text);
+ *     return true;
+ * } catch (Exception e) {
+ *     return false;
+ * }
+ * </pre></code>
+ *
+ * <p>
+ *     This approach, however, is frowned upon for two important reasons. Firstly, the performance is poor. A micro-benchmark that involves executing
+ *     the above code (even reusing the SimpleDateFormat object) to evaluate whether or not <code>text</code> is a timestamp took approximately 125-130 seconds
+ *     to iterate 1,000,000 times (after discarding the first 1,000,000 iterations as a 'warmup'). As a comparison, this utility takes about 8-11 seconds against
+ *     the same data and on the same machine.
+ * </p>
+ *
+ * <p>
+ *     Secondly, the above snippet has a very expensive side effect of throwing an Exception if the text does not match the format. This Exception is silently ignored,
+ *     but can have devastating effects on the JVM as a whole, as creating the Exception object can result in requiring a Safepoint, which means that all threads in the JVM
+ *     may be forced to pause.
+ * </p>
+ *
+ * <p>
+ *     Note, however, that this class is not intended to replace SimpleDateFormat, as it does not perform the actual parsing but instead only determines whether or not
+ *     a given input text matches the pattern, so that if it does, a SimpleDateFormat can be used parse the input.
+ * </p>
+ */
+public interface DateTimeMatcher {
+    /**
+     * Determines whether or not the text matches the pattern
+     * @param text the text to evaluate
+     * @return <code>true</code> if the text matches the pattern, <code>false</code> otherwise
+     */
+    boolean matches(String text);
+
+    static DateTimeMatcher compile(String format) {
+        if (format == null) {
+            return t -> false;
+        }
+
+        return new DateTimeMatcherCompiler().compile(format);
+    }
+}
diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/DateTimeMatcherCompiler.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/DateTimeMatcherCompiler.java
new file mode 100644
index 0000000..5bf854b
--- /dev/null
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/DateTimeMatcherCompiler.java
@@ -0,0 +1,56 @@
+/*
+ * 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.util.text;
+
+import java.util.ArrayList;
+import java.util.List;
+
+class DateTimeMatcherCompiler {
+
+    DateTimeMatcher compile(final String format) {
+        final RegexDateTimeMatcher regexMatcher = new RegexDateTimeMatcher.Compiler().compile(format);
+
+        final List<DateTimeMatcher> matchers = new ArrayList<>(4);
+
+        // Add a matcher that will filter out any input if it's too short or too long to match the regex.
+        // This allows us to save on the expense of evaluating the Regular Expression in some cases.
+        final int minLength = regexMatcher.getMinInputLength();
+        final int maxLength = regexMatcher.getMaxInputLength();
+        matchers.add(input -> input.length() >= minLength && input.length() <= maxLength);
+
+        // Look for common patterns in date/time formats that allow us to quickly determine if some input text
+        // will match the pattern. For example, a common date pattern is yyyy/MM/dd or MM/dd/yyyy. In the first
+        // case, we know that it is not going to match unless the first 4 characters of the input are digits.
+        // In the later case, we know that it will not match if the first 2 characters are not digits.
+        if (format.startsWith("yyyy")) {
+            matchers.add(new StartsWithDigitsDateTimeMatcher(4));
+        } else if (format.startsWith("yy") || format.startsWith("mm")) {
+            matchers.add(new StartsWithDigitsDateTimeMatcher(2));
+        } else if (format.startsWith("H") || format.startsWith("h")) {
+            matchers.add(new StartsWithDigitsDateTimeMatcher(1));
+        } else if (format.startsWith("M") && !format.startsWith("MMM")) {
+            // If format begins with M, it could be a number of a month name. So we have to check if it starts with at least 3 M's to determine if the month is a number of a name.
+            matchers.add(new StartsWithDigitsDateTimeMatcher(1));
+        }
+
+        matchers.add(regexMatcher);
+
+        // Use the SimpleDateFormatMatcher only if our regex matches. This allows us to parse the date only to guarantee that we are correct if we say that the input text matches.
+        matchers.add(new SimpleDateFormatMatcher(format));
+        return new ListDateTimeMatcher(matchers);
+    }
+}
diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/ListDateTimeMatcher.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/ListDateTimeMatcher.java
new file mode 100644
index 0000000..dd3e0d0
--- /dev/null
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/ListDateTimeMatcher.java
@@ -0,0 +1,44 @@
+/*
+ * 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.util.text;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * An implementation of the DateTimeMatcher that accepts in its constructor a List of delegate DateTimeMatchers.
+ * This matcher will return <code>true</code> if and only if ALL matchers in the constructor return <code>true</code> for
+ * the same input.
+ */
+class ListDateTimeMatcher implements DateTimeMatcher {
+    private final List<DateTimeMatcher> matchers;
+
+    public ListDateTimeMatcher(final List<DateTimeMatcher> matchers) {
+        this.matchers = new ArrayList<>(matchers);
+    }
+
+    @Override
+    public boolean matches(final String text) {
+        for (final DateTimeMatcher matcher : matchers) {
+            if (!matcher.matches(text)) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+}
diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/RegexDateTimeMatcher.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/RegexDateTimeMatcher.java
new file mode 100644
index 0000000..fe1b919
--- /dev/null
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/RegexDateTimeMatcher.java
@@ -0,0 +1,538 @@
+/*
+ * 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.util.text;
+
+import java.text.DateFormatSymbols;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class RegexDateTimeMatcher implements DateTimeMatcher {
+    private final Pattern pattern;
+    private final List<String> subPatterns;
+    private final int minLength;
+    private final int maxLength;
+
+    private RegexDateTimeMatcher(final Pattern pattern, final List<String> subPatterns, final int minLength, final int maxLength) {
+        this.pattern = pattern;
+        this.subPatterns = subPatterns;
+        this.minLength = minLength;
+        this.maxLength = maxLength;
+    }
+
+    public int getMinInputLength() {
+        return minLength;
+    }
+
+    public int getMaxInputLength() {
+        return maxLength;
+    }
+
+    @Override
+    public boolean matches(final String text) {
+        if (text.length() < minLength || text.length() > maxLength) {
+            return false;
+        }
+
+        return pattern.matcher(text).matches();
+    }
+
+    // This method is not used except in order to help debugging. If a pattern is not matching a given input, this can be used
+    // to help determine which part of the compiled regular expression is not matching the input
+    public String determineMismatch(final String text) {
+        for (int patternsToUse = subPatterns.size() - 1; patternsToUse >= 0; patternsToUse--) {
+            final StringBuilder sb = new StringBuilder();
+
+            for (int i=0; i < patternsToUse; i++) {
+                sb.append(subPatterns.get(i));
+            }
+
+            final String regex = "^" + sb.toString();
+            final Pattern pattern = Pattern.compile(regex);
+            final Matcher matcher = pattern.matcher(text);
+            final boolean found = matcher.find();
+            if (found) {
+                return "Longest Match: <" + matcher.group(0) + "> based on pattern <" + regex + ">. The following portions did not match: " + subPatterns.subList(patternsToUse, subPatterns.size());
+            }
+        }
+
+        return "Could not match any part of the pattern";
+    }
+
+
+    public static class Compiler {
+        private final List<String> patterns = new ArrayList<>();
+
+        private char currentPattern;
+        private int charCount;
+        private boolean patternStarted = false;
+
+        private static final String AMPM_PATTERN;
+        private static final String ERAS_PATTERN;
+        private static final String MONTHS_PATTERN;
+        private static final String LONG_WEEKDAY_PATTERN;
+        private static final String SHORT_WEEKDAY_PATTERN;
+        private static final String ZONE_NAME_PATTERN;
+
+        private static final LengthRange AMPM_RANGE;
+        private static final LengthRange ERAS_RANGE;
+        private static final LengthRange MONTH_NAME_RANGE;
+        private static final LengthRange LONG_WEEKDAY_RANGE;
+        private static final LengthRange SHORT_WEEKDAY_RANGE;
+        private static final LengthRange ZONE_NAME_RANGE;
+
+        private LengthRange range = new LengthRange(0, 0);
+
+        static {
+            final DateFormatSymbols dateFormatSymbols = DateFormatSymbols.getInstance(Locale.US);
+
+            final String[] ampm = dateFormatSymbols.getAmPmStrings();
+            AMPM_PATTERN = joinRegex(ampm);
+            AMPM_RANGE = lengthRange(ampm);
+
+            final String[] eras = dateFormatSymbols.getEras();
+            ERAS_PATTERN = joinRegex(eras);
+            ERAS_RANGE = lengthRange(eras);
+
+            final List<String> monthNames = new ArrayList<>();
+            monthNames.addAll(Arrays.asList(dateFormatSymbols.getMonths()));
+            monthNames.addAll(Arrays.asList(dateFormatSymbols.getShortMonths()));
+            final String[] monthNameArray = monthNames.toArray(new String[0]);
+            MONTHS_PATTERN = joinRegex(monthNameArray);
+            MONTH_NAME_RANGE = lengthRange(monthNameArray);
+
+            final String[] longWeekdays = dateFormatSymbols.getWeekdays();
+            LONG_WEEKDAY_PATTERN = joinRegex(longWeekdays);
+            LONG_WEEKDAY_RANGE = lengthRange(longWeekdays);
+
+            final String[] shortWeekdays = dateFormatSymbols.getShortWeekdays();
+            SHORT_WEEKDAY_PATTERN = joinRegex(shortWeekdays);
+            SHORT_WEEKDAY_RANGE = lengthRange(shortWeekdays);
+
+            int maxTimeZoneLength = 0;
+            final String[][] zoneStrings = dateFormatSymbols.getZoneStrings();
+            final StringBuilder zoneNamePatternBuilder = new StringBuilder();
+            for (final String[] zoneNames : zoneStrings) {
+                for (final String zoneName : zoneNames) {
+                    zoneNamePatternBuilder.append(Pattern.quote(zoneName)).append("|");
+
+                    maxTimeZoneLength = Math.max(maxTimeZoneLength, zoneName.length());
+                }
+            }
+
+            zoneNamePatternBuilder.deleteCharAt(zoneNamePatternBuilder.length() - 1);
+            ZONE_NAME_PATTERN = zoneNamePatternBuilder.toString();
+            ZONE_NAME_RANGE = new LengthRange(1, maxTimeZoneLength);
+        }
+
+        public RegexDateTimeMatcher compile(final String format) {
+            currentPattern = 0;
+            charCount = 0;
+
+            char lastChar = 0;
+
+            for (int i = 0; i < format.length(); i++) {
+                final char c = format.charAt(i);
+
+                if (c != lastChar) {
+                    endPattern();
+                } else {
+                    charCount++;
+                }
+
+                try {
+                    switch (c) {
+                        case '\'':
+                            i = copyText(format, i);
+                            break;
+                        default:
+                            if ((c >= 'A' && c <= 'Z') || (c >= 'a' && c <= 'z')) {
+                                if (c != lastChar) {
+                                    beginPattern(c);
+                                }
+
+                                continue;
+                            }
+                            appendChar(c);
+                            break;
+                    }
+                } finally {
+                    lastChar = c;
+                }
+            }
+
+            endPattern();
+
+            final StringBuilder sb = new StringBuilder();
+            for (final String pattern : patterns) {
+                sb.append(pattern);
+            }
+
+            final String regex = sb.toString();
+            final Pattern pattern = Pattern.compile(regex);
+            return new RegexDateTimeMatcher(pattern, patterns, range.getMinLength(), range.getMaxLength());
+        }
+
+
+        private static LengthRange lengthRange(final String[] values) {
+            return new LengthRange(minLength(values), maxLength(values));
+        }
+
+        private static int minLength(final String[] values) {
+            if (values.length == 0) {
+                return 0;
+            }
+
+            int minLength = values[0].length();
+            for (final String value : values) {
+                minLength = Math.min(minLength, value.length());
+            }
+
+            return minLength;
+        }
+
+        private static int maxLength(final String[] values) {
+            if (values.length == 0) {
+                return 0;
+            }
+
+            int maxLength = values[0].length();
+            for (final String value : values) {
+                maxLength = Math.max(maxLength, value.length());
+            }
+
+            return maxLength;
+        }
+
+        private static String joinRegex(final String[] values) {
+            final StringBuilder sb = new StringBuilder("(?:");
+
+            for (final String value : values) {
+                sb.append(Pattern.quote(value)).append("|");
+            }
+
+            sb.deleteCharAt(sb.length() - 1);
+            sb.append(")");
+            return sb.toString();
+        }
+
+        private int copyText(final String formatString, final int startChar) {
+            boolean lastCharQuote = false;
+
+            final StringBuilder textBuilder = new StringBuilder();
+
+            try {
+                for (int i = startChar + 1; i < formatString.length(); i++) {
+                    final char c = formatString.charAt(i);
+                    if (c == '\'') {
+                        // We found a quote char. If the last character is also a quote, then it was an escape character. Copy a single quote, set lastCharQuote = false because we've finished
+                        // the escape sequence, and then continue to the next character.
+                        if (lastCharQuote) {
+                            textBuilder.append("'");
+                            lastCharQuote = false;
+                            continue;
+                        }
+
+                        // We found a quote character. The last character is not a quote. This character may or may not be an escape character, so we have to move on to the next character to find out.
+                        lastCharQuote = true;
+                        continue;
+                    } else if (lastCharQuote) {
+                        // The current character is not a quote character but the last character was. This means that the last character was ending the quotation.
+                        return i - 1;
+                    }
+
+                    textBuilder.append(c);
+                    lastCharQuote = false;
+                }
+
+                return formatString.length();
+            } finally {
+                if (textBuilder.length() == 0) {
+                    patterns.add("'");
+                } else {
+                    final String text = textBuilder.toString();
+                    if (text.length() > 0) {
+                        patterns.add(Pattern.quote(textBuilder.toString()));
+                    }
+                }
+            }
+        }
+
+        private void beginPattern(final char c) {
+            this.patternStarted = true;
+            this.charCount = 1;
+            this.currentPattern = c;
+        }
+
+        private void appendChar(final char c) {
+            patterns.add(Pattern.quote(String.valueOf(c)));
+            range = range.plus(1, 1);
+        }
+
+        private void endPattern() {
+            if (!patternStarted) {
+                return;
+            }
+
+            patternStarted = false;
+            switch (currentPattern) {
+                case 'G':
+                    addEraDesignator();
+                    break;
+                case 'y':
+                case 'Y':
+                    if (this.charCount == 2) {
+                        addYear(2);
+                    } else {
+                        addYear(this.charCount);
+                    }
+                    break;
+                case 'M':
+                    if (this.charCount <= 2) {
+                        addShortMonth();
+                    } else {
+                        addLongMonth();
+                    }
+                    break;
+                case 'w':
+                    addWeekInYear();
+                    break;
+                case 'W':
+                    addWeekInMonth();
+                    break;
+                case 'D':
+                    addDayInYear();
+                    break;
+                case 'd':
+                    addDayInMonth();
+                    break;
+                case 'F':
+                    addDayOfWeekInMonth();
+                    break;
+                case 'E':
+                    if (this.charCount <= 3) {
+                        addShortDayNameInWeek();
+                    } else {
+                        addLongDayNameInWeek();
+                    }
+                    break;
+                case 'u':
+                    addDayNumberInWeek();
+                    break;
+                case 'a':
+                    addAmPmMarker();
+                    break;
+                case 'H':
+                    addHourInDayBaseZero();
+                    break;
+                case 'k':
+                    addHourInDayBaseOne();
+                    break;
+                case 'K':
+                    add12HourBaseZero();
+                    break;
+                case 'h':
+                    add12HourBaseOne();
+                    break;
+                case 'm':
+                    addMinuteInHour();
+                    break;
+                case 's':
+                    addSecondInMinute();
+                    break;
+                case 'S':
+                    addMillisecond();
+                    break;
+                case 'z':
+                    addGeneralTimeZone();
+                    break;
+                case 'Z':
+                    addRFC822TimeZone();
+                    break;
+                case 'X':
+                    addISO8601TimeZone();
+                    break;
+            }
+        }
+
+        private void addEraDesignator() {
+            patterns.add(ERAS_PATTERN);
+            range = range.plus(ERAS_RANGE);
+        }
+
+        private void addYear(final int maxDigits) {
+            patterns.add("(?:-?\\d{1," + maxDigits + "})");
+            range = range.plus(1, maxDigits);
+        }
+
+        private void addShortMonth() {
+            patterns.add("(?:0[1-9]|1[0-2])");
+            range = range.plus(1, 2);
+        }
+
+        private void addLongMonth() {
+            patterns.add("(?:" + MONTHS_PATTERN + ")");
+            range = range.plus(MONTH_NAME_RANGE);
+        }
+
+        private void addWeekInYear() {
+            patterns.add("\\d{1,2}");
+            range = range.plus(1, 4);
+        }
+
+        private void addWeekInMonth() {
+            patterns.add("[0-4]");
+            range = range.plus(1, 1);
+        }
+
+        private void addDayInYear() {
+            patterns.add("\\d{1,3}");
+            range = range.plus(1, 3);
+        }
+
+        private void addDayInMonth() {
+            patterns.add("[0-3]?[0-9]");
+            range = range.plus(1, 2);
+        }
+
+        private void addDayOfWeekInMonth() {
+            patterns.add("[0-7]");
+            range = range.plus(1, 1);
+        }
+
+        private void addShortDayNameInWeek() {
+            patterns.add(SHORT_WEEKDAY_PATTERN);
+            range = range.plus(SHORT_WEEKDAY_RANGE);
+        }
+
+        private void addLongDayNameInWeek() {
+            patterns.add(LONG_WEEKDAY_PATTERN);
+            range = range.plus(LONG_WEEKDAY_RANGE);
+        }
+
+        private void addDayNumberInWeek() {
+            patterns.add("[1-7]");
+            range = range.plus(1, 1);
+        }
+
+        private void addAmPmMarker() {
+            patterns.add(AMPM_PATTERN);
+            range = range.plus(AMPM_RANGE);
+        }
+
+        private void addHourInDayBaseZero() {
+            patterns.add("(?:[0-9]|[01][0-9]|2[0-3])");
+            range = range.plus(1, 2);
+        }
+
+        private void addHourInDayBaseOne() {
+            patterns.add("(?:[1-9]|0[1-9]|1[0-9]|2[0-4])");
+            range = range.plus(1, 2);
+        }
+
+        private void add12HourBaseZero() {
+            patterns.add("(?:0?[0-9]|1[01])");
+            range = range.plus(1, 2);
+        }
+
+        private void add12HourBaseOne() {
+            patterns.add("(?:[1-9]|0[1-9]|1[012])");
+            range = range.plus(1, 2);
+        }
+
+        private void addMinuteInHour() {
+            patterns.add("(?:[0-9]|[0-5][0-9])");
+            range = range.plus(1, 2);
+        }
+
+        private void addSecondInMinute() {
+            addMinuteInHour(); // Same pattern
+            range = range.plus(1, 2);
+        }
+
+        private void addMillisecond() {
+            patterns.add("\\d{1,3}");
+            range = range.plus(1, 3);
+        }
+
+        private void addGeneralTimeZone() {
+            final StringBuilder sb = new StringBuilder();
+
+            sb.append("(?:"); // begin non-capturing group
+            sb.append(getGMTOffsetTimeZone());
+            sb.append("|");
+            sb.append(getNamedTimeZone());
+            sb.append(")"); // end non-capturing group
+
+            patterns.add(sb.toString());
+            range = range.plus(ZONE_NAME_RANGE);
+        }
+
+        private String getGMTOffsetTimeZone() {
+            // From SimpleDateFormat JavaDocs, GMTOffsetTimeZone defined as: GMT Sign Hours : Minutes
+            // Sign defined as '-' or '+'
+            // Hours defined as 1 or 2 digits, Minutes defined as 1 or 2 digits
+            // Digit defined as number between 0-9
+            return "(?:GMT[-+]\\d{1,2}:\\d{2})";
+        }
+
+        private String getNamedTimeZone() {
+            return ZONE_NAME_PATTERN;
+        }
+
+        private void addRFC822TimeZone() {
+            patterns.add("(?:[-+]\\d{4})");
+            range = range.plus(5, 5);
+        }
+
+        private void addISO8601TimeZone() {
+            patterns.add("(?:Z|(?:[-+](?:\\d{2}|\\d{4}|\\d{2}\\:\\d{2})))");
+            range = range.plus(1, 6);
+        }
+
+
+        private static class LengthRange {
+            private final int min;
+            private final int max;
+
+            public LengthRange(final int min, final int max) {
+                this.min = min;
+                this.max = max;
+            }
+
+            public int getMinLength() {
+                return min;
+            }
+
+            public int getMaxLength() {
+                return max;
+            }
+
+            public LengthRange plus(final LengthRange other) {
+                return new LengthRange(getMinLength() + other.getMinLength(), getMaxLength() + other.getMaxLength());
+            }
+
+            public LengthRange plus(final int min, final int max) {
+                return new LengthRange(getMinLength() + min, getMaxLength() + max);
+            }
+
+        }
+    }
+}
diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/SimpleDateFormatMatcher.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/SimpleDateFormatMatcher.java
new file mode 100644
index 0000000..721d140
--- /dev/null
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/SimpleDateFormatMatcher.java
@@ -0,0 +1,38 @@
+/*
+ * 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.util.text;
+
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+
+class SimpleDateFormatMatcher implements DateTimeMatcher {
+    private final DateFormat dateFormat;
+
+    public SimpleDateFormatMatcher(final String format) {
+        this.dateFormat = new SimpleDateFormat(format);
+    }
+
+    @Override
+    public boolean matches(final String text) {
+        try {
+            dateFormat.parse(text);
+            return true;
+        } catch (final Exception e) {
+            return false;
+        }
+    }
+}
diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/StartsWithDigitsDateTimeMatcher.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/StartsWithDigitsDateTimeMatcher.java
new file mode 100644
index 0000000..2ccbac1
--- /dev/null
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/text/StartsWithDigitsDateTimeMatcher.java
@@ -0,0 +1,43 @@
+/*
+ * 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.util.text;
+
+/**
+ * A DateTimeMatcher that bases its decision purely on whether or not the first X number of characters in the input text are digits.
+ */
+public class StartsWithDigitsDateTimeMatcher implements DateTimeMatcher {
+    private final int expectedNumberOfDigits;
+
+    public StartsWithDigitsDateTimeMatcher(final int expectedNumberOfDigits) {
+        this.expectedNumberOfDigits = expectedNumberOfDigits;
+    }
+
+    @Override
+    public boolean matches(final String text) {
+        if (text.length() < expectedNumberOfDigits) {
+            return false;
+        }
+
+        for (int i=0; i < expectedNumberOfDigits; i++) {
+            if (!Character.isDigit(text.charAt(i))) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+}
diff --git a/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/text/TestRegexDateTimeMatcher.java b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/text/TestRegexDateTimeMatcher.java
new file mode 100644
index 0000000..4a5e168
--- /dev/null
+++ b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/text/TestRegexDateTimeMatcher.java
@@ -0,0 +1,69 @@
+/*
+ * 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.util.text;
+
+import org.junit.Test;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertTrue;
+
+public class TestRegexDateTimeMatcher {
+
+    @Test
+    public void testCommonFormatsExpectedToPass() {
+        final Map<String, String> exampleToPattern = new LinkedHashMap<>();
+
+        // Following examples are intended to test specific functions in the regex generation.
+        exampleToPattern.put("2018-12-12", "yyyy-MM-dd");
+        exampleToPattern.put("2018/12/12", "yyyy/MM/dd");
+        exampleToPattern.put("12/12/2018", "MM/dd/yyyy");
+        exampleToPattern.put("12/12/18", "MM/dd/yy");
+        exampleToPattern.put("1:40:55", "HH:mm:ss");
+        exampleToPattern.put("01:0:5", "HH:mm:ss");
+        exampleToPattern.put("12/12/2018 13:04:08 GMT-05:00", "MM/dd/yyyy HH:mm:ss z");
+        exampleToPattern.put("12/12/2018 13:04:08 -0500", "MM/dd/yyyy HH:mm:ss Z");
+        exampleToPattern.put("12/12/2018 13:04:08 EST", "MM/dd/yyyy HH:mm:ss zzzz");
+        exampleToPattern.put("12/12/2018 13:04:08 -05", "MM/dd/yyyy HH:mm:ss X");
+        exampleToPattern.put("0:08 PM", "K:mm a");
+        exampleToPattern.put("Dec 12, 2018", "MMM dd, yyyy");
+        exampleToPattern.put("12 Dec 2018", "dd MMM yyyy");
+        exampleToPattern.put("12 December 2018", "dd MMM yyyy");
+
+        // TODO: The following examples are taken from the SimpleDateFormat's JavaDoc. Ensure that this is not a licensing concern,
+        // since it is not being distributed.
+        exampleToPattern.put("2001.07.04 AD at 12:08:56 PDT", "yyyy.MM.dd G 'at' HH:mm:ss z");
+        exampleToPattern.put("Wed, Jul 4, '01", "EEE, MMM d, ''yy");
+        exampleToPattern.put("12:08 PM", "h:mm a");
+        exampleToPattern.put("12 o'clock PM, Pacific Daylight Time", "hh 'o''clock' a, zzzz");
+        exampleToPattern.put("0:08 PM, PDT", "K:mm a, z");
+        exampleToPattern.put("02001.July.04 AD 12:08 PM", "yyyyy.MMMMM.dd GGG hh:mm aaa");
+        exampleToPattern.put("Wed, 4 Jul 2001 12:08:56 -0700", "EEE, d MMM yyyy HH:mm:ss Z");
+        exampleToPattern.put("010704120856-0700", "yyMMddHHmmssZ");
+        exampleToPattern.put("2001-07-04T12:08:56.235-0700", "yyyy-MM-dd'T'HH:mm:ss.SSSZ");
+        exampleToPattern.put("2001-07-04T12:08:56.235-07:00", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX");
+        exampleToPattern.put("2001-W27-3", "YYYY-'W'ww-u");
+
+        for (final Map.Entry<String, String> entry : exampleToPattern.entrySet()) {
+            final RegexDateTimeMatcher matcher = new RegexDateTimeMatcher.Compiler().compile(entry.getValue());
+            final boolean matches = matcher.matches(entry.getKey());
+
+            assertTrue("Pattern <" + entry.getValue() + "> did not match <" + entry.getKey() + ">", matches);
+        }
+    }
+}
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
index f48b919..78ff117 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
@@ -16,6 +16,22 @@
  */
 package org.apache.nifi.util;
 
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.FlowFileFilter;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.FlowFileAccessException;
+import org.apache.nifi.processor.exception.FlowFileHandlingException;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.provenance.ProvenanceReporter;
+import org.junit.Assert;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
@@ -43,22 +59,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
-import org.apache.nifi.controller.queue.QueueSize;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.processor.FlowFileFilter;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.FlowFileAccessException;
-import org.apache.nifi.processor.exception.FlowFileHandlingException;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
-import org.apache.nifi.processor.io.OutputStreamCallback;
-import org.apache.nifi.processor.io.StreamCallback;
-import org.apache.nifi.provenance.ProvenanceReporter;
-import org.junit.Assert;
-
 public class MockProcessSession implements ProcessSession {
 
     private final Map<Relationship, List<MockFlowFile>> transferMap = new ConcurrentHashMap<>();
@@ -607,6 +607,16 @@ public class MockProcessSession implements ProcessSession {
             }
 
             @Override
+            public void mark(final int readlimit) {
+                bais.mark(readlimit);
+            }
+
+            @Override
+            public void reset() {
+                bais.reset();
+            }
+
+            @Override
             public String toString() {
                 return "ErrorHandlingInputStream[flowFile=" + mock + "]";
             }
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
index 2e2c8f7..043f7ab 100755
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
@@ -119,7 +119,17 @@ public class AvroTypeUtil {
 
     private static Field buildAvroField(final RecordField recordField) {
         final Schema schema = buildAvroSchema(recordField.getDataType(), recordField.getFieldName(), recordField.isNullable());
-        final Field field = new Field(recordField.getFieldName(), schema, null, recordField.getDefaultValue());
+
+        final Field field;
+        final String recordFieldName = recordField.getFieldName();
+        if (isValidAvroFieldName(recordFieldName)) {
+            field = new Field(recordField.getFieldName(), schema, null, recordField.getDefaultValue());
+        } else {
+            final String validName = createValidAvroFieldName(recordField.getFieldName());
+            field = new Field(validName, schema, null, recordField.getDefaultValue());
+            field.addAlias(recordField.getFieldName());
+        }
+
         for (final String alias : recordField.getAliases()) {
             field.addAlias(alias);
         }
@@ -127,6 +137,56 @@ public class AvroTypeUtil {
         return field;
     }
 
+    private static boolean isValidAvroFieldName(final String fieldName) {
+        // Avro field names must match the following criteria:
+        // 1. Must be non-empty
+        // 2. Must begin with a letter or an underscore
+        // 3. Must consist only of letters, underscores, and numbers.
+        if (fieldName.isEmpty()) {
+            return false;
+        }
+
+        final char firstChar = fieldName.charAt(0);
+        if (firstChar != '_' && !Character.isLetter(firstChar)) {
+            return false;
+        }
+
+        for (int i=1; i < fieldName.length(); i++) {
+            final char c = fieldName.charAt(i);
+            if (c != '_' && !Character.isLetterOrDigit(c)) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static String createValidAvroFieldName(final String fieldName) {
+        if (fieldName.isEmpty()) {
+            return "UNNAMED_FIELD";
+        }
+
+        final StringBuilder sb = new StringBuilder();
+
+        final char firstChar = fieldName.charAt(0);
+        if (firstChar == '_' || Character.isLetter(firstChar)) {
+            sb.append(firstChar);
+        } else {
+            sb.append("_");
+        }
+
+        for (int i=1; i < fieldName.length(); i++) {
+            final char c = fieldName.charAt(i);
+            if (c == '_' || Character.isLetterOrDigit(c)) {
+                sb.append(c);
+            } else {
+                sb.append("_");
+            }
+        }
+
+        return sb.toString();
+    }
+
     private static Schema buildAvroSchema(final DataType dataType, final String fieldName, final boolean nullable) {
         final Schema schema;
 
@@ -462,7 +522,7 @@ public class AvroTypeUtil {
         Field field = avroSchema.getField(fieldName);
         if (field == null) {
             // No straight mapping was found, so check the aliases to see if it can be mapped
-            for(final String alias: recordField.getAliases()) {
+            for (final String alias : recordField.getAliases()) {
                 field = avroSchema.getField(alias);
                 if (field != null) {
                     fieldName = alias;
@@ -471,6 +531,28 @@ public class AvroTypeUtil {
             }
         }
 
+        if (field == null) {
+            for (final Field childField : avroSchema.getFields()) {
+                final Set<String> aliases = childField.aliases();
+                if (aliases.isEmpty()) {
+                    continue;
+                }
+
+                if (aliases.contains(fieldName)) {
+                    field = childField;
+                    break;
+                }
+
+                for (final String alias : recordField.getAliases()) {
+                    if (aliases.contains(alias)) {
+                        field = childField;
+                        fieldName = alias;
+                        break;
+                    }
+                }
+            }
+        }
+
         return new ImmutablePair<>(fieldName, field);
     }
 
@@ -493,7 +575,7 @@ public class AvroTypeUtil {
             }
 
             final Object converted = convertToAvroObject(rawValue, field.schema(), fieldName, charset);
-            rec.put(fieldName, converted);
+            rec.put(field.name(), converted);
         }
 
         // see if the Avro schema has any fields that aren't in the RecordSchema, and if those fields have a default
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/JsonInferenceSchemaRegistryService.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/JsonInferenceSchemaRegistryService.java
index b3819cf..73d4185 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/JsonInferenceSchemaRegistryService.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/JsonInferenceSchemaRegistryService.java
@@ -21,6 +21,7 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.schema.access.AvroSchemaTextStrategy;
 import org.apache.nifi.schema.access.InferenceSchemaStrategy;
@@ -60,7 +61,7 @@ public class JsonInferenceSchemaRegistryService extends SchemaRegistryService {
     }
 
     @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final ConfigurationContext context) {
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final PropertyContext context) {
         if (strategy == null) {
             return null;
         }
@@ -102,7 +103,7 @@ public class JsonInferenceSchemaRegistryService extends SchemaRegistryService {
         if (schemaAccess.equalsIgnoreCase(SCHEMA_NAME_PROPERTY.getValue()) || schemaAccess.equalsIgnoreCase(SCHEMA_TEXT_PROPERTY.getValue())) {
             return getSchema(variables, readSchema);
         } else {
-            return ((JsonSchemaAccessStrategy)schemaAccessStrategy).getSchema(variables, content, readSchema);
+            return ((JsonSchemaAccessStrategy) schemaAccessStrategy).getSchema(variables, content, readSchema);
         }
     }
 }
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java
index 6923f48..1beb50e 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java
@@ -22,6 +22,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.schema.access.SchemaAccessStrategy;
@@ -43,17 +44,17 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.nifi.schema.access.SchemaAccessUtils.CONFLUENT_ENCODED_SCHEMA;
 import static org.apache.nifi.schema.access.SchemaAccessUtils.HWX_CONTENT_ENCODED_SCHEMA;
 import static org.apache.nifi.schema.access.SchemaAccessUtils.HWX_SCHEMA_REF_ATTRIBUTES;
 import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY;
-import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME;
 import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_BRANCH_NAME;
-import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_VERSION;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME;
 import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME_PROPERTY;
 import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REGISTRY;
 import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT;
 import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT_PROPERTY;
-import static org.apache.nifi.schema.access.SchemaAccessUtils.CONFLUENT_ENCODED_SCHEMA;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_VERSION;
 
 public abstract class SchemaRegistryService extends AbstractControllerService {
 
@@ -151,15 +152,7 @@ public abstract class SchemaRegistryService extends AbstractControllerService {
         return suppliedFields;
     }
 
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final ConfigurationContext context) {
-        if (strategy == null) {
-            return null;
-        }
-
-        return SchemaAccessUtils.getSchemaAccessStrategy(strategy, schemaRegistry, context);
-    }
-
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry, final ValidationContext context) {
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry, final PropertyContext context) {
         if (allowableValue == null) {
             return null;
         }
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java
index 1f6c29b..004999f 100755
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java
@@ -17,23 +17,6 @@
 
 package org.apache.nifi.avro;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
 import org.apache.avro.Conversions;
 import org.apache.avro.LogicalTypes;
 import org.apache.avro.Schema;
@@ -41,11 +24,11 @@ import org.apache.avro.Schema.Field;
 import org.apache.avro.Schema.Type;
 import org.apache.avro.file.DataFileStream;
 import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericData.Record;
 import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.generic.GenericFixed;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.GenericRecordBuilder;
-import org.apache.avro.generic.GenericData.Record;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.DataType;
@@ -56,6 +39,25 @@ import org.apache.nifi.serialization.record.type.RecordDataType;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class TestAvroTypeUtil {
 
     @Test
@@ -450,4 +452,40 @@ public class TestAvroTypeUtil {
         assertTrue(o instanceof String);
         assertEquals("Hello", o);
     }
+
+    @Test
+    public void testAliasCreatedForInvalidField() {
+       final List<RecordField> fields = new ArrayList<>();
+       fields.add(new RecordField("valid", RecordFieldType.STRING.getDataType()));
+       fields.add(new RecordField("$invalid2", RecordFieldType.STRING.getDataType()));
+       fields.add(new RecordField("3invalid3", RecordFieldType.STRING.getDataType()));
+       fields.add(new RecordField("  __ Another ONE!!", RecordFieldType.STRING.getDataType()));
+
+       final RecordSchema recordSchema = new SimpleRecordSchema(fields);
+
+       final Schema avroSchema = AvroTypeUtil.extractAvroSchema(recordSchema);
+       assertNotNull(avroSchema.getField("valid"));
+
+       assertNull(avroSchema.getField("$invalid"));
+       final Field field2 = avroSchema.getField("_invalid2");
+       assertNotNull(field2);
+       assertEquals("_invalid2", field2.name());
+       assertEquals(1, field2.aliases().size());
+       assertTrue(field2.aliases().contains("$invalid2"));
+
+        assertNull(avroSchema.getField("$invalid3"));
+        final Field field3 = avroSchema.getField("_invalid3");
+        assertNotNull(field3);
+        assertEquals("_invalid3", field3.name());
+        assertEquals(1, field3.aliases().size());
+        assertTrue(field3.aliases().contains("3invalid3"));
+
+        assertNull(avroSchema.getField("  __ Another ONE!!"));
+        final Field field4 = avroSchema.getField("_____Another_ONE__");
+        assertNotNull(field4);
+        assertEquals("_____Another_ONE__", field4.name());
+        assertEquals(1, field4.aliases().size());
+        assertTrue(field4.aliases().contains("  __ Another ONE!!"));
+
+    }
 }
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractPutHDFSRecord.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractPutHDFSRecord.java
index 22f61b2..62e7231 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractPutHDFSRecord.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-hadoop-record-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractPutHDFSRecord.java
@@ -16,21 +16,6 @@
  */
 package org.apache.nifi.processors.hadoop;
 
-import java.io.BufferedInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.security.PrivilegedAction;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -63,6 +48,20 @@ import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 import org.apache.nifi.util.StopWatch;
 
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.PrivilegedAction;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
 /**
  * Base class for processors that write Records to HDFS.
  */
@@ -302,12 +301,11 @@ public abstract class AbstractPutHDFSRecord extends AbstractHadoopProcessor {
                 final StopWatch stopWatch = new StopWatch(true);
 
                 // Read records from the incoming FlowFile and write them the tempFile
-                session.read(putFlowFile, (final InputStream rawIn) -> {
+                session.read(putFlowFile, (final InputStream in) -> {
                     RecordReader recordReader = null;
                     HDFSRecordWriter recordWriter = null;
 
-                    try (final BufferedInputStream in = new BufferedInputStream(rawIn)) {
-
+                    try {
                         // if we fail to create the RecordReader then we want to route to failure, so we need to
                         // handle this separately from the other IOExceptions which normally route to retry
                         try {
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SSLSocketChannelRecordReader.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SSLSocketChannelRecordReader.java
index 873297e..12a3f3e 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SSLSocketChannelRecordReader.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/record/listen/SSLSocketChannelRecordReader.java
@@ -25,6 +25,7 @@ import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
 import org.apache.nifi.serialization.RecordReaderFactory;
 
+import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.InetAddress;
@@ -58,7 +59,8 @@ public class SSLSocketChannelRecordReader implements SocketChannelRecordReader {
             throw new IllegalStateException("Cannot create RecordReader because already created");
         }
 
-        final InputStream in = new SSLSocketChannelInputStream(sslSocketChannel);
+        final InputStream socketIn = new SSLSocketChannelInputStream(sslSocketChannel);
+        final InputStream in = new BufferedInputStream(socketIn);
         recordReader = readerFactory.createRecordReader(flowFile, in, logger);
         return recordReader;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index 216449c..071a052 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@ -25,6 +25,7 @@ import org.apache.nifi.controller.queue.QueueSize;
 import org.apache.nifi.controller.repository.claim.ContentClaim;
 import org.apache.nifi.controller.repository.claim.ContentClaimWriteCache;
 import org.apache.nifi.controller.repository.claim.ResourceClaim;
+import org.apache.nifi.controller.repository.io.ContentClaimInputStream;
 import org.apache.nifi.controller.repository.io.DisableOnCloseInputStream;
 import org.apache.nifi.controller.repository.io.DisableOnCloseOutputStream;
 import org.apache.nifi.controller.repository.io.FlowFileAccessInputStream;
@@ -144,7 +145,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
     private long contentSizeIn = 0L, contentSizeOut = 0L;
 
     private ContentClaim currentReadClaim = null;
-    private ByteCountingInputStream currentReadClaimStream = null;
+    private ContentClaimInputStream currentReadClaimStream = null;
     private long processingStartTime;
 
     // List of InputStreams that have been opened by calls to {@link #read(FlowFile)} and not yet closed
@@ -2183,36 +2184,22 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
                 }
 
                 claimCache.flush(claim);
-                final InputStream rawInStream = context.getContentRepository().read(claim);
 
                 if (currentReadClaimStream != null) {
                     currentReadClaimStream.close();
                 }
 
                 currentReadClaim = claim;
-
-                currentReadClaimStream = new ByteCountingInputStream(rawInStream);
-                StreamUtils.skip(currentReadClaimStream, offset);
+                currentReadClaimStream = new ContentClaimInputStream(context.getContentRepository(), claim, offset);
 
                 // Use a non-closeable stream because we want to keep it open after the callback has finished so that we can
                 // reuse the same InputStream for the next FlowFile
                 final InputStream disableOnClose = new DisableOnCloseInputStream(currentReadClaimStream);
-
                 return disableOnClose;
             } else {
                 claimCache.flush(claim);
-                final InputStream rawInStream = context.getContentRepository().read(claim);
-                try {
-                    StreamUtils.skip(rawInStream, offset);
-                } catch(IOException ioe) {
-                    try {
-                        rawInStream.close();
-                    } catch (final Exception e) {
-                        ioe.addSuppressed(ioe);
-                    }
 
-                    throw ioe;
-                }
+                final InputStream rawInStream = new ContentClaimInputStream(context.getContentRepository(), claim, offset);
                 return rawInStream;
             }
         } catch (final ContentNotFoundException cnfe) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ContentClaimInputStream.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ContentClaimInputStream.java
new file mode 100644
index 0000000..94b9d2e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ContentClaimInputStream.java
@@ -0,0 +1,158 @@
+/*
+ * 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.controller.repository.io;
+
+import org.apache.nifi.controller.repository.ContentRepository;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.stream.io.StreamUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * An InputStream that is provided a Content Repository, Content Claim, and offset into the Content Claim where a FlowFile's
+ * content begins, and is capable of reading the content from the Content Repository, as well as providing the ability to use
+ * {@link #mark(int)}/{@link #reset()} in order to re-read content without buffering it.
+ */
+public class ContentClaimInputStream extends InputStream {
+    private final ContentRepository contentRepository;
+    private final ContentClaim contentClaim;
+    private final long claimOffset;
+
+    private InputStream delegate;
+    private long bytesConsumed;
+    private long currentOffset; // offset into the Content Claim; will differ from bytesRead if reset() is called after reading at least one byte or if claimOffset > 0
+    private long markOffset;
+
+    public ContentClaimInputStream(final ContentRepository contentRepository, final ContentClaim contentClaim, final long claimOffset) {
+        this.contentRepository = contentRepository;
+        this.contentClaim = contentClaim;
+        this.claimOffset = claimOffset;
+
+        this.currentOffset = claimOffset;
+    }
+
+    private InputStream getDelegate() throws IOException {
+        if (delegate == null) {
+            formDelegate();
+        }
+
+        return delegate;
+    }
+
+    public long getBytesConsumed() {
+        return bytesConsumed;
+    }
+
+    public long getCurrentOffset() {
+        return currentOffset;
+    }
+
+    @Override
+    public int read() throws IOException {
+        final int value = getDelegate().read();
+        if (value != -1) {
+            bytesConsumed++;
+            currentOffset++;
+        }
+
+        return value;
+    }
+
+    @Override
+    public int read(final byte[] b) throws IOException {
+        final int count = getDelegate().read(b);
+        if (count != -1) {
+            bytesConsumed += count;
+            currentOffset += count;
+        }
+
+        return count;
+    }
+
+    @Override
+    public int read(final byte[] b, final int off, final int len) throws IOException {
+        final int count = getDelegate().read(b, off, len);
+        if (count != -1) {
+            bytesConsumed += count;
+            currentOffset += count;
+        }
+
+        return count;
+    }
+
+    @Override
+    public long skip(final long n) throws IOException {
+        final long count = getDelegate().skip(n);
+        if (count > 0) {
+            bytesConsumed += count;
+            currentOffset += count;
+        }
+
+        return count;
+    }
+
+    @Override
+    public int available() throws IOException {
+        if (delegate == null) {
+            return 0;
+        }
+
+        return delegate.available();
+    }
+
+    @Override
+    public boolean markSupported() {
+        return true;
+    }
+
+    @Override
+    public void mark(final int readlimit) {
+        markOffset = currentOffset;
+    }
+
+    @Override
+    public void reset() throws IOException {
+        if (markOffset < 0) {
+            throw new IOException("Stream has not been marked");
+        }
+
+        if (currentOffset != markOffset) {
+            delegate.close();
+            formDelegate();
+            StreamUtils.skip(delegate, markOffset - claimOffset);
+            currentOffset = markOffset;
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (delegate != null) {
+            delegate.close();
+        }
+    }
+
+    private void formDelegate() throws IOException {
+        if (delegate != null) {
+            delegate.close();
+        }
+
+        delegate = contentRepository.read(contentClaim);
+        StreamUtils.skip(delegate, claimOffset);
+        currentOffset = claimOffset;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardConfigurationContext.java
index 3c6a003..3ab7486 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardConfigurationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardConfigurationContext.java
@@ -16,22 +16,22 @@
  */
 package org.apache.nifi.controller.service;
 
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.nifi.attribute.expression.language.PreparedQuery;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
-import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ComponentNode;
+import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.util.FormatUtils;
 
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 public class StandardConfigurationContext implements ConfigurationContext {
 
     private final ComponentNode component;
@@ -74,7 +74,17 @@ public class StandardConfigurationContext implements ConfigurationContext {
     @Override
     public PropertyValue getProperty(final PropertyDescriptor property) {
         final String configuredValue = component.getProperty(property);
-        return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, serviceLookup, preparedQueries.get(property), variableRegistry);
+        final String resolvedValue = (configuredValue == null) ? property.getDefaultValue() : configuredValue;
+
+        if (resolvedValue == null) {
+            // We need to get the 'canonical representation' of the property descriptor from the component itself,
+            // since the supplied PropertyDescriptor may have been built using only the name, and without the proper
+            // default value.
+            final PropertyDescriptor resolvedDescriptor = component.getPropertyDescriptor(property.getName());
+            return new StandardPropertyValue(resolvedDescriptor.getDefaultValue(), serviceLookup, preparedQueries.get(property), variableRegistry);
+        }
+
+        return new StandardPropertyValue(resolvedValue, serviceLookup, preparedQueries.get(property), variableRegistry);
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
index 1880864..d21cc5c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
@@ -1223,6 +1223,7 @@ public class TestStandardProcessSession {
             session.read(ff1, new InputStreamCallback() {
                 @Override
                 public void process(InputStream in) throws IOException {
+                    in.read();
                 }
             });
             Assert.fail("Expected MissingFlowFileException");
@@ -1401,6 +1402,7 @@ public class TestStandardProcessSession {
             session.write(ff1, new StreamCallback() {
                 @Override
                 public void process(InputStream in, OutputStream out) throws IOException {
+                    in.read();
                 }
             });
             Assert.fail("Expected MissingFlowFileException");
@@ -1444,6 +1446,7 @@ public class TestStandardProcessSession {
             session.write(ff2, new StreamCallback() {
                 @Override
                 public void process(InputStream in, OutputStream out) throws IOException {
+                    in.read();
                 }
             });
             Assert.fail("Expected ContentNotFoundException");
@@ -1486,6 +1489,7 @@ public class TestStandardProcessSession {
             session.read(ff2, new InputStreamCallback() {
                 @Override
                 public void process(InputStream in) throws IOException {
+                    in.read();
                 }
             });
             Assert.fail("Expected MissingFlowFileException");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/io/TestContentClaimInputStream.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/io/TestContentClaimInputStream.java
new file mode 100644
index 0000000..4cba0b3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/io/TestContentClaimInputStream.java
@@ -0,0 +1,192 @@
+/*
+ * 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.controller.repository.io;
+
+import org.apache.nifi.controller.repository.ContentRepository;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class TestContentClaimInputStream {
+
+    private ContentRepository repo;
+    private ContentClaim contentClaim;
+    private AtomicBoolean closed = new AtomicBoolean();
+
+    @Before
+    public void setup() throws IOException {
+        repo = mock(ContentRepository.class);
+        contentClaim = mock(ContentClaim.class);
+
+        closed.set(false);
+        Mockito.when(repo.read(contentClaim)).thenAnswer(invocation -> new ByteArrayInputStream("hello".getBytes()) {
+            @Override
+            public void close() throws IOException {
+                super.close();
+                closed.set(true);
+            }
+        });
+    }
+
+
+    @Test
+    public void testStreamCreatedFromRepository() throws IOException {
+        final ContentClaimInputStream in = new ContentClaimInputStream(repo, contentClaim, 0L);
+
+        final byte[] buff = new byte[5];
+        StreamUtils.fillBuffer(in, buff);
+
+        Mockito.verify(repo, Mockito.times(1)).read(contentClaim);
+        Mockito.verifyNoMoreInteractions(repo);
+
+        final String contentRead = new String(buff);
+        assertEquals("hello", contentRead);
+
+        assertEquals(5, in.getBytesConsumed());
+        assertFalse(closed.get());
+
+        // Ensure that underlying stream is closed
+        in.close();
+        assertTrue(closed.get());
+    }
+
+
+    @Test
+    public void testThatContentIsSkipped() throws IOException {
+        final ContentClaimInputStream in = new ContentClaimInputStream(repo, contentClaim, 3L);
+
+        final byte[] buff = new byte[2];
+        StreamUtils.fillBuffer(in, buff);
+
+        Mockito.verify(repo, Mockito.times(1)).read(contentClaim);
+        Mockito.verifyNoMoreInteractions(repo);
+
+        final String contentRead = new String(buff);
+        assertEquals("lo", contentRead);
+
+        assertEquals(2, in.getBytesConsumed());
+        assertFalse(closed.get());
+
+        // Ensure that underlying stream is closed
+        in.close();
+        assertTrue(closed.get());
+    }
+
+
+    @Test
+    public void testRereadEntireClaim() throws IOException {
+        final ContentClaimInputStream in = new ContentClaimInputStream(repo, contentClaim, 0L);
+
+        final byte[] buff = new byte[5];
+
+        final int invocations = 10;
+        for (int i=0; i < invocations; i++) {
+            in.mark(5);
+
+            StreamUtils.fillBuffer(in, buff, true);
+
+            final String contentRead = new String(buff);
+            assertEquals("hello", contentRead);
+
+            assertEquals(5 * (i+1), in.getBytesConsumed());
+            assertEquals(5, in.getCurrentOffset());
+            assertEquals(-1, in.read());
+
+            in.reset();
+        }
+
+        Mockito.verify(repo, Mockito.times(invocations + 1)).read(contentClaim); // Will call reset() 'invocations' times plus the initial read
+        Mockito.verifyNoMoreInteractions(repo);
+
+        // Ensure that underlying stream is closed
+        in.close();
+        assertTrue(closed.get());
+    }
+
+
+    @Test
+    public void testMultipleResetCallsAfterMark() throws IOException {
+        final ContentClaimInputStream in = new ContentClaimInputStream(repo, contentClaim, 0L);
+
+        final byte[] buff = new byte[5];
+
+        final int invocations = 10;
+        in.mark(5);
+
+        for (int i=0; i < invocations; i++) {
+            StreamUtils.fillBuffer(in, buff, true);
+
+            final String contentRead = new String(buff);
+            assertEquals("hello", contentRead);
+
+            assertEquals(5 * (i+1), in.getBytesConsumed());
+            assertEquals(5, in.getCurrentOffset());
+            assertEquals(-1, in.read());
+
+            in.reset();
+        }
+
+        Mockito.verify(repo, Mockito.times(invocations + 1)).read(contentClaim); // Will call reset() 'invocations' times plus the initial read
+        Mockito.verifyNoMoreInteractions(repo);
+
+        // Ensure that underlying stream is closed
+        in.close();
+        assertTrue(closed.get());
+    }
+
+
+    @Test
+    public void testRereadWithOffset() throws IOException {
+        final ContentClaimInputStream in = new ContentClaimInputStream(repo, contentClaim, 3L);
+
+        final byte[] buff = new byte[2];
+
+        final int invocations = 10;
+        for (int i=0; i < invocations; i++) {
+            in.mark(5);
+
+            StreamUtils.fillBuffer(in, buff, true);
+
+            final String contentRead = new String(buff);
+            assertEquals("lo", contentRead);
+
+            assertEquals(2 * (i+1), in.getBytesConsumed());
+            assertEquals(5, in.getCurrentOffset());
+            assertEquals(-1, in.read());
+
+            in.reset();
+        }
+
+        Mockito.verify(repo, Mockito.times(invocations + 1)).read(contentClaim); // Will call reset() 'invocations' times plus the initial read
+        Mockito.verifyNoMoreInteractions(repo);
+
+        // Ensure that underlying stream is closed
+        in.close();
+        assertTrue(closed.get());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java
index affbe11..5f3445a 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hive.streaming.ConnectionError;
+import org.apache.hive.streaming.HiveRecordWriter;
 import org.apache.hive.streaming.HiveStreamingConnection;
 import org.apache.hive.streaming.InvalidTable;
 import org.apache.hive.streaming.SerializationError;
@@ -59,11 +60,9 @@ import org.apache.nifi.util.StringUtils;
 import org.apache.nifi.util.hive.AuthenticationFailedException;
 import org.apache.nifi.util.hive.HiveConfigurator;
 import org.apache.nifi.util.hive.HiveOptions;
-import org.apache.hive.streaming.HiveRecordWriter;
 import org.apache.nifi.util.hive.HiveUtils;
 import org.apache.nifi.util.hive.ValidationResources;
 
-import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
@@ -396,11 +395,10 @@ public class PutHive3Streaming extends AbstractProcessor {
 
         StreamingConnection hiveStreamingConnection = null;
 
-        try (final InputStream rawIn = session.read(flowFile)) {
+        try (final InputStream in = session.read(flowFile)) {
             final RecordReader reader;
 
-            try (final BufferedInputStream in = new BufferedInputStream(rawIn)) {
-
+            try {
                 // if we fail to create the RecordReader then we want to route to failure, so we need to
                 // handle this separately from the other IOExceptions which normally route to retry
                 try {
@@ -415,7 +413,7 @@ public class PutHive3Streaming extends AbstractProcessor {
                 hiveStreamingConnection.beginTransaction();
                 hiveStreamingConnection.write(in);
                 hiveStreamingConnection.commitTransaction();
-                rawIn.close();
+                in.close();
 
                 Map<String, String> updateAttributes = new HashMap<>();
                 updateAttributes.put(HIVE_STREAMING_RECORD_COUNT_ATTR, Long.toString(hiveStreamingConnection.getConnectionStats().getRecordsWritten()));
diff --git a/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/main/java/org/apache/nifi/processors/jolt/record/JoltTransformRecord.java b/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/main/java/org/apache/nifi/processors/jolt/record/JoltTransformRecord.java
index 6ce4161..2e1ef71 100644
--- a/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/main/java/org/apache/nifi/processors/jolt/record/JoltTransformRecord.java
+++ b/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/main/java/org/apache/nifi/processors/jolt/record/JoltTransformRecord.java
@@ -69,8 +69,8 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
@@ -296,36 +296,67 @@ public class JoltTransformRecord extends AbstractProcessor {
         final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
 
         final RecordSchema schema;
-        final ClassLoader originalContextClassLoader = Thread.currentThread().getContextClassLoader();
         try (final InputStream in = session.read(original);
              final RecordReader reader = readerFactory.createRecordReader(original, in, getLogger())) {
             schema = writerFactory.getSchema(original.getAttributes(), reader.getSchema());
-            Record record;
 
             FlowFile transformed = session.create(original);
             final Map<String, String> attributes = new HashMap<>();
             final WriteResult writeResult;
-            try (final OutputStream out = session.write(transformed);
-                 final RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, out)) {
 
-                final JoltTransform transform = getTransform(context, original);
-                writer.beginRecordSet();
-                while ((record = reader.nextRecord()) != null) {
-                    Map<String, Object> recordMap = (Map<String, Object>) DataTypeUtils.convertRecordFieldtoObject(record, RecordFieldType.RECORD.getRecordDataType(record.getSchema()));
-                    // JOLT expects arrays to be of type List where our Record code uses Object[].
-                    // Make another pass of the transformed objects to change Object[] to List.
-                    recordMap = (Map<String, Object>) normalizeJoltObjects(recordMap);
-                    Object transformedObject = transform(transform, recordMap);
-                    // JOLT expects arrays to be of type List where our Record code uses Object[].
-                    // Make another pass of the transformed objects to change List to Object[].
-                    Record r = DataTypeUtils.toRecord(normalizeRecordObjects(transformedObject), schema, "r");
-                    writer.write(r);
+            try {
+                // We want to transform the first record before creating the Record Writer. We do this because the Record will likely end up with a different structure
+                // and therefore a difference Schema after being transformed. As a result, we want to transform the Record and then provide the transformed schema to the
+                // Record Writer so that if the Record Writer chooses to inherit the Record Schema from the Record itself, it will inherit the transformed schema, not the
+                // schema determined by the Record Reader.
+                final Record firstRecord = reader.nextRecord();
+                if (firstRecord == null) {
+                    try (final OutputStream out = session.write(transformed);
+                         final RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, out)) {
+
+                        writer.beginRecordSet();
+                        writeResult = writer.finishRecordSet();
+
+                        attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
+                        attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
+                        attributes.putAll(writeResult.getAttributes());
+                    }
+
+                    transformed = session.putAllAttributes(transformed, attributes);
+                    session.transfer(transformed, REL_SUCCESS);
+                    session.transfer(original, REL_ORIGINAL);
+                    logger.info("{} had no Records to transform", new Object[]{original});
+                    return;
                 }
-                writeResult = writer.finishRecordSet();
 
-                attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
-                attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
-                attributes.putAll(writeResult.getAttributes());
+                final JoltTransform transform = getTransform(context, original);
+                final Record transformedFirstRecord = transform(firstRecord, transform);
+
+                final RecordSchema writeSchema = writerFactory.getSchema(original.getAttributes(), transformedFirstRecord.getSchema());
+
+                // TODO: Is it possible that two Records with the same input schema could have different schemas after transformation?
+                // If so, then we need to avoid this pattern of writing all Records from the input FlowFile to the same output FlowFile
+                // and instead use a Map<RecordSchema, RecordSetWriter>. This way, even if many different output schemas are possible,
+                // the output FlowFiles will each only contain records that have the same schema.
+                try (final OutputStream out = session.write(transformed);
+                     final RecordSetWriter writer = writerFactory.createWriter(getLogger(), writeSchema, out)) {
+
+                    writer.beginRecordSet();
+
+                    writer.write(transformedFirstRecord);
+
+                    Record record;
+                    while ((record = reader.nextRecord()) != null) {
+                        final Record transformedRecord = transform(record, transform);
+                        writer.write(transformedRecord);
+                    }
+
+                    writeResult = writer.finishRecordSet();
+
+                    attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
+                    attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
+                    attributes.putAll(writeResult.getAttributes());
+                }
             } catch (Exception e) {
                 logger.error("Unable to write transformed records {} due to {}", new Object[]{original, e.toString(), e});
                 session.remove(transformed);
@@ -339,15 +370,27 @@ public class JoltTransformRecord extends AbstractProcessor {
             session.getProvenanceReporter().modifyContent(transformed, "Modified With " + transformType, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
             session.transfer(original, REL_ORIGINAL);
             logger.debug("Transformed {}", new Object[]{original});
-
-
         } catch (final Exception ex) {
             logger.error("Unable to transform {} due to {}", new Object[]{original, ex.toString(), ex});
             session.transfer(original, REL_FAILURE);
-            return;
         }
     }
 
+    private Record transform(final Record record, final JoltTransform transform) {
+        Map<String, Object> recordMap = (Map<String, Object>) DataTypeUtils.convertRecordFieldtoObject(record, RecordFieldType.RECORD.getRecordDataType(record.getSchema()));
+
+        // JOLT expects arrays to be of type List where our Record code uses Object[].
+        // Make another pass of the transformed objects to change Object[] to List.
+        recordMap = (Map<String, Object>) normalizeJoltObjects(recordMap);
+        final Object transformedObject = transform(transform, recordMap);
+
+        // JOLT expects arrays to be of type List where our Record code uses Object[].
+        // Make another pass of the transformed objects to change List to Object[].
+        final Object normalizedRecordValues = normalizeRecordObjects(transformedObject);
+        final Record updatedRecord = DataTypeUtils.toRecord(normalizedRecordValues, "r");
+        return updatedRecord;
+    }
+
     private JoltTransform getTransform(final ProcessContext context, final FlowFile flowFile) {
         final Optional<String> specString;
         if (context.getProperty(JOLT_SPEC).isSet()) {
diff --git a/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/test/java/org/apache/nifi/processors/jolt/record/TestJoltTransformRecord.java b/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/test/java/org/apache/nifi/processors/jolt/record/TestJoltTransformRecord.java
index 313781d..e7b43b5 100644
--- a/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/test/java/org/apache/nifi/processors/jolt/record/TestJoltTransformRecord.java
+++ b/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/test/java/org/apache/nifi/processors/jolt/record/TestJoltTransformRecord.java
@@ -167,15 +167,19 @@ public class TestJoltTransformRecord {
     @Test
     public void testInvalidFlowFileContent() throws IOException {
         generateTestData(1, null);
+
         final String outputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformRecord/chainrOutputSchema.avsc")));
+        final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformRecord/chainrSpec.json")));
+
         runner.setProperty(writer, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY);
         runner.setProperty(writer, SchemaAccessUtils.SCHEMA_TEXT, outputSchemaText);
         runner.setProperty(writer, "Pretty Print JSON", "true");
-        runner.enableControllerService(writer);
-        final String spec = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformRecord/chainrSpec.json")));
         runner.setProperty(JoltTransformRecord.JOLT_SPEC, spec);
+
+        runner.enableControllerService(writer);
         parser.failAfter(0);
         runner.enqueue("invalid json");
+
         runner.run();
         runner.assertAllFlowFilesTransferred(JoltTransformRecord.REL_FAILURE);
     }
@@ -494,19 +498,23 @@ runner.assertTransferCount(JoltTransformRecord.REL_ORIGINAL, 1);
     public void testJoltSpecEL() throws IOException {
         generateTestData(1, null);
         final String outputSchemaText = new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformRecord/defaultrOutputSchema.avsc")));
+
         runner.setProperty(writer, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY);
         runner.setProperty(writer, SchemaAccessUtils.SCHEMA_TEXT, outputSchemaText);
         runner.setProperty(writer, "Pretty Print JSON", "true");
         runner.enableControllerService(writer);
-        final String spec = "${joltSpec}";
-        runner.setProperty(JoltTransformRecord.JOLT_SPEC, spec);
+
+        runner.setProperty(JoltTransformRecord.JOLT_SPEC, "${joltSpec}");
         runner.setProperty(JoltTransformRecord.JOLT_TRANSFORM, JoltTransformRecord.DEFAULTR);
+
         final Map<String, String> attributes = Collections.singletonMap("joltSpec",
                 "{\"RatingRange\":5,\"rating\":{\"*\":{\"MaxLabel\":\"High\",\"MinLabel\":\"Low\",\"DisplayType\":\"NORMAL\"}}}");
         runner.enqueue(new byte[0], attributes);
+
         runner.run();
         runner.assertTransferCount(JoltTransformRecord.REL_SUCCESS, 1);
-runner.assertTransferCount(JoltTransformRecord.REL_ORIGINAL, 1);
+        runner.assertTransferCount(JoltTransformRecord.REL_ORIGINAL, 1);
+
         final MockFlowFile transformed = runner.getFlowFilesForRelationship(JoltTransformRecord.REL_SUCCESS).get(0);
         assertEquals(new String(Files.readAllBytes(Paths.get("src/test/resources/TestJoltTransformRecord/defaultrOutput.json"))),
                 new String(transformed.toByteArray()));
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_10.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_10.java
index e670c7d..6d6af61 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_10.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_10.java
@@ -17,19 +17,6 @@
 
 package org.apache.nifi.processors.kafka.pubsub;
 
-import java.io.BufferedInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
@@ -62,6 +49,18 @@ import org.apache.nifi.serialization.RecordSetWriterFactory;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
 @Tags({"Apache", "Kafka", "Record", "csv", "json", "avro", "logs", "Put", "Send", "Message", "PubSub", "0.10.x"})
 @CapabilityDescription("Sends the contents of a FlowFile as individual records to Apache Kafka using the Kafka 0.10.x Producer API. "
     + "The contents of the FlowFile are expected to be record-oriented data that can be read by the configured Record Reader. "
@@ -329,8 +328,8 @@ public class PublishKafkaRecord_0_10 extends AbstractProcessor {
                 try {
                     session.read(flowFile, new InputStreamCallback() {
                         @Override
-                        public void process(final InputStream rawIn) throws IOException {
-                            try (final InputStream in = new BufferedInputStream(rawIn)) {
+                        public void process(final InputStream in) throws IOException {
+                            try {
                                 final RecordReader reader = readerFactory.createRecordReader(attributes, in, getLogger());
                                 final RecordSet recordSet = reader.createRecordSet();
 
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java
index 8bbec17..0815518 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java
@@ -17,22 +17,6 @@
 
 package org.apache.nifi.processors.kafka.pubsub;
 
-import java.io.BufferedInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
-
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
@@ -65,6 +49,21 @@ import org.apache.nifi.serialization.RecordSetWriterFactory;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
 @Tags({"Apache", "Kafka", "Record", "csv", "json", "avro", "logs", "Put", "Send", "Message", "PubSub", "0.11.x"})
 @CapabilityDescription("Sends the contents of a FlowFile as individual records to Apache Kafka using the Kafka 0.11.x Producer API. "
     + "The contents of the FlowFile are expected to be record-oriented data that can be read by the configured Record Reader. "
@@ -404,8 +403,8 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
                 try {
                     session.read(flowFile, new InputStreamCallback() {
                         @Override
-                        public void process(final InputStream rawIn) throws IOException {
-                            try (final InputStream in = new BufferedInputStream(rawIn)) {
+                        public void process(final InputStream in) throws IOException {
+                            try {
                                 final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger());
                                 final RecordSet recordSet = reader.createRecordSet();
 
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java
index 82a3918..bb0478c 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java
@@ -17,22 +17,6 @@
 
 package org.apache.nifi.processors.kafka.pubsub;
 
-import java.io.BufferedInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
-
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
@@ -65,6 +49,21 @@ import org.apache.nifi.serialization.RecordSetWriterFactory;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
 @Tags({"Apache", "Kafka", "Record", "csv", "json", "avro", "logs", "Put", "Send", "Message", "PubSub", "1.0"})
 @CapabilityDescription("Sends the contents of a FlowFile as individual records to Apache Kafka using the Kafka 1.0 Producer API. "
     + "The contents of the FlowFile are expected to be record-oriented data that can be read by the configured Record Reader. "
@@ -406,8 +405,8 @@ public class PublishKafkaRecord_1_0 extends AbstractProcessor {
                 try {
                     session.read(flowFile, new InputStreamCallback() {
                         @Override
-                        public void process(final InputStream rawIn) throws IOException {
-                            try (final InputStream in = new BufferedInputStream(rawIn)) {
+                        public void process(final InputStream in) throws IOException {
+                            try {
                                 final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger());
                                 final RecordSet recordSet = reader.createRecordSet();
 
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_0.java
index 88a6b95..2dde669 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_0.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_0.java
@@ -49,7 +49,6 @@ import org.apache.nifi.serialization.RecordSetWriterFactory;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 
-import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.charset.Charset;
@@ -404,8 +403,8 @@ public class PublishKafkaRecord_2_0 extends AbstractProcessor {
                 try {
                     session.read(flowFile, new InputStreamCallback() {
                         @Override
-                        public void process(final InputStream rawIn) throws IOException {
-                            try (final InputStream in = new BufferedInputStream(rawIn)) {
+                        public void process(final InputStream in) throws IOException {
+                            try {
                                 final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger());
                                 final RecordSet recordSet = reader.createRecordSet();
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java
index 9dbe8c1..e4371f6 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java
@@ -125,13 +125,36 @@ public abstract class AbstractRecordProcessor extends AbstractProcessor {
 
                     try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, getLogger())) {
 
-                        final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        // Get the first record and process it before we create the Record Writer. We do this so that if the Processor
+                        // updates the Record's schema, we can provide an updated schema to the Record Writer. If there are no records,
+                        // then we can simply create the Writer with the Reader's schema and begin & end the Record Set.
+                        Record firstRecord = reader.nextRecord();
+                        if (firstRecord == null) {
+                            final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                            try (final RecordSetWriter writer = writerFactory.createWriter(getLogger(), writeSchema, out)) {
+                                writer.beginRecordSet();
+
+                                final WriteResult writeResult = writer.finishRecordSet();
+                                attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
+                                attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
+                                attributes.putAll(writeResult.getAttributes());
+                                recordCount.set(writeResult.getRecordCount());
+                            }
+
+                            return;
+                        }
+
+                        firstRecord = AbstractRecordProcessor.this.process(firstRecord, original, context);
+
+                        final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, firstRecord.getSchema());
                         try (final RecordSetWriter writer = writerFactory.createWriter(getLogger(), writeSchema, out)) {
                             writer.beginRecordSet();
 
+                            writer.write(firstRecord);
+
                             Record record;
                             while ((record = reader.nextRecord()) != null) {
-                                final Record processed = AbstractRecordProcessor.this.process(record, writeSchema, original, context);
+                                final Record processed = AbstractRecordProcessor.this.process(record, original, context);
                                 writer.write(processed);
                             }
 
@@ -166,5 +189,5 @@ public abstract class AbstractRecordProcessor extends AbstractProcessor {
         getLogger().info("Successfully converted {} records for {}", new Object[] {count, flowFile});
     }
 
-    protected abstract Record process(Record record, RecordSchema writeSchema, FlowFile flowFile, ProcessContext context);
+    protected abstract Record process(Record record, FlowFile flowFile, ProcessContext context);
 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRouteRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRouteRecord.java
index 374ed48..9d96d34 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRouteRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRouteRecord.java
@@ -17,17 +17,6 @@
 
 package org.apache.nifi.processors.standard;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
@@ -48,6 +37,17 @@ import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.util.Tuple;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
 public abstract class AbstractRouteRecord<T> extends AbstractProcessor {
     static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
         .name("record-reader")
@@ -112,18 +112,29 @@ public abstract class AbstractRouteRecord<T> extends AbstractProcessor {
         final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
         final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
 
-
         final AtomicInteger numRecords = new AtomicInteger(0);
         final Map<Relationship, Tuple<FlowFile, RecordSetWriter>> writers = new HashMap<>();
         final FlowFile original = flowFile;
         final Map<String, String> originalAttributes = original.getAttributes();
+
         try {
             session.read(flowFile, new InputStreamCallback() {
                 @Override
                 public void process(final InputStream in) throws IOException {
                     try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, getLogger())) {
 
-                        final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final Record firstRecord = reader.nextRecord();
+                        if (firstRecord == null) {
+                            getLogger().info("{} has no Records, so routing just the original FlowFile to 'original'", new Object[] {original});
+                            return;
+                        }
+
+                        final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, firstRecord.getSchema());
+
+                        final Set<Relationship> firstRecordRelationships = route(firstRecord, writeSchema, original, context, flowFileContext);
+                        for (final Relationship relationship : firstRecordRelationships) {
+                            writeRecord(firstRecord, relationship, writers, session, original, originalAttributes, writerFactory);
+                        }
 
                         Record record;
                         while ((record = reader.nextRecord()) != null) {
@@ -131,21 +142,7 @@ public abstract class AbstractRouteRecord<T> extends AbstractProcessor {
                             numRecords.incrementAndGet();
 
                             for (final Relationship relationship : relationships) {
-                                final RecordSetWriter recordSetWriter;
-                                Tuple<FlowFile, RecordSetWriter> tuple = writers.get(relationship);
-                                if (tuple == null) {
-                                    FlowFile outFlowFile = session.create(original);
-                                    final OutputStream out = session.write(outFlowFile);
-                                    recordSetWriter = writerFactory.createWriter(getLogger(), writeSchema, out);
-                                    recordSetWriter.beginRecordSet();
-
-                                    tuple = new Tuple<>(outFlowFile, recordSetWriter);
-                                    writers.put(relationship, tuple);
-                                } else {
-                                    recordSetWriter = tuple.getValue();
-                                }
-
-                                recordSetWriter.write(record);
+                                writeRecord(record, relationship, writers, session, original, originalAttributes, writerFactory);
                             }
                         }
                     } catch (final SchemaNotFoundException | MalformedRecordException e) {
@@ -216,6 +213,28 @@ public abstract class AbstractRouteRecord<T> extends AbstractProcessor {
         getLogger().info("Successfully processed {}, creating {} derivative FlowFiles and processing {} records", new Object[] {flowFile, writers.size(), numRecords});
     }
 
+    private void writeRecord(final Record record, final Relationship relationship, final Map<Relationship, Tuple<FlowFile, RecordSetWriter>> writers, final ProcessSession session,
+                             final FlowFile original, final Map<String, String> originalAttributes, final RecordSetWriterFactory writerFactory) throws IOException, SchemaNotFoundException {
+        final RecordSetWriter recordSetWriter;
+        Tuple<FlowFile, RecordSetWriter> tuple = writers.get(relationship);
+
+        if (tuple == null) {
+            FlowFile outFlowFile = session.create(original);
+            final OutputStream out = session.write(outFlowFile);
+
+            final RecordSchema recordWriteSchema = writerFactory.getSchema(originalAttributes, record.getSchema());
+            recordSetWriter = writerFactory.createWriter(getLogger(), recordWriteSchema, out);
+            recordSetWriter.beginRecordSet();
+
+            tuple = new Tuple<>(outFlowFile, recordSetWriter);
+            writers.put(relationship, tuple);
+        } else {
+            recordSetWriter = tuple.getValue();
+        }
+
+        recordSetWriter.write(record);
+    }
+
     protected abstract Set<Relationship> route(Record record, RecordSchema writeSchema, FlowFile flowFile, ProcessContext context, T flowFileContext);
 
     protected abstract boolean isRouteOriginal();
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertRecord.java
index a2d4e69..3b24d79 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertRecord.java
@@ -30,7 +30,6 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.serialization.record.Record;
-import org.apache.nifi.serialization.record.RecordSchema;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -60,7 +59,7 @@ public class ConvertRecord extends AbstractRecordProcessor {
     }
 
     @Override
-    protected Record process(final Record record, final RecordSchema writeSchema, final FlowFile flowFile, final ProcessContext context) {
+    protected Record process(final Record record, final FlowFile flowFile, final ProcessContext context) {
         return record;
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java
index c687f74..b9686b2 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java
@@ -44,7 +44,10 @@ import org.apache.nifi.record.path.RecordPath;
 import org.apache.nifi.record.path.RecordPathResult;
 import org.apache.nifi.record.path.util.RecordPathCache;
 import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.util.DataTypeUtils;
 import org.apache.nifi.util.Tuple;
@@ -157,7 +160,7 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
     private static final Set<Relationship> UNMATCHED_COLLECTION = Collections.singleton(REL_UNMATCHED);
     private static final Set<Relationship> SUCCESS_COLLECTION = Collections.singleton(REL_SUCCESS);
 
-    private volatile Set<Relationship> relationships = new HashSet<>(Arrays.asList(new Relationship[] {REL_SUCCESS, REL_FAILURE}));
+    private volatile Set<Relationship> relationships = new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE));
     private volatile boolean routeToMatchedUnmatched = false;
 
     @OnScheduled
@@ -197,8 +200,8 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
     @SuppressWarnings("unchecked")
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
         final Set<String> dynamicPropNames = validationContext.getProperties().keySet().stream()
-            .filter(prop -> prop.isDynamic())
-            .map(prop -> prop.getName())
+            .filter(PropertyDescriptor::isDynamic)
+            .map(PropertyDescriptor::getName)
             .collect(Collectors.toSet());
 
         if (dynamicPropNames.isEmpty()) {
@@ -305,8 +308,6 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
         // Ensure that the Record has the appropriate schema to account for the newly added values
         final RecordPath resultPath = flowFileContext.getValue();
         if (resultPath != null) {
-            record.incorporateSchema(writeSchema);
-
             final Object lookupValue = lookupValueOption.get();
             final RecordPathResult resultPathResult = flowFileContext.getValue().evaluate(record);
 
@@ -327,19 +328,25 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
 
                         for (final String fieldName : lookupRecord.getRawFieldNames()) {
                             final Object value = lookupRecord.getValue(fieldName);
-                            destinationRecord.setValue(fieldName, value);
+
+                            final Optional<RecordField> recordFieldOption = lookupRecord.getSchema().getField(fieldName);
+                            if (recordFieldOption.isPresent()) {
+                                destinationRecord.setValue(recordFieldOption.get(), value);
+                            } else {
+                                destinationRecord.setValue(fieldName, value);
+                            }
                         }
                     } else {
                         final Optional<Record> parentOption = fieldVal.getParentRecord();
-
-                        if (parentOption.isPresent()) {
-                            parentOption.get().setValue(fieldVal.getField().getFieldName(), lookupRecord);
-                        }
+                        parentOption.ifPresent(parent -> parent.setValue(fieldVal.getField(), lookupRecord));
                     }
                 });
             } else {
-                resultPathResult.getSelectedFields().forEach(fieldVal -> fieldVal.updateValue(lookupValue));
+                final DataType inferredDataType = DataTypeUtils.inferDataType(lookupValue, RecordFieldType.STRING.getDataType());
+                resultPathResult.getSelectedFields().forEach(fieldVal -> fieldVal.updateValue(lookupValue, inferredDataType));
             }
+
+            record.incorporateInactiveFields();
         }
 
         final Set<Relationship> rels = routeToMatchedUnmatched ? MATCHED_COLLECTION : SUCCESS_COLLECTION;
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java
index 7d9981b..c5273d8 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java
@@ -16,31 +16,9 @@
  */
 package org.apache.nifi.processors.standard;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.Supplier;
-
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
 import org.apache.calcite.config.CalciteConnectionProperty;
 import org.apache.calcite.config.Lex;
 import org.apache.calcite.jdbc.CalciteConnection;
@@ -58,7 +36,6 @@ import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
@@ -84,6 +61,30 @@ import org.apache.nifi.serialization.WriteResult;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.ResultSetRecordSet;
 import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.Tuple;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
 
 @EventDriven
 @SideEffectFree
@@ -141,10 +142,9 @@ public class QueryRecord extends AbstractProcessor {
     static final PropertyDescriptor CACHE_SCHEMA = new PropertyDescriptor.Builder()
         .name("cache-schema")
         .displayName("Cache Schema")
-        .description("Parsing the SQL query and deriving the FlowFile's schema is relatively expensive. If this value is set to true, "
-            + "the Processor will cache these values so that the Processor is much more efficient and much faster. However, if this is done, "
-            + "then the schema that is derived for the first FlowFile processed must apply to all FlowFiles. If all FlowFiles will not have the exact "
-            + "same schema, or if the SQL SELECT statement uses the Expression Language, this value should be set to false.")
+        .description("This property is no longer used. It remains solely for backward compatibility in order to avoid making existing Processors invalid upon upgrade. This property will be" +
+            " removed in future versions. Now, instead of forcing the user to understand the semantics of schema caching, the Processor caches up to 25 schemas and automatically rolls off the" +
+            " old schemas. This provides the same performance when caching was enabled previously and in some cases very significant performance improvements if caching was previously disabled.")
         .expressionLanguageSupported(ExpressionLanguageScope.NONE)
         .allowableValues("true", "false")
         .defaultValue("true")
@@ -165,7 +165,10 @@ public class QueryRecord extends AbstractProcessor {
     private List<PropertyDescriptor> properties;
     private final Set<Relationship> relationships = Collections.synchronizedSet(new HashSet<>());
 
-    private final Map<String, BlockingQueue<CachedStatement>> statementQueues = new HashMap<>();
+    private final Cache<Tuple<String, RecordSchema>, BlockingQueue<CachedStatement>> statementQueues = Caffeine.newBuilder()
+        .maximumSize(25)
+        .removalListener(this::onCacheEviction)
+        .build();
 
     @Override
     protected void init(final ProcessorInitializationContext context) {
@@ -215,25 +218,6 @@ public class QueryRecord extends AbstractProcessor {
     }
 
     @Override
-    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        final boolean cache = validationContext.getProperty(CACHE_SCHEMA).asBoolean();
-        if (cache) {
-            for (final PropertyDescriptor descriptor : validationContext.getProperties().keySet()) {
-                if (descriptor.isDynamic() && validationContext.isExpressionLanguagePresent(validationContext.getProperty(descriptor).getValue())) {
-                    return Collections.singleton(new ValidationResult.Builder()
-                        .subject("Cache Schema")
-                        .input("true")
-                        .valid(false)
-                        .explanation("Cannot have 'Cache Schema' property set to true if any SQL statement makes use of the Expression Language")
-                        .build());
-                }
-            }
-        }
-
-        return Collections.emptyList();
-    }
-
-    @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
             .name(propertyDescriptorName)
@@ -246,6 +230,26 @@ public class QueryRecord extends AbstractProcessor {
             .build();
     }
 
+    @OnStopped
+    public synchronized void cleanup() {
+        for (final BlockingQueue<CachedStatement> statementQueue : statementQueues.asMap().values()) {
+            clearQueue(statementQueue);
+        }
+
+        statementQueues.invalidateAll();
+    }
+
+    private void onCacheEviction(final Tuple<String, RecordSchema> key, final BlockingQueue<CachedStatement> queue, final RemovalCause cause) {
+        clearQueue(queue);
+    }
+
+    private void clearQueue(final BlockingQueue<CachedStatement> statementQueue) {
+        CachedStatement stmt;
+        while ((stmt = statementQueue.poll()) != null) {
+            closeQuietly(stmt.getStatement(), stmt.getConnection());
+        }
+    }
+
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
         final FlowFile original = session.get();
@@ -262,13 +266,14 @@ public class QueryRecord extends AbstractProcessor {
         final Set<FlowFile> createdFlowFiles = new HashSet<>();
 
         // Determine the Record Reader's schema
+        final RecordSchema writerSchema;
         final RecordSchema readerSchema;
         try (final InputStream rawIn = session.read(original)) {
             final Map<String, String> originalAttributes = original.getAttributes();
             final RecordReader reader = recordReaderFactory.createRecordReader(originalAttributes, rawIn, getLogger());
-            final RecordSchema inputSchema = reader.getSchema();
+            readerSchema = reader.getSchema();
 
-            readerSchema = recordSetWriterFactory.getSchema(originalAttributes, inputSchema);
+            writerSchema = recordSetWriterFactory.getSchema(originalAttributes, readerSchema);
         } catch (final Exception e) {
             getLogger().error("Failed to determine Record Schema from {}; routing to failure", new Object[] {original, e});
             session.transfer(original, REL_FAILURE);
@@ -296,12 +301,7 @@ public class QueryRecord extends AbstractProcessor {
                 try {
                     final String sql = context.getProperty(descriptor).evaluateAttributeExpressions(original).getValue();
                     final AtomicReference<WriteResult> writeResultRef = new AtomicReference<>();
-                    final QueryResult queryResult;
-                    if (context.getProperty(CACHE_SCHEMA).asBoolean()) {
-                        queryResult = queryWithCache(session, original, sql, context, recordReaderFactory);
-                    } else {
-                        queryResult = query(session, original, sql, context, recordReaderFactory);
-                    }
+                    final QueryResult queryResult = query(session, original, readerSchema, sql, recordReaderFactory);
 
                     final AtomicReference<String> mimeTypeRef = new AtomicReference<>();
                     try {
@@ -313,7 +313,7 @@ public class QueryRecord extends AbstractProcessor {
                                 final RecordSchema writeSchema;
 
                                 try {
-                                    recordSet = new ResultSetRecordSet(rs, readerSchema);
+                                    recordSet = new ResultSetRecordSet(rs, writerSchema);
                                     final RecordSchema resultSetSchema = recordSet.getSchema();
                                     writeSchema = recordSetWriterFactory.getSchema(originalAttributes, resultSetSchema);
                                 } catch (final SQLException | SchemaNotFoundException e) {
@@ -389,82 +389,59 @@ public class QueryRecord extends AbstractProcessor {
     }
 
 
-    private synchronized CachedStatement getStatement(final String sql, final Supplier<CalciteConnection> connectionSupplier, final ProcessSession session,
-        final FlowFile flowFile, final RecordReaderFactory recordReaderFactory) throws SQLException {
-
-        final BlockingQueue<CachedStatement> statementQueue = statementQueues.get(sql);
-        if (statementQueue == null) {
-            return buildCachedStatement(sql, connectionSupplier, session, flowFile, recordReaderFactory);
-        }
+    private synchronized CachedStatement getStatement(final String sql, final RecordSchema schema, final Supplier<CachedStatement> statementBuilder) {
+        final Tuple<String, RecordSchema> tuple = new Tuple<>(sql, schema);
+        final BlockingQueue<CachedStatement> statementQueue = statementQueues.get(tuple, key -> new LinkedBlockingQueue<>());
 
         final CachedStatement cachedStmt = statementQueue.poll();
         if (cachedStmt != null) {
             return cachedStmt;
         }
 
-        return buildCachedStatement(sql, connectionSupplier, session, flowFile, recordReaderFactory);
+        return statementBuilder.get();
     }
 
-    private CachedStatement buildCachedStatement(final String sql, final Supplier<CalciteConnection> connectionSupplier, final ProcessSession session,
-        final FlowFile flowFile, final RecordReaderFactory recordReaderFactory) throws SQLException {
+    private CachedStatement buildCachedStatement(final String sql, final ProcessSession session,  final FlowFile flowFile, final RecordSchema schema,
+                                                 final RecordReaderFactory recordReaderFactory) {
 
-        final CalciteConnection connection = connectionSupplier.get();
+        final CalciteConnection connection = createConnection();
         final SchemaPlus rootSchema = connection.getRootSchema();
 
-        final FlowFileTable<?, ?> flowFileTable = new FlowFileTable<>(session, flowFile, recordReaderFactory, getLogger());
+        final FlowFileTable flowFileTable = new FlowFileTable(session, flowFile, schema, recordReaderFactory, getLogger());
         rootSchema.add("FLOWFILE", flowFileTable);
         rootSchema.setCacheEnabled(false);
 
-        final PreparedStatement stmt = connection.prepareStatement(sql);
-        return new CachedStatement(stmt, flowFileTable, connection);
-    }
-
-    @OnStopped
-    public synchronized void cleanup() {
-        for (final BlockingQueue<CachedStatement> statementQueue : statementQueues.values()) {
-            CachedStatement stmt;
-            while ((stmt = statementQueue.poll()) != null) {
-                closeQuietly(stmt.getStatement(), stmt.getConnection());
-            }
+        try {
+            final PreparedStatement stmt = connection.prepareStatement(sql);
+            return new CachedStatement(stmt, flowFileTable, connection);
+        } catch (final SQLException e) {
+            throw new ProcessException(e);
         }
-
-        statementQueues.clear();
     }
 
-    @OnScheduled
-    public synchronized void setupQueues(final ProcessContext context) {
-        // Create a Queue of PreparedStatements for each property that is user-defined. This allows us to easily poll the
-        // queue and add as necessary, knowing that the queue already exists.
-        for (final PropertyDescriptor descriptor : context.getProperties().keySet()) {
-            if (!descriptor.isDynamic()) {
-                continue;
-            }
 
-            final String sql = context.getProperty(descriptor).evaluateAttributeExpressions().getValue();
-            final BlockingQueue<CachedStatement> queue = new LinkedBlockingQueue<>(context.getMaxConcurrentTasks());
-            statementQueues.put(sql, queue);
+    private CalciteConnection createConnection() {
+        final Properties properties = new Properties();
+        properties.put(CalciteConnectionProperty.LEX.camelName(), Lex.MYSQL_ANSI.name());
+
+        try {
+            final Connection connection = DriverManager.getConnection("jdbc:calcite:", properties);
+            final CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class);
+            return calciteConnection;
+        } catch (final Exception e) {
+            throw new ProcessException(e);
         }
     }
 
-    protected QueryResult queryWithCache(final ProcessSession session, final FlowFile flowFile, final String sql, final ProcessContext context,
-        final RecordReaderFactory recordParserFactory) throws SQLException {
 
-        final Supplier<CalciteConnection> connectionSupplier = () -> {
-            final Properties properties = new Properties();
-            properties.put(CalciteConnectionProperty.LEX.camelName(), Lex.MYSQL_ANSI.name());
+    protected QueryResult query(final ProcessSession session, final FlowFile flowFile, final RecordSchema schema, final String sql, final RecordReaderFactory recordReaderFactory)
+                throws SQLException {
 
-            try {
-                final Connection connection = DriverManager.getConnection("jdbc:calcite:", properties);
-                final CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class);
-                return calciteConnection;
-            } catch (final Exception e) {
-                throw new ProcessException(e);
-            }
-        };
+        final Supplier<CachedStatement> statementBuilder = () -> buildCachedStatement(sql, session, flowFile, schema, recordReaderFactory);
 
-        final CachedStatement cachedStatement = getStatement(sql, connectionSupplier, session, flowFile, recordParserFactory);
+        final CachedStatement cachedStatement = getStatement(sql, schema, statementBuilder);
         final PreparedStatement stmt = cachedStatement.getStatement();
-        final FlowFileTable<?, ?> table = cachedStatement.getTable();
+        final FlowFileTable table = cachedStatement.getTable();
         table.setFlowFile(session, flowFile);
 
         final ResultSet rs;
@@ -480,7 +457,7 @@ public class QueryRecord extends AbstractProcessor {
             public void close() throws IOException {
                 table.close();
 
-                final BlockingQueue<CachedStatement> statementQueue = statementQueues.get(sql);
+                final BlockingQueue<CachedStatement> statementQueue = statementQueues.getIfPresent(new Tuple<>(sql, schema));
                 if (statementQueue == null || !statementQueue.offer(cachedStatement)) {
                     try {
                         cachedStatement.getConnection().close();
@@ -503,58 +480,6 @@ public class QueryRecord extends AbstractProcessor {
         };
     }
 
-    protected QueryResult query(final ProcessSession session, final FlowFile flowFile, final String sql, final ProcessContext context,
-        final RecordReaderFactory recordParserFactory) throws SQLException {
-
-        final Properties properties = new Properties();
-        properties.put(CalciteConnectionProperty.LEX.camelName(), Lex.MYSQL_ANSI.name());
-
-        Connection connection = null;
-        ResultSet resultSet = null;
-        Statement statement = null;
-        try {
-            connection = DriverManager.getConnection("jdbc:calcite:", properties);
-            final CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class);
-            final SchemaPlus rootSchema = calciteConnection.getRootSchema();
-
-            final FlowFileTable<?, ?> flowFileTable = new FlowFileTable<>(session, flowFile, recordParserFactory, getLogger());
-            rootSchema.add("FLOWFILE", flowFileTable);
-            rootSchema.setCacheEnabled(false);
-
-            statement = connection.createStatement();
-
-            try {
-                resultSet = statement.executeQuery(sql);
-            } catch (final Throwable t) {
-                flowFileTable.close();
-                throw t;
-            }
-
-            final ResultSet rs = resultSet;
-            final Statement stmt = statement;
-            final Connection conn = connection;
-
-            return new QueryResult() {
-                @Override
-                public void close() throws IOException {
-                    closeQuietly(rs, stmt, conn);
-                }
-
-                @Override
-                public ResultSet getResultSet() {
-                    return rs;
-                }
-
-                @Override
-                public int getRecordsRead() {
-                    return flowFileTable.getRecordsRead();
-                }
-            };
-        } catch (final Exception e) {
-            closeQuietly(resultSet, statement, connection);
-            throw e;
-        }
-    }
 
     private void closeQuietly(final AutoCloseable... closeables) {
         if (closeables == null) {
@@ -611,24 +536,24 @@ public class QueryRecord extends AbstractProcessor {
         }
     }
 
-    private static interface QueryResult extends Closeable {
+    private interface QueryResult extends Closeable {
         ResultSet getResultSet();
 
         int getRecordsRead();
     }
 
     private static class CachedStatement {
-        private final FlowFileTable<?, ?> table;
+        private final FlowFileTable table;
         private final PreparedStatement statement;
         private final Connection connection;
 
-        public CachedStatement(final PreparedStatement statement, final FlowFileTable<?, ?> table, final Connection connection) {
+        public CachedStatement(final PreparedStatement statement, final FlowFileTable table, final Connection connection) {
             this.statement = statement;
             this.table = table;
             this.connection = connection;
         }
 
-        public FlowFileTable<?, ?> getTable() {
+        public FlowFileTable getTable() {
             return table;
         }
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateRecord.java
index 9b5d209..20e5dd1 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateRecord.java
@@ -17,15 +17,6 @@
 
 package org.apache.nifi.processors.standard;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@@ -52,9 +43,19 @@ import org.apache.nifi.serialization.SimpleRecordSchema;
 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.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.util.DataTypeUtils;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
 
 @EventDriven
 @SideEffectFree
@@ -113,8 +114,7 @@ public class UpdateRecord extends AbstractRecordProcessor {
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        final boolean containsDynamic = validationContext.getProperties().keySet().stream()
-            .anyMatch(property -> property.isDynamic());
+        final boolean containsDynamic = validationContext.getProperties().keySet().stream().anyMatch(PropertyDescriptor::isDynamic);
 
         if (containsDynamic) {
             return Collections.emptyList();
@@ -142,13 +142,9 @@ public class UpdateRecord extends AbstractRecordProcessor {
     }
 
     @Override
-    protected Record process(Record record, final RecordSchema writeSchema, final FlowFile flowFile, final ProcessContext context) {
+    protected Record process(Record record, final FlowFile flowFile, final ProcessContext context) {
         final boolean evaluateValueAsRecordPath = context.getProperty(REPLACEMENT_VALUE_STRATEGY).getValue().equals(RECORD_PATH_VALUES.getValue());
 
-        // Incorporate the RecordSchema that we will use for writing records into the Schema that we have
-        // for the record, because it's possible that the updates to the record will not be valid otherwise.
-        record.incorporateSchema(writeSchema);
-
         for (final String recordPathText : recordPaths) {
             final RecordPath recordPath = recordPathCache.getCompiled(recordPathText);
             final RecordPathResult result = recordPath.evaluate(record);
@@ -177,15 +173,17 @@ public class UpdateRecord extends AbstractRecordProcessor {
                         fieldVariables.put(FIELD_TYPE, fieldVal.getField().getDataType().getFieldType().name());
 
                         final String evaluatedReplacementVal = replacementValue.evaluateAttributeExpressions(flowFile, fieldVariables).getValue();
-                        fieldVal.updateValue(evaluatedReplacementVal);
+                        fieldVal.updateValue(evaluatedReplacementVal, RecordFieldType.STRING.getDataType());
                     });
                 } else {
                     final String evaluatedReplacementVal = replacementValue.evaluateAttributeExpressions(flowFile).getValue();
-                    result.getSelectedFields().forEach(fieldVal -> fieldVal.updateValue(evaluatedReplacementVal));
+                    result.getSelectedFields().forEach(fieldVal -> fieldVal.updateValue(evaluatedReplacementVal, RecordFieldType.STRING.getDataType()));
                 }
             }
         }
 
+        record.incorporateInactiveFields();
+
         return record;
     }
 
@@ -204,7 +202,7 @@ public class UpdateRecord extends AbstractRecordProcessor {
             final RecordPathResult replacementResult = replacementRecordPath.evaluate(record, fieldVal);
             final List<FieldValue> selectedFields = replacementResult.getSelectedFields().collect(Collectors.toList());
             final Object replacementObject = getReplacementObject(selectedFields);
-            fieldVal.updateValue(replacementObject);
+            updateFieldValue(fieldVal, replacementObject);
 
             record = updateRecord(destinationFieldValues, selectedFields, record);
         }
@@ -222,29 +220,44 @@ public class UpdateRecord extends AbstractRecordProcessor {
                 return (Record) replacement;
             }
 
+            final FieldValue replacementFieldValue = (FieldValue) replacement;
+            if (replacementFieldValue.getValue() instanceof Record) {
+                return (Record) replacementFieldValue.getValue();
+            }
+
             final List<RecordField> fields = selectedFields.stream().map(FieldValue::getField).collect(Collectors.toList());
             final RecordSchema schema = new SimpleRecordSchema(fields);
             final Record mapRecord = new MapRecord(schema, new HashMap<>());
             for (final FieldValue selectedField : selectedFields) {
-                mapRecord.setValue(selectedField.getField().getFieldName(), selectedField.getValue());
+                mapRecord.setValue(selectedField.getField(), selectedField.getValue());
             }
 
             return mapRecord;
         } else {
             for (final FieldValue fieldVal : destinationFields) {
-                fieldVal.updateValue(getReplacementObject(selectedFields));
+                final Object replacementObject = getReplacementObject(selectedFields);
+                updateFieldValue(fieldVal, replacementObject);
             }
             return record;
         }
     }
 
+    private void updateFieldValue(final FieldValue fieldValue, final Object replacement) {
+        if (replacement instanceof FieldValue) {
+            final FieldValue replacementFieldValue = (FieldValue) replacement;
+            fieldValue.updateValue(replacementFieldValue.getValue(), replacementFieldValue.getField().getDataType());
+        } else {
+            fieldValue.updateValue(replacement);
+        }
+    }
+
     private Object getReplacementObject(final List<FieldValue> selectedFields) {
         if (selectedFields.size() > 1) {
             final List<RecordField> fields = selectedFields.stream().map(FieldValue::getField).collect(Collectors.toList());
             final RecordSchema schema = new SimpleRecordSchema(fields);
             final Record record = new MapRecord(schema, new HashMap<>());
             for (final FieldValue fieldVal : selectedFields) {
-                record.setValue(fieldVal.getField().getFieldName(), fieldVal.getValue());
+                record.setValue(fieldVal.getField(), fieldVal.getValue());
             }
 
             return record;
@@ -253,7 +266,7 @@ public class UpdateRecord extends AbstractRecordProcessor {
         if (selectedFields.isEmpty()) {
             return null;
         } else {
-            return selectedFields.get(0).getValue();
+            return selectedFields.get(0);
         }
     }
 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java
index 5f92311..06ffb76 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java
@@ -28,7 +28,7 @@ import org.apache.nifi.serialization.record.Record;
 
 import java.io.InputStream;
 
-public class FlowFileEnumerator<InternalType> implements Enumerator<Object> {
+public class FlowFileEnumerator implements Enumerator<Object> {
     private final ProcessSession session;
     private final FlowFile flowFile;
     private final ComponentLog logger;
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java
index 9e10377..8c0e2ce 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java
@@ -16,15 +16,6 @@
  */
 package org.apache.nifi.queryrecord;
 
-import java.io.InputStream;
-import java.lang.reflect.Type;
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
@@ -46,18 +37,24 @@ import org.apache.calcite.util.Pair;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.serialization.RecordReader;
 import org.apache.nifi.serialization.RecordReaderFactory;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordSchema;
 
+import java.lang.reflect.Type;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 
-public class FlowFileTable<S, E> extends AbstractTable implements QueryableTable, TranslatableTable {
+public class FlowFileTable extends AbstractTable implements QueryableTable, TranslatableTable {
 
-    private final RecordReaderFactory recordParserFactory;
+    private final RecordReaderFactory recordReaderFactory;
     private final ComponentLog logger;
 
     private RecordSchema recordSchema;
@@ -67,15 +64,16 @@ public class FlowFileTable<S, E> extends AbstractTable implements QueryableTable
     private volatile FlowFile flowFile;
     private volatile int maxRecordsRead;
 
-    private final Set<FlowFileEnumerator<?>> enumerators = new HashSet<>();
+    private final Set<FlowFileEnumerator> enumerators = new HashSet<>();
 
     /**
      * Creates a FlowFile table.
      */
-    public FlowFileTable(final ProcessSession session, final FlowFile flowFile, final RecordReaderFactory recordParserFactory, final ComponentLog logger) {
+    public FlowFileTable(final ProcessSession session, final FlowFile flowFile, final RecordSchema schema, final RecordReaderFactory recordReaderFactory, final ComponentLog logger) {
         this.session = session;
         this.flowFile = flowFile;
-        this.recordParserFactory = recordParserFactory;
+        this.recordSchema = schema;
+        this.recordReaderFactory = recordReaderFactory;
         this.logger = logger;
     }
 
@@ -93,7 +91,7 @@ public class FlowFileTable<S, E> extends AbstractTable implements QueryableTable
 
     public void close() {
         synchronized (enumerators) {
-            for (final FlowFileEnumerator<?> enumerator : enumerators) {
+            for (final FlowFileEnumerator enumerator : enumerators) {
                 enumerator.close();
             }
         }
@@ -110,7 +108,7 @@ public class FlowFileTable<S, E> extends AbstractTable implements QueryableTable
             @Override
             @SuppressWarnings({"unchecked", "rawtypes"})
             public Enumerator<Object> enumerator() {
-                final FlowFileEnumerator flowFileEnumerator = new FlowFileEnumerator(session, flowFile, logger, recordParserFactory, fields) {
+                final FlowFileEnumerator flowFileEnumerator = new FlowFileEnumerator(session, flowFile, logger, recordReaderFactory, fields) {
                     @Override
                     protected void onFinish() {
                         final int recordCount = getRecordsRead();
@@ -175,30 +173,16 @@ public class FlowFileTable<S, E> extends AbstractTable implements QueryableTable
             return relDataType;
         }
 
-        RecordSchema schema;
-        try (final InputStream in = session.read(flowFile)) {
-            final RecordReader recordParser = recordParserFactory.createRecordReader(flowFile, in, logger);
-            schema = recordParser.getSchema();
-        } catch (final Exception e) {
-            throw new ProcessException("Failed to determine schema of data records for " + flowFile, e);
-        }
-
         final List<String> names = new ArrayList<>();
         final List<RelDataType> types = new ArrayList<>();
 
         final JavaTypeFactory javaTypeFactory = (JavaTypeFactory) typeFactory;
-        for (final RecordField field : schema.getFields()) {
+        for (final RecordField field : recordSchema.getFields()) {
             names.add(field.getFieldName());
             final RelDataType relDataType = getRelDataType(field.getDataType(), javaTypeFactory);
             types.add(javaTypeFactory.createTypeWithNullability(relDataType, field.isNullable()));
         }
 
-        logger.debug("Found Schema: {}", new Object[] {schema});
-
-        if (recordSchema == null) {
-            recordSchema = schema;
-        }
-
         relDataType = typeFactory.createStructType(Pair.zip(names, types));
         return relDataType;
     }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTableScan.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTableScan.java
index afca202..b1b656f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTableScan.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTableScan.java
@@ -16,8 +16,6 @@
  */
 package org.apache.nifi.queryrecord;
 
-import java.util.List;
-
 import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 import org.apache.calcite.adapter.enumerable.EnumerableRel;
 import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
@@ -37,6 +35,8 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 
+import java.util.List;
+
 /**
  * Relational expression representing a scan of a FlowFile.
  *
@@ -45,10 +45,10 @@ import org.apache.calcite.rel.type.RelDataTypeField;
  * </p>
  */
 public class FlowFileTableScan extends TableScan implements EnumerableRel {
-    final FlowFileTable<?, ?> flowFileTable;
+    final FlowFileTable flowFileTable;
     final int[] fields;
 
-    protected FlowFileTableScan(final RelOptCluster cluster, final RelOptTable table, final FlowFileTable<?, ?> flowFileTable, final int[] fields) {
+    protected FlowFileTableScan(final RelOptCluster cluster, final RelOptTable table, final FlowFileTable flowFileTable, final int[] fields) {
         super(cluster, cluster.traitSetOf(EnumerableConvention.INSTANCE), table);
 
         this.flowFileTable = flowFileTable;
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLookupRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLookupRecord.java
index 30b2b24..3efd9d1 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLookupRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLookupRecord.java
@@ -17,17 +17,6 @@
 
 package org.apache.nifi.processors.standard;
 
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.lookup.RecordLookupService;
@@ -48,6 +37,16 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.junit.Assert.assertTrue;
+
 public class TestLookupRecord {
 
     private TestRunner runner;
@@ -112,7 +111,7 @@ public class TestLookupRecord {
     }
 
     @Test
-    public void testAllMatch() throws InitializationException {
+    public void testAllMatch() {
         lookupService.addValue("John Doe", "Soccer");
         lookupService.addValue("Jane Doe", "Basketball");
         lookupService.addValue("Jimmy Doe", "Football");
@@ -129,7 +128,7 @@ public class TestLookupRecord {
     }
 
     @Test
-    public void testAllUnmatched() throws InitializationException {
+    public void testAllUnmatched() {
         runner.enqueue("");
         runner.run();
 
@@ -142,7 +141,7 @@ public class TestLookupRecord {
     }
 
     @Test
-    public void testMixtureOfMatch() throws InitializationException {
+    public void testMixtureOfMatch() {
         lookupService.addValue("John Doe", "Soccer");
         lookupService.addValue("Jimmy Doe", "Football");
 
@@ -166,7 +165,7 @@ public class TestLookupRecord {
 
 
     @Test
-    public void testResultPathNotFound() throws InitializationException {
+    public void testResultPathNotFound() {
         runner.setProperty(LookupRecord.RESULT_RECORD_PATH, "/other");
 
         lookupService.addValue("John Doe", "Soccer");
@@ -181,11 +180,11 @@ public class TestLookupRecord {
 
         out.assertAttributeEquals("record.count", "3");
         out.assertAttributeEquals("mime.type", "text/plain");
-        out.assertContentEquals("John Doe,48,\nJane Doe,47,\nJimmy Doe,14,\n");
+        out.assertContentEquals("John Doe,48,,Soccer\nJane Doe,47,,Basketball\nJimmy Doe,14,,Football\n");
     }
 
     @Test
-    public void testLookupPathNotFound() throws InitializationException {
+    public void testLookupPathNotFound() {
         runner.setProperty("lookup", "/other");
 
         runner.enqueue("");
@@ -200,7 +199,7 @@ public class TestLookupRecord {
     }
 
     @Test
-    public void testUnparseableData() throws InitializationException {
+    public void testUnparseableData() {
         recordReader.failAfter(1);
 
         runner.enqueue("");
@@ -213,7 +212,7 @@ public class TestLookupRecord {
     }
 
     @Test
-    public void testNoResultPath() throws InitializationException {
+    public void testNoResultPath() {
         lookupService.addValue("John Doe", "Soccer");
         lookupService.addValue("Jane Doe", "Basketball");
         lookupService.addValue("Jimmy Doe", "Football");
@@ -233,7 +232,7 @@ public class TestLookupRecord {
 
 
     @Test
-    public void testMultipleLookupPaths() throws InitializationException {
+    public void testMultipleLookupPaths() {
         lookupService.addValue("John Doe", "Soccer");
         lookupService.addValue("Jane Doe", "Basketball");
         lookupService.addValue("Jimmy Doe", "Football");
@@ -252,7 +251,7 @@ public class TestLookupRecord {
     }
 
     @Test
-    public void testInvalidUnlessAllRequiredPropertiesAdded() throws InitializationException {
+    public void testInvalidUnlessAllRequiredPropertiesAdded() {
         runner.removeProperty(new PropertyDescriptor.Builder().name("lookup").build());
         runner.setProperty("hello", "/name");
         runner.assertNotValid();
@@ -266,7 +265,7 @@ public class TestLookupRecord {
 
 
     @Test
-    public void testAddFieldsToExistingRecord() throws InitializationException, IOException {
+    public void testAddFieldsToExistingRecord() throws InitializationException {
         final RecordLookup lookupService = new RecordLookup();
         runner.addControllerService("lookup", lookupService);
         runner.enableControllerService(lookupService);
@@ -275,7 +274,7 @@ public class TestLookupRecord {
         fields.add(new RecordField("favorite", RecordFieldType.STRING.getDataType()));
         fields.add(new RecordField("least", RecordFieldType.STRING.getDataType()));
         final RecordSchema schema = new SimpleRecordSchema(fields);
-        final Record sports = new MapRecord(schema, new HashMap<String, Object>());
+        final Record sports = new MapRecord(schema, new HashMap<>());
 
         sports.setValue("favorite", "basketball");
         sports.setValue("least", "soccer");
@@ -318,7 +317,7 @@ public class TestLookupRecord {
         fields.add(new RecordField("favorite", RecordFieldType.STRING.getDataType()));
         fields.add(new RecordField("least", RecordFieldType.STRING.getDataType()));
         final RecordSchema schema = new SimpleRecordSchema(fields);
-        final Record sports = new MapRecord(schema, new HashMap<String, Object>());
+        final Record sports = new MapRecord(schema, new HashMap<>());
 
         sports.setValue("favorite", "basketball");
         sports.setValue("least", "soccer");
@@ -364,7 +363,7 @@ public class TestLookupRecord {
         fields.add(new RecordField("favorite", RecordFieldType.STRING.getDataType()));
         fields.add(new RecordField("least", RecordFieldType.STRING.getDataType()));
         final RecordSchema schema = new SimpleRecordSchema(fields);
-        final Record sports = new MapRecord(schema, new HashMap<String, Object>());
+        final Record sports = new MapRecord(schema, new HashMap<>());
 
         sports.setValue("favorite", "basketball");
         sports.setValue("least", "soccer");
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java
index 656ad5f..58387ef 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java
@@ -59,6 +59,7 @@ import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.util.StringUtils;
 
 import javax.net.ssl.SSLContext;
+import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.Proxy;
@@ -298,8 +299,11 @@ public class RestLookupService extends AbstractControllerService implements Reco
                 return Optional.empty();
             }
 
-            InputStream is = responseBody.byteStream();
-            Record record = handleResponse(is, context);
+            final Record record;
+            try (final InputStream is = responseBody.byteStream();
+                final InputStream bufferedIn = new BufferedInputStream(is)) {
+                record = handleResponse(bufferedIn, context);
+            }
 
             return Optional.ofNullable(record);
         } catch (Exception e) {
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java
index 6ff380d..cb42fed 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java
@@ -20,6 +20,7 @@ package org.apache.nifi.serialization;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
 
 import java.io.IOException;
@@ -38,8 +39,10 @@ public interface RecordReaderFactory extends ControllerService {
      * Create a RecordReader instance to read records from specified InputStream.
      * This method calls {@link #createRecordReader(Map, InputStream, ComponentLog)} with Attributes of the specified FlowFile.
      * @param flowFile Attributes of this FlowFile are used to resolve Record Schema via Expression Language dynamically. This can be null.
-     * @param in InputStream containing Records. This can be null or empty stream.
-     * @param logger A logger bind to a component
+     *
+     * @param in InputStream containing Records.
+     * @param logger A logger bound to a component
+     *
      * @return Created RecordReader instance
      */
     default RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException {
@@ -47,11 +50,26 @@ public interface RecordReaderFactory extends ControllerService {
     }
 
     /**
+     * <p>
      * Create a RecordReader instance to read records from specified InputStream.
-     * @param variables A map contains variables which is used to resolve Record Schema via Expression Language dynamically.
+     * <p>
+     *
+     * <p>
+     * Many Record Readers will need to read from the Input Stream in order to ascertain the appropriate Schema, and then
+     * re-read some of the data in order to read the Records. As a result, it is common for Readers to use
+     * {@link InputStream#mark(int) mark}/{@link InputStream#reset() reset}, so this should be considered when providing an
+     * InputStream. The {@link InputStream} that is provided by {@link org.apache.nifi.processor.ProcessSession#read(FlowFile) SessionProcess.read} /
+     * {@link org.apache.nifi.processor.ProcessSession#write(FlowFile, StreamCallback) ProcessSession.write} does provide the ability to use mark/reset
+     * and does so in a way that allows any number of bytes to be read before resetting without requiring that data be buffered. Therefore, it is recommended
+     * that when providing an InputStream from {@link org.apache.nifi.processor.ProcessSession ProcessSession} that the InputStream not be wrapped in a
+     * BufferedInputStream. However, if the stream is coming from elsewhere, it may be necessary.
+     * </p>
+     *
+     * @param variables A map containing variables which is used to resolve the Record Schema dynamically via Expression Language.
      *                 This can be null or empty.
-     * @param in InputStream containing Records. This can be null or empty stream.
-     * @param logger A logger bind to a component
+     * @param in InputStream containing Records.
+     * @param logger A logger bound to a component
+     *
      * @return Created RecordReader instance
      */
     RecordReader createRecordReader(Map<String, String> variables, InputStream in, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException;
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSchemaCacheService.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSchemaCacheService.java
new file mode 100644
index 0000000..87925ae
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSchemaCacheService.java
@@ -0,0 +1,51 @@
+/*
+ * 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.controller.ControllerService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.util.Optional;
+
+public interface RecordSchemaCacheService extends ControllerService {
+    public static final String CACHE_IDENTIFIER_ATTRIBUTE = "schema.cache.identifier";
+
+    /**
+     * Updates the cache to include the given Record Schema and returns an identifier
+     * for the Schema. If the schema already exists in the cache, the existing identifier
+     * is returned. Otherwise, the schema is added to the cache and a new identifier is
+     * created and returned. This identifier can then be used to retrieve the Record Schema
+     * via the {@link #getSchema(String)} method
+     *
+     * @param schema the schema to cache
+     * @return a unique identifier for the schema
+     */
+    String cacheSchema(RecordSchema schema);
+
+    /**
+     * Returns the Schema with the given identifier, if it can be found in the cache.
+     * Note that the cache may choose to evict schemas for any number of reasons and, as such,
+     * the service may return an empty Optional even immediately after the Schema is cached
+     * via the {@link #cacheSchema(RecordSchema)}.
+     *
+     * @param schemaIdentifier the identifier of the schema
+     * @return an Optional holding the Record Schema with the given identifier, if it can be found,
+     * or an empty Optional if the schema cannot be found
+     */
+    Optional<RecordSchema> getSchema(String schemaIdentifier);
+
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml
index b16464e..d7c3d5b 100755
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml
@@ -1,13 +1,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <!-- 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 
+    <!-- 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. -->
     <modelVersion>4.0.0</modelVersion>
     <parent>
@@ -140,6 +140,7 @@
                         <exclude>src/test/resources/csv/multi-bank-account.csv</exclude>
                         <exclude>src/test/resources/csv/single-bank-account.csv</exclude>
                         <exclude>src/test/resources/csv/multi-bank-account_escapedchar.csv</exclude>
+                        <exclude>src/test/resources/csv/prov-events.csv</exclude>
                         <exclude>src/test/resources/grok/error-with-stack-trace.log</exclude>
                         <exclude>src/test/resources/grok/nifi-log-sample-multiline-with-stacktrace.log</exclude>
                         <exclude>src/test/resources/grok/nifi-log-sample.log</exclude>
@@ -160,6 +161,8 @@
                         <exclude>src/test/resources/json/output/dataTypes.json</exclude>
                         <exclude>src/test/resources/json/elements-for-record-choice.json</exclude>
                         <exclude>src/test/resources/json/record-choice.avsc</exclude>
+                        <exclude>src/test/resources/json/prov-events.json</exclude>
+                        <exclude>src/test/resources/json/docs-example.json</exclude>
                         <exclude>src/test/resources/syslog/syslog5424/log.txt</exclude>
                         <exclude>src/test/resources/syslog/syslog5424/log_all.txt</exclude>
                         <exclude>src/test/resources/syslog/syslog5424/log_mix.txt</exclude>
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java
index 434d73f..11379b9 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java
@@ -25,7 +25,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -85,7 +85,7 @@ public class AvroReader extends SchemaRegistryService implements RecordReaderFac
     }
 
     @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(String strategy, SchemaRegistry schemaRegistry, ConfigurationContext context) {
+    protected SchemaAccessStrategy getSchemaAccessStrategy(String strategy, SchemaRegistry schemaRegistry, PropertyContext context) {
         if (EMBEDDED_AVRO_SCHEMA.getValue().equals(strategy)) {
             return new EmbeddedAvroSchemaAccessStrategy();
         } else {
@@ -94,15 +94,6 @@ public class AvroReader extends SchemaRegistryService implements RecordReaderFac
     }
 
     @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(String allowableValue, SchemaRegistry schemaRegistry, ValidationContext context) {
-        if (EMBEDDED_AVRO_SCHEMA.getValue().equals(allowableValue)) {
-            return new EmbeddedAvroSchemaAccessStrategy();
-        } else {
-            return super.getSchemaAccessStrategy(allowableValue, schemaRegistry, context);
-        }
-    }
-
-    @Override
     public RecordReader createRecordReader(final Map<String, String> variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException {
         final String schemaAccessStrategy = getConfigurationContext().getProperty(getSchemaAcessStrategyDescriptor()).getValue();
         if (EMBEDDED_AVRO_SCHEMA.getValue().equals(schemaAccessStrategy)) {
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java
index 9c31cca..624981b 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java
@@ -17,20 +17,10 @@
 
 package org.apache.nifi.csv;
 
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.Reader;
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.commons.csv.CSVFormat;
 import org.apache.commons.csv.CSVParser;
 import org.apache.commons.io.input.BOMInputStream;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.schema.access.SchemaAccessStrategy;
 import org.apache.nifi.schema.access.SchemaField;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
@@ -39,19 +29,24 @@ import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
 
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 public class CSVHeaderSchemaStrategy implements SchemaAccessStrategy {
     private static final Set<SchemaField> schemaFields = EnumSet.noneOf(SchemaField.class);
 
-    private final ConfigurationContext context;
+    private final PropertyContext context;
 
-    public CSVHeaderSchemaStrategy(final ConfigurationContext context) {
+    public CSVHeaderSchemaStrategy(final PropertyContext context) {
         this.context = context;
     }
 
-    public CSVHeaderSchemaStrategy(final ValidationContext context) {
-        this.context = null;
-    }
-
     @Override
     public RecordSchema getSchema(Map<String, String> variables, final InputStream contentStream, final RecordSchema readSchema) throws SchemaNotFoundException {
         if (this.context == null) {
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java
index 858ef70..af46147 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java
@@ -17,26 +17,25 @@
 
 package org.apache.nifi.csv;
 
-import java.io.BufferedInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVRecord;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.schema.access.SchemaAccessStrategy;
 import org.apache.nifi.schema.access.SchemaAccessUtils;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schema.inference.InferSchemaAccessStrategy;
+import org.apache.nifi.schema.inference.SchemaInferenceEngine;
+import org.apache.nifi.schema.inference.RecordSourceFactory;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.schema.inference.TimeValueInference;
 import org.apache.nifi.schemaregistry.services.SchemaRegistry;
 import org.apache.nifi.serialization.DateTimeUtils;
 import org.apache.nifi.serialization.RecordReader;
@@ -45,6 +44,12 @@ import org.apache.nifi.serialization.SchemaRegistryService;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.stream.io.NonCloseableInputStream;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
 @Tags({"csv", "parse", "record", "row", "reader", "delimited", "comma", "separated", "values"})
 @CapabilityDescription("Parses CSV-formatted data, returning each row in the CSV file as a separate record. "
     + "This reader assumes that the first line in the content is the column names and all subsequent lines are "
@@ -117,7 +122,7 @@ public class CSVReader extends SchemaRegistryService implements RecordReaderFact
         // Ensure that if we are deriving schema from header that we always treat the first line as a header,
         // regardless of the 'First Line is Header' property
         final String accessStrategy = context.getProperty(SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY).getValue();
-        if (headerDerivedAllowableValue.getValue().equals(accessStrategy)) {
+        if (headerDerivedAllowableValue.getValue().equals(accessStrategy) || SchemaInferenceUtil.INFER_SCHEMA.getValue().equals(accessStrategy)) {
             this.csvFormat = this.csvFormat.withFirstRecordAsHeader();
             this.firstLineIsHeader = true;
         }
@@ -126,33 +131,27 @@ public class CSVReader extends SchemaRegistryService implements RecordReaderFact
     @Override
     public RecordReader createRecordReader(final Map<String, String> variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException {
         // Use Mark/Reset of a BufferedInputStream in case we read from the Input Stream for the header.
-        final BufferedInputStream bufferedIn = new BufferedInputStream(in);
-        bufferedIn.mark(1024 * 1024);
-        final RecordSchema schema = getSchema(variables, new NonCloseableInputStream(bufferedIn), null);
-        bufferedIn.reset();
+        in.mark(1024 * 1024);
+        final RecordSchema schema = getSchema(variables, new NonCloseableInputStream(in), null);
+        in.reset();
 
         if(APACHE_COMMONS_CSV.getValue().equals(csvParser)) {
-            return new CSVRecordReader(bufferedIn, logger, schema, csvFormat, firstLineIsHeader, ignoreHeader, dateFormat, timeFormat, timestampFormat, charSet);
+            return new CSVRecordReader(in, logger, schema, csvFormat, firstLineIsHeader, ignoreHeader, dateFormat, timeFormat, timestampFormat, charSet);
         } else if(JACKSON_CSV.getValue().equals(csvParser)) {
-            return new JacksonCSVRecordReader(bufferedIn, logger, schema, csvFormat, firstLineIsHeader, ignoreHeader, dateFormat, timeFormat, timestampFormat, charSet);
+            return new JacksonCSVRecordReader(in, logger, schema, csvFormat, firstLineIsHeader, ignoreHeader, dateFormat, timeFormat, timestampFormat, charSet);
         } else {
             throw new IOException("Parser not supported");
         }
     }
 
     @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final ConfigurationContext context) {
-        if (strategy.equalsIgnoreCase(headerDerivedAllowableValue.getValue())) {
-            return new CSVHeaderSchemaStrategy(context);
-        }
-
-        return super.getSchemaAccessStrategy(strategy, schemaRegistry, context);
-    }
-
-    @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry, final ValidationContext context) {
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry, final PropertyContext context) {
         if (allowableValue.equalsIgnoreCase(headerDerivedAllowableValue.getValue())) {
             return new CSVHeaderSchemaStrategy(context);
+        } else if (allowableValue.equalsIgnoreCase(SchemaInferenceUtil.INFER_SCHEMA.getValue())) {
+            final RecordSourceFactory<CSVRecord> sourceFactory = (var, in) -> new CSVRecordSource(in, context);
+            final SchemaInferenceEngine<CSVRecord> inference = new CSVSchemaInference(new TimeValueInference(dateFormat, timeFormat, timestampFormat));
+            return new InferSchemaAccessStrategy<>(sourceFactory, inference, getLogger());
         }
 
         return super.getSchemaAccessStrategy(allowableValue, schemaRegistry, context);
@@ -162,11 +161,12 @@ public class CSVReader extends SchemaRegistryService implements RecordReaderFact
     protected List<AllowableValue> getSchemaAccessStrategyValues() {
         final List<AllowableValue> allowableValues = new ArrayList<>(super.getSchemaAccessStrategyValues());
         allowableValues.add(headerDerivedAllowableValue);
+        allowableValues.add(SchemaInferenceUtil.INFER_SCHEMA);
         return allowableValues;
     }
 
     @Override
     protected AllowableValue getDefaultSchemaAccessStrategy() {
-        return headerDerivedAllowableValue;
+        return SchemaInferenceUtil.INFER_SCHEMA;
     }
 }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSource.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSource.java
new file mode 100644
index 0000000..ab4362a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSource.java
@@ -0,0 +1,61 @@
+/*
+ * 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.csv;
+
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.csv.CSVRecord;
+import org.apache.commons.io.input.BOMInputStream;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.schema.inference.RecordSource;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.io.UnsupportedEncodingException;
+import java.util.Iterator;
+
+public class CSVRecordSource implements RecordSource<CSVRecord> {
+    private final Iterator<CSVRecord> csvRecordIterator;
+
+    public CSVRecordSource(final InputStream in, final PropertyContext context) throws IOException {
+        final String charset = context.getProperty(CSVUtils.CHARSET).getValue();
+
+        final Reader reader;
+        try {
+            reader = new InputStreamReader(new BOMInputStream(in), charset);
+        } catch (UnsupportedEncodingException e) {
+            throw new ProcessException(e);
+        }
+
+        final CSVFormat csvFormat = CSVUtils.createCSVFormat(context).withFirstRecordAsHeader().withTrim();
+        final CSVParser csvParser = new CSVParser(reader, csvFormat);
+        csvRecordIterator = csvParser.iterator();
+    }
+
+    @Override
+    public CSVRecord next() {
+        if (csvRecordIterator.hasNext()) {
+            final CSVRecord record = csvRecordIterator.next();
+            return record;
+        }
+
+        return null;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVSchemaInference.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVSchemaInference.java
new file mode 100644
index 0000000..b81ddc0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVSchemaInference.java
@@ -0,0 +1,130 @@
+/*
+ * 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.csv;
+
+import org.apache.commons.csv.CSVRecord;
+import org.apache.commons.lang3.math.NumberUtils;
+import org.apache.nifi.schema.inference.FieldTypeInference;
+import org.apache.nifi.schema.inference.SchemaInferenceEngine;
+import org.apache.nifi.schema.inference.RecordSource;
+import org.apache.nifi.schema.inference.TimeValueInference;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+public class CSVSchemaInference implements SchemaInferenceEngine<CSVRecord> {
+
+    private final TimeValueInference timeValueInference;
+
+    public CSVSchemaInference(final TimeValueInference timeValueInference) {
+        this.timeValueInference = timeValueInference;
+    }
+
+
+    @Override
+    public RecordSchema inferSchema(final RecordSource<CSVRecord> recordSource) throws IOException {
+        final Map<String, FieldTypeInference> typeMap = new LinkedHashMap<>();
+        while (true) {
+            final CSVRecord rawRecord = recordSource.next();
+            if (rawRecord == null) {
+                break;
+            }
+
+            inferSchema(rawRecord, typeMap);
+        }
+
+        return createSchema(typeMap);
+    }
+
+
+    private void inferSchema(final CSVRecord csvRecord, final Map<String, FieldTypeInference> typeMap) {
+        final Map<String, String> values = csvRecord.toMap();
+        for (final Map.Entry<String, String> entry : values.entrySet()) {
+            final String value = entry.getValue();
+            if (value == null) {
+                return;
+            }
+
+            final String fieldName = entry.getKey();
+            final FieldTypeInference typeInference = typeMap.computeIfAbsent(fieldName, key -> new FieldTypeInference());
+            final String trimmed = trim(value);
+            final DataType dataType = getDataType(trimmed);
+            typeInference.addPossibleDataType(dataType);
+        }
+    }
+
+    private String trim(String value) {
+        return (value.length() > 1) && value.startsWith("\"") && value.endsWith("\"") ? value.substring(1, value.length() - 1) : value;
+    }
+
+
+    private DataType getDataType(final String value) {
+        if (value == null || value.isEmpty()) {
+            return null;
+        }
+
+        if (NumberUtils.isParsable(value)) {
+            if (value.contains(".")) {
+                try {
+                    final double doubleValue = Double.parseDouble(value);
+                    if (doubleValue > Float.MAX_VALUE || doubleValue < Float.MIN_VALUE) {
+                        return RecordFieldType.DOUBLE.getDataType();
+                    }
+
+                    return RecordFieldType.FLOAT.getDataType();
+                } catch (final NumberFormatException nfe) {
+                    return RecordFieldType.STRING.getDataType();
+                }
+            }
+
+            try {
+                final long longValue = Long.parseLong(value);
+                if (longValue > Integer.MAX_VALUE || longValue < Integer.MIN_VALUE) {
+                    return RecordFieldType.LONG.getDataType();
+                }
+
+                return RecordFieldType.INT.getDataType();
+            } catch (final NumberFormatException nfe) {
+                return RecordFieldType.STRING.getDataType();
+            }
+        }
+
+        if (value.equalsIgnoreCase("true") || value.equalsIgnoreCase("false")) {
+            return RecordFieldType.BOOLEAN.getDataType();
+        }
+
+        final Optional<DataType> timeDataType = timeValueInference.getDataType(value);
+        return timeDataType.orElse(RecordFieldType.STRING.getDataType());
+    }
+
+
+    private RecordSchema createSchema(final Map<String, FieldTypeInference> inferences) {
+        final List<RecordField> recordFields = new ArrayList<>(inferences.size());
+        inferences.forEach((fieldName, type) -> recordFields.add(new RecordField(fieldName, type.toDataType(), true)));
+        return new SimpleRecordSchema(recordFields);
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java
index 9e7293b..a462c72 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java
@@ -28,6 +28,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
@@ -240,16 +241,7 @@ public class GrokReader extends SchemaRegistryService implements RecordReaderFac
     }
 
     @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final ConfigurationContext context) {
-        if (strategy.equalsIgnoreCase(STRING_FIELDS_FROM_GROK_EXPRESSION.getValue())) {
-            return createAccessStrategy();
-        } else {
-            return super.getSchemaAccessStrategy(strategy, schemaRegistry, context);
-        }
-    }
-
-    @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final ValidationContext context) {
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final PropertyContext context) {
         if (strategy.equalsIgnoreCase(STRING_FIELDS_FROM_GROK_EXPRESSION.getValue())) {
             return createAccessStrategy();
         } else {
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
index 4f9a791..9874a02 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
@@ -17,14 +17,6 @@
 
 package org.apache.nifi.json;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Optional;
-
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
@@ -46,11 +38,16 @@ import org.codehaus.jackson.JsonToken;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.node.ArrayNode;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
 
 public abstract class AbstractJsonRowRecordReader implements RecordReader {
     private final ComponentLog logger;
     private final JsonParser jsonParser;
-    private final boolean array;
     private final JsonNode firstJsonNode;
 
     private boolean firstObjectConsumed = false;
@@ -58,7 +55,10 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
     private static final JsonFactory jsonFactory = new JsonFactory();
     private static final ObjectMapper codec = new ObjectMapper();
 
-    public AbstractJsonRowRecordReader(final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
+
+    public AbstractJsonRowRecordReader(final InputStream in, final ComponentLog logger, final String dateFormat, final String timeFormat, final String timestampFormat)
+            throws IOException, MalformedRecordException {
+
         this.logger = logger;
 
         try {
@@ -67,10 +67,7 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
 
             JsonToken token = jsonParser.nextToken();
             if (token == JsonToken.START_ARRAY) {
-                array = true;
                 token = jsonParser.nextToken(); // advance to START_OBJECT token
-            } else {
-                array = false;
             }
 
             if (token == JsonToken.START_OBJECT) { // could be END_ARRAY also
@@ -87,13 +84,15 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
     @Override
     public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException {
         final JsonNode nextNode = getNextJsonNode();
+        if (nextNode == null) {
+            return null;
+        }
+
         final RecordSchema schema = getSchema();
         try {
             return convertJsonNodeToRecord(nextNode, schema, coerceTypes, dropUnknownFields);
         } catch (final MalformedRecordException mre) {
             throw mre;
-        } catch (final IOException ioe) {
-            throw ioe;
         } catch (final Exception e) {
             logger.debug("Failed to convert JSON Element {} into a Record object using schema {} due to {}", new Object[] {nextNode, schema, e.toString(), e});
             throw new MalformedRecordException("Successfully parsed a JSON object from input but failed to convert into a Record object with the given schema", e);
@@ -200,7 +199,7 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
     }
 
 
-    private JsonNode getNextJsonNode() throws JsonParseException, IOException, MalformedRecordException {
+    protected JsonNode getNextJsonNode() throws IOException, MalformedRecordException {
         if (!firstObjectConsumed) {
             firstObjectConsumed = true;
             return firstJsonNode;
@@ -227,23 +226,10 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
         }
     }
 
-
     @Override
     public void close() throws IOException {
         jsonParser.close();
     }
 
-    protected JsonParser getJsonParser() {
-        return jsonParser;
-    }
-
-    protected JsonFactory getJsonFactory() {
-        return jsonFactory;
-    }
-
-    protected Optional<JsonNode> getFirstJsonNode() {
-        return Optional.ofNullable(firstJsonNode);
-    }
-
     protected abstract Record convertJsonNodeToRecord(JsonNode nextNode, RecordSchema schema, boolean coerceTypes, boolean dropUnknownFields) throws IOException, MalformedRecordException;
 }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java
index d8f21ca..a59bc19 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java
@@ -17,35 +17,45 @@
 
 package org.apache.nifi.json;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
+import com.jayway.jsonpath.JsonPath;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaAccessStrategy;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schema.inference.SchemaInferenceEngine;
+import org.apache.nifi.schema.inference.RecordSourceFactory;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.schema.inference.TimeValueInference;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
 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 org.codehaus.jackson.JsonNode;
 
-import com.jayway.jsonpath.JsonPath;
+import java.io.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
 
 @Tags({"json", "jsonpath", "record", "reader", "parser"})
 @CapabilityDescription("Parses JSON records and evaluates user-defined JSON Path's against each JSON object. While the reader expects each record "
@@ -61,6 +71,7 @@ import com.jayway.jsonpath.JsonPath;
     expressionLanguageScope=ExpressionLanguageScope.NONE)
 public class JsonPathReader extends SchemaRegistryService implements RecordReaderFactory {
 
+
     private volatile String dateFormat;
     private volatile String timeFormat;
     private volatile String timestampFormat;
@@ -130,9 +141,31 @@ public class JsonPathReader extends SchemaRegistryService implements RecordReade
     }
 
     @Override
+    protected List<AllowableValue> getSchemaAccessStrategyValues() {
+        final List<AllowableValue> allowableValues = new ArrayList<>(super.getSchemaAccessStrategyValues());
+        allowableValues.add(SchemaInferenceUtil.INFER_SCHEMA);
+        return allowableValues;
+    }
+
+    @Override
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final PropertyContext context) {
+        final RecordSourceFactory<JsonNode> jsonSourceFactory = (var, in) -> new JsonRecordSource(in);
+        final Supplier<SchemaInferenceEngine<JsonNode>> inferenceSupplier = () -> new JsonSchemaInference(new TimeValueInference(dateFormat, timeFormat, timestampFormat));
+
+        return SchemaInferenceUtil.getSchemaAccessStrategy(strategy, context, getLogger(), jsonSourceFactory, inferenceSupplier,
+            () -> super.getSchemaAccessStrategy(strategy, schemaRegistry, context));
+    }
+
+    @Override
+    protected AllowableValue getDefaultSchemaAccessStrategy() {
+        return SchemaInferenceUtil.INFER_SCHEMA;
+    }
+
+    @Override
     public RecordReader createRecordReader(final Map<String, String> variables, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException {
-        final RecordSchema schema = getSchema(variables, in, null);
-        return new JsonPathRowRecordReader(jsonPaths, schema, in, logger, dateFormat, timeFormat, timestampFormat);
+        final InputStream bufferedIn = new BufferedInputStream(in);
+        final RecordSchema schema = getSchema(variables, bufferedIn, null);
+        return new JsonPathRowRecordReader(jsonPaths, schema, bufferedIn, logger, dateFormat, timeFormat, timestampFormat);
     }
 
 }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java
index 90ab799..a664b88 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java
@@ -17,17 +17,11 @@
 
 package org.apache.nifi.json;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.text.DateFormat;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.function.Supplier;
-
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.DocumentContext;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.PathNotFoundException;
+import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.SimpleRecordSchema;
@@ -43,11 +37,16 @@ import org.apache.nifi.serialization.record.util.DataTypeUtils;
 import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
 import org.codehaus.jackson.JsonNode;
 
-import com.jayway.jsonpath.Configuration;
-import com.jayway.jsonpath.DocumentContext;
-import com.jayway.jsonpath.JsonPath;
-import com.jayway.jsonpath.PathNotFoundException;
-import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
 
 public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
     private static final Configuration STRICT_PROVIDER_CONFIGURATION = Configuration.builder().jsonProvider(new JacksonJsonProvider()).build();
@@ -62,9 +61,9 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
     private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
 
     public JsonPathRowRecordReader(final LinkedHashMap<String, JsonPath> jsonPaths, final RecordSchema schema, final InputStream in, final ComponentLog logger,
-        final String dateFormat, final String timeFormat, final String timestampFormat)
-        throws MalformedRecordException, IOException {
-        super(in, logger);
+                final String dateFormat, final String timeFormat, final String timestampFormat)
+                throws MalformedRecordException, IOException {
+        super(in, logger, dateFormat, timeFormat, timestampFormat);
 
         final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
         final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
@@ -91,7 +90,7 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
     }
 
     @Override
-    protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema, final boolean coerceTypes, final boolean dropUnknownFields) throws IOException {
+    protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema, final boolean coerceTypes, final boolean dropUnknownFields) {
         if (jsonNode == null) {
             return null;
         }
@@ -118,12 +117,12 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
             }
 
             final Optional<RecordField> field = schema.getField(fieldName);
-            final Object defaultValue = field.isPresent() ? field.get().getDefaultValue() : null;
+            final Object defaultValue = field.map(RecordField::getDefaultValue).orElse(null);
 
             if (coerceTypes && desiredType != null) {
                 value = convert(value, desiredType, fieldName, defaultValue);
             } else {
-                final DataType dataType = field.isPresent() ? field.get().getDataType() : null;
+                final DataType dataType = field.map(RecordField::getDataType).orElse(null);
                 value = convert(value, dataType);
             }
 
@@ -232,7 +231,7 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
                 final Optional<DataType> desiredTypeOption = childSchema.getDataType(key);
                 if (desiredTypeOption.isPresent()) {
                     final Optional<RecordField> field = childSchema.getField(key);
-                    final Object defaultFieldValue = field.isPresent() ? field.get().getDefaultValue() : null;
+                    final Object defaultFieldValue = field.map(RecordField::getDefaultValue).orElse(null);
 
                     final Object coercedValue = convert(entry.getValue(), desiredTypeOption.get(), fieldName + "." + key, defaultFieldValue);
                     coercedValues.put(key, coercedValue);
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSource.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSource.java
new file mode 100644
index 0000000..3887fb0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSource.java
@@ -0,0 +1,55 @@
+/*
+ * 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.json;
+
+import org.apache.nifi.schema.inference.RecordSource;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class JsonRecordSource implements RecordSource<JsonNode> {
+    private static final JsonFactory jsonFactory;
+    private final JsonParser jsonParser;
+
+    static {
+        jsonFactory = new JsonFactory();
+        jsonFactory.setCodec(new ObjectMapper());
+    }
+
+    public JsonRecordSource(final InputStream in) throws IOException {
+        jsonParser = jsonFactory.createJsonParser(in);
+    }
+
+    @Override
+    public JsonNode next() throws IOException {
+        while (true) {
+            final JsonToken token = jsonParser.nextToken();
+            if (token == null) {
+                return null;
+            }
+
+            if (token == JsonToken.START_OBJECT) {
+                return jsonParser.readValueAsTree();
+            }
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonSchemaInference.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonSchemaInference.java
new file mode 100644
index 0000000..b09c79f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonSchemaInference.java
@@ -0,0 +1,112 @@
+/*
+ * 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.json;
+
+import org.apache.nifi.schema.inference.HierarchicalSchemaInference;
+import org.apache.nifi.schema.inference.TimeValueInference;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.node.ArrayNode;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class JsonSchemaInference extends HierarchicalSchemaInference<JsonNode> {
+
+    private final TimeValueInference timeValueInference;
+
+    public JsonSchemaInference(final TimeValueInference timeValueInference) {
+        this.timeValueInference = timeValueInference;
+    }
+
+
+    protected DataType getDataType(final JsonNode jsonNode) {
+        if (jsonNode.isTextual()) {
+            final String text = jsonNode.getTextValue();
+            if (text == null) {
+                return RecordFieldType.STRING.getDataType();
+            }
+
+            final Optional<DataType> timeDataType = timeValueInference.getDataType(text);
+            return timeDataType.orElse(RecordFieldType.STRING.getDataType());
+        }
+
+        if (jsonNode.isObject()) {
+            final RecordSchema schema = createSchema(jsonNode);
+            return RecordFieldType.RECORD.getRecordDataType(schema);
+        }
+
+        if (jsonNode.isIntegralNumber()) {
+            return RecordFieldType.LONG.getDataType();
+        }
+
+        if (jsonNode.isFloatingPointNumber()) {
+            return RecordFieldType.FLOAT.getDataType();
+        }
+        if (jsonNode.isDouble()) {
+            return RecordFieldType.DOUBLE.getDataType();
+        }
+        if (jsonNode.isBinary()) {
+            return RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType());
+        }
+        if (jsonNode.isBoolean()) {
+            return RecordFieldType.BOOLEAN.getDataType();
+        }
+
+        return null;
+    }
+
+    @Override
+    protected boolean isObject(final JsonNode value) {
+        return value.isObject();
+    }
+
+    @Override
+    protected boolean isArray(final JsonNode value) {
+        return value.isArray();
+    }
+
+    @Override
+    protected void forEachFieldInRecord(final JsonNode rawRecord, final BiConsumer<String, JsonNode> fieldConsumer) {
+        final Iterator<Map.Entry<String, JsonNode>> itr = rawRecord.getFields();
+        while (itr.hasNext()) {
+            final Map.Entry<String, JsonNode> entry = itr.next();
+            final String fieldName = entry.getKey();
+            final JsonNode value = entry.getValue();
+
+            fieldConsumer.accept(fieldName, value);
+        }
+    }
+
+    @Override
+    protected void forEachRawRecordInArray(final JsonNode arrayRecord, final Consumer<JsonNode> rawRecordConsumer) {
+        final ArrayNode arrayNode = (ArrayNode) arrayRecord;
+        for (final JsonNode element : arrayNode) {
+            rawRecordConsumer.accept(element);
+        }
+    }
+
+    @Override
+    protected String getRootName(final JsonNode rawRecord) {
+        return null;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java
index 27833ad..c805421 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java
@@ -17,25 +17,39 @@
 
 package org.apache.nifi.json;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaAccessStrategy;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schema.inference.SchemaInferenceEngine;
+import org.apache.nifi.schema.inference.RecordSourceFactory;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.schema.inference.TimeValueInference;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
 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 org.codehaus.jackson.JsonNode;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+
+import static org.apache.nifi.schema.inference.SchemaInferenceUtil.INFER_SCHEMA;
+import static org.apache.nifi.schema.inference.SchemaInferenceUtil.SCHEMA_CACHE;
 
 @Tags({"json", "tree", "record", "reader", "parser"})
 @CapabilityDescription("Parses JSON into individual Record objects. While the reader expects each record "
@@ -55,6 +69,7 @@ public class JsonTreeReader extends SchemaRegistryService implements RecordReade
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(SCHEMA_CACHE);
         properties.add(DateTimeUtils.DATE_FORMAT);
         properties.add(DateTimeUtils.TIME_FORMAT);
         properties.add(DateTimeUtils.TIMESTAMP_FORMAT);
@@ -69,7 +84,29 @@ public class JsonTreeReader extends SchemaRegistryService implements RecordReade
     }
 
     @Override
+    protected List<AllowableValue> getSchemaAccessStrategyValues() {
+        final List<AllowableValue> allowableValues = new ArrayList<>(super.getSchemaAccessStrategyValues());
+        allowableValues.add(INFER_SCHEMA);
+        return allowableValues;
+    }
+
+    @Override
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final PropertyContext context) {
+        final RecordSourceFactory<JsonNode> jsonSourceFactory = (var, in) -> new JsonRecordSource(in);
+        final Supplier<SchemaInferenceEngine<JsonNode>> inferenceSupplier = () -> new JsonSchemaInference(new TimeValueInference(dateFormat, timeFormat, timestampFormat));
+
+        return SchemaInferenceUtil.getSchemaAccessStrategy(strategy, context, getLogger(), jsonSourceFactory, inferenceSupplier,
+            () -> super.getSchemaAccessStrategy(strategy, schemaRegistry, context));
+    }
+
+    @Override
+    protected AllowableValue getDefaultSchemaAccessStrategy() {
+        return INFER_SCHEMA;
+    }
+
+    @Override
     public RecordReader createRecordReader(final Map<String, String> variables, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException {
-        return new JsonTreeRowRecordReader(in, logger, getSchema(variables, in, null), dateFormat, timeFormat, timestampFormat);
+        final RecordSchema schema = getSchema(variables, in, null);
+        return new JsonTreeRowRecordReader(in, logger, schema, dateFormat, timeFormat, timestampFormat);
     }
 }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java
index e53fcc0..058d9ee 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java
@@ -17,16 +17,6 @@
 
 package org.apache.nifi.json;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.text.DateFormat;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.function.Supplier;
-
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.SimpleRecordSchema;
@@ -44,6 +34,15 @@ import org.apache.nifi.serialization.record.util.DataTypeUtils;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.node.ArrayNode;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
 
 public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
     private final RecordSchema schema;
@@ -55,7 +54,7 @@ public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
 
     public JsonTreeRowRecordReader(final InputStream in, final ComponentLog logger, final RecordSchema schema,
         final String dateFormat, final String timeFormat, final String timestampFormat) throws IOException, MalformedRecordException {
-        super(in, logger);
+        super(in, logger, dateFormat, timeFormat, timestampFormat);
         this.schema = schema;
 
         final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
@@ -68,6 +67,7 @@ public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
     }
 
 
+
     @Override
     protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema, final boolean coerceTypes, final boolean dropUnknownFields)
         throws IOException, MalformedRecordException {
@@ -143,7 +143,7 @@ public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
             }
         }
 
-        final Supplier<String> supplier = () -> jsonNode.toString();
+        final Supplier<String> supplier = jsonNode::toString;
         return new MapRecord(schema, values, SerializedForm.of(supplier, "application/json"), false, dropUnknown);
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java
index 4ce6ceb..2b65eea 100755
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java
@@ -36,7 +36,6 @@ import org.apache.nifi.serialization.record.type.MapDataType;
 import org.apache.nifi.serialization.record.type.RecordDataType;
 import org.apache.nifi.serialization.record.util.DataTypeUtils;
 import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.JsonGenerator;
 import org.codehaus.jackson.util.MinimalPrettyPrinter;
 
@@ -53,7 +52,6 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
     private final ComponentLog logger;
     private final SchemaAccessWriter schemaAccess;
     private final RecordSchema recordSchema;
-    private final JsonFactory factory = new JsonFactory();
     private final JsonGenerator generator;
     private final NullSuppression nullSuppression;
     private final OutputGrouping outputGrouping;
@@ -87,6 +85,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
         LAZY_TIME_FORMAT = () -> tf;
         LAZY_TIMESTAMP_FORMAT = () -> tsf;
 
+        final JsonFactory factory = new JsonFactory();
         this.generator = factory.createJsonGenerator(out);
         if (prettyPrint) {
             generator.useDefaultPrettyPrinter();
@@ -141,7 +140,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
             schemaAccess.writeHeader(recordSchema, getOutputStream());
         }
 
-        writeRecord(record, recordSchema, generator, g -> g.writeStartObject(), g -> g.writeEndObject(), true);
+        writeRecord(record, recordSchema, generator, JsonGenerator::writeStartObject, JsonGenerator::writeEndObject, true);
         return schemaAccess.getAttributes(recordSchema);
     }
 
@@ -154,13 +153,13 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
             schemaAccess.writeHeader(recordSchema, getOutputStream());
         }
 
-        writeRecord(record, recordSchema, generator, g -> g.writeStartObject(), g -> g.writeEndObject(), false);
+        writeRecord(record, recordSchema, generator, JsonGenerator::writeStartObject, JsonGenerator::writeEndObject, false);
         final Map<String, String> attributes = schemaAccess.getAttributes(recordSchema);
         return WriteResult.of(incrementRecordCount(), attributes);
     }
 
     private void writeRecord(final Record record, final RecordSchema writeSchema, final JsonGenerator generator,
-        final GeneratorTask startTask, final GeneratorTask endTask, final boolean schemaAware) throws JsonGenerationException, IOException {
+        final GeneratorTask startTask, final GeneratorTask endTask, final boolean schemaAware) throws IOException {
 
         final Optional<SerializedForm> serializedForm = record.getSerializedForm();
         if (serializedForm.isPresent()) {
@@ -233,8 +232,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
     }
 
     @SuppressWarnings("unchecked")
-    private void writeRawValue(final JsonGenerator generator, final Object value, final String fieldName)
-        throws JsonGenerationException, IOException {
+    private void writeRawValue(final JsonGenerator generator, final Object value, final String fieldName) throws IOException {
 
         if (value == null) {
             generator.writeNull();
@@ -243,7 +241,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
 
         if (value instanceof Record) {
             final Record record = (Record) value;
-            writeRecord(record, record.getSchema(), generator, gen -> gen.writeStartObject(), gen -> gen.writeEndObject(), false);
+            writeRecord(record, record.getSchema(), generator, JsonGenerator::writeStartObject, JsonGenerator::writeEndObject, false);
             return;
         }
 
@@ -276,7 +274,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
     }
 
     @SuppressWarnings("unchecked")
-    private void writeValue(final JsonGenerator generator, final Object value, final String fieldName, final DataType dataType) throws JsonGenerationException, IOException {
+    private void writeValue(final JsonGenerator generator, final Object value, final String fieldName, final DataType dataType) throws IOException {
         if (value == null) {
             generator.writeNull();
             return;
@@ -362,7 +360,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
                 final Record record = (Record) coercedValue;
                 final RecordDataType recordDataType = (RecordDataType) chosenDataType;
                 final RecordSchema childSchema = recordDataType.getChildSchema();
-                writeRecord(record, childSchema, generator, gen -> gen.writeStartObject(), gen -> gen.writeEndObject(), true);
+                writeRecord(record, childSchema, generator, JsonGenerator::writeStartObject, JsonGenerator::writeEndObject, true);
                 break;
             }
             case MAP: {
@@ -394,11 +392,9 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
         }
     }
 
-    private void writeArray(final Object[] values, final String fieldName, final JsonGenerator generator, final DataType elementType)
-        throws JsonGenerationException, IOException {
+    private void writeArray(final Object[] values, final String fieldName, final JsonGenerator generator, final DataType elementType) throws IOException {
         generator.writeStartArray();
-        for (int i = 0; i < values.length; i++) {
-            final Object element = values[i];
+        for (final Object element : values) {
             writeValue(generator, element, fieldName, elementType);
         }
         generator.writeEndArray();
@@ -410,7 +406,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
         return this.mimeType;
     }
 
-    private static interface GeneratorTask {
-        void apply(JsonGenerator generator) throws JsonGenerationException, IOException;
+    private interface GeneratorTask {
+        void apply(JsonGenerator generator) throws IOException;
     }
 }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/CachedSchemaAccessStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/CachedSchemaAccessStrategy.java
new file mode 100644
index 0000000..c09325b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/CachedSchemaAccessStrategy.java
@@ -0,0 +1,66 @@
+/*
+ * 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.schema.inference;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaAccessStrategy;
+import org.apache.nifi.schema.access.SchemaField;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.RecordSchemaCacheService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class CachedSchemaAccessStrategy implements SchemaAccessStrategy {
+    private final RecordSchemaCacheService schemaCacheService;
+    private final SchemaAccessStrategy backupStrategy;
+    private final ComponentLog logger;
+
+    public CachedSchemaAccessStrategy(final RecordSchemaCacheService schemaCacheService, final SchemaAccessStrategy backupStrategy, final ComponentLog logger) {
+        this.schemaCacheService = schemaCacheService;
+        this.backupStrategy = backupStrategy;
+        this.logger = logger;
+    }
+
+    @Override
+    public RecordSchema getSchema(final Map<String, String> variables, final InputStream contentStream, final RecordSchema readSchema) throws SchemaNotFoundException, IOException {
+        final String cacheIdentifier = variables.get(RecordSchemaCacheService.CACHE_IDENTIFIER_ATTRIBUTE);
+        if (cacheIdentifier == null) {
+            logger.debug("Cache Identifier not found. Will delegate to backup Schema Access Strategy");
+            return backupStrategy.getSchema(variables, contentStream, readSchema);
+        }
+
+        final Optional<RecordSchema> schemaOption = schemaCacheService.getSchema(cacheIdentifier);
+        if (schemaOption.isPresent()) {
+            logger.debug("Found Cached Record Schema with identifier {}", new Object[] {cacheIdentifier});
+            return schemaOption.get();
+        }
+
+        logger.debug("Encountered Cache Miss with identifier {}. Will delegate to backup Schema Access Strategy", new Object[] {cacheIdentifier});
+        return backupStrategy.getSchema(variables, contentStream, readSchema);
+    }
+
+    @Override
+    public Set<SchemaField> getSuppliedSchemaFields() {
+        return EnumSet.noneOf(SchemaField.class);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/FieldTypeInference.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/FieldTypeInference.java
new file mode 100644
index 0000000..e148baf
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/FieldTypeInference.java
@@ -0,0 +1,109 @@
+/*
+ * 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.schema.inference;
+
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class FieldTypeInference {
+    private static final DataType DEFAULT_DATA_TYPE = RecordFieldType.STRING.getDataType();
+
+    // We don't actually need a singleDataType and a Set of DataTypes - we could use
+    // just the Set. However, the most common case will be the case where there is only a single
+    // unique value for the data type, and so this paradigm allows us to avoid the cost of creating
+    // and using the HashSet.
+    private DataType singleDataType = null;
+    private Set<DataType> possibleDataTypes;
+
+    public void addPossibleDataType(final DataType dataType) {
+        if (dataType == null) {
+            return;
+        }
+
+        if (singleDataType == null) {
+            singleDataType = dataType;
+            return;
+        }
+
+        if (possibleDataTypes == null && singleDataType.equals(dataType)) {
+            return;
+        }
+
+        final RecordFieldType singleFieldType = singleDataType.getFieldType();
+        final RecordFieldType additionalFieldType = dataType.getFieldType();
+
+        if (singleFieldType == RecordFieldType.RECORD && additionalFieldType == RecordFieldType.RECORD) {
+            // If we currently believe the field must be a Record, and the new possibility is also a record but the schemas
+            // are different, then consider the inferred type to be a Record with all possible fields. This is done, in comparison
+            // to using a UNION of the two because we can have a case where we have Records with many optional fields, and using a
+            // UNION could result in a UNION whose possible types are as long as number of permutations of those, which can be very
+            // expensive and not any more correct than just having a Record all of whose fields are optional.
+            final RecordSchema singleDataTypeSchema = ((RecordDataType) singleDataType).getChildSchema();
+            final RecordSchema newSchema = ((RecordDataType) dataType).getChildSchema();
+
+            final RecordSchema mergedSchema = DataTypeUtils.merge(singleDataTypeSchema, newSchema);
+            singleDataType = RecordFieldType.RECORD.getRecordDataType(mergedSchema);
+            return;
+        }
+
+        if (singleFieldType.isWiderThan(additionalFieldType)) {
+            // Assigned type is already wide enough to encompass the given type
+            return;
+        }
+
+        if (additionalFieldType.isWiderThan(singleFieldType)) {
+            // The given type is wide enough to encompass the assigned type. So changed the assigned type to the given type.
+            singleDataType = dataType;
+            return;
+        }
+
+        if (possibleDataTypes == null) {
+            possibleDataTypes = new HashSet<>();
+            possibleDataTypes.add(singleDataType);
+        }
+
+        possibleDataTypes.add(dataType);
+    }
+
+
+    /**
+     * Creates a single DataType that represents the field
+     * @return a single DataType that represents the field
+     */
+    public DataType toDataType() {
+        if (possibleDataTypes == null) {
+            if (singleDataType == null) {
+                return DEFAULT_DATA_TYPE;
+            }
+
+            return singleDataType;
+        }
+
+        DataType aggregate = null;
+        for (final DataType dataType : possibleDataTypes) {
+            aggregate = DataTypeUtils.mergeDataTypes(aggregate, dataType);
+        }
+
+        return aggregate;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/HierarchicalSchemaInference.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/HierarchicalSchemaInference.java
new file mode 100644
index 0000000..78791bb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/HierarchicalSchemaInference.java
@@ -0,0 +1,137 @@
+/*
+ * 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.schema.inference;
+
+import org.apache.nifi.serialization.SimpleRecordSchema;
+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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public abstract class HierarchicalSchemaInference<T> implements SchemaInferenceEngine<T> {
+
+    public RecordSchema inferSchema(final RecordSource<T> recordSource) throws IOException {
+        final Map<String, FieldTypeInference> typeMap = new LinkedHashMap<>();
+        String rootElementName = null;
+
+        while (true) {
+            final T rawRecord = recordSource.next();
+            if (rawRecord == null) {
+                break;
+            }
+
+            inferSchema(rawRecord, typeMap);
+
+            final String name = getRootName(rawRecord);
+            if (rootElementName == null) {
+                rootElementName = name;
+            } else if (!rootElementName.equals(name)) {
+                rootElementName = null;
+            }
+        }
+
+        return createSchema(typeMap, rootElementName);
+    }
+
+    protected void inferSchema(final T rawRecord, final Map<String, FieldTypeInference> inferences) {
+        if (isObject(rawRecord)) {
+            final BiConsumer<String, T> inferType = (fieldName, value) -> inferType(fieldName, value, inferences);
+            forEachFieldInRecord(rawRecord, inferType);
+        } else if (isArray(rawRecord)) {
+            forEachRawRecordInArray(rawRecord, arrayElement -> inferSchema(arrayElement, inferences));
+        } else {
+            throw new IllegalArgumentException("Cannot derive a Record Schema : expected an Array or Complex Object but got " + rawRecord);
+        }
+    }
+
+    private void inferType(final String fieldName, final T value, final Map<String, FieldTypeInference> inferences) {
+        if (value == null) {
+            return;
+        }
+
+        final FieldTypeInference typeInference = inferences.computeIfAbsent(fieldName, key -> new FieldTypeInference());
+
+        if (isObject(value)) {
+            final RecordSchema schema = createSchema(value);
+            final DataType fieldDataType = RecordFieldType.RECORD.getRecordDataType(schema);
+            typeInference.addPossibleDataType(fieldDataType);
+        } else if (isArray(value)) {
+            final FieldTypeInference arrayElementTypeInference = new FieldTypeInference();
+            forEachRawRecordInArray(value, arrayElement -> inferType(arrayElement, arrayElementTypeInference));
+
+            final DataType elementDataType = arrayElementTypeInference.toDataType();
+            final DataType arrayDataType = RecordFieldType.ARRAY.getArrayDataType(elementDataType);
+            typeInference.addPossibleDataType(arrayDataType);
+        } else {
+            typeInference.addPossibleDataType(getDataType(value));
+        }
+    }
+
+    private void inferType(final T value, final FieldTypeInference typeInference) {
+        if (isObject(value)) {
+            final RecordSchema schema = createSchema(value);
+            final DataType fieldDataType = RecordFieldType.RECORD.getRecordDataType(schema);
+            typeInference.addPossibleDataType(fieldDataType);
+        } else if (isArray(value)) {
+            final FieldTypeInference arrayElementTypeInference = new FieldTypeInference();
+            forEachRawRecordInArray(value, arrayElement -> inferType(arrayElement, arrayElementTypeInference));
+
+            final DataType elementDataType = arrayElementTypeInference.toDataType();
+            final DataType arrayDataType = RecordFieldType.ARRAY.getArrayDataType(elementDataType);
+            typeInference.addPossibleDataType(arrayDataType);
+        } else {
+            typeInference.addPossibleDataType(getDataType(value));
+        }
+    }
+
+    private RecordSchema createSchema(final Map<String, FieldTypeInference> inferences, final String rootElementName) {
+        final List<RecordField> recordFields = new ArrayList<>(inferences.size());
+        inferences.forEach((fieldName, type) -> recordFields.add(new RecordField(fieldName, type.toDataType())));
+        final SimpleRecordSchema schema = new SimpleRecordSchema(recordFields);
+        schema.setSchemaName(rootElementName);
+        return schema;
+    }
+
+    protected RecordSchema createSchema(final T rawRecord) {
+        final Map<String, FieldTypeInference> typeMap = new LinkedHashMap<>();
+        inferSchema(rawRecord, typeMap);
+
+        final RecordSchema schema = createSchema(typeMap, getRootName(rawRecord));
+        return schema;
+    }
+
+
+    protected abstract DataType getDataType(T value);
+
+    protected abstract boolean isObject(T value);
+
+    protected abstract boolean isArray(T value);
+
+    protected abstract void forEachFieldInRecord(T rawRecord, BiConsumer<String, T> fieldConsumer);
+
+    protected abstract void forEachRawRecordInArray(T arrayRecord, Consumer<T> rawRecordConsumer);
+
+    protected abstract String getRootName(T rawRecord);
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/InferSchemaAccessStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/InferSchemaAccessStrategy.java
new file mode 100644
index 0000000..1338386
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/InferSchemaAccessStrategy.java
@@ -0,0 +1,62 @@
+/*
+ * 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.schema.inference;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaAccessStrategy;
+import org.apache.nifi.schema.access.SchemaField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.stream.io.NonCloseableInputStream;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Set;
+
+public class InferSchemaAccessStrategy<T> implements SchemaAccessStrategy {
+    private final RecordSourceFactory<T> recordSourceFactory;
+    private final SchemaInferenceEngine<T> schemaInference;
+    private final ComponentLog logger;
+
+    public InferSchemaAccessStrategy(final RecordSourceFactory<T> recordSourceFactory, final SchemaInferenceEngine<T> schemaInference, final ComponentLog logger) {
+        this.recordSourceFactory = recordSourceFactory;
+        this.schemaInference = schemaInference;
+        this.logger = logger;
+    }
+
+    @Override
+    public RecordSchema getSchema(final Map<String, String> variables, final InputStream contentStream, final RecordSchema readSchema) throws IOException {
+        // We expect to be able to mark/reset any length because we expect that the underlying stream here will be a ContentClaimInputStream, which is able to
+        // re-read the content regardless of how much data is read.
+        contentStream.mark(10_000_000);
+        try {
+            final RecordSource<T> recordSource = recordSourceFactory.create(variables, new NonCloseableInputStream(contentStream));
+            final RecordSchema schema = schemaInference.inferSchema(recordSource);
+
+            logger.debug("Successfully inferred schema {}", new Object[] {schema});
+            return schema;
+        } finally {
+            contentStream.reset();
+        }
+    }
+
+    @Override
+    public Set<SchemaField> getSuppliedSchemaFields() {
+        return EnumSet.noneOf(SchemaField.class);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/RecordSource.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/RecordSource.java
new file mode 100644
index 0000000..c7e2583
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/RecordSource.java
@@ -0,0 +1,24 @@
+/*
+ * 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.schema.inference;
+
+import java.io.IOException;
+
+@FunctionalInterface
+public interface RecordSource<T> {
+    T next() throws IOException;
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/RecordSourceFactory.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/RecordSourceFactory.java
new file mode 100644
index 0000000..8d933ee
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/RecordSourceFactory.java
@@ -0,0 +1,27 @@
+/*
+ * 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.schema.inference;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+public interface RecordSourceFactory<T> {
+
+    RecordSource<T> create(Map<String, String> variables, InputStream contentStream) throws IOException;
+
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/SchemaInferenceEngine.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/SchemaInferenceEngine.java
new file mode 100644
index 0000000..3dbd26b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/SchemaInferenceEngine.java
@@ -0,0 +1,27 @@
+/*
+ * 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.schema.inference;
+
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+
+public interface SchemaInferenceEngine<T> {
+
+    RecordSchema inferSchema(RecordSource<T> recordSource) throws IOException;
+
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/SchemaInferenceUtil.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/SchemaInferenceUtil.java
new file mode 100644
index 0000000..9097ed6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/SchemaInferenceUtil.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.schema.inference;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaAccessStrategy;
+import org.apache.nifi.serialization.RecordSchemaCacheService;
+
+import java.util.function.Supplier;
+
+public class SchemaInferenceUtil {
+    public static final AllowableValue INFER_SCHEMA = new AllowableValue("infer-schema", "Infer Schema",
+        "The Schema of the data will be inferred automatically when the data is read. See component Usage and Additional Details for information about how the schema is inferred.");
+
+    public static final PropertyDescriptor SCHEMA_CACHE = new Builder()
+        .name("schema-inference-cache")
+        .displayName("Schema Inference Cache")
+        .description("Specifies a Schema Cache to use when inferring the schema. If not populated, the schema will be inferred each time. " +
+            "However, if a cache is specified, the cache will first be consulted and if the applicable schema can be found, it will be used instead of inferring the schema.")
+        .required(false)
+        .identifiesControllerService(RecordSchemaCacheService.class)
+        .build();
+
+
+    public static <T> SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final PropertyContext context,  final ComponentLog logger,
+                                                                   final RecordSourceFactory<T> recordSourceFactory, final Supplier<SchemaInferenceEngine<T>> inferenceSupplier,
+                                                                   final Supplier<SchemaAccessStrategy> defaultSupplier) {
+        if (INFER_SCHEMA.getValue().equalsIgnoreCase(strategy)) {
+            final SchemaAccessStrategy inferenceStrategy = new InferSchemaAccessStrategy<>(recordSourceFactory, inferenceSupplier.get(), logger);
+            final RecordSchemaCacheService schemaCache = context.getProperty(SCHEMA_CACHE).asControllerService(RecordSchemaCacheService.class);
+            if (schemaCache == null) {
+                return inferenceStrategy;
+            }
+
+            return new CachedSchemaAccessStrategy(schemaCache, inferenceStrategy, logger);
+        }
+
+        return defaultSupplier.get();
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/TimeValueInference.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/TimeValueInference.java
new file mode 100644
index 0000000..518cbb8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/TimeValueInference.java
@@ -0,0 +1,69 @@
+/*
+ * 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.schema.inference;
+
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.text.DateTimeMatcher;
+
+import java.util.Optional;
+
+public class TimeValueInference {
+    private final Optional<DataType> dateDataType;
+    private final Optional<DataType> timeDataType;
+    private final Optional<DataType> timestampDataType;
+
+    private final DateTimeMatcher dateMatcher;
+    private final DateTimeMatcher timeMatcher;
+    private final DateTimeMatcher timestampMatcher;
+
+    public TimeValueInference(final String dateFormat, final String timeFormat, final String timestampFormat) {
+        this.dateDataType = Optional.of(RecordFieldType.DATE.getDataType(dateFormat));
+        this.timeDataType = Optional.of(RecordFieldType.TIME.getDataType(timeFormat));
+        this.timestampDataType = Optional.of(RecordFieldType.TIMESTAMP.getDataType(timestampFormat));
+
+        this.dateMatcher = DateTimeMatcher.compile(dateFormat);
+        this.timeMatcher = DateTimeMatcher.compile(timeFormat);
+        this.timestampMatcher = DateTimeMatcher.compile(timestampFormat);
+    }
+
+    public String getDateFormat() {
+        return dateDataType.map(DataType::getFormat).orElse(null);
+    }
+
+    public String getTimeFormat() {
+        return timeDataType.map(DataType::getFormat).orElse(null);
+    }
+
+    public String getTimestampFormat() {
+        return timestampDataType.map(DataType::getFormat).orElse(null);
+    }
+
+    public Optional<DataType> getDataType(final String value) {
+        if (timestampMatcher.matches(value)) {
+            return timestampDataType;
+        }
+        if (dateMatcher.matches(value)) {
+            return dateDataType;
+        }
+        if (timeMatcher.matches(value)) {
+            return timeDataType;
+        }
+
+        return Optional.empty();
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/VolatileSchemaCache.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/VolatileSchemaCache.java
new file mode 100644
index 0000000..926d2d0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/inference/VolatileSchemaCache.java
@@ -0,0 +1,154 @@
+/*
+ * 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.schema.inference;
+
+import avro.shaded.com.google.common.annotations.VisibleForTesting;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.commons.codec.binary.Hex;
+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.components.PropertyDescriptor.Builder;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.serialization.RecordSchemaCacheService;
+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;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.VARIABLE_REGISTRY;
+import static org.apache.nifi.processor.util.StandardValidators.POSITIVE_INTEGER_VALIDATOR;
+
+@CapabilityDescription("Provides a Schema Cache that evicts elements based on a Least-Recently-Used algorithm. This cache is not persisted, so any restart of NiFi will result in " +
+    "the cache being cleared. Additionally, the cache will be cleared any time that the Controller Service is stopped and restarted.")
+@Tags({"record", "schema", "cache"})
+public class VolatileSchemaCache extends AbstractControllerService implements RecordSchemaCacheService {
+
+    static final PropertyDescriptor MAX_SIZE = new Builder()
+        .name("max-cache-size")
+        .displayName("Maximum Cache Size")
+        .description("The maximum number of Schemas to cache.")
+        .required(true)
+        .addValidator(POSITIVE_INTEGER_VALIDATOR)
+        .expressionLanguageSupported(VARIABLE_REGISTRY)
+        .defaultValue("100")
+        .build();
+
+    private volatile Cache<String, RecordSchema> cache;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Collections.singletonList(MAX_SIZE);
+    }
+
+    @OnEnabled
+    public void setup(final ConfigurationContext context) {
+        final int maxSize = context.getProperty(MAX_SIZE).evaluateAttributeExpressions().asInteger();
+
+        cache = Caffeine.newBuilder()
+            .maximumSize(maxSize)
+            .build();
+    }
+
+
+    @Override
+    public String cacheSchema(final RecordSchema schema) {
+        final String identifier = createIdentifier(schema);
+        final RecordSchema existingSchema = cache.get(identifier, id -> schema);
+
+        if (existingSchema == null) {
+            // We successfully inserted into the cache.
+            getLogger().debug("Successfully cached schema with ID {} (no existing schema with this ID)", new Object[] {identifier});
+            return identifier;
+        }
+
+        // There was already a Schema in the cache with that identifier.
+        if (existingSchema.equals(schema)) {
+            // Schemas match. Already cached successfully.
+            getLogger().debug("Successfully cached schema with ID {} (existing schema with this ID was equal)", new Object[] {identifier});
+            return identifier;
+        }
+
+        // Schemas hashed to same value but do not equal one another. Append a randomly generated UUID
+        // and add that to the cache.
+        final String updatedIdentifier = identifier + "-" + UUID.randomUUID().toString();
+        cache.put(updatedIdentifier, schema);
+
+        getLogger().debug("Schema with ID {} conflicted with new Schema. Resolved by using generated identifier {}", new Object[] {identifier, updatedIdentifier});
+        return updatedIdentifier;
+    }
+
+    @Override
+    public Optional<RecordSchema> getSchema(final String schemaIdentifier) {
+        final RecordSchema cachedSchema = cache.getIfPresent(schemaIdentifier);
+        return Optional.ofNullable(cachedSchema);
+    }
+
+    @VisibleForTesting
+    protected String createIdentifier(final RecordSchema schema) {
+        final MessageDigest digest;
+        try {
+            digest = MessageDigest.getInstance("SHA-256");
+        } catch (NoSuchAlgorithmException e) {
+            throw new AssertionError(e);
+        }
+
+        final Optional<String> suppliedText = schema.getSchemaText();
+        if (suppliedText.isPresent()) {
+            digest.update(suppliedText.get().getBytes(StandardCharsets.UTF_8));
+        } else {
+            computeHash(schema, digest);
+        }
+
+        final byte[] digestBytes = digest.digest();
+        return Hex.encodeHexString(digestBytes);
+    }
+
+    private void computeHash(final RecordSchema schema, final MessageDigest digest) {
+        for (final RecordField field : schema.getFields()) {
+            digest.update(field.getFieldName().getBytes(StandardCharsets.UTF_8));
+
+            final DataType dataType = field.getDataType();
+            final RecordFieldType fieldType = dataType.getFieldType();
+            digest.update(fieldType.name().getBytes(StandardCharsets.UTF_8));
+
+            final String format = dataType.getFormat();
+            if (format != null) {
+                digest.update(format.getBytes(StandardCharsets.UTF_8));
+            }
+
+            if (fieldType == RecordFieldType.RECORD) {
+                final RecordSchema childSchema = ((RecordDataType) dataType).getChildSchema();
+                if (childSchema != null) {
+                    computeHash(childSchema, digest);
+                }
+            }
+        }
+    }
+}
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/CacheIdSchemaAccessWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/CacheIdSchemaAccessWriter.java
new file mode 100644
index 0000000..f229493
--- /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/CacheIdSchemaAccessWriter.java
@@ -0,0 +1,62 @@
+/*
+ * 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.schema.access.NopSchemaAccessWriter;
+import org.apache.nifi.schema.access.SchemaAccessWriter;
+import org.apache.nifi.schema.access.SchemaField;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+public class CacheIdSchemaAccessWriter implements SchemaAccessWriter {
+    private final RecordSchemaCacheService cache;
+    private final SchemaAccessWriter delegate;
+
+    public CacheIdSchemaAccessWriter(final RecordSchemaCacheService cache, final SchemaAccessWriter delegate) {
+        this.cache = cache;
+        this.delegate = delegate == null ? new NopSchemaAccessWriter() : delegate;
+    }
+
+    @Override
+    public void writeHeader(final RecordSchema schema, final OutputStream out) throws IOException {
+        delegate.writeHeader(schema, out);
+    }
+
+    @Override
+    public Map<String, String> getAttributes(final RecordSchema schema) {
+        final Map<String, String> attributes = new HashMap<>(delegate.getAttributes(schema));
+        final String identifier = cache.cacheSchema(schema);
+        attributes.put(RecordSchemaCacheService.CACHE_IDENTIFIER_ATTRIBUTE, identifier);
+        return attributes;
+    }
+
+    @Override
+    public void validateSchema(final RecordSchema schema) throws SchemaNotFoundException {
+        delegate.validateSchema(schema);
+    }
+
+    @Override
+    public Set<SchemaField> getRequiredSchemaFields() {
+        return delegate.getRequiredSchemaFields();
+    }
+}
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/SchemaRegistryRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java
index 4853e4b..734168b 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java
@@ -17,22 +17,10 @@
 
 package org.apache.nifi.serialization;
 
-import static org.apache.nifi.schema.access.SchemaAccessUtils.INHERIT_RECORD_SCHEMA;
-import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME_PROPERTY;
-import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT_PROPERTY;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.ConfigurationContext;
@@ -47,6 +35,19 @@ import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.schema.access.WriteAvroSchemaAttributeStrategy;
 import org.apache.nifi.serialization.record.RecordSchema;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.nifi.schema.access.SchemaAccessUtils.INHERIT_RECORD_SCHEMA;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT_PROPERTY;
+
 public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryService {
 
     static final AllowableValue SCHEMA_NAME_ATTRIBUTE = new AllowableValue("schema-name", "Set 'schema.name' Attribute",
@@ -71,12 +72,20 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic
             + "an Exception will be thrown when attempting to write the data. This is based on the encoding used by version 3.2.x of the Confluent Schema Registry.");
     static final AllowableValue NO_SCHEMA = new AllowableValue("no-schema", "Do Not Write Schema", "Do not add any schema-related information to the FlowFile.");
 
+    static final PropertyDescriptor SCHEMA_CACHE = new Builder()
+        .name("schema-cache")
+        .displayName("Schema Cache")
+        .description("Specifies a Schema Cache to add the Record Schema to so that Record Readers can quickly lookup the schema.")
+        .required(false)
+        .identifiesControllerService(RecordSchemaCacheService.class)
+        .build();
+
     /**
      * This constant is just a base spec for the actual PropertyDescriptor.
      * As it can be overridden by subclasses with different AllowableValues and default value,
      * {@link #getSchemaWriteStrategyDescriptor()} should be used to get the actual descriptor, instead of using this constant directly.
      */
-    private static final PropertyDescriptor SCHEMA_WRITE_STRATEGY = new PropertyDescriptor.Builder()
+    private static final PropertyDescriptor SCHEMA_WRITE_STRATEGY = new Builder()
         .name("Schema Write Strategy")
         .description("Specifies how the schema for a Record should be added to the data.")
         .required(true)
@@ -97,18 +106,19 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic
         final List<PropertyDescriptor> properties = new ArrayList<>();
 
         final AllowableValue[] strategies = getSchemaWriteStrategyValues().toArray(new AllowableValue[0]);
-        properties.add(new PropertyDescriptor.Builder()
+        properties.add(new Builder()
             .fromPropertyDescriptor(SCHEMA_WRITE_STRATEGY)
             .defaultValue(getDefaultSchemaWriteStrategy().getValue())
             .allowableValues(strategies)
             .build());
+        properties.add(SCHEMA_CACHE);
         properties.addAll(super.getSupportedPropertyDescriptors());
 
         return properties;
     }
 
     protected AllowableValue getDefaultSchemaWriteStrategy() {
-        return SCHEMA_NAME_ATTRIBUTE;
+        return NO_SCHEMA;
     }
 
     @Override
@@ -124,8 +134,9 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic
     public void storeSchemaWriteStrategy(final ConfigurationContext context) {
         this.configurationContext = context;
 
-        final String writerValue = context.getProperty(getSchemaWriteStrategyDescriptor()).getValue();
-        this.schemaAccessWriter = getSchemaWriteStrategy(writerValue);
+        final String strategy = context.getProperty(getSchemaWriteStrategyDescriptor()).getValue();
+        final RecordSchemaCacheService recordSchemaCacheService = context.getProperty(SCHEMA_CACHE).asControllerService(RecordSchemaCacheService.class);
+        this.schemaAccessWriter = createSchemaWriteStrategy(strategy, recordSchemaCacheService);
     }
 
     @Override
@@ -147,11 +158,20 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic
         return schemaAccessStrategyList;
     }
 
-    protected SchemaAccessWriter getSchemaWriteStrategy(final String strategy) {
-        if (strategy == null) {
-            return null;
+    protected SchemaAccessWriter getSchemaWriteStrategy() {
+        return schemaAccessWriter;
+    }
+
+    private SchemaAccessWriter createSchemaWriteStrategy(final String strategy, final RecordSchemaCacheService recordSchemaCacheService) {
+        final SchemaAccessWriter writer = createRawSchemaWriteStrategy(strategy);
+        if (recordSchemaCacheService == null) {
+            return writer;
+        } else {
+            return new CacheIdSchemaAccessWriter(recordSchemaCacheService, writer);
         }
+    }
 
+    private SchemaAccessWriter createRawSchemaWriteStrategy(final String strategy) {
         if (strategy.equalsIgnoreCase(SCHEMA_NAME_ATTRIBUTE.getValue())) {
             return new SchemaNameAsAttribute();
         } else if (strategy.equalsIgnoreCase(AVRO_SCHEMA_ATTRIBUTE.getValue())) {
@@ -170,8 +190,7 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic
     }
 
     protected Set<SchemaField> getRequiredSchemaFields(final ValidationContext validationContext) {
-        final String writeStrategyValue = validationContext.getProperty(getSchemaWriteStrategyDescriptor()).getValue();
-        final SchemaAccessWriter writer = getSchemaWriteStrategy(writeStrategyValue);
+        final SchemaAccessWriter writer = getSchemaWriteStrategy();
         if (writer == null) {
             return EnumSet.noneOf(SchemaField.class);
         }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/Syslog5424Reader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/Syslog5424Reader.java
index fdff68b..7f29db2 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/Syslog5424Reader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/Syslog5424Reader.java
@@ -22,7 +22,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -101,12 +101,7 @@ public class Syslog5424Reader extends SchemaRegistryService implements RecordRea
     }
 
     @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final ConfigurationContext context) {
-        return createAccessStrategy();
-    }
-
-    @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final ValidationContext context) {
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final PropertyContext context) {
         return createAccessStrategy();
     }
 
@@ -136,7 +131,7 @@ public class Syslog5424Reader extends SchemaRegistryService implements RecordRea
 
 
             @Override
-            public RecordSchema getSchema(Map<String, String> variables, InputStream contentStream, RecordSchema readSchema) throws SchemaNotFoundException {
+            public RecordSchema getSchema(Map<String, String> variables, InputStream contentStream, RecordSchema readSchema) {
                 return recordSchema;
             }
 
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/SyslogReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/SyslogReader.java
index a03e66e..9b7eda9 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/SyslogReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/syslog/SyslogReader.java
@@ -22,7 +22,7 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -99,12 +99,7 @@ public class SyslogReader extends SchemaRegistryService implements RecordReaderF
     }
 
     @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final ConfigurationContext context) {
-        return createAccessStrategy();
-    }
-
-    @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final ValidationContext context) {
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final PropertyContext context) {
         return createAccessStrategy();
     }
 
@@ -129,7 +124,7 @@ public class SyslogReader extends SchemaRegistryService implements RecordReaderF
 
 
             @Override
-            public RecordSchema getSchema(Map<String, String> variables, InputStream contentStream, RecordSchema readSchema) throws SchemaNotFoundException {
+            public RecordSchema getSchema(Map<String, String> variables, InputStream contentStream, RecordSchema readSchema) {
                 return recordSchema;
             }
 
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 e5f8514..3971baa 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
@@ -17,16 +17,12 @@
 
 package org.apache.nifi.text;
 
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.List;
-
 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.components.PropertyValue;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
@@ -39,6 +35,11 @@ import org.apache.nifi.serialization.RecordSetWriterFactory;
 import org.apache.nifi.serialization.SchemaRegistryRecordSetWriter;
 import org.apache.nifi.serialization.record.RecordSchema;
 
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+
 @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 fields that are available "
@@ -83,7 +84,7 @@ public class FreeFormTextRecordSetWriter extends SchemaRegistryRecordSetWriter i
     }
 
     @Override
-    protected SchemaAccessStrategy getSchemaAccessStrategy(String strategy, SchemaRegistry schemaRegistry, ConfigurationContext context) {
+    protected SchemaAccessStrategy getSchemaAccessStrategy(String strategy, SchemaRegistry schemaRegistry, PropertyContext context) {
         return new InheritSchemaFromRecord();
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/WriteXMLResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/WriteXMLResult.java
index 382067c..16a1e9e 100755
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/WriteXMLResult.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/WriteXMLResult.java
@@ -19,7 +19,6 @@ package org.apache.nifi.xml;
 
 import javanet.staxutils.IndentingXMLStreamWriter;
 import org.apache.nifi.record.NullSuppression;
-import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.schema.access.SchemaAccessWriter;
 import org.apache.nifi.serialization.AbstractRecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriter;
@@ -48,14 +47,15 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.function.Supplier;
+import java.util.regex.Pattern;
 
 import static org.apache.nifi.xml.XMLRecordSetWriter.RECORD_TAG_NAME;
 import static org.apache.nifi.xml.XMLRecordSetWriter.ROOT_TAG_NAME;
 
 
 public class WriteXMLResult extends AbstractRecordSetWriter implements RecordSetWriter, RawRecordWriter {
+    private static final Pattern TAG_NAME_CHARS_TO_STRIP = Pattern.compile("[/<>!&'\"]");
 
-    private final ComponentLog logger;
     private final RecordSchema recordSchema;
     private final SchemaAccessWriter schemaAccess;
     private final XMLStreamWriter writer;
@@ -64,7 +64,6 @@ public class WriteXMLResult extends AbstractRecordSetWriter implements RecordSet
     private final String arrayTagName;
     private final String recordTagName;
     private final String rootTagName;
-    private final String charSet;
     private final boolean allowWritingMultipleRecords;
     private boolean hasWrittenRecord;
 
@@ -72,13 +71,12 @@ public class WriteXMLResult extends AbstractRecordSetWriter implements RecordSet
     private final Supplier<DateFormat> LAZY_TIME_FORMAT;
     private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
 
-    public WriteXMLResult(final ComponentLog logger, final RecordSchema recordSchema, final SchemaAccessWriter schemaAccess, final OutputStream out, final boolean prettyPrint,
+    public WriteXMLResult(final RecordSchema recordSchema, final SchemaAccessWriter schemaAccess, final OutputStream out, final boolean prettyPrint,
                           final NullSuppression nullSuppression, final ArrayWrapping arrayWrapping, final String arrayTagName, final String rootTagName, final String recordTagName,
                           final String charSet, final String dateFormat, final String timeFormat, final String timestampFormat) throws IOException {
 
         super(out);
 
-        this.logger = logger;
         this.recordSchema = recordSchema;
         this.schemaAccess = schemaAccess;
         this.nullSuppression = nullSuppression;
@@ -95,18 +93,13 @@ public class WriteXMLResult extends AbstractRecordSetWriter implements RecordSet
             if (recordTagNameOptional.isPresent()) {
                 this.recordTagName = recordTagNameOptional.get();
             } else {
-                StringBuilder message = new StringBuilder();
-                message.append("The property \'")
-                        .append(RECORD_TAG_NAME.getDisplayName())
-                        .append("\' has not been set and the writer does not find a record name in the schema.");
-                throw new IOException(message.toString());
+                final String message = "The property '" + RECORD_TAG_NAME.getDisplayName() +
+                    "' has not been set and the writer does not find a record name in the schema.";
+                throw new IOException(message);
             }
         }
 
         this.allowWritingMultipleRecords = !(this.rootTagName == null);
-
-        this.charSet = charSet;
-
         hasWrittenRecord = false;
 
         final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
@@ -190,13 +183,9 @@ public class WriteXMLResult extends AbstractRecordSetWriter implements RecordSet
 
     private void checkWritingMultipleRecords() throws IOException {
         if (!allowWritingMultipleRecords && hasWrittenRecord) {
-            StringBuilder message = new StringBuilder();
-            message.append("The writer attempts to write multiple record although property \'")
-                    .append(ROOT_TAG_NAME.getDisplayName())
-                    .append("\' has not been set. If the XMLRecordSetWriter is supposed to write multiple records into one ")
-                    .append("FlowFile, this property is required to be configured.");
-            throw new IOException(message.toString()
-            );
+            final String message = "The writer attempts to write multiple record although property \'" + ROOT_TAG_NAME.getDisplayName() +
+                "\' has not been set. If the XMLRecordSetWriter is supposed to write multiple records into one FlowFile, this property is required to be configured.";
+            throw new IOException(message);
         }
     }
 
@@ -441,9 +430,13 @@ public class WriteXMLResult extends AbstractRecordSetWriter implements RecordSet
         writeAllTags(tagsToOpen);
     }
 
+    private String escapeTagName(final String tagName) {
+        return TAG_NAME_CHARS_TO_STRIP.matcher(tagName).replaceAll("");
+    }
+
     private void writeAllTags(Deque<String> tagsToOpen) throws XMLStreamException {
         for (String tagName : tagsToOpen) {
-            writer.writeStartElement(tagName);
+            writer.writeStartElement(escapeTagName(tagName));
         }
         tagsToOpen.clear();
     }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
index d8216df..d96d397 100755
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
@@ -22,23 +22,37 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaAccessStrategy;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schema.inference.SchemaInferenceEngine;
+import org.apache.nifi.schema.inference.RecordSourceFactory;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.schema.inference.TimeValueInference;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
 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 org.apache.nifi.xml.inference.XmlNode;
+import org.apache.nifi.xml.inference.XmlRecordSource;
+import org.apache.nifi.xml.inference.XmlSchemaInference;
 
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.function.Supplier;
+
+import static org.apache.nifi.schema.inference.SchemaInferenceUtil.INFER_SCHEMA;
 
 @Tags({"xml", "record", "reader", "parser"})
 @CapabilityDescription("Reads XML content and creates Record objects. Records are expected in the second level of " +
@@ -102,6 +116,7 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(SchemaInferenceUtil.SCHEMA_CACHE);
         properties.add(RECORD_FORMAT);
         properties.add(ATTRIBUTE_PREFIX);
         properties.add(CONTENT_FIELD_NAME);
@@ -112,29 +127,52 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact
     }
 
     @Override
-    public RecordReader createRecordReader(final Map<String, String> variables, final InputStream in, final ComponentLog logger)
-            throws IOException, SchemaNotFoundException, MalformedRecordException {
-        final ConfigurationContext context = getConfigurationContext();
-
-        final RecordSchema schema = getSchema(variables, in, null);
+    protected List<AllowableValue> getSchemaAccessStrategyValues() {
+        final List<AllowableValue> allowableValues = new ArrayList<>(super.getSchemaAccessStrategyValues());
+        allowableValues.add(INFER_SCHEMA);
+        return allowableValues;
+    }
 
-        final String attributePrefix = context.getProperty(ATTRIBUTE_PREFIX).isSet()
-                ? context.getProperty(ATTRIBUTE_PREFIX).evaluateAttributeExpressions(variables).getValue().trim() : null;
+    @Override
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String strategy, final SchemaRegistry schemaRegistry, final PropertyContext context) {
+        final RecordSourceFactory<XmlNode> sourceFactory = (variables, contentStream) -> new XmlRecordSource(contentStream, isMultipleRecords(context, variables));
+        final Supplier<SchemaInferenceEngine<XmlNode>> schemaInference = () -> new XmlSchemaInference(new TimeValueInference(dateFormat, timeFormat, timestampFormat));
 
-        final String contentFieldName = context.getProperty(CONTENT_FIELD_NAME).isSet()
-                ? context.getProperty(CONTENT_FIELD_NAME).evaluateAttributeExpressions(variables).getValue().trim() : null;
+        return SchemaInferenceUtil.getSchemaAccessStrategy(strategy, context, getLogger(), sourceFactory, schemaInference,
+            () -> super.getSchemaAccessStrategy(strategy, schemaRegistry, context));
+    }
 
-        final boolean isArray;
+    private boolean isMultipleRecords(final PropertyContext context, final Map<String, String> variables) {
         final String recordFormat = context.getProperty(RECORD_FORMAT).evaluateAttributeExpressions(variables).getValue().trim();
         if ("true".equalsIgnoreCase(recordFormat)) {
-            isArray = true;
+            return true;
         } else if ("false".equalsIgnoreCase(recordFormat)) {
-            isArray = false;
+            return false;
         } else {
-            throw new IOException("Cannot parse XML Records because the '" + RECORD_FORMAT.getDisplayName() + "' property evaluates to '"
+            throw new ProcessException("Cannot parse XML Records because the '" + RECORD_FORMAT.getDisplayName() + "' property evaluates to '"
                 + recordFormat + "', which is neither 'true' nor 'false'");
         }
+    }
+
+    @Override
+    protected AllowableValue getDefaultSchemaAccessStrategy() {
+        return INFER_SCHEMA;
+    }
+
+    @Override
+    public RecordReader createRecordReader(final Map<String, String> variables, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException, MalformedRecordException {
+        final ConfigurationContext context = getConfigurationContext();
+
+        final RecordSchema schema = getSchema(variables, in, null);
+
+        final String attributePrefix = trim(context.getProperty(ATTRIBUTE_PREFIX).evaluateAttributeExpressions(variables).getValue());
+        final String contentFieldName = trim(context.getProperty(CONTENT_FIELD_NAME).evaluateAttributeExpressions(variables).getValue());
+        final boolean isArray = isMultipleRecords(context, variables);
 
         return new XMLRecordReader(in, schema, isArray, attributePrefix, contentFieldName, dateFormat, timeFormat, timestampFormat, logger);
     }
+
+    private String trim(final String value) {
+        return value == null ? null : value.trim();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
index e819b92..4a2eacb 100755
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
@@ -84,6 +84,10 @@ public class XMLRecordReader implements RecordReader {
         try {
             final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
 
+            // Avoid XXE Vulnerabilities
+            xmlInputFactory.setProperty(XMLInputFactory.SUPPORT_DTD, false);
+            xmlInputFactory.setProperty("javax.xml.stream.isSupportingExternalEntities", false);
+
             xmlEventReader = xmlInputFactory.createXMLEventReader(in);
 
             if (isArray) {
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordSetWriter.java
index 6867af0..1f718d6 100755
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordSetWriter.java
@@ -166,7 +166,6 @@ public class XMLRecordSetWriter extends DateTimeTextRecordSetWriter implements R
 
     @Override
     public RecordSetWriter createWriter(final ComponentLog logger, final RecordSchema schema, final OutputStream out) throws SchemaNotFoundException, IOException {
-
         final String nullSuppression = getConfigurationContext().getProperty(SUPPRESS_NULLS).getValue();
         final NullSuppression nullSuppressionEnum;
         if (nullSuppression.equals(ALWAYS_SUPPRESS.getValue())) {
@@ -203,7 +202,7 @@ public class XMLRecordSetWriter extends DateTimeTextRecordSetWriter implements R
 
         final String charSet = getConfigurationContext().getProperty(CHARACTER_SET).getValue();
 
-        return new WriteXMLResult(logger, schema, getSchemaAccessWriter(schema),
+        return new WriteXMLResult(schema, getSchemaAccessWriter(schema),
                 out, prettyPrint, nullSuppressionEnum, arrayWrappingEnum, arrayTagName, rootTagName, recordTagName, charSet,
                 getDateFormat().orElse(null), getTimeFormat().orElse(null), getTimestampFormat().orElse(null));
     }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlArrayNode.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlArrayNode.java
new file mode 100644
index 0000000..6351fc5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlArrayNode.java
@@ -0,0 +1,54 @@
+/*
+ * 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.inference;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Consumer;
+
+public class XmlArrayNode implements XmlNode {
+    private final String nodeName;
+    private final List<XmlNode> elements = new ArrayList<>();
+
+    public XmlArrayNode(final String nodeName) {
+        this.nodeName = nodeName;
+    }
+
+    @Override
+    public XmlNodeType getNodeType() {
+        return XmlNodeType.ARRAY;
+    }
+
+    void addElement(final XmlNode node) {
+        elements.add(node);
+    }
+
+    public List<XmlNode> getElements() {
+        return Collections.unmodifiableList(elements);
+    }
+
+    public void forEach(final Consumer<XmlNode> consumer) {
+        elements.forEach(consumer);
+    }
+
+    @Override
+    public String getName() {
+        return nodeName;
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlContainerNode.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlContainerNode.java
new file mode 100644
index 0000000..d6bb2c7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlContainerNode.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.xml.inference;
+
+import java.util.Map;
+import java.util.function.BiConsumer;
+
+public class XmlContainerNode implements XmlNode {
+    private final String nodeName;
+    private final Map<String, XmlNode> children;
+
+    public XmlContainerNode(final String nodeName, final Map<String, XmlNode> children) {
+        this.nodeName = nodeName;
+        this.children = children;
+    }
+
+    @Override
+    public XmlNodeType getNodeType() {
+        return XmlNodeType.CONTAINER;
+    }
+
+    public XmlNode getChildNode(final String fieldName) {
+        return children.get(fieldName);
+    }
+
+    public void forEach(final BiConsumer<String, XmlNode> consumer) {
+        for (final Map.Entry<String, XmlNode> child : children.entrySet()) {
+            consumer.accept(child.getKey(), child.getValue());
+        }
+    }
+
+    @Override
+    public String getName() {
+        return nodeName;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlNode.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlNode.java
new file mode 100644
index 0000000..1d9511d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlNode.java
@@ -0,0 +1,25 @@
+/*
+ * 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.inference;
+
+public interface XmlNode {
+
+    XmlNodeType getNodeType();
+
+    String getName();
+
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlNodeType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlNodeType.java
new file mode 100644
index 0000000..fb5835e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlNodeType.java
@@ -0,0 +1,23 @@
+/*
+ * 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.inference;
+
+public enum XmlNodeType {
+    CONTAINER,
+    TEXT,
+    ARRAY;
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlRecordSource.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlRecordSource.java
new file mode 100644
index 0000000..663d75d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlRecordSource.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.inference;
+
+import org.apache.nifi.schema.inference.RecordSource;
+
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+public class XmlRecordSource implements RecordSource<XmlNode> {
+
+    private final XMLEventReader xmlEventReader;
+
+    public XmlRecordSource(final InputStream in, final boolean ignoreWrapper) throws IOException {
+        try {
+            final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
+
+            // Avoid XXE Vulnerabilities
+            xmlInputFactory.setProperty(XMLInputFactory.SUPPORT_DTD, false);
+            xmlInputFactory.setProperty("javax.xml.stream.isSupportingExternalEntities", false);
+
+            xmlEventReader = xmlInputFactory.createXMLEventReader(in);
+
+            if (ignoreWrapper) {
+                readStartElement();
+            }
+        } catch (XMLStreamException e) {
+            throw new IOException("Could not parse XML", e);
+        }
+    }
+
+    @Override
+    public XmlNode next() throws IOException {
+        try {
+            // Find a start element
+            final StartElement startElement = readStartElement();
+            if (startElement == null) {
+                return null;
+            }
+
+            final XmlNode xmlNode = readNext(startElement);
+            return xmlNode;
+        } catch (final XMLStreamException xmle) {
+            throw new IOException(xmle);
+        }
+    }
+
+    private XmlNode readNext(final StartElement startElement) throws XMLStreamException, IOException {
+        // Parse everything until we encounter the end element
+        final StringBuilder content = new StringBuilder();
+        final Map<String, XmlNode> childNodes = new LinkedHashMap<>();
+
+        while (xmlEventReader.hasNext()) {
+            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
+
+            if (xmlEvent.isEndDocument()) {
+                throw new EOFException("Expected to encounter End-of-Element tag for start tag '" + startElement.getName() + "'");
+            }
+
+            if (xmlEvent.isEndElement()) {
+                break;
+            }
+
+            if (xmlEvent.isCharacters()) {
+                final Characters characters = xmlEvent.asCharacters();
+                if (!characters.isWhiteSpace()) {
+                    content.append(characters.getData());
+                }
+            }
+
+            if (xmlEvent.isStartElement()) {
+                final StartElement childStartElement = xmlEvent.asStartElement();
+                final XmlNode childNode = readNext(childStartElement);
+                final String childName = childStartElement.getName().getLocalPart();
+
+                final XmlNode existingNode = childNodes.get(childName);
+                if (existingNode == null) {
+                    childNodes.put(childName, childNode);
+                } else if (existingNode.getNodeType() == XmlNodeType.ARRAY) {
+                    ((XmlArrayNode) existingNode).addElement(childNode);
+                } else {
+                    final XmlArrayNode arrayNode = new XmlArrayNode(childStartElement.getName().getLocalPart());
+                    arrayNode.addElement(existingNode);
+                    arrayNode.addElement(childNode);
+                    childNodes.put(childName, arrayNode);
+                }
+            }
+        }
+
+        final String nodeName = startElement.getName().getLocalPart();
+        if (childNodes.isEmpty()) {
+            return new XmlTextNode(nodeName, content.toString().trim());
+        } else {
+            return new XmlContainerNode(nodeName, childNodes);
+        }
+    }
+
+    private StartElement readStartElement() throws XMLStreamException {
+        while (xmlEventReader.hasNext()) {
+            final XMLEvent xmlEvent = xmlEventReader.nextEvent();
+
+            if (xmlEvent.isStartElement()) {
+                final StartElement startElement = xmlEvent.asStartElement();
+                return startElement;
+            }
+        }
+
+        return null;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlSchemaInference.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlSchemaInference.java
new file mode 100644
index 0000000..134e9a2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlSchemaInference.java
@@ -0,0 +1,117 @@
+/*
+ * 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.inference;
+
+import org.apache.commons.lang3.math.NumberUtils;
+import org.apache.nifi.schema.inference.HierarchicalSchemaInference;
+import org.apache.nifi.schema.inference.TimeValueInference;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class XmlSchemaInference extends HierarchicalSchemaInference<XmlNode> {
+    private static final Logger logger = LoggerFactory.getLogger(XmlSchemaInference.class);
+    private final TimeValueInference timeValueInference;
+
+    public XmlSchemaInference(final TimeValueInference timeValueInference) {
+        this.timeValueInference = timeValueInference;
+    }
+
+    @Override
+    protected DataType getDataType(final XmlNode xmlNode) {
+        final XmlNodeType nodeType = xmlNode.getNodeType();
+        if (nodeType != XmlNodeType.TEXT) {
+            logger.debug("When inferring XML Schema, expected to get an XmlTextNode but received a {} node instead; will ignore this node.", nodeType);
+            return null;
+        }
+
+        final String text = ((XmlTextNode) xmlNode).getText();
+        return inferTextualDataType(text);
+    }
+
+    private DataType inferTextualDataType(final String text) {
+        if (text == null || text.isEmpty()) {
+            return null;
+        }
+
+        if (NumberUtils.isParsable(text)) {
+            if (text.contains(".")) {
+                try {
+                    final double doubleValue = Double.parseDouble(text);
+                    if (doubleValue > Float.MAX_VALUE || doubleValue < Float.MIN_VALUE) {
+                        return RecordFieldType.DOUBLE.getDataType();
+                    }
+
+                    return RecordFieldType.FLOAT.getDataType();
+                } catch (final NumberFormatException nfe) {
+                    return RecordFieldType.STRING.getDataType();
+                }
+            }
+
+            try {
+                final long longValue = Long.parseLong(text);
+                if (longValue > Integer.MAX_VALUE || longValue < Integer.MIN_VALUE) {
+                    return RecordFieldType.LONG.getDataType();
+                }
+
+                return RecordFieldType.INT.getDataType();
+            } catch (final NumberFormatException nfe) {
+                return RecordFieldType.STRING.getDataType();
+            }
+        }
+
+        if (text.equalsIgnoreCase("true") || text.equalsIgnoreCase("false")) {
+            return RecordFieldType.BOOLEAN.getDataType();
+        }
+
+        final Optional<DataType> timeDataType = timeValueInference.getDataType(text);
+        return timeDataType.orElse(RecordFieldType.STRING.getDataType());
+    }
+
+
+    @Override
+    protected boolean isObject(final XmlNode value) {
+        return value.getNodeType() == XmlNodeType.CONTAINER;
+    }
+
+    @Override
+    protected boolean isArray(final XmlNode value) {
+        return value.getNodeType() == XmlNodeType.ARRAY;
+    }
+
+    @Override
+    protected void forEachFieldInRecord(final XmlNode rawRecord, final BiConsumer<String, XmlNode> fieldConsumer) {
+        final XmlContainerNode container = (XmlContainerNode) rawRecord;
+        container.forEach(fieldConsumer);
+    }
+
+    @Override
+    protected void forEachRawRecordInArray(final XmlNode arrayRecord, final Consumer<XmlNode> rawRecordConsumer) {
+        final XmlArrayNode arrayNode = (XmlArrayNode) arrayRecord;
+        arrayNode.forEach(rawRecordConsumer);
+    }
+
+    @Override
+    protected String getRootName(final XmlNode rawRecord) {
+        return rawRecord.getName();
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlTextNode.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlTextNode.java
new file mode 100644
index 0000000..0b87d88
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlTextNode.java
@@ -0,0 +1,41 @@
+/*
+ * 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.inference;
+
+public class XmlTextNode implements XmlNode {
+    private final String nodeName;
+    private final String text;
+
+    public XmlTextNode(final String nodeName, final String text) {
+        this.nodeName = nodeName;
+        this.text = text;
+    }
+
+    @Override
+    public XmlNodeType getNodeType() {
+        return XmlNodeType.TEXT;
+    }
+
+    public String getText() {
+        return text;
+    }
+
+    @Override
+    public String getName() {
+        return nodeName;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
index fc9f6d1..39a59aa 100755
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -31,4 +31,6 @@ org.apache.nifi.syslog.SyslogReader
 org.apache.nifi.syslog.Syslog5424Reader
 
 org.apache.nifi.xml.XMLReader
-org.apache.nifi.xml.XMLRecordSetWriter
\ No newline at end of file
+org.apache.nifi.xml.XMLRecordSetWriter
+
+org.apache.nifi.schema.inference.VolatileSchemaCache
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.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 8e18934..ee712bf 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
@@ -31,7 +31,7 @@
 
 
 		<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
@@ -40,11 +40,11 @@
 			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>
@@ -63,18 +63,106 @@
 			<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>Schema Inference</h2>
+
+        <p>
+            While NiFi's Record API does require that each Record have a schema, it is often convenient to infer the schema based on the values in the data,
+            rather than having to manually create a schema. This is accomplished by selecting a value of "Infer Schema" for the "Schema Access Strategy" property.
+            When using this strategy, the Reader will determine the schema by first parsing all data in the FlowFile, keeping track of all fields that it has encountered
+            and the type of each field. Once all data has been parsed, a schema is formed that encompasses all fields that have been encountered.
+        </p>
+
+        <p>
+            A common concern when inferring schemas is how to handle the condition of two values that have different types. For example, consider a FlowFile with the following two records:
+        </p>
+<code><pre>
+name, age
+John, 8
+Jane, Ten
+</pre></code>
+
+        <p>
+            It is clear that the "name" field will be inferred as a STRING type. However, how should we handle the "age" field? Should the field be an CHOICE between INT and STRING? Should we
+            prefer LONG over INT? Should we just use a STRING? Should the field be considered nullable?
+        </p>
+
+        <p>
+            To help understand how this Record Reader infers schemas, we have the following list of rules that are followed in the inference logic:
+        </p>
+
+        <ul>
+            <li>All fields are inferred to be nullable.</li>
+            <li>
+                When two values are encountered for the same field in two different records (or two values are encountered for an ARRAY type), the inference engine prefers
+                to use a "wider" data type over using a CHOICE data type. A data type "A" is said to be wider than data type "B" if and only if data type "A" encompasses all
+                values of "B" in addition to other values. For example, the LONG type is wider than the INT type but not wider than the BOOLEAN type (and BOOLEAN is also not wider
+                than LONG). INT is wider than SHORT. The STRING type is considered wider than all other types with the Exception of MAP, RECORD, ARRAY, and CHOICE.
+            </li>
+            <li>
+                Before inferring the type of a value, leading and trailing whitespace are removed. Additionally, if the value is surrounded by double-quotes ("), the double-quotes
+                are removed. Therefore, the value <code>16</code> is interpreted the same as <code>         "16"</code>. Both will be interpreted as an INT. However, the value
+                <code>"         16"</code> will be inferred as a STRING type because the white space is enclosed within double-quotes, which means that the white space is considered
+                part of the value.
+            </li>
+            <li>
+                If the "Time Format," "Timestamp Format," or "Date Format" properties are configured, any value that would otherwise be considered a STRING type is first checked against
+                the configured formats to see if it matches any of them. If the value matches the Timestamp Format, the value is considered a Timestamp field. If it matches the Date Format,
+                it is considered a Date field. If it matches the Time Format, it is considered a Time field. In the unlikely event that the value matches more than one of the configured
+                formats, they will be matched in the order: Timestamp, Date, Time. I.e., if a value matched both the Timestamp Format and the Date Format, the type that is inferred will be
+                Timestamp. Because parsing dates and times can be expensive, it is advisable not to configure these formats if dates, times, and timestamps are not expected, or if processing
+                the data as a STRING is acceptable. For use cases when this is important, though, the inference engine is intelligent enough to optimize the parsing by first checking several
+                very cheap conditions. For example, the string's length is examined to see if it is too long or too short to match the pattern. This results in far more efficient processing
+                than would result if attempting to parse each string value as a timestamp.
+            </li>
+            <li>The MAP type is never inferred.</li>
+            <li>The ARRAY type is never inferred.</li>
+            <li>The RECORD type is never inferred.</li>
+            <li>If a field exists but all values are null, then the field is inferred to be of type STRING.</li>
+        </ul>
+
+
+
+        <h2>Caching of Inferred Schemas</h2>
+
+        <p>
+            This Record Reader requires that if a schema is to be inferred, that all records be read in order to ensure that the schema that gets inferred is applicable for all
+            records in the FlowFile. However, this can become expensive, especially if the data undergoes many different transformations. To alleviate the cost of inferring schemas,
+            the Record Reader can be configured with a "Schema Inference Cache" by populating the property with that name. This is a Controller Service that can be shared by Record
+            Readers and Record Writers.
+        </p>
+
+        <p>
+            Whenever a Record Writer is used to write data, if it is configured with a "Schema Cache," it will also add the schema to the Schema Cache. This will result in an
+            identifier for that schema being added as an attribute to the FlowFile.
+        </p>
+
+        <p>
+            Whenever a Record Reader is used to read data, if it is configured with a "Schema Inference Cache", it will first look for a "schema.cache.identifier" attribute on the FlowFile.
+            If the attribute exists, it will use the value of that attribute to lookup the schema in the schema cache. If it is able to find a schema in the cache with that identifier,
+            then it will use that schema instead of reading, parsing, and analyzing the data to infer the schema. If the attribute is not available on the FlowFile, or if the attribute is
+            available but the cache does not have a schema with that identifier, then the Record Reader will proceed to infer the schema as described above.
+        </p>
+
+        <p>
+            The end result is that users are able to chain together many different Processors to operate on Record-oriented data. Typically, only the first such Processor in the chain will
+            incur the "penalty" of inferring the schema. For all other Processors in the chain, the Record Reader is able to simply lookup the schema in the Schema Cache by identifier.
+            This allows the Record Reader to infer a schema accurately, since it is inferred based on all data in the FlowFile, and still allows this to happen efficiently since the schema
+            will typically only be inferred once, regardless of how many Processors handle the data.
+        </p>
+
+
 
 		<h2>Examples</h2>
-		
+
 		<h3>Example 1</h3>
-		
+
         <p>
         	As an example, consider a FlowFile whose contents consists of the following:
         </p>
@@ -86,12 +174,12 @@ first customer!"<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 Schema Registry pointing to an AvroSchemaRegistry and the schema is
         	configured as the following:
         </p>
-        
+
 <code>
 <pre>
 {
@@ -147,11 +235,11 @@ first customer!"<br />
     			</tr>
     		</body>
     	</table>
-    	
+
     	<p>
     		The second record will contain the following values:
     	</p>
-    	
+
 		<table>
     		<head>
     			<th>Field Name</th>
@@ -180,11 +268,11 @@ first customer!"<br />
     			</tr>
     		</body>
     	</table>
-    	
+
 		<p>
 			The third record will contain the following values:
-		</p>    	
-    	
+		</p>
+
 		<table>
     		<head>
     			<th>Field Name</th>
@@ -227,7 +315,7 @@ first customer!"<br />
     		a schema will be referenced elsewhere. But what happens if a schema is obtained from a Schema Registry, for instance,
     		and the CSV Header indicates a different set of column names?
     	</p>
-    	
+
     	<p>
     		For example, let's say that the following schema is obtained from the Schema Registry:
     	</p>
@@ -247,22 +335,22 @@ first customer!"<br />
 }
 </pre>
 </code>
-		
+
 		<p>
 			And the CSV contains the following data:
 		</p>
-		
+
 <code>
 <pre>
 id, name, balance, notes
 1, John Doe, 123.45, First Customer
 </pre>
 </code>
-		
+
 		<p>
 		Note here that our schema indicates that the final column is named "memo" whereas the CSV Header indicates that it is named "notes."
 		</p>
-    	
+
     	<p>
     	In this case, the reader will look at the "Ignore CSV Header Column Names" property. If this property is set to "true" then the column names
     	provided in the CSV will simply be ignored and the last column will be called "memo." However, if the "Ignore CSV Header Column Names" property
@@ -296,8 +384,8 @@ id, name, balance, notes
     		</body>
     	</table>
 		</p>
-		
-		
+
+
 		<p>
 		With "Ignore CSV Header Column Names" property set to "true":<br />
 				<table>
@@ -329,6 +417,6 @@ id, name, balance, notes
     		</body>
     	</table>
 		</p>
-		
+
     </body>
 </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 75a93a3..87ca8c3 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
@@ -28,13 +28,13 @@
         	<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>
-        
+
         <p>
         	If the root of the FlowFile's JSON is a JSON Array, each JSON Object found in that array will be treated as a separate
         	Record, not as a single record made up of an array. If the root of the FlowFile's JSON is a JSON Object, it will be
         	evaluated as a single Record.
         </p>
-        
+
         <p>
         	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
@@ -42,7 +42,7 @@
         	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>
 			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
@@ -51,7 +51,7 @@
 
 
 		<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
@@ -60,11 +60,11 @@
 			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>
@@ -84,20 +84,110 @@
 				property (Date Format, Time Format, Timestamp Format property). If no value is specified, then the value will be converted into a String
 				representation of the number of milliseconds since epoch (Midnight GMT, January 1, 1970).</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>Schema Inference</h2>
+
+		<p>
+			While NiFi's Record API does require that each Record have a schema, it is often convenient to infer the schema based on the values in the data,
+			rather than having to manually create a schema. This is accomplished by selecting a value of "Infer Schema" for the "Schema Access Strategy" property.
+			When using this strategy, the Reader will determine the schema by first parsing all data in the FlowFile, keeping track of all fields that it has encountered
+			and the type of each field. Once all data has been parsed, a schema is formed that encompasses all fields that have been encountered.
+		</p>
+
+		<p>
+			A common concern when inferring schemas is how to handle the condition of two values that have different types. For example, consider a FlowFile with the following two records:
+		</p>
+		<code><pre>
+[{
+    "name": "John",
+    "age": 8,
+    "values": "N/A"
+}, {
+    "name": "Jane",
+    "age": "Ten",
+    "values": [ 8, "Ten" ]
+}]
+</pre></code>
+
+		<p>
+			It is clear that the "name" field will be inferred as a STRING type. However, how should we handle the "age" field? Should the field be an CHOICE between INT and STRING? Should we
+			prefer LONG over INT? Should we just use a STRING? Should the field be considered nullable?
+		</p>
+
+		<p>
+			To help understand how this Record Reader infers schemas, we have the following list of rules that are followed in the inference logic:
+		</p>
+
+		<ul>
+			<li>All fields are inferred to be nullable.</li>
+			<li>
+				When two values are encountered for the same field in two different records (or two values are encountered for an ARRAY type), the inference engine prefers
+				to use a "wider" data type over using a CHOICE data type. A data type "A" is said to be wider than data type "B" if and only if data type "A" encompasses all
+				values of "B" in addition to other values. For example, the LONG type is wider than the INT type but not wider than the BOOLEAN type (and BOOLEAN is also not wider
+				than LONG). INT is wider than SHORT. The STRING type is considered wider than all other types with the Exception of MAP, RECORD, ARRAY, and CHOICE.
+			</li>
+			<li>
+				If two values are encountered for the same field in two different records (or two values are encountered for an ARRAY type), but neither value is of a type that
+				is wider than the other, then a CHOICE type is used. In the example above, the "values" field will be inferred as a CHOICE between a STRING or an ARRRAY&lt;STRING&gt;.
+			</li>
+            <li>
+                If the "Time Format," "Timestamp Format," or "Date Format" properties are configured, any value that would otherwise be considered a STRING type is first checked against
+                the configured formats to see if it matches any of them. If the value matches the Timestamp Format, the value is considered a Timestamp field. If it matches the Date Format,
+                it is considered a Date field. If it matches the Time Format, it is considered a Time field. In the unlikely event that the value matches more than one of the configured
+                formats, they will be matched in the order: Timestamp, Date, Time. I.e., if a value matched both the Timestamp Format and the Date Format, the type that is inferred will be
+                Timestamp. Because parsing dates and times can be expensive, it is advisable not to configure these formats if dates, times, and timestamps are not expected, or if processing
+                the data as a STRING is acceptable. For use cases when this is important, though, the inference engine is intelligent enough to optimize the parsing by first checking several
+                very cheap conditions. For example, the string's length is examined to see if it is too long or too short to match the pattern. This results in far more efficient processing
+                than would result if attempting to parse each string value as a timestamp.
+            </li>
+			<li>The MAP type is never inferred. Instead, the RECORD type is used.</li>
+			<li>If a field exists but all values are null, then the field is inferred to be of type STRING.</li>
+		</ul>
+
+
+
+		<h2>Caching of Inferred Schemas</h2>
+
+		<p>
+			This Record Reader requires that if a schema is to be inferred, that all records be read in order to ensure that the schema that gets inferred is applicable for all
+			records in the FlowFile. However, this can become expensive, especially if the data undergoes many different transformations. To alleviate the cost of inferring schemas,
+			the Record Reader can be configured with a "Schema Inference Cache" by populating the property with that name. This is a Controller Service that can be shared by Record
+			Readers and Record Writers.
+		</p>
+
+		<p>
+			Whenever a Record Writer is used to write data, if it is configured with a "Schema Cache," it will also add the schema to the Schema Cache. This will result in an
+			identifier for that schema being added as an attribute to the FlowFile.
+		</p>
+
+		<p>
+			Whenever a Record Reader is used to read data, if it is configured with a "Schema Inference Cache", it will first look for a "schema.cache.identifier" attribute on the FlowFile.
+			If the attribute exists, it will use the value of that attribute to lookup the schema in the schema cache. If it is able to find a schema in the cache with that identifier,
+			then it will use that schema instead of reading, parsing, and analyzing the data to infer the schema. If the attribute is not available on the FlowFile, or if the attribute is
+			available but the cache does not have a schema with that identifier, then the Record Reader will proceed to infer the schema as described above.
+		</p>
+
+		<p>
+			The end result is that users are able to chain together many different Processors to operate on Record-oriented data. Typically, only the first such Processor in the chain will
+			incur the "penalty" of inferring the schema. For all other Processors in the chain, the Record Reader is able to simply lookup the schema in the Schema Cache by identifier.
+			This allows the Record Reader to infer a schema accurately, since it is inferred based on all data in the FlowFile, and still allows this to happen efficiently since the schema
+			will typically only be inferred once, regardless of how many Processors handle the data.
+		</p>
+
+
 
 		<h2>Examples</h2>
 
         <p>
         	As an example, consider a FlowFile whose content contains the following JSON:
         </p>
-        
+
         <code>
         <pre>
 [{
@@ -124,11 +214,11 @@
   }]
 		</pre>
         </code>
-        
+
         <p>
     		And the following schema has been configured:
         </p>
-        
+
         <code>
         <pre>
 {
@@ -148,10 +238,10 @@
 }
         </pre>
         </code>
-        
+
         <p>
         	If we configure this Controller Service with the following user-defined properties:
-        	
+
         	<table>
         		<tr>
         			<th>Property Name</th>
@@ -179,7 +269,7 @@
     			</tr>
         	</table>
         </p>
-        
+
 		<p>
 			In this case, the FlowFile will generate two Records. The first record will consist of the following key/value pairs:
 
@@ -210,7 +300,7 @@
     			</tr>
 			</table>
 		</p>
-		
+
 		<p>
 			The second record will consist of the following key/value pairs:
 
@@ -241,6 +331,6 @@
     			</tr>
 			</table>
 		</p>
-		
+
     </body>
 </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 6c717c3..b5415c3 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
@@ -28,7 +28,7 @@
         	If the schema contains a field for which no JSON field exists, a null value will be used in the Record
         	(or the default value defined in the schema, if applicable).
         </p>
-        
+
         <p>
         	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
@@ -37,7 +37,7 @@
 
 
 		<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
@@ -46,11 +46,11 @@
 			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>
@@ -70,15 +70,107 @@
 				property (Date Format, Time Format, Timestamp Format property). If no value is specified, then the value will be converted into a String
 				representation of the number of milliseconds since epoch (Midnight GMT, January 1, 1970).</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>Schema Inference</h2>
+
+        <p>
+            While NiFi's Record API does require that each Record have a schema, it is often convenient to infer the schema based on the values in the data,
+            rather than having to manually create a schema. This is accomplished by selecting a value of "Infer Schema" for the "Schema Access Strategy" property.
+            When using this strategy, the Reader will determine the schema by first parsing all data in the FlowFile, keeping track of all fields that it has encountered
+            and the type of each field. Once all data has been parsed, a schema is formed that encompasses all fields that have been encountered.
+        </p>
+
+        <p>
+            A common concern when inferring schemas is how to handle the condition of two values that have different types. For example, consider a FlowFile with the following two records:
+        </p>
+<code><pre>
+[{
+    "name": "John",
+    "age": 8,
+    "values": "N/A"
+}, {
+    "name": "Jane",
+    "age": "Ten",
+    "values": [ 8, "Ten" ]
+}]
+</pre></code>
+
+        <p>
+            It is clear that the "name" field will be inferred as a STRING type. However, how should we handle the "age" field? Should the field be an CHOICE between INT and STRING? Should we
+            prefer LONG over INT? Should we just use a STRING? Should the field be considered nullable?
+        </p>
+
+        <p>
+            To help understand how this Record Reader infers schemas, we have the following list of rules that are followed in the inference logic:
+        </p>
+
+        <ul>
+            <li>All fields are inferred to be nullable.</li>
+            <li>
+                When two values are encountered for the same field in two different records (or two values are encountered for an ARRAY type), the inference engine prefers
+                to use a "wider" data type over using a CHOICE data type. A data type "A" is said to be wider than data type "B" if and only if data type "A" encompasses all
+                values of "B" in addition to other values. For example, the LONG type is wider than the INT type but not wider than the BOOLEAN type (and BOOLEAN is also not wider
+                than LONG). INT is wider than SHORT. The STRING type is considered wider than all other types with the Exception of MAP, RECORD, ARRAY, and CHOICE.
+            </li>
+            <li>
+                If two values are encountered for the same field in two different records (or two values are encountered for an ARRAY type), but neither value is of a type that
+                is wider than the other, then a CHOICE type is used. In the example above, the "values" field will be inferred as a CHOICE between a STRING or an ARRRAY&lt;STRING&gt;.
+            </li>
+            <li>
+                If the "Time Format," "Timestamp Format," or "Date Format" properties are configured, any value that would otherwise be considered a STRING type is first checked against
+                the configured formats to see if it matches any of them. If the value matches the Timestamp Format, the value is considered a Timestamp field. If it matches the Date Format,
+                it is considered a Date field. If it matches the Time Format, it is considered a Time field. In the unlikely event that the value matches more than one of the configured
+                formats, they will be matched in the order: Timestamp, Date, Time. I.e., if a value matched both the Timestamp Format and the Date Format, the type that is inferred will be
+                Timestamp. Because parsing dates and times can be expensive, it is advisable not to configure these formats if dates, times, and timestamps are not expected, or if processing
+                the data as a STRING is acceptable. For use cases when this is important, though, the inference engine is intelligent enough to optimize the parsing by first checking several
+                very cheap conditions. For example, the string's length is examined to see if it is too long or too short to match the pattern. This results in far more efficient processing
+                than would result if attempting to parse each string value as a timestamp.
+            </li>
+            <li>The MAP type is never inferred. Instead, the RECORD type is used.</li>
+            <li>If a field exists but all values are null, then the field is inferred to be of type STRING.</li>
+        </ul>
+
+
+
+        <h2>Caching of Inferred Schemas</h2>
+
+        <p>
+            This Record Reader requires that if a schema is to be inferred, that all records be read in order to ensure that the schema that gets inferred is applicable for all
+            records in the FlowFile. However, this can become expensive, especially if the data undergoes many different transformations. To alleviate the cost of inferring schemas,
+            the Record Reader can be configured with a "Schema Inference Cache" by populating the property with that name. This is a Controller Service that can be shared by Record
+            Readers and Record Writers.
+        </p>
+
+        <p>
+            Whenever a Record Writer is used to write data, if it is configured with a "Schema Cache," it will also add the schema to the Schema Cache. This will result in an
+            identifier for that schema being added as an attribute to the FlowFile.
+        </p>
+
+        <p>
+            Whenever a Record Reader is used to read data, if it is configured with a "Schema Inference Cache", it will first look for a "schema.cache.identifier" attribute on the FlowFile.
+            If the attribute exists, it will use the value of that attribute to lookup the schema in the schema cache. If it is able to find a schema in the cache with that identifier,
+            then it will use that schema instead of reading, parsing, and analyzing the data to infer the schema. If the attribute is not available on the FlowFile, or if the attribute is
+            available but the cache does not have a schema with that identifier, then the Record Reader will proceed to infer the schema as described above.
+        </p>
+
+        <p>
+            The end result is that users are able to chain together many different Processors to operate on Record-oriented data. Typically, only the first such Processor in the chain will
+            incur the "penalty" of inferring the schema. For all other Processors in the chain, the Record Reader is able to simply lookup the schema in the Schema Cache by identifier.
+            This allows the Record Reader to infer a schema accurately, since it is inferred based on all data in the FlowFile, and still allows this to happen efficiently since the schema
+            will typically only be inferred once, regardless of how many Processors handle the data.
+        </p>
+
+
+
         <h2>Examples</h2>
-        
+
         <p>
         	As an example, consider the following JSON is read:
         </p>
@@ -109,10 +201,10 @@
   }]
 </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:
+        	Controller Service is chosen to denote the Schema):
         </p>
 
 <code>
@@ -142,17 +234,17 @@
 }
 </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>
@@ -205,11 +297,11 @@
     			</td>
     		</tr>
         </table>
-        
+
         <p>
         	The second Record will consist of the following values:
         </p>
-        
+
 		<table>
 			<tr>
     			<th>Field Name</th>
@@ -236,6 +328,6 @@
     			<td><i>empty array</i></td>
     		</tr>
         </table>
-        
+
     </body>
 </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.xml.XMLReader/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html
index 46cd259..1444d48 100755
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html
@@ -27,6 +27,7 @@
         that are not defined in the schema will be skipped. Depending on whether the property "Expect Records as Array"
         is set to "false" or "true", the reader either expects a single record or an array of records for each FlowFile.
     </p>
+
     <p>
         Example: Single record
     </p>
@@ -429,5 +430,102 @@
             </pre>
     </code>
 
+
+    <h2>Schema Inference</h2>
+
+    <p>
+        While NiFi's Record API does require that each Record have a schema, it is often convenient to infer the schema based on the values in the data,
+        rather than having to manually create a schema. This is accomplished by selecting a value of "Infer Schema" for the "Schema Access Strategy" property.
+        When using this strategy, the Reader will determine the schema by first parsing all data in the FlowFile, keeping track of all fields that it has encountered
+        and the type of each field. Once all data has been parsed, a schema is formed that encompasses all fields that have been encountered.
+    </p>
+
+    <p>
+        A common concern when inferring schemas is how to handle the condition of two values that have different types. For example, consider a FlowFile with the following two records:
+    </p>
+    <code><pre>
+<root>
+    <record>
+        <name>John</name>
+        <age>8</age>
+        <values>N/A</values>
+    </record>
+    <record>
+        <name>Jane</name>
+        <age>Ten</age>
+        <values>8</values>
+        <values>Ten</values>
+    </record>
+</root>
+</pre></code>
+
+    <p>
+        It is clear that the "name" field will be inferred as a STRING type. However, how should we handle the "age" field? Should the field be an CHOICE between INT and STRING? Should we
+        prefer LONG over INT? Should we just use a STRING? Should the field be considered nullable?
+    </p>
+
+    <p>
+        To help understand how this Record Reader infers schemas, we have the following list of rules that are followed in the inference logic:
+    </p>
+
+    <ul>
+        <li>All fields are inferred to be nullable.</li>
+        <li>
+            When two values are encountered for the same field in two different records (or two values are encountered for an ARRAY type), the inference engine prefers
+            to use a "wider" data type over using a CHOICE data type. A data type "A" is said to be wider than data type "B" if and only if data type "A" encompasses all
+            values of "B" in addition to other values. For example, the LONG type is wider than the INT type but not wider than the BOOLEAN type (and BOOLEAN is also not wider
+            than LONG). INT is wider than SHORT. The STRING type is considered wider than all other types with the Exception of MAP, RECORD, ARRAY, and CHOICE.
+        </li>
+        <li>
+            If two values are encountered for the same field in two different records (or two values are encountered for an ARRAY type), but neither value is of a type that
+            is wider than the other, then a CHOICE type is used. In the example above, the "values" field will be inferred as a CHOICE between a STRING or an ARRRAY&lt;STRING&gt;.
+        </li>
+        <li>
+            If the "Time Format," "Timestamp Format," or "Date Format" properties are configured, any value that would otherwise be considered a STRING type is first checked against
+            the configured formats to see if it matches any of them. If the value matches the Timestamp Format, the value is considered a Timestamp field. If it matches the Date Format,
+            it is considered a Date field. If it matches the Time Format, it is considered a Time field. In the unlikely event that the value matches more than one of the configured
+            formats, they will be matched in the order: Timestamp, Date, Time. I.e., if a value matched both the Timestamp Format and the Date Format, the type that is inferred will be
+            Timestamp. Because parsing dates and times can be expensive, it is advisable not to configure these formats if dates, times, and timestamps are not expected, or if processing
+            the data as a STRING is acceptable. For use cases when this is important, though, the inference engine is intelligent enough to optimize the parsing by first checking several
+            very cheap conditions. For example, the string's length is examined to see if it is too long or too short to match the pattern. This results in far more efficient processing
+            than would result if attempting to parse each string value as a timestamp.
+        </li>
+        <li>The MAP type is never inferred. Instead, the RECORD type is used.</li>
+        <li>If two elements exist with the same name and the same parent (i.e., two sibling elements have the same name), the field will be inferred to be of type ARRAY.</li>
+        <li>If a field exists but all values are null, then the field is inferred to be of type STRING.</li>
+    </ul>
+
+
+
+    <h2>Caching of Inferred Schemas</h2>
+
+    <p>
+        This Record Reader requires that if a schema is to be inferred, that all records be read in order to ensure that the schema that gets inferred is applicable for all
+        records in the FlowFile. However, this can become expensive, especially if the data undergoes many different transformations. To alleviate the cost of inferring schemas,
+        the Record Reader can be configured with a "Schema Inference Cache" by populating the property with that name. This is a Controller Service that can be shared by Record
+        Readers and Record Writers.
+    </p>
+
+    <p>
+        Whenever a Record Writer is used to write data, if it is configured with a "Schema Cache," it will also add the schema to the Schema Cache. This will result in an
+        identifier for that schema being added as an attribute to the FlowFile.
+    </p>
+
+    <p>
+        Whenever a Record Reader is used to read data, if it is configured with a "Schema Inference Cache", it will first look for a "schema.cache.identifier" attribute on the FlowFile.
+        If the attribute exists, it will use the value of that attribute to lookup the schema in the schema cache. If it is able to find a schema in the cache with that identifier,
+        then it will use that schema instead of reading, parsing, and analyzing the data to infer the schema. If the attribute is not available on the FlowFile, or if the attribute is
+        available but the cache does not have a schema with that identifier, then the Record Reader will proceed to infer the schema as described above.
+    </p>
+
+    <p>
+        The end result is that users are able to chain together many different Processors to operate on Record-oriented data. Typically, only the first such Processor in the chain will
+        incur the "penalty" of inferring the schema. For all other Processors in the chain, the Record Reader is able to simply lookup the schema in the Schema Cache by identifier.
+        This allows the Record Reader to infer a schema accurately, since it is inferred based on all data in the FlowFile, and still allows this to happen efficiently since the schema
+        will typically only be inferred once, regardless of how many Processors handle the data.
+    </p>
+
+
+
     </body>
 </html>
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java
index 7ddcc29..dd25a3b 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java
@@ -437,7 +437,7 @@ public class TestCSVRecordReader {
         try (final InputStream bais = new ByteArrayInputStream(inputData);
             final CSVRecordReader reader = createReader(bais, schema, format)) {
 
-            final Record record = reader.nextRecord();
+            final Record record = reader.nextRecord(true, true);
             assertNotNull(record);
 
             assertEquals("1", record.getValue("id"));
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVSchemaInference.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVSchemaInference.java
new file mode 100644
index 0000000..ce801d6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVSchemaInference.java
@@ -0,0 +1,78 @@
+/*
+ * 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.csv;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.inference.InferSchemaAccessStrategy;
+import org.apache.nifi.schema.inference.TimeValueInference;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.util.MockConfigurationContext;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+
+public class TestCSVSchemaInference {
+
+    private final TimeValueInference timestampInference = new TimeValueInference("yyyy-MM-dd", "HH:mm:ss", "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'");
+
+    @Test
+    public void testInferenceIncludesAllRecords() throws IOException {
+        final File file = new File("src/test/resources/csv/prov-events.csv");
+
+        final Map<PropertyDescriptor, String> properties = new HashMap<>();
+        new CSVReader().getSupportedPropertyDescriptors().forEach(prop -> properties.put(prop, prop.getDefaultValue()));
+        properties.put(CSVUtils.TRIM_FIELDS, "true");
+        final PropertyContext context = new MockConfigurationContext(properties, null);
+
+        final RecordSchema schema;
+        try (final InputStream in = new FileInputStream(file);
+             final InputStream bufferedIn = new BufferedInputStream(in)) {
+
+            final InferSchemaAccessStrategy<?> accessStrategy = new InferSchemaAccessStrategy<>(
+                (var, content) -> new CSVRecordSource(content, context),
+                new CSVSchemaInference(timestampInference), Mockito.mock(ComponentLog.class));
+            schema = accessStrategy.getSchema(null, bufferedIn, null);
+        }
+
+        assertSame(RecordFieldType.STRING, schema.getDataType("eventId").get().getFieldType());
+        assertSame(RecordFieldType.INT, schema.getDataType("eventOrdinal").get().getFieldType());
+        assertSame(RecordFieldType.STRING, schema.getDataType("eventType").get().getFieldType());
+        assertSame(RecordFieldType.LONG, schema.getDataType("timestampMillis").get().getFieldType());
+
+        assertEquals(RecordFieldType.TIMESTAMP.getDataType("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"), schema.getDataType("timestamp").get());
+        assertEquals(RecordFieldType.TIME.getDataType("HH:mm:ss"), schema.getDataType("eventTime").get());
+        assertEquals(RecordFieldType.DATE.getDataType("yyyy-MM-dd"), schema.getDataType("eventDate").get());
+        assertEquals(RecordFieldType.STRING.getDataType(), schema.getDataType("maybeTime").get());
+        assertEquals(RecordFieldType.DATE.getDataType("yyyy-MM-dd"), schema.getDataType("maybeDate").get());
+
+        assertSame(RecordFieldType.INT, schema.getDataType("parentIds").get().getFieldType());
+        assertSame(RecordFieldType.STRING, schema.getDataType("numeric string").get().getFieldType());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestJacksonCSVRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestJacksonCSVRecordReader.java
index d83cbfb..935fe73 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestJacksonCSVRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestJacksonCSVRecordReader.java
@@ -235,7 +235,7 @@ public class TestJacksonCSVRecordReader {
         try (final InputStream bais = new ByteArrayInputStream(inputData);
             final JacksonCSVRecordReader reader = createReader(bais, schema, format)) {
 
-            final Record record = reader.nextRecord();
+            final Record record = reader.nextRecord(true, true);
             assertNotNull(record);
 
             assertEquals("1", record.getValue("id"));
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java
index 6f4269a..d0ac6b3 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java
@@ -17,7 +17,18 @@
 
 package org.apache.nifi.csv;
 
-import static org.junit.Assert.assertEquals;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.QuoteMode;
+import org.apache.nifi.schema.access.SchemaNameAsAttribute;
+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.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.junit.Test;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -35,18 +46,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.TimeZone;
 
-import org.apache.commons.csv.CSVFormat;
-import org.apache.commons.csv.QuoteMode;
-import org.apache.nifi.schema.access.SchemaNameAsAttribute;
-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.RecordFieldType;
-import org.apache.nifi.serialization.record.RecordSchema;
-import org.apache.nifi.serialization.record.RecordSet;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 
 public class TestWriteCSVResult {
@@ -111,18 +111,14 @@ public class TestWriteCSVResult {
         assertEquals(2, splits.length);
         assertEquals(headerLine, splits[0]);
 
-        final String values = splits[1];
-        final StringBuilder expectedBuilder = new StringBuilder();
-        expectedBuilder.append("\"a孟bc李12儒3\",\"true\",\"1\",\"c\",\"8\",\"9\",\"8\",\"8\",\"8.0\",\"8.0\",");
-
         final String dateValue = getDateFormat(RecordFieldType.DATE.getDefaultFormat()).format(now);
         final String timeValue = getDateFormat(RecordFieldType.TIME.getDefaultFormat()).format(now);
         final String timestampValue = getDateFormat(RecordFieldType.TIMESTAMP.getDefaultFormat()).format(now);
 
-        expectedBuilder.append('"').append(dateValue).append('"').append(',');
-        expectedBuilder.append('"').append(timeValue).append('"').append(',');
-        expectedBuilder.append('"').append(timestampValue).append('"').append(',');
-        expectedBuilder.append(",\"48\",,");
+        final String values = splits[1];
+        final StringBuilder expectedBuilder = new StringBuilder();
+        expectedBuilder.append("\"true\",\"1\",\"8\",\"9\",\"8\",\"8\",\"8.0\",\"8.0\",\"" + dateValue + "\",\"" + timeValue + "\",\"" + timestampValue + "\",\"c\",\"a孟bc李12儒3\",,\"48\",,");
+
         final String expectedValues = expectedBuilder.toString();
 
         assertEquals(expectedValues, values);
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestInferJsonSchemaAccessStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestInferJsonSchemaAccessStrategy.java
new file mode 100644
index 0000000..4a10356
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestInferJsonSchemaAccessStrategy.java
@@ -0,0 +1,196 @@
+/*
+ * 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.json;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.inference.InferSchemaAccessStrategy;
+import org.apache.nifi.schema.inference.SchemaInferenceEngine;
+import org.apache.nifi.schema.inference.TimeValueInference;
+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;
+import org.apache.nifi.serialization.record.type.ChoiceDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.codehaus.jackson.JsonNode;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.BufferedInputStream;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+public class TestInferJsonSchemaAccessStrategy {
+    private final String dateFormat = RecordFieldType.DATE.getDefaultFormat();
+    private final String timeFormat = RecordFieldType.TIME.getDefaultFormat();
+    private final String timestampFormat = "yyyy-MM-DD'T'HH:mm:ss.SSS'Z'";
+
+    private final SchemaInferenceEngine<JsonNode> timestampInference = new JsonSchemaInference(new TimeValueInference("yyyy-MM-dd", "HH:mm:ss", "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"));
+    private final SchemaInferenceEngine<JsonNode> noTimestampInference = new JsonSchemaInference(new TimeValueInference("yyyy-MM-dd", "HH:mm:ss", "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"));
+
+    @Test
+    @Ignore
+    public void testPerformanceOfSchemaInferenceWithTimestamp() throws IOException {
+        final File file = new File("src/test/resources/json/prov-events.json");
+        final byte[] data = Files.readAllBytes(file.toPath());
+        final ComponentLog logger = Mockito.mock(ComponentLog.class);
+
+        final byte[] manyCopies = new byte[data.length * 20];
+        for (int i=0; i < 20; i++) {
+            System.arraycopy(data, 0, manyCopies, data.length * i, data.length);
+        }
+
+        final InferSchemaAccessStrategy<?> accessStrategy = new InferSchemaAccessStrategy<>((var,content) -> new JsonRecordSource(content), timestampInference, Mockito.mock(ComponentLog.class));
+
+        for (int j = 0; j < 10; j++) {
+            final long start = System.nanoTime();
+
+            for (int i = 0; i < 10_000; i++) {
+                try (final InputStream in = new ByteArrayInputStream(manyCopies)) {
+                    final RecordSchema schema = accessStrategy.getSchema(null, in, null);
+                }
+            }
+
+            final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+            System.out.println(millis);
+        }
+    }
+
+    @Test
+    @Ignore
+    public void testPerformanceOfSchemaInferenceWithoutTimestamp() throws IOException {
+        final File file = new File("src/test/resources/json/prov-events.json");
+        final byte[] data = Files.readAllBytes(file.toPath());
+        final ComponentLog logger = Mockito.mock(ComponentLog.class);
+
+        final byte[] manyCopies = new byte[data.length * 20];
+        for (int i=0; i < 20; i++) {
+            System.arraycopy(data, 0, manyCopies, data.length * i, data.length);
+        }
+
+        for (int j = 0; j < 10; j++) {
+            final long start = System.nanoTime();
+
+            for (int i = 0; i < 10_000; i++) {
+                try (final InputStream in = new ByteArrayInputStream(manyCopies)) {
+                    final InferSchemaAccessStrategy<?> accessStrategy = new InferSchemaAccessStrategy<>((var,content) -> new JsonRecordSource(content),
+                         noTimestampInference, Mockito.mock(ComponentLog.class));
+
+                    final RecordSchema schema = accessStrategy.getSchema(null, in, null);
+                }
+            }
+
+            final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+            System.out.println(millis);
+        }
+    }
+
+    @Test
+    public void testInferenceIncludesAllRecords() throws IOException {
+        final File file = new File("src/test/resources/json/prov-events.json");
+        final RecordSchema schema = inferSchema(file);
+
+        final RecordField extraField1 = schema.getField("extra field 1").get();
+        assertSame(RecordFieldType.STRING, extraField1.getDataType().getFieldType());
+
+        final RecordField extraField2 = schema.getField("extra field 2").get();
+        assertSame(RecordFieldType.STRING, extraField2.getDataType().getFieldType());
+
+        final RecordField updatedAttributesField = schema.getField("updatedAttributes").get();
+        final DataType updatedAttributesDataType = updatedAttributesField.getDataType();
+        assertSame(RecordFieldType.RECORD, updatedAttributesDataType.getFieldType());
+
+        final List<String> expectedAttributeNames = Arrays.asList("path", "filename", "drop reason", "uuid", "reporting.task.type", "s2s.address", "schema.cache.identifier", "reporting.task.uuid",
+            "record.count", "s2s.host", "reporting.task.transaction.id", "reporting.task.name", "mime.type");
+
+        final RecordSchema updatedAttributesSchema = ((RecordDataType) updatedAttributesDataType).getChildSchema();
+        assertEquals(expectedAttributeNames.size(), updatedAttributesSchema.getFieldCount());
+
+        for (final String attributeName : expectedAttributeNames) {
+            assertSame(RecordFieldType.STRING, updatedAttributesSchema.getDataType(attributeName).get().getFieldType());
+        }
+    }
+
+    @Test
+    public void testDateAndTimestampsInferred() throws IOException {
+        final File file = new File("src/test/resources/json/prov-events.json");
+        final RecordSchema schema = inferSchema(file);
+
+        final RecordField timestampField = schema.getField("timestamp").get();
+        assertEquals(RecordFieldType.TIMESTAMP.getDataType("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"), timestampField.getDataType());
+
+        final RecordField dateField = schema.getField("eventDate").get();
+        assertEquals(RecordFieldType.DATE.getDataType("yyyy-MM-dd"), dateField.getDataType());
+
+        final RecordField timeField = schema.getField("eventTime").get();
+        assertEquals(RecordFieldType.TIME.getDataType("HH:mm:ss"), timeField.getDataType());
+
+        // TIME value and a STRING should be inferred as a STRING field
+        final RecordField maybeTimeField = schema.getField("maybeTime").get();
+        assertEquals(RecordFieldType.STRING, maybeTimeField.getDataType().getFieldType());
+
+        // DATE value and a null value should be inferred as a DATE field
+        final RecordField maybeDateField = schema.getField("maybeDate").get();
+        assertEquals(RecordFieldType.DATE.getDataType("yyyy-MM-dd"), maybeDateField.getDataType());
+    }
+
+    /**
+     * Test is intended to ensure that all inference rules that are explained in the readers' additionalDetails.html are correct
+     */
+    @Test
+    public void testDocsExample() throws IOException {
+        final File file = new File("src/test/resources/json/docs-example.json");
+        final RecordSchema schema = inferSchema(file);
+
+        assertSame(RecordFieldType.STRING, schema.getDataType("name").get().getFieldType());
+        assertSame(RecordFieldType.STRING, schema.getDataType("age").get().getFieldType());
+
+        final DataType valuesDataType = schema.getDataType("values").get();
+        assertSame(RecordFieldType.CHOICE, valuesDataType.getFieldType());
+
+        final ChoiceDataType valuesChoiceType = (ChoiceDataType) valuesDataType;
+        final List<DataType> possibleTypes = valuesChoiceType.getPossibleSubTypes();
+        assertEquals(2, possibleTypes.size());
+        assertTrue(possibleTypes.contains(RecordFieldType.STRING.getDataType()));
+        assertTrue(possibleTypes.contains(RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.STRING.getDataType())));
+
+        assertSame(RecordFieldType.STRING, schema.getDataType("nullValue").get().getFieldType());
+    }
+
+    private RecordSchema inferSchema(final File file) throws IOException {
+        try (final InputStream in = new FileInputStream(file);
+             final InputStream bufferedIn = new BufferedInputStream(in)) {
+
+            final InferSchemaAccessStrategy<?> accessStrategy = new InferSchemaAccessStrategy<>((var,content) -> new JsonRecordSource(content),
+                timestampInference, Mockito.mock(ComponentLog.class));
+
+            return accessStrategy.getSchema(null, bufferedIn, null);
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java
index d71fd32..adbdc02 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java
@@ -17,10 +17,21 @@
 
 package org.apache.nifi.json;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import org.apache.avro.Schema;
+import org.apache.nifi.avro.AvroTypeUtil;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ChoiceDataType;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.mockito.Mockito;
 
 import java.io.ByteArrayInputStream;
 import java.io.File;
@@ -37,21 +48,10 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
-import org.apache.avro.Schema;
-import org.apache.nifi.avro.AvroTypeUtil;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.serialization.MalformedRecordException;
-import org.apache.nifi.serialization.SimpleRecordSchema;
-import org.apache.nifi.serialization.record.DataType;
-import org.apache.nifi.serialization.record.Record;
-import org.apache.nifi.serialization.record.RecordField;
-import org.apache.nifi.serialization.record.RecordFieldType;
-import org.apache.nifi.serialization.record.RecordSchema;
-import org.apache.nifi.serialization.record.type.ChoiceDataType;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.mockito.Mockito;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 public class TestJsonTreeRowRecordReader {
     private final String dateFormat = RecordFieldType.DATE.getDefaultFormat();
@@ -80,6 +80,7 @@ public class TestJsonTreeRowRecordReader {
         return accountSchema;
     }
 
+
     @Test
     @Ignore("Intended only for manual testing to determine performance before/after modifications")
     public void testPerformanceOnLocalFile() throws IOException, MalformedRecordException {
@@ -347,7 +348,7 @@ public class TestJsonTreeRowRecordReader {
         try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array.json"));
             final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
 
-            final Record schemaValidatedRecord = reader.nextRecord();
+            final Record schemaValidatedRecord = reader.nextRecord(true, true);
             assertEquals(1, schemaValidatedRecord.getValue("id"));
             assertEquals("John Doe", schemaValidatedRecord.getValue("name"));
             assertNull(schemaValidatedRecord.getValue("balance"));
@@ -379,7 +380,7 @@ public class TestJsonTreeRowRecordReader {
         try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array.json"));
             final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
 
-            final Record schemaValidatedRecord = reader.nextRecord();
+            final Record schemaValidatedRecord = reader.nextRecord(true, true);
             assertEquals("1", schemaValidatedRecord.getValue("id")); // will be coerced into a STRING as per the schema
             assertEquals("John Doe", schemaValidatedRecord.getValue("name"));
             assertNull(schemaValidatedRecord.getValue("balance"));
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/schema/inference/TestVolatileSchemaCache.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/schema/inference/TestVolatileSchemaCache.java
new file mode 100644
index 0000000..c6eb9aa
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/schema/inference/TestVolatileSchemaCache.java
@@ -0,0 +1,103 @@
+/*
+ * 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.schema.inference;
+
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.util.MockConfigurationContext;
+import org.apache.nifi.util.MockControllerServiceInitializationContext;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+public class TestVolatileSchemaCache {
+
+    @Test
+    public void testEqualSchemasSameIdentifier() throws InitializationException {
+        final List<RecordField> fields = new ArrayList<>();
+        for (int i=0; i < 3; i++) {
+            fields.add(new RecordField(String.valueOf(i), RecordFieldType.STRING.getDataType()));
+        }
+
+        final ConfigurationContext configContext = new MockConfigurationContext(Collections.singletonMap(VolatileSchemaCache.MAX_SIZE, "100"), null);
+        final VolatileSchemaCache cache = new VolatileSchemaCache();
+        cache.initialize(new MockControllerServiceInitializationContext(cache, "id"));
+        cache.setup(configContext);
+
+        final String firstId = cache.cacheSchema(new SimpleRecordSchema(fields));
+        final String secondId = cache.cacheSchema(new SimpleRecordSchema(fields));
+        assertEquals(firstId, secondId);
+    }
+
+    @Test
+    public void testDifferentSchemasDifferentIdentifier() throws InitializationException {
+        final List<RecordField> stringFields = new ArrayList<>();
+        final List<RecordField> intFields = new ArrayList<>();
+
+        for (int i=0; i < 3; i++) {
+            stringFields.add(new RecordField(String.valueOf(i), RecordFieldType.STRING.getDataType()));
+            intFields.add(new RecordField(String.valueOf(i), RecordFieldType.INT.getDataType()));
+        }
+
+        final ConfigurationContext configContext = new MockConfigurationContext(Collections.singletonMap(VolatileSchemaCache.MAX_SIZE, "100"), null);
+        final VolatileSchemaCache cache = new VolatileSchemaCache();
+        cache.initialize(new MockControllerServiceInitializationContext(cache, "id"));
+        cache.setup(configContext);
+
+        final String firstId = cache.cacheSchema(new SimpleRecordSchema(stringFields));
+        final String secondId = cache.cacheSchema(new SimpleRecordSchema(intFields));
+        assertNotEquals(firstId, secondId);
+    }
+
+    @Test
+    public void testIdentifierCollission() throws InitializationException {
+        final List<RecordField> stringFields = new ArrayList<>();
+        final List<RecordField> intFields = new ArrayList<>();
+
+        for (int i=0; i < 3; i++) {
+            stringFields.add(new RecordField(String.valueOf(i), RecordFieldType.STRING.getDataType()));
+            intFields.add(new RecordField(String.valueOf(i), RecordFieldType.INT.getDataType()));
+        }
+
+        final ConfigurationContext configContext = new MockConfigurationContext(Collections.singletonMap(VolatileSchemaCache.MAX_SIZE, "100"), null);
+        final VolatileSchemaCache cache = new VolatileSchemaCache() {
+            @Override
+            protected String createIdentifier(final RecordSchema schema) {
+                return "identifier";
+            }
+        };
+
+        cache.initialize(new MockControllerServiceInitializationContext(cache, "id"));
+        cache.setup(configContext);
+
+        final String firstId = cache.cacheSchema(new SimpleRecordSchema(stringFields));
+        final String secondId = cache.cacheSchema(new SimpleRecordSchema(intFields));
+        assertNotEquals(firstId, secondId);
+
+        assertEquals(new SimpleRecordSchema(stringFields), cache.getSchema(firstId).get());
+        assertEquals(new SimpleRecordSchema(intFields), cache.getSchema(secondId).get());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestInferXmlSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestInferXmlSchema.java
new file mode 100644
index 0000000..75d6988
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestInferXmlSchema.java
@@ -0,0 +1,99 @@
+/*
+ * 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.schema.inference.InferSchemaAccessStrategy;
+import org.apache.nifi.schema.inference.SchemaInferenceEngine;
+import org.apache.nifi.schema.inference.RecordSourceFactory;
+import org.apache.nifi.schema.inference.TimeValueInference;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.xml.inference.XmlNode;
+import org.apache.nifi.xml.inference.XmlRecordSource;
+import org.apache.nifi.xml.inference.XmlSchemaInference;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collections;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+
+public class TestInferXmlSchema {
+
+    private final TimeValueInference timeValueInference = new TimeValueInference("MM/dd/yyyy", "HH:mm:ss", "MM/dd/yyyy HH:mm:ss.SSS");
+
+    @Test
+    public void testFlatXml() throws IOException {
+        final File file = new File("src/test/resources/xml/person.xml");
+        final RecordSourceFactory<XmlNode> xmlSourceFactory = (var, in) ->  new XmlRecordSource(in, false);
+        final SchemaInferenceEngine<XmlNode> schemaInference = new XmlSchemaInference(timeValueInference);
+        final InferSchemaAccessStrategy<XmlNode> inferStrategy = new InferSchemaAccessStrategy<>(xmlSourceFactory, schemaInference, Mockito.mock(ComponentLog.class));
+
+        final RecordSchema schema;
+        try (final InputStream fis = new FileInputStream(file);
+             final InputStream in = new BufferedInputStream(fis)) {
+            schema = inferStrategy.getSchema(Collections.emptyMap(), in, null);
+        }
+
+        assertSame(RecordFieldType.STRING, schema.getDataType("NAME").get().getFieldType());
+        assertSame(RecordFieldType.INT, schema.getDataType("AGE").get().getFieldType());
+        assertSame(RecordFieldType.STRING, schema.getDataType("COUNTRY").get().getFieldType());
+
+        assertEquals(RecordFieldType.DATE.getDataType(timeValueInference.getDateFormat()), schema.getDataType("DOB").get());
+        assertEquals(RecordFieldType.TIME.getDataType(timeValueInference.getTimeFormat()), schema.getDataType("TOB").get());
+        assertEquals(RecordFieldType.TIMESTAMP.getDataType(timeValueInference.getTimestampFormat()), schema.getDataType("TSOB").get());
+    }
+
+    @Test
+    public void testFieldsFromAllRecordsIncluded() throws IOException {
+        final File file = new File("src/test/resources/xml/people_nested.xml");
+        final RecordSourceFactory<XmlNode> xmlSourceFactory = (var, in) ->  new XmlRecordSource(in, true);
+        final SchemaInferenceEngine<XmlNode> schemaInference = new XmlSchemaInference(timeValueInference);
+        final InferSchemaAccessStrategy<XmlNode> inferStrategy = new InferSchemaAccessStrategy<>(xmlSourceFactory, schemaInference, Mockito.mock(ComponentLog.class));
+
+        final RecordSchema schema;
+        try (final InputStream fis = new FileInputStream(file);
+             final InputStream in = new BufferedInputStream(fis)) {
+            schema = inferStrategy.getSchema(Collections.emptyMap(), in, null);
+        }
+
+        assertSame(RecordFieldType.STRING, schema.getDataType("NAME").get().getFieldType());
+        assertSame(RecordFieldType.INT, schema.getDataType("AGE").get().getFieldType());
+        assertSame(RecordFieldType.STRING, schema.getDataType("COUNTRY").get().getFieldType());
+
+        assertEquals(RecordFieldType.DATE.getDataType(timeValueInference.getDateFormat()), schema.getDataType("DOB").get());
+        assertEquals(RecordFieldType.STRING.getDataType(), schema.getDataType("TOB").get());
+        assertEquals(RecordFieldType.TIMESTAMP.getDataType(timeValueInference.getTimestampFormat()), schema.getDataType("TSOB").get());
+
+        final DataType addressDataType = schema.getDataType("ADDRESS").get();
+        final RecordSchema addressSchema = ((RecordDataType) addressDataType).getChildSchema();
+
+        assertSame(RecordFieldType.STRING, addressSchema.getDataType("STREET").get().getFieldType());
+        assertSame(RecordFieldType.STRING, addressSchema.getDataType("CITY").get().getFieldType());
+        assertSame(RecordFieldType.STRING, addressSchema.getDataType("STATE").get().getFieldType());
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestWriteXMLResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestWriteXMLResult.java
index 37fff12..6ab9229 100755
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestWriteXMLResult.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestWriteXMLResult.java
@@ -17,7 +17,6 @@
 
 package org.apache.nifi.xml;
 
-import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.schema.access.SchemaNameAsAttribute;
 import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.DataType;
@@ -29,7 +28,6 @@ import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Mockito;
 import org.xmlunit.diff.DefaultNodeMatcher;
 import org.xmlunit.diff.ElementSelectors;
 import org.xmlunit.matchers.CompareMatcher;
@@ -50,12 +48,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.TimeZone;
 
-import static org.apache.nifi.xml.ArrayWrapping.NO_WRAPPING;
-import static org.apache.nifi.xml.ArrayWrapping.USE_PROPERTY_AS_WRAPPER;
-import static org.apache.nifi.xml.ArrayWrapping.USE_PROPERTY_FOR_ELEMENTS;
 import static org.apache.nifi.record.NullSuppression.ALWAYS_SUPPRESS;
 import static org.apache.nifi.record.NullSuppression.NEVER_SUPPRESS;
 import static org.apache.nifi.record.NullSuppression.SUPPRESS_MISSING;
+import static org.apache.nifi.xml.ArrayWrapping.NO_WRAPPING;
+import static org.apache.nifi.xml.ArrayWrapping.USE_PROPERTY_AS_WRAPPER;
+import static org.apache.nifi.xml.ArrayWrapping.USE_PROPERTY_FOR_ELEMENTS;
 import static org.apache.nifi.xml.TestWriteXMLResultUtils.DATE_FORMAT;
 import static org.apache.nifi.xml.TestWriteXMLResultUtils.SCHEMA_IDENTIFIER_RECORD;
 import static org.apache.nifi.xml.TestWriteXMLResultUtils.TIMESTAMP_FORMAT;
@@ -64,8 +62,8 @@ import static org.apache.nifi.xml.TestWriteXMLResultUtils.getEmptyNestedRecordDe
 import static org.apache.nifi.xml.TestWriteXMLResultUtils.getEmptyNestedRecordEmptyNestedSchema;
 import static org.apache.nifi.xml.TestWriteXMLResultUtils.getEmptyRecordsWithEmptySchema;
 import static org.apache.nifi.xml.TestWriteXMLResultUtils.getNestedRecords;
-import static org.apache.nifi.xml.TestWriteXMLResultUtils.getNestedRecordsWithOnlyNullValues;
 import static org.apache.nifi.xml.TestWriteXMLResultUtils.getNestedRecordsWithNullValues;
+import static org.apache.nifi.xml.TestWriteXMLResultUtils.getNestedRecordsWithOnlyNullValues;
 import static org.apache.nifi.xml.TestWriteXMLResultUtils.getRecordWithSimpleArray;
 import static org.apache.nifi.xml.TestWriteXMLResultUtils.getRecordWithSimpleMap;
 import static org.apache.nifi.xml.TestWriteXMLResultUtils.getSimpleRecords;
@@ -78,7 +76,7 @@ import static org.junit.Assert.assertThat;
 public class TestWriteXMLResult {
 
     @Test
-    public void testRecordNameIsNullSchemaIdentifierMissing() throws IOException {
+    public void testRecordNameIsNullSchemaIdentifierMissing() {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecordsWithoutIdentifierInSchema();
 
@@ -86,7 +84,7 @@ public class TestWriteXMLResult {
         final StringBuilder actualMessage = new StringBuilder();
 
         try {
-            new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+            new WriteXMLResult(recordSet.getSchema(), new SchemaNameAsAttribute(),
                     out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "root", null, "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         } catch (IOException e) {
@@ -100,7 +98,7 @@ public class TestWriteXMLResult {
     public void testRecordNameIsNullSchemaIdentifierExists() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecords();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult(recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", null, "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -117,7 +115,7 @@ public class TestWriteXMLResult {
     public void testRootNameIsNull() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecords();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult(recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, null, "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         final String expectedMessage = "The writer attempts to write multiple record although property \'Name of Root Tag\' " +
@@ -137,7 +135,7 @@ public class TestWriteXMLResult {
     public void testSingleRecord() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSingleRecord();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, null, "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -202,7 +200,7 @@ public class TestWriteXMLResult {
         final Record record = new MapRecord(schema, valueMap);
         final RecordSet rs = RecordSet.of(schema, record);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), rs.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( rs.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "RECORD", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(rs);
@@ -220,7 +218,7 @@ public class TestWriteXMLResult {
     public void testSimpleRecord() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecords();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -236,7 +234,7 @@ public class TestWriteXMLResult {
     public void testSimpleRecordWithNullValuesAlwaysSuppress() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -252,7 +250,7 @@ public class TestWriteXMLResult {
     public void testSimpleRecordWithNullValuesNeverSuppress() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -268,7 +266,7 @@ public class TestWriteXMLResult {
     public void testSimpleRecordWithNullValuesSuppressMissings() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, SUPPRESS_MISSING, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -284,7 +282,7 @@ public class TestWriteXMLResult {
     public void testEmptyRecordWithEmptySchema() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getEmptyRecordsWithEmptySchema();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -299,7 +297,7 @@ public class TestWriteXMLResult {
     public void testNestedRecord() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getNestedRecords();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -317,7 +315,7 @@ public class TestWriteXMLResult {
     public void testNestedRecordWithNullValuesAlwaysSuppress() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getNestedRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -333,7 +331,7 @@ public class TestWriteXMLResult {
     public void testNestedRecordWithNullValuesNeverSuppress() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getNestedRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -351,7 +349,7 @@ public class TestWriteXMLResult {
     public void testNestedRecordWithNullValuesSuppressMissings() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getNestedRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, SUPPRESS_MISSING, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -371,7 +369,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getNestedRecordsWithOnlyNullValues();
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -389,7 +387,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getNestedRecordsWithOnlyNullValues();
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -409,7 +407,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getEmptyNestedRecordEmptyNestedSchema();
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -427,7 +425,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getEmptyNestedRecordEmptyNestedSchema();
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -445,7 +443,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getEmptyNestedRecordDefinedSchema();
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, SUPPRESS_MISSING, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -463,7 +461,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.WITHOUT_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -483,7 +481,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.HAS_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -503,7 +501,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.HAS_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -523,7 +521,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.ONLY_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -541,7 +539,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.ONLY_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, USE_PROPERTY_AS_WRAPPER, "ARRAY", "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -559,7 +557,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.EMPTY);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -577,7 +575,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.EMPTY);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, USE_PROPERTY_AS_WRAPPER, "ARRAY", "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -595,7 +593,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.WITHOUT_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, USE_PROPERTY_AS_WRAPPER, "ARRAY", "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -615,7 +613,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.WITHOUT_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, USE_PROPERTY_FOR_ELEMENTS, "ELEM", "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -635,7 +633,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.WITHOUT_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -655,7 +653,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.HAS_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -675,7 +673,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.ONLY_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -693,7 +691,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.EMPTY);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -713,7 +711,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.HAS_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -733,7 +731,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.EMPTY);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -749,7 +747,7 @@ public class TestWriteXMLResult {
     public void testChoice() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecordsWithChoice();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -775,7 +773,7 @@ public class TestWriteXMLResult {
     public void testWriteWithoutSchemaSimpleRecord() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecords();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -799,7 +797,7 @@ public class TestWriteXMLResult {
     public void testWriteWithoutSchemaSimpleRecordWithNullValuesAlwaysSuppress() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -823,7 +821,7 @@ public class TestWriteXMLResult {
     public void testWriteWithoutSchemaSimpleRecordWithNullValuesNeverSuppress() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -847,7 +845,7 @@ public class TestWriteXMLResult {
     public void testWriteWithoutSchemaSimpleRecordWithNullValuesSuppressMissings() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getSimpleRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, SUPPRESS_MISSING, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -871,7 +869,7 @@ public class TestWriteXMLResult {
     public void testWriteWithoutSchemaNestedRecord() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getNestedRecords();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -897,7 +895,7 @@ public class TestWriteXMLResult {
     public void testWriteWithoutSchemaNestedRecordWithNullValuesAlwaysSuppress() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getNestedRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -923,7 +921,7 @@ public class TestWriteXMLResult {
     public void testWriteWithoutSchemaNestedRecordWithNullValuesNeverSuppress() throws IOException {
         OutputStream out = new ByteArrayOutputStream();
         RecordSet recordSet = getNestedRecordsWithNullValues();
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -951,7 +949,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getNestedRecordsWithOnlyNullValues();
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -977,7 +975,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getNestedRecordsWithOnlyNullValues();
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1005,7 +1003,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.WITHOUT_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1033,7 +1031,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.HAS_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1061,7 +1059,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.HAS_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1089,7 +1087,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.ONLY_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1115,7 +1113,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.ONLY_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, USE_PROPERTY_AS_WRAPPER, "ARRAY", "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1141,7 +1139,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.EMPTY);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1167,7 +1165,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.EMPTY);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, NEVER_SUPPRESS, USE_PROPERTY_AS_WRAPPER, "ARRAY", "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1193,7 +1191,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.WITHOUT_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, USE_PROPERTY_AS_WRAPPER, "ARRAY", "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1221,7 +1219,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleArray(TestWriteXMLResultUtils.NullValues.WITHOUT_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, USE_PROPERTY_FOR_ELEMENTS, "ELEM", "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1249,7 +1247,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.WITHOUT_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1277,7 +1275,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.HAS_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.write(recordSet);
@@ -1297,7 +1295,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.ONLY_NULL);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1323,7 +1321,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.EMPTY);
 
-        WriteXMLResult writer = new WriteXMLResult(Mockito.mock(ComponentLog.class), recordSet.getSchema(), new SchemaNameAsAttribute(),
+        WriteXMLResult writer = new WriteXMLResult( recordSet.getSchema(), new SchemaNameAsAttribute(),
                 out, true, ALWAYS_SUPPRESS, NO_WRAPPING, null, "ROOT", "PERSON", "UTF-8", DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
 
         writer.onBeginRecordSet();
@@ -1349,7 +1347,7 @@ public class TestWriteXMLResult {
 
         RecordSet recordSet = getRecordWithSimpleMap(TestWriteXMLResultUtils.NullValues.HAS_NULL);
... 199 lines suppressed ...