You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2017/04/24 21:03:47 UTC

[1/7] nifi git commit: NIFI-3682: This closes #1682. Add Schema Access Strategy and Schema Write Strategy Record Readers and Writers; bug fixes.

Repository: nifi
Updated Branches:
  refs/heads/master 57ccf97c5 -> 50ea1083e


http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithEmbeddedSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithEmbeddedSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithEmbeddedSchema.java
new file mode 100644
index 0000000..da9f70b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithEmbeddedSchema.java
@@ -0,0 +1,290 @@
+/*
+ * 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.avro;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.Schema.Type;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+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.junit.Test;
+
+public class TestAvroReaderWithEmbeddedSchema {
+
+
+    @Test
+    public void testLogicalTypes() throws IOException, ParseException, MalformedRecordException, SchemaNotFoundException {
+        final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/logical-types.avsc"));
+
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        final String expectedTime = "2017-04-04 14:20:33.000";
+        final DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
+        df.setTimeZone(TimeZone.getTimeZone("gmt"));
+        final long timeLong = df.parse(expectedTime).getTime();
+
+        final long secondsSinceMidnight = 33 + (20 * 60) + (14 * 60 * 60);
+        final long millisSinceMidnight = secondsSinceMidnight * 1000L;
+
+
+        final byte[] serialized;
+        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter);
+            final DataFileWriter<GenericRecord> writer = dataFileWriter.create(schema, baos)) {
+
+            final GenericRecord record = new GenericData.Record(schema);
+            record.put("timeMillis", millisSinceMidnight);
+            record.put("timeMicros", millisSinceMidnight * 1000L);
+            record.put("timestampMillis", timeLong);
+            record.put("timestampMicros", timeLong * 1000L);
+            record.put("date", 17260);
+
+            writer.append(record);
+            writer.flush();
+
+            serialized = baos.toByteArray();
+        }
+
+        try (final InputStream in = new ByteArrayInputStream(serialized)) {
+            final AvroRecordReader reader = new AvroReaderWithEmbeddedSchema(in);
+            final RecordSchema recordSchema = reader.getSchema();
+
+            assertEquals(RecordFieldType.TIME, recordSchema.getDataType("timeMillis").get().getFieldType());
+            assertEquals(RecordFieldType.TIME, recordSchema.getDataType("timeMicros").get().getFieldType());
+            assertEquals(RecordFieldType.TIMESTAMP, recordSchema.getDataType("timestampMillis").get().getFieldType());
+            assertEquals(RecordFieldType.TIMESTAMP, recordSchema.getDataType("timestampMicros").get().getFieldType());
+            assertEquals(RecordFieldType.DATE, recordSchema.getDataType("date").get().getFieldType());
+
+            final Record record = reader.nextRecord();
+            assertEquals(new java.sql.Time(millisSinceMidnight), record.getValue("timeMillis"));
+            assertEquals(new java.sql.Time(millisSinceMidnight), record.getValue("timeMicros"));
+            assertEquals(new java.sql.Timestamp(timeLong), record.getValue("timestampMillis"));
+            assertEquals(new java.sql.Timestamp(timeLong), record.getValue("timestampMicros"));
+            final DateFormat noTimeOfDayDateFormat = new SimpleDateFormat("yyyy-MM-dd");
+            noTimeOfDayDateFormat.setTimeZone(TimeZone.getTimeZone("gmt"));
+            assertEquals(new java.sql.Date(timeLong).toString(), noTimeOfDayDateFormat.format(record.getValue("date")));
+        }
+    }
+
+    @Test
+    public void testDataTypes() throws IOException, MalformedRecordException, SchemaNotFoundException {
+        final List<Field> accountFields = new ArrayList<>();
+        accountFields.add(new Field("accountId", Schema.create(Type.LONG), null, (Object) null));
+        accountFields.add(new Field("accountName", Schema.create(Type.STRING), null, (Object) null));
+        final Schema accountSchema = Schema.createRecord("account", null, null, false);
+        accountSchema.setFields(accountFields);
+
+        final List<Field> catFields = new ArrayList<>();
+        catFields.add(new Field("catTailLength", Schema.create(Type.INT), null, (Object) null));
+        catFields.add(new Field("catName", Schema.create(Type.STRING), null, (Object) null));
+        final Schema catSchema = Schema.createRecord("cat", null, null, false);
+        catSchema.setFields(catFields);
+
+        final List<Field> dogFields = new ArrayList<>();
+        dogFields.add(new Field("dogTailLength", Schema.create(Type.INT), null, (Object) null));
+        dogFields.add(new Field("dogName", Schema.create(Type.STRING), null, (Object) null));
+        final Schema dogSchema = Schema.createRecord("dog", null, null, false);
+        dogSchema.setFields(dogFields);
+
+        final List<Field> fields = new ArrayList<>();
+        fields.add(new Field("name", Schema.create(Type.STRING), null, (Object) null));
+        fields.add(new Field("age", Schema.create(Type.INT), null, (Object) null));
+        fields.add(new Field("balance", Schema.create(Type.DOUBLE), null, (Object) null));
+        fields.add(new Field("rate", Schema.create(Type.FLOAT), null, (Object) null));
+        fields.add(new Field("debt", Schema.create(Type.BOOLEAN), null, (Object) null));
+        fields.add(new Field("nickname", Schema.create(Type.NULL), null, (Object) null));
+        fields.add(new Field("binary", Schema.create(Type.BYTES), null, (Object) null));
+        fields.add(new Field("fixed", Schema.createFixed("fixed", null, null, 5), null, (Object) null));
+        fields.add(new Field("map", Schema.createMap(Schema.create(Type.STRING)), null, (Object) null));
+        fields.add(new Field("array", Schema.createArray(Schema.create(Type.LONG)), null, (Object) null));
+        fields.add(new Field("account", accountSchema, null, (Object) null));
+        fields.add(new Field("desiredbalance", Schema.createUnion( // test union of NULL and other type with no value
+            Arrays.asList(Schema.create(Type.NULL), Schema.create(Type.DOUBLE))),
+            null, (Object) null));
+        fields.add(new Field("dreambalance", Schema.createUnion( // test union of NULL and other type with a value
+            Arrays.asList(Schema.create(Type.NULL), Schema.create(Type.DOUBLE))),
+            null, (Object) null));
+        fields.add(new Field("favAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, (Object) null));
+        fields.add(new Field("otherFavAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, (Object) null));
+
+        final Schema schema = Schema.createRecord("record", null, null, false);
+        schema.setFields(fields);
+
+        final byte[] source;
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        final Map<String, String> map = new HashMap<>();
+        map.put("greeting", "hello");
+        map.put("salutation", "good-bye");
+
+        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter);
+            final DataFileWriter<GenericRecord> writer = dataFileWriter.create(schema, baos)) {
+
+            final GenericRecord record = new GenericData.Record(schema);
+            record.put("name", "John");
+            record.put("age", 33);
+            record.put("balance", 1234.56D);
+            record.put("rate", 0.045F);
+            record.put("debt", false);
+            record.put("binary", ByteBuffer.wrap("binary".getBytes(StandardCharsets.UTF_8)));
+            record.put("fixed", new GenericData.Fixed(Schema.create(Type.BYTES), "fixed".getBytes(StandardCharsets.UTF_8)));
+            record.put("map", map);
+            record.put("array", Arrays.asList(1L, 2L));
+            record.put("dreambalance", 10_000_000.00D);
+
+            final GenericRecord accountRecord = new GenericData.Record(accountSchema);
+            accountRecord.put("accountId", 83L);
+            accountRecord.put("accountName", "Checking");
+            record.put("account", accountRecord);
+
+            final GenericRecord catRecord = new GenericData.Record(catSchema);
+            catRecord.put("catTailLength", 1);
+            catRecord.put("catName", "Meow");
+            record.put("otherFavAnimal", catRecord);
+
+            final GenericRecord dogRecord = new GenericData.Record(dogSchema);
+            dogRecord.put("dogTailLength", 14);
+            dogRecord.put("dogName", "Fido");
+            record.put("favAnimal", dogRecord);
+
+            writer.append(record);
+        }
+
+        source = baos.toByteArray();
+
+        try (final InputStream in = new ByteArrayInputStream(source)) {
+            final AvroRecordReader reader = new AvroReaderWithEmbeddedSchema(in);
+            final RecordSchema recordSchema = reader.getSchema();
+            assertEquals(15, recordSchema.getFieldCount());
+
+            assertEquals(RecordFieldType.STRING, recordSchema.getDataType("name").get().getFieldType());
+            assertEquals(RecordFieldType.INT, recordSchema.getDataType("age").get().getFieldType());
+            assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("balance").get().getFieldType());
+            assertEquals(RecordFieldType.FLOAT, recordSchema.getDataType("rate").get().getFieldType());
+            assertEquals(RecordFieldType.BOOLEAN, recordSchema.getDataType("debt").get().getFieldType());
+            assertEquals(RecordFieldType.STRING, recordSchema.getDataType("nickname").get().getFieldType());
+            assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("binary").get().getFieldType());
+            assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("fixed").get().getFieldType());
+            assertEquals(RecordFieldType.MAP, recordSchema.getDataType("map").get().getFieldType());
+            assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("array").get().getFieldType());
+            assertEquals(RecordFieldType.RECORD, recordSchema.getDataType("account").get().getFieldType());
+            assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("desiredbalance").get().getFieldType());
+            assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("dreambalance").get().getFieldType());
+            assertEquals(RecordFieldType.CHOICE, recordSchema.getDataType("favAnimal").get().getFieldType());
+            assertEquals(RecordFieldType.CHOICE, recordSchema.getDataType("otherFavAnimal").get().getFieldType());
+
+            final Object[] values = reader.nextRecord().getValues();
+            assertEquals(15, values.length);
+            assertEquals("John", values[0]);
+            assertEquals(33, values[1]);
+            assertEquals(1234.56D, values[2]);
+            assertEquals(0.045F, values[3]);
+            assertEquals(false, values[4]);
+            assertEquals(null, values[5]);
+            assertArrayEquals(toObjectArray("binary".getBytes(StandardCharsets.UTF_8)), (Object[]) values[6]);
+            assertArrayEquals(toObjectArray("fixed".getBytes(StandardCharsets.UTF_8)), (Object[]) values[7]);
+            assertEquals(map, values[8]);
+            assertArrayEquals(new Object[] {1L, 2L}, (Object[]) values[9]);
+
+            final Map<String, Object> accountValues = new HashMap<>();
+            accountValues.put("accountName", "Checking");
+            accountValues.put("accountId", 83L);
+
+            final List<RecordField> accountRecordFields = new ArrayList<>();
+            accountRecordFields.add(new RecordField("accountId", RecordFieldType.LONG.getDataType()));
+            accountRecordFields.add(new RecordField("accountName", RecordFieldType.STRING.getDataType()));
+
+            final RecordSchema accountRecordSchema = new SimpleRecordSchema(accountRecordFields);
+            final Record mapRecord = new MapRecord(accountRecordSchema, accountValues);
+
+            assertEquals(mapRecord, values[10]);
+
+            assertNull(values[11]);
+            assertEquals(10_000_000.0D, values[12]);
+
+            final Map<String, Object> dogMap = new HashMap<>();
+            dogMap.put("dogName", "Fido");
+            dogMap.put("dogTailLength", 14);
+
+            final List<RecordField> dogRecordFields = new ArrayList<>();
+            dogRecordFields.add(new RecordField("dogTailLength", RecordFieldType.INT.getDataType()));
+            dogRecordFields.add(new RecordField("dogName", RecordFieldType.STRING.getDataType()));
+            final RecordSchema dogRecordSchema = new SimpleRecordSchema(dogRecordFields);
+            final Record dogRecord = new MapRecord(dogRecordSchema, dogMap);
+
+            assertEquals(dogRecord, values[13]);
+
+            final Map<String, Object> catMap = new HashMap<>();
+            catMap.put("catName", "Meow");
+            catMap.put("catTailLength", 1);
+
+            final List<RecordField> catRecordFields = new ArrayList<>();
+            catRecordFields.add(new RecordField("catTailLength", RecordFieldType.INT.getDataType()));
+            catRecordFields.add(new RecordField("catName", RecordFieldType.STRING.getDataType()));
+            final RecordSchema catRecordSchema = new SimpleRecordSchema(catRecordFields);
+            final Record catRecord = new MapRecord(catRecordSchema, catMap);
+
+            assertEquals(catRecord, values[14]);
+        }
+    }
+
+    private Object[] toObjectArray(final byte[] bytes) {
+        final Object[] array = new Object[bytes.length];
+        for (int i = 0; i < bytes.length; i++) {
+            array[i] = Byte.valueOf(bytes[i]);
+        }
+        return array;
+    }
+
+    public static enum Status {
+        GOOD, BAD;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java
deleted file mode 100644
index 56e2e3d..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.nifi.avro;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TimeZone;
-
-import org.apache.avro.Schema;
-import org.apache.avro.Schema.Field;
-import org.apache.avro.Schema.Type;
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumWriter;
-import org.apache.nifi.serialization.MalformedRecordException;
-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.junit.Test;
-
-public class TestAvroRecordReader {
-
-
-    @Test
-    public void testLogicalTypes() throws IOException, ParseException, MalformedRecordException {
-        final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/logical-types.avsc"));
-
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        final String expectedTime = "2017-04-04 14:20:33.000";
-        final DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
-        df.setTimeZone(TimeZone.getTimeZone("gmt"));
-        final long timeLong = df.parse(expectedTime).getTime();
-
-        final long secondsSinceMidnight = 33 + (20 * 60) + (14 * 60 * 60);
-        final long millisSinceMidnight = secondsSinceMidnight * 1000L;
-
-
-        final byte[] serialized;
-        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
-        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter);
-            final DataFileWriter<GenericRecord> writer = dataFileWriter.create(schema, baos)) {
-
-            final GenericRecord record = new GenericData.Record(schema);
-            record.put("timeMillis", millisSinceMidnight);
-            record.put("timeMicros", millisSinceMidnight * 1000L);
-            record.put("timestampMillis", timeLong);
-            record.put("timestampMicros", timeLong * 1000L);
-            record.put("date", 17260);
-
-            writer.append(record);
-            writer.flush();
-
-            serialized = baos.toByteArray();
-        }
-
-        try (final InputStream in = new ByteArrayInputStream(serialized)) {
-            final AvroRecordReader reader = new AvroRecordReader(in);
-            final RecordSchema recordSchema = reader.getSchema();
-
-            assertEquals(RecordFieldType.TIME, recordSchema.getDataType("timeMillis").get().getFieldType());
-            assertEquals(RecordFieldType.TIME, recordSchema.getDataType("timeMicros").get().getFieldType());
-            assertEquals(RecordFieldType.TIMESTAMP, recordSchema.getDataType("timestampMillis").get().getFieldType());
-            assertEquals(RecordFieldType.TIMESTAMP, recordSchema.getDataType("timestampMicros").get().getFieldType());
-            assertEquals(RecordFieldType.DATE, recordSchema.getDataType("date").get().getFieldType());
-
-            final Record record = reader.nextRecord();
-            assertEquals(new java.sql.Time(millisSinceMidnight), record.getValue("timeMillis"));
-            assertEquals(new java.sql.Time(millisSinceMidnight), record.getValue("timeMicros"));
-            assertEquals(new java.sql.Timestamp(timeLong), record.getValue("timestampMillis"));
-            assertEquals(new java.sql.Timestamp(timeLong), record.getValue("timestampMicros"));
-            final DateFormat noTimeOfDayDateFormat = new SimpleDateFormat("yyyy-MM-dd");
-            noTimeOfDayDateFormat.setTimeZone(TimeZone.getTimeZone("gmt"));
-            assertEquals(new java.sql.Date(timeLong).toString(), noTimeOfDayDateFormat.format(record.getValue("date")));
-        }
-    }
-
-    @Test
-    @SuppressWarnings({"unchecked", "rawtypes"})
-    public void testDataTypes() throws IOException, MalformedRecordException {
-        final List<Field> accountFields = new ArrayList<>();
-        accountFields.add(new Field("accountId", Schema.create(Type.LONG), null, (Object) null));
-        accountFields.add(new Field("accountName", Schema.create(Type.STRING), null, (Object) null));
-        final Schema accountSchema = Schema.createRecord("account", null, null, false);
-        accountSchema.setFields(accountFields);
-
-        final List<Field> catFields = new ArrayList<>();
-        catFields.add(new Field("catTailLength", Schema.create(Type.INT), null, (Object) null));
-        catFields.add(new Field("catName", Schema.create(Type.STRING), null, (Object) null));
-        final Schema catSchema = Schema.createRecord("cat", null, null, false);
-        catSchema.setFields(catFields);
-
-        final List<Field> dogFields = new ArrayList<>();
-        dogFields.add(new Field("dogTailLength", Schema.create(Type.INT), null, (Object) null));
-        dogFields.add(new Field("dogName", Schema.create(Type.STRING), null, (Object) null));
-        final Schema dogSchema = Schema.createRecord("dog", null, null, false);
-        dogSchema.setFields(dogFields);
-
-        final List<Field> fields = new ArrayList<>();
-        fields.add(new Field("name", Schema.create(Type.STRING), null, (Object) null));
-        fields.add(new Field("age", Schema.create(Type.INT), null, (Object) null));
-        fields.add(new Field("balance", Schema.create(Type.DOUBLE), null, (Object) null));
-        fields.add(new Field("rate", Schema.create(Type.FLOAT), null, (Object) null));
-        fields.add(new Field("debt", Schema.create(Type.BOOLEAN), null, (Object) null));
-        fields.add(new Field("nickname", Schema.create(Type.NULL), null, (Object) null));
-        fields.add(new Field("binary", Schema.create(Type.BYTES), null, (Object) null));
-        fields.add(new Field("fixed", Schema.createFixed("fixed", null, null, 5), null, (Object) null));
-        fields.add(new Field("map", Schema.createMap(Schema.create(Type.STRING)), null, (Object) null));
-        fields.add(new Field("array", Schema.createArray(Schema.create(Type.LONG)), null, (Object) null));
-        fields.add(new Field("account", accountSchema, null, (Object) null));
-        fields.add(new Field("desiredbalance", Schema.createUnion( // test union of NULL and other type with no value
-            Arrays.asList(Schema.create(Type.NULL), Schema.create(Type.DOUBLE))),
-            null, (Object) null));
-        fields.add(new Field("dreambalance", Schema.createUnion( // test union of NULL and other type with a value
-            Arrays.asList(Schema.create(Type.NULL), Schema.create(Type.DOUBLE))),
-            null, (Object) null));
-        fields.add(new Field("favAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, (Object) null));
-        fields.add(new Field("otherFavAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, (Object) null));
-
-        final Schema schema = Schema.createRecord("record", null, null, false);
-        schema.setFields(fields);
-
-        final byte[] source;
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        final Map<String, String> map = new HashMap<>();
-        map.put("greeting", "hello");
-        map.put("salutation", "good-bye");
-
-        final List<RecordField> mapFields = new ArrayList<>();
-        mapFields.add(new RecordField("greeting", RecordFieldType.STRING.getDataType()));
-        mapFields.add(new RecordField("salutation", RecordFieldType.STRING.getDataType()));
-        final RecordSchema mapSchema = new SimpleRecordSchema(mapFields);
-        final Record expectedRecord = new MapRecord(mapSchema, (Map) map);
-
-        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
-        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter);
-            final DataFileWriter<GenericRecord> writer = dataFileWriter.create(schema, baos)) {
-
-            final GenericRecord record = new GenericData.Record(schema);
-            record.put("name", "John");
-            record.put("age", 33);
-            record.put("balance", 1234.56D);
-            record.put("rate", 0.045F);
-            record.put("debt", false);
-            record.put("binary", ByteBuffer.wrap("binary".getBytes(StandardCharsets.UTF_8)));
-            record.put("fixed", new GenericData.Fixed(Schema.create(Type.BYTES), "fixed".getBytes(StandardCharsets.UTF_8)));
-            record.put("map", map);
-            record.put("array", Arrays.asList(1L, 2L));
-            record.put("dreambalance", 10_000_000.00D);
-
-            final GenericRecord accountRecord = new GenericData.Record(accountSchema);
-            accountRecord.put("accountId", 83L);
-            accountRecord.put("accountName", "Checking");
-            record.put("account", accountRecord);
-
-            final GenericRecord catRecord = new GenericData.Record(catSchema);
-            catRecord.put("catTailLength", 1);
-            catRecord.put("catName", "Meow");
-            record.put("otherFavAnimal", catRecord);
-
-            final GenericRecord dogRecord = new GenericData.Record(dogSchema);
-            dogRecord.put("dogTailLength", 14);
-            dogRecord.put("dogName", "Fido");
-            record.put("favAnimal", dogRecord);
-
-            writer.append(record);
-        }
-
-        source = baos.toByteArray();
-
-        try (final InputStream in = new ByteArrayInputStream(source)) {
-            final AvroRecordReader reader = new AvroRecordReader(in);
-            final RecordSchema recordSchema = reader.getSchema();
-            assertEquals(15, recordSchema.getFieldCount());
-
-            assertEquals(RecordFieldType.STRING, recordSchema.getDataType("name").get().getFieldType());
-            assertEquals(RecordFieldType.INT, recordSchema.getDataType("age").get().getFieldType());
-            assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("balance").get().getFieldType());
-            assertEquals(RecordFieldType.FLOAT, recordSchema.getDataType("rate").get().getFieldType());
-            assertEquals(RecordFieldType.BOOLEAN, recordSchema.getDataType("debt").get().getFieldType());
-            assertEquals(RecordFieldType.RECORD, recordSchema.getDataType("nickname").get().getFieldType());
-            assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("binary").get().getFieldType());
-            assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("fixed").get().getFieldType());
-            assertEquals(RecordFieldType.RECORD, recordSchema.getDataType("map").get().getFieldType());
-            assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("array").get().getFieldType());
-            assertEquals(RecordFieldType.RECORD, recordSchema.getDataType("account").get().getFieldType());
-            assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("desiredbalance").get().getFieldType());
-            assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("dreambalance").get().getFieldType());
-            assertEquals(RecordFieldType.CHOICE, recordSchema.getDataType("favAnimal").get().getFieldType());
-            assertEquals(RecordFieldType.CHOICE, recordSchema.getDataType("otherFavAnimal").get().getFieldType());
-
-            final Object[] values = reader.nextRecord().getValues();
-            assertEquals(15, values.length);
-            assertEquals("John", values[0]);
-            assertEquals(33, values[1]);
-            assertEquals(1234.56D, values[2]);
-            assertEquals(0.045F, values[3]);
-            assertEquals(false, values[4]);
-            assertEquals(null, values[5]);
-            assertArrayEquals(toObjectArray("binary".getBytes(StandardCharsets.UTF_8)), (Object[]) values[6]);
-            assertArrayEquals(toObjectArray("fixed".getBytes(StandardCharsets.UTF_8)), (Object[]) values[7]);
-            assertEquals(expectedRecord, values[8]);
-            assertArrayEquals(new Object[] {1L, 2L}, (Object[]) values[9]);
-
-            final Map<String, Object> accountValues = new HashMap<>();
-            accountValues.put("accountName", "Checking");
-            accountValues.put("accountId", 83L);
-
-            final List<RecordField> accountRecordFields = new ArrayList<>();
-            accountRecordFields.add(new RecordField("accountId", RecordFieldType.LONG.getDataType()));
-            accountRecordFields.add(new RecordField("accountName", RecordFieldType.STRING.getDataType()));
-
-            final RecordSchema accountRecordSchema = new SimpleRecordSchema(accountRecordFields);
-            final Record mapRecord = new MapRecord(accountRecordSchema, accountValues);
-
-            assertEquals(mapRecord, values[10]);
-
-            assertNull(values[11]);
-            assertEquals(10_000_000.0D, values[12]);
-
-            final Map<String, Object> dogMap = new HashMap<>();
-            dogMap.put("dogName", "Fido");
-            dogMap.put("dogTailLength", 14);
-
-            final List<RecordField> dogRecordFields = new ArrayList<>();
-            dogRecordFields.add(new RecordField("dogTailLength", RecordFieldType.INT.getDataType()));
-            dogRecordFields.add(new RecordField("dogName", RecordFieldType.STRING.getDataType()));
-            final RecordSchema dogRecordSchema = new SimpleRecordSchema(dogRecordFields);
-            final Record dogRecord = new MapRecord(dogRecordSchema, dogMap);
-
-            assertEquals(dogRecord, values[13]);
-
-            final Map<String, Object> catMap = new HashMap<>();
-            catMap.put("catName", "Meow");
-            catMap.put("catTailLength", 1);
-
-            final List<RecordField> catRecordFields = new ArrayList<>();
-            catRecordFields.add(new RecordField("catTailLength", RecordFieldType.INT.getDataType()));
-            catRecordFields.add(new RecordField("catName", RecordFieldType.STRING.getDataType()));
-            final RecordSchema catRecordSchema = new SimpleRecordSchema(catRecordFields);
-            final Record catRecord = new MapRecord(catRecordSchema, catMap);
-
-            assertEquals(catRecord, values[14]);
-        }
-    }
-
-    private Object[] toObjectArray(final byte[] bytes) {
-        final Object[] array = new Object[bytes.length];
-        for (int i = 0; i < bytes.length; i++) {
-            array[i] = Byte.valueOf(bytes[i]);
-        }
-        return array;
-    }
-
-    public static enum Status {
-        GOOD, BAD;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java
index 2102813..409ede2 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java
@@ -41,13 +41,10 @@ import java.util.Map;
 import java.util.TimeZone;
 
 import org.apache.avro.Schema;
-import org.apache.avro.file.DataFileStream;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.GenericData.Array;
-import org.apache.avro.generic.GenericData.StringType;
+import org.apache.avro.generic.GenericRecord;
 import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.WriteResult;
 import org.apache.nifi.serialization.record.MapRecord;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordField;
@@ -56,12 +53,19 @@ import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 import org.junit.Test;
 
-public class TestWriteAvroResult {
+public abstract class TestWriteAvroResult {
+
+    protected abstract WriteAvroResult createWriter(Schema schema);
+
+    protected abstract GenericRecord readRecord(InputStream in, Schema schema) throws IOException;
+
+    protected void verify(final WriteResult writeResult) {
+    }
 
     @Test
     public void testLogicalTypes() throws IOException, ParseException {
         final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/logical-types.avsc"));
-        final WriteAvroResult writer = new WriteAvroResult(schema);
+        final WriteAvroResult writer = createWriter(schema);
 
         final List<RecordField> fields = new ArrayList<>();
         fields.add(new RecordField("timeMillis", RecordFieldType.TIME.getDataType()));
@@ -91,11 +95,7 @@ public class TestWriteAvroResult {
         }
 
         try (final InputStream in = new ByteArrayInputStream(data)) {
-            final DataFileStream<GenericRecord> dataFileStream = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>());
-            final Schema avroSchema = dataFileStream.getSchema();
-            GenericData.setStringType(avroSchema, StringType.String);
-
-            final GenericRecord avroRecord = dataFileStream.next();
+            final GenericRecord avroRecord = readRecord(in, schema);
             final long secondsSinceMidnight = 33 + (20 * 60) + (14 * 60 * 60);
             final long millisSinceMidnight = secondsSinceMidnight * 1000L;
 
@@ -110,9 +110,8 @@ public class TestWriteAvroResult {
 
     @Test
     public void testDataTypes() throws IOException {
-        // TODO: Test Enums
         final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/datatypes.avsc"));
-        final WriteAvroResult writer = new WriteAvroResult(schema);
+        final WriteAvroResult writer = createWriter(schema);
 
         final List<RecordField> subRecordFields = Collections.singletonList(new RecordField("field1", RecordFieldType.STRING.getDataType()));
         final RecordSchema subRecordSchema = new SimpleRecordSchema(subRecordFields);
@@ -124,7 +123,7 @@ public class TestWriteAvroResult {
         fields.add(new RecordField("double", RecordFieldType.DOUBLE.getDataType()));
         fields.add(new RecordField("float", RecordFieldType.FLOAT.getDataType()));
         fields.add(new RecordField("boolean", RecordFieldType.BOOLEAN.getDataType()));
-        fields.add(new RecordField("bytes", RecordFieldType.ARRAY.getChoiceDataType(Collections.singletonList(RecordFieldType.BYTE.getDataType()))));
+        fields.add(new RecordField("bytes", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType())));
         fields.add(new RecordField("nullOrLong", RecordFieldType.LONG.getDataType()));
         fields.add(new RecordField("array", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.INT.getDataType())));
         fields.add(new RecordField("record", RecordFieldType.RECORD.getRecordDataType(subRecordSchema)));
@@ -148,21 +147,18 @@ public class TestWriteAvroResult {
 
         final byte[] data;
         try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
-            writer.write(RecordSet.of(record.getSchema(), record), baos);
+            final WriteResult writeResult = writer.write(RecordSet.of(record.getSchema(), record), baos);
+            verify(writeResult);
             data = baos.toByteArray();
         }
 
         try (final InputStream in = new ByteArrayInputStream(data)) {
-            final DataFileStream<GenericRecord> dataFileStream = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>());
-            final Schema avroSchema = dataFileStream.getSchema();
-            GenericData.setStringType(avroSchema, StringType.String);
-
-            final GenericRecord avroRecord = dataFileStream.next();
+            final GenericRecord avroRecord = readRecord(in, schema);
             assertMatch(record, avroRecord);
         }
     }
 
-    private void assertMatch(final Record record, final GenericRecord avroRecord) {
+    protected void assertMatch(final Record record, final GenericRecord avroRecord) {
         for (final String fieldName : record.getSchema().getFieldNames()) {
             Object avroValue = avroRecord.get(fieldName);
             final Object recordValue = record.getValue(fieldName);

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithSchema.java
new file mode 100644
index 0000000..6ace012
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithSchema.java
@@ -0,0 +1,46 @@
+/*
+ * 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.avro;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericData.StringType;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+
+public class TestWriteAvroResultWithSchema extends TestWriteAvroResult {
+
+    @Override
+    protected WriteAvroResult createWriter(final Schema schema) {
+        return new WriteAvroResultWithSchema(schema);
+    }
+
+    @Override
+    protected GenericRecord readRecord(final InputStream in, final Schema schema) throws IOException {
+        final DataFileStream<GenericRecord> dataFileStream = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>());
+        final Schema avroSchema = dataFileStream.getSchema();
+        GenericData.setStringType(avroSchema, StringType.String);
+        final GenericRecord avroRecord = dataFileStream.next();
+
+        return avroRecord;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithoutSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithoutSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithoutSchema.java
new file mode 100644
index 0000000..d40bb55
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithoutSchema.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.avro;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.nifi.schema.access.SchemaTextAsAttribute;
+import org.apache.nifi.serialization.WriteResult;
+import org.junit.Assert;
+
+public class TestWriteAvroResultWithoutSchema extends TestWriteAvroResult {
+
+    @Override
+    protected WriteAvroResult createWriter(final Schema schema) {
+        return new WriteAvroResultWithExternalSchema(schema, AvroTypeUtil.createSchema(schema), new SchemaTextAsAttribute());
+    }
+
+    @Override
+    protected GenericRecord readRecord(final InputStream in, final Schema schema) throws IOException {
+        final BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(in, null);
+        final GenericDatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema);
+        return reader.read(null, decoder);
+    }
+
+    @Override
+    protected void verify(final WriteResult writeResult) {
+        final Map<String, String> attributes = writeResult.getAttributes();
+
+        final String schemaText = attributes.get("avro.schema");
+        Assert.assertNotNull(schemaText);
+        new Schema.Parser().parse(schemaText);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVHeaderSchemaStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVHeaderSchemaStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVHeaderSchemaStrategy.java
new file mode 100644
index 0000000..3eed784
--- /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/TestCSVHeaderSchemaStrategy.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.csv;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.util.MockConfigurationContext;
+import org.junit.Test;
+
+public class TestCSVHeaderSchemaStrategy {
+
+    @Test
+    public void testSimple() throws SchemaNotFoundException, IOException {
+        final CSVHeaderSchemaStrategy strategy = new CSVHeaderSchemaStrategy();
+        final String headerLine = "a, b, c, d, e\\,z, f";
+        final byte[] headerBytes = headerLine.getBytes();
+
+        final Map<PropertyDescriptor, String> properties = new HashMap<>();
+        properties.put(CSVUtils.CSV_FORMAT, CSVUtils.CUSTOM.getValue());
+        properties.put(CSVUtils.COMMENT_MARKER, "#");
+        properties.put(CSVUtils.VALUE_SEPARATOR, ",");
+        properties.put(CSVUtils.TRIM_FIELDS, "true");
+        properties.put(CSVUtils.QUOTE_CHAR, "\"");
+        properties.put(CSVUtils.ESCAPE_CHAR, "\\");
+
+        final ConfigurationContext context = new MockConfigurationContext(properties, null);
+
+        final RecordSchema schema;
+        try (final InputStream bais = new ByteArrayInputStream(headerBytes)) {
+            schema = strategy.getSchema(null, bais, context);
+        }
+
+        final List<String> expectedFieldNames = Arrays.asList("a", "b", "c", "d", "e,z", "f");
+        assertEquals(expectedFieldNames, schema.getFieldNames());
+
+        assertTrue(schema.getFields().stream()
+            .allMatch(field -> field.getDataType().equals(RecordFieldType.STRING.getDataType())));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/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 1e8997b..9424e79 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
@@ -36,6 +36,7 @@ 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;
@@ -52,7 +53,6 @@ public class TestWriteCSVResult {
     @Test
     public void testDataTypes() throws IOException {
         final CSVFormat csvFormat = CSVFormat.DEFAULT.withQuoteMode(QuoteMode.ALL).withRecordSeparator("\n");
-        final WriteCSVResult result = new WriteCSVResult(csvFormat, RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
 
         final StringBuilder headerBuilder = new StringBuilder();
         final List<RecordField> fields = new ArrayList<>();
@@ -71,6 +71,9 @@ public class TestWriteCSVResult {
         }
         final RecordSchema schema = new SimpleRecordSchema(fields);
 
+        final WriteCSVResult result = new WriteCSVResult(csvFormat, schema, new SchemaNameAsAttribute(),
+            RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat(), true);
+
         final long now = System.currentTimeMillis();
         final Map<String, Object> valueMap = new HashMap<>();
         valueMap.put("string", "string");
@@ -117,7 +120,7 @@ public class TestWriteCSVResult {
         expectedBuilder.append('"').append(dateValue).append('"').append(',');
         expectedBuilder.append('"').append(timeValue).append('"').append(',');
         expectedBuilder.append('"').append(timestampValue).append('"').append(',');
-        expectedBuilder.append(",\"48\",");
+        expectedBuilder.append(",\"48\",,");
         final String expectedValues = expectedBuilder.toString();
 
         assertEquals(expectedValues, values);

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/grok/TestGrokRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/grok/TestGrokRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/grok/TestGrokRecordReader.java
index a741ad1..ae5d433 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/grok/TestGrokRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/grok/TestGrokRecordReader.java
@@ -45,7 +45,7 @@ public class TestGrokRecordReader {
             grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt");
             grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} %{GREEDYDATA:message}");
 
-            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null);
+            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, GrokReader.createRecordSchema(grok), true);
 
             final String[] logLevels = new String[] {"INFO", "WARN", "ERROR", "FATAL", "FINE"};
             final String[] messages = new String[] {"Test Message 1", "Red", "Green", "Blue", "Yellow"};
@@ -75,7 +75,7 @@ public class TestGrokRecordReader {
         final String msg = "2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.LoggerClass \n"
             + "org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces";
         final InputStream bais = new ByteArrayInputStream(msg.getBytes(StandardCharsets.UTF_8));
-        final GrokRecordReader deserializer = new GrokRecordReader(bais, grok, null);
+        final GrokRecordReader deserializer = new GrokRecordReader(bais, grok, GrokReader.createRecordSchema(grok), true);
 
         final Object[] values = deserializer.nextRecord().getValues();
 
@@ -98,7 +98,7 @@ public class TestGrokRecordReader {
             grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt");
             grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \\[%{DATA:thread}\\] %{DATA:class} %{GREEDYDATA:message}");
 
-            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null);
+            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, GrokReader.createRecordSchema(grok), true);
 
             final String[] logLevels = new String[] {"INFO", "INFO", "INFO", "WARN", "WARN"};
 
@@ -122,7 +122,7 @@ public class TestGrokRecordReader {
             grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt");
             grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \\[%{DATA:thread}\\] %{DATA:class} %{GREEDYDATA:message}?");
 
-            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null);
+            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, GrokReader.createRecordSchema(grok), true);
 
             final String[] logLevels = new String[] {"INFO", "INFO", "ERROR", "WARN", "WARN"};
 
@@ -154,7 +154,7 @@ public class TestGrokRecordReader {
             grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt");
             grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} %{GREEDYDATA:message}");
 
-            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null);
+            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, GrokReader.createRecordSchema(grok), true);
 
             final String[] logLevels = new String[] {"INFO", "ERROR", "INFO"};
             final String[] messages = new String[] {"message without stack trace",

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/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 2422206..75e4d31 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
@@ -309,4 +309,27 @@ public class TestJsonTreeRowRecordReader {
         }
     }
 
+    @Test
+    public void testIncorrectSchema() throws IOException, MalformedRecordException {
+        final DataType accountType = RecordFieldType.RECORD.getRecordDataType(getAccountSchema());
+        final List<RecordField> fields = getDefaultFields();
+        fields.add(new RecordField("account", accountType));
+        fields.remove(new RecordField("balance", RecordFieldType.DOUBLE.getDataType()));
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-bank-account-wrong-field-type.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
+
+            reader.nextRecord().getValues();
+            Assert.fail("Was able to read record with invalid schema.");
+
+        } catch (final MalformedRecordException mre) {
+            final String msg = mre.getCause().getMessage();
+            assertTrue(msg.contains("account.balance"));
+            assertTrue(msg.contains("true"));
+            assertTrue(msg.contains("Double"));
+            assertTrue(msg.contains("Boolean"));
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java
index 6119d36..5c8bc49 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java
@@ -37,6 +37,7 @@ import java.util.Map;
 import java.util.TimeZone;
 
 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;
 import org.apache.nifi.serialization.record.MapRecord;
@@ -52,9 +53,6 @@ public class TestWriteJsonResult {
 
     @Test
     public void testDataTypes() throws IOException, ParseException {
-        final WriteJsonResult writer = new WriteJsonResult(Mockito.mock(ComponentLog.class), true, RecordFieldType.DATE.getDefaultFormat(),
-            RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
-
         final List<RecordField> fields = new ArrayList<>();
         for (final RecordFieldType fieldType : RecordFieldType.values()) {
             if (fieldType == RecordFieldType.CHOICE) {
@@ -63,16 +61,25 @@ public class TestWriteJsonResult {
                 possibleTypes.add(RecordFieldType.LONG.getDataType());
 
                 fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getChoiceDataType(possibleTypes)));
+            } else if (fieldType == RecordFieldType.MAP) {
+                fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getMapDataType(RecordFieldType.INT.getDataType())));
             } else {
                 fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType()));
             }
         }
         final RecordSchema schema = new SimpleRecordSchema(fields);
 
+        final WriteJsonResult writer = new WriteJsonResult(Mockito.mock(ComponentLog.class), schema, new SchemaNameAsAttribute(), true, RecordFieldType.DATE.getDefaultFormat(),
+            RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
+
         final DateFormat df = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS");
         df.setTimeZone(TimeZone.getTimeZone("gmt"));
         final long time = df.parse("2017/01/01 17:00:00.000").getTime();
 
+        final Map<String, Object> map = new LinkedHashMap<>();
+        map.put("height", 48);
+        map.put("width", 96);
+
         final Map<String, Object> valueMap = new LinkedHashMap<>();
         valueMap.put("string", "string");
         valueMap.put("boolean", true);
@@ -90,6 +97,7 @@ public class TestWriteJsonResult {
         valueMap.put("record", null);
         valueMap.put("array", null);
         valueMap.put("choice", 48L);
+        valueMap.put("map", map);
 
         final Record record = new MapRecord(schema, valueMap);
         final RecordSet rs = RecordSet.of(schema, record);

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/output/dataTypes.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/output/dataTypes.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/output/dataTypes.json
index 40c28dd..881925c 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/output/dataTypes.json
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/output/dataTypes.json
@@ -14,5 +14,9 @@
   "timestamp" : "2017-01-01 17:00:00",
   "record" : null,
   "choice" : 48,
-  "array" : null
+  "array" : null,
+  "map" : {
+    "height" : 48,
+    "width" : 96
+  }
 } ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-bank-account-wrong-field-type.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-bank-account-wrong-field-type.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-bank-account-wrong-field-type.json
new file mode 100644
index 0000000..50d676c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-bank-account-wrong-field-type.json
@@ -0,0 +1,13 @@
+{
+	"id": 1,
+	"name": "John Doe",
+	"address": "123 My Street",
+	"city": "My City", 
+	"state": "MS",
+	"zipCode": "11111",
+	"country": "USA",
+	"account": {
+		"id": 42,
+		"balance": true
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java b/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java
index 68c2461..88362b8 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java
@@ -16,38 +16,76 @@
  */
 package org.apache.nifi.schemaregistry.services;
 
-import java.util.Map;
+import java.io.IOException;
+import java.util.Set;
 
 import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.schema.access.SchemaField;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
 
 /**
  * Represents {@link ControllerService} strategy to expose internal and/or
  * integrate with external Schema Registry
  */
-public interface SchemaRegistry extends ControllerService, AutoCloseable {
-
-    public static final String SCHEMA_NAME_ATTR = "schema.name";
+public interface SchemaRegistry extends ControllerService {
 
 
     /**
      * Retrieves and returns the textual representation of the schema based on
-     * the provided name of the schema available in Schema Registry. Will throw
-     * an runtime exception if schema can not be found.
+     * the provided name of the schema available in Schema Registry.
+     *
+     * @return the text that corresponds to the latest version of the schema with the given name
+     *
+     * @throws IOException if unable to communicate with the backing store
+     * @throws SchemaNotFoundException if unable to find the schema with the given name
      */
-    String retrieveSchemaText(String schemaName);
+    String retrieveSchemaText(String schemaName) throws IOException, SchemaNotFoundException;
 
     /**
-     * Retrieves and returns the textual representation of the schema based on
-     * the provided name of the schema available in Schema Registry and optional
-     * additional attributes. Will throw an runtime exception if schema can not
-     * be found.
+     * Retrieves the textual representation of the schema with the given ID and version
+     *
+     * @param schemaId the unique identifier for the desired schema
+     * @param version the version of the desired schema
+     * @return the textual representation of the schema with the given ID and version
+     *
+     * @throws IOException if unable to communicate with the backing store
+     * @throws SchemaNotFoundException if unable to find the schema with the given id and version
      */
-    String retrieveSchemaText(String schemaName, Map<String, String> attributes);
+    String retrieveSchemaText(long schemaId, int version) throws IOException, SchemaNotFoundException;
 
+    /**
+     * Retrieves and returns the RecordSchema based on the provided name of the schema available in Schema Registry. The RecordSchema
+     * that is returned must have the Schema's name populated in its SchemaIdentifier. I.e., a call to
+     * {@link RecordSchema}.{@link RecordSchema#getIdentifier() getIdentifier()}.{@link SchemaIdentifier#getName() getName()}
+     * will always return an {@link Optional} that is not empty.
+     *
+     * @return the latest version of the schema with the given name, or <code>null</code> if no schema can be found with the given name.
+     * @throws SchemaNotFoundException if unable to find the schema with the given name
+     */
+    RecordSchema retrieveSchema(String schemaName) throws IOException, SchemaNotFoundException;
 
-    RecordSchema retrieveSchema(String schemaName);
 
+    /**
+     * Retrieves the schema with the given ID and version. The RecordSchema that is returned must have the Schema's identifier and version
+     * populated in its SchemaIdentifier. I.e., a call to
+     * {@link RecordSchema}.{@link RecordSchema#getIdentifier() getIdentifier()}.{@link SchemaIdentifier#getIdentifier() getIdentifier()}
+     * will always return an {@link Optional} that is not empty, as will a call to
+     * {@link RecordSchema}.{@link RecordSchema#getIdentifier() getIdentifier()}.{@link SchemaIdentifier#getVersion() getVersion()}.
+     *
+     * @param schemaId the unique identifier for the desired schema
+     * @param version the version of the desired schema
+     * @return the schema with the given ID and version or <code>null</code> if no schema
+     *         can be found with the given ID and version
+     *
+     * @throws IOException if unable to communicate with the backing store
+     * @throws SchemaNotFoundException if unable to find the schema with the given id and version
+     */
+    RecordSchema retrieveSchema(long schemaId, int version) throws IOException, SchemaNotFoundException;
 
-    RecordSchema retrieveSchema(String schemaName, Map<String, String> attributes);
+    /**
+     * @return the set of all Schema Fields that are supplied by the RecordSchema that is returned from {@link #retrieveSchema(String)} and {@link #retrieveSchema(long, int)}
+     */
+    Set<SchemaField> getSuppliedSchemaFields();
 }


[5/7] nifi git commit: NIFI-3682: This closes #1682. Add Schema Access Strategy and Schema Write Strategy Record Readers and Writers; bug fixes.

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryFlowFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryFlowFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryFlowFile.java
deleted file mode 100644
index 8e1c7ed..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryFlowFile.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.standard;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.nifi.controller.AbstractControllerService;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.processors.standard.util.record.MockRecordParser;
-import org.apache.nifi.processors.standard.util.record.MockRecordWriter;
-import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.serialization.RecordSetWriter;
-import org.apache.nifi.serialization.RecordSetWriterFactory;
-import org.apache.nifi.serialization.WriteResult;
-import org.apache.nifi.serialization.record.Record;
-import org.apache.nifi.serialization.record.RecordFieldType;
-import org.apache.nifi.serialization.record.RecordSet;
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestQueryFlowFile {
-
-    static {
-        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
-        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
-        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
-        System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.processors.standard.SQLTransform", "debug");
-    }
-
-    private static final String REL_NAME = "success";
-
-    @Test
-    public void testSimple() throws InitializationException, IOException, SQLException {
-        final MockRecordParser parser = new MockRecordParser();
-        parser.addSchemaField("name", RecordFieldType.STRING);
-        parser.addSchemaField("age", RecordFieldType.INT);
-        parser.addRecord("Tom", 49);
-
-        final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
-
-        final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class);
-        runner.addControllerService("parser", parser);
-        runner.enableControllerService(parser);
-        runner.addControllerService("writer", writer);
-        runner.enableControllerService(writer);
-
-        runner.setProperty(REL_NAME, "select name, age from FLOWFILE WHERE name <> ''");
-        runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser");
-        runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer");
-
-        final int numIterations = 1;
-        for (int i = 0; i < numIterations; i++) {
-            runner.enqueue(new byte[0]);
-        }
-
-        runner.setThreadCount(4);
-        runner.run(2 * numIterations);
-
-        runner.assertTransferCount(REL_NAME, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(REL_NAME).get(0);
-        System.out.println(new String(out.toByteArray()));
-        out.assertContentEquals("\"name\",\"points\"\n\"Tom\",\"49\"\n");
-    }
-
-    @Test
-    public void testParseFailure() throws InitializationException, IOException, SQLException {
-        final MockRecordParser parser = new MockRecordParser();
-        parser.addSchemaField("name", RecordFieldType.STRING);
-        parser.addSchemaField("age", RecordFieldType.INT);
-        parser.addRecord("Tom", 49);
-
-        final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
-
-        final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class);
-        runner.addControllerService("parser", parser);
-        runner.enableControllerService(parser);
-        runner.addControllerService("writer", writer);
-        runner.enableControllerService(writer);
-
-        runner.setProperty(REL_NAME, "select name, age from FLOWFILE WHERE name <> ''");
-        runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser");
-        runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer");
-
-        final int numIterations = 1;
-        for (int i = 0; i < numIterations; i++) {
-            runner.enqueue(new byte[0]);
-        }
-
-        runner.setThreadCount(4);
-        runner.run(2 * numIterations);
-
-        runner.assertTransferCount(REL_NAME, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(REL_NAME).get(0);
-        System.out.println(new String(out.toByteArray()));
-        out.assertContentEquals("\"name\",\"points\"\n\"Tom\",\"49\"\n");
-    }
-
-
-    @Test
-    public void testTransformCalc() throws InitializationException, IOException, SQLException {
-        final MockRecordParser parser = new MockRecordParser();
-        parser.addSchemaField("ID", RecordFieldType.INT);
-        parser.addSchemaField("AMOUNT1", RecordFieldType.FLOAT);
-        parser.addSchemaField("AMOUNT2", RecordFieldType.FLOAT);
-        parser.addSchemaField("AMOUNT3", RecordFieldType.FLOAT);
-
-        parser.addRecord("008", 10.05F, 15.45F, 89.99F);
-        parser.addRecord("100", 20.25F, 25.25F, 45.25F);
-        parser.addRecord("105", 20.05F, 25.05F, 45.05F);
-        parser.addRecord("200", 34.05F, 25.05F, 75.05F);
-
-        final MockRecordWriter writer = new MockRecordWriter("\"NAME\",\"POINTS\"");
-
-        final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class);
-        runner.addControllerService("parser", parser);
-        runner.enableControllerService(parser);
-        runner.addControllerService("writer", writer);
-        runner.enableControllerService(writer);
-
-        runner.setProperty(REL_NAME, "select ID, AMOUNT1+AMOUNT2+AMOUNT3 as TOTAL from FLOWFILE where ID=100");
-        runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser");
-        runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer");
-
-        runner.enqueue(new byte[0]);
-        runner.run();
-
-        runner.assertTransferCount(REL_NAME, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(REL_NAME).get(0);
-
-        out.assertContentEquals("\"NAME\",\"POINTS\"\n\"100\",\"90.75\"\n");
-    }
-
-
-    @Test
-    public void testAggregateFunction() throws InitializationException, IOException {
-        final MockRecordParser parser = new MockRecordParser();
-        parser.addSchemaField("name", RecordFieldType.STRING);
-        parser.addSchemaField("points", RecordFieldType.INT);
-        parser.addRecord("Tom", 1);
-        parser.addRecord("Jerry", 2);
-        parser.addRecord("Tom", 99);
-
-        final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
-
-        final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class);
-        runner.addControllerService("parser", parser);
-        runner.enableControllerService(parser);
-        runner.addControllerService("writer", writer);
-        runner.enableControllerService(writer);
-
-        runner.setProperty(REL_NAME, "select name, sum(points) as points from FLOWFILE GROUP BY name");
-        runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser");
-        runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer");
-
-        runner.enqueue("");
-        runner.run();
-
-        runner.assertTransferCount(REL_NAME, 1);
-        final MockFlowFile flowFileOut = runner.getFlowFilesForRelationship(ExecuteProcess.REL_SUCCESS).get(0);
-        flowFileOut.assertContentEquals("\"name\",\"points\"\n\"Tom\",\"100\"\n\"Jerry\",\"2\"\n");
-    }
-
-    @Test
-    public void testColumnNames() throws InitializationException, IOException {
-        final MockRecordParser parser = new MockRecordParser();
-        parser.addSchemaField("name", RecordFieldType.STRING);
-        parser.addSchemaField("points", RecordFieldType.INT);
-        parser.addSchemaField("greeting", RecordFieldType.STRING);
-        parser.addRecord("Tom", 1, "Hello");
-        parser.addRecord("Jerry", 2, "Hi");
-        parser.addRecord("Tom", 99, "Howdy");
-
-        final List<String> colNames = new ArrayList<>();
-        colNames.add("name");
-        colNames.add("points");
-        colNames.add("greeting");
-        colNames.add("FAV_GREETING");
-        final ResultSetValidatingRecordWriter writer = new ResultSetValidatingRecordWriter(colNames);
-
-        final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class);
-        runner.addControllerService("parser", parser);
-        runner.enableControllerService(parser);
-        runner.addControllerService("writer", writer);
-        runner.enableControllerService(writer);
-
-        runner.setProperty(REL_NAME, "select *, greeting AS FAV_GREETING from FLOWFILE");
-        runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser");
-        runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer");
-
-        runner.enqueue("");
-        runner.run();
-
-        runner.assertTransferCount(REL_NAME, 1);
-    }
-
-
-    private static class ResultSetValidatingRecordWriter extends AbstractControllerService implements RecordSetWriterFactory {
-        private final List<String> columnNames;
-
-        public ResultSetValidatingRecordWriter(final List<String> colNames) {
-            this.columnNames = new ArrayList<>(colNames);
-        }
-
-        @Override
-        public RecordSetWriter createWriter(ComponentLog logger) {
-            return new RecordSetWriter() {
-                @Override
-                public WriteResult write(final RecordSet rs, final OutputStream out) throws IOException {
-                    final int colCount = rs.getSchema().getFieldCount();
-                    Assert.assertEquals(columnNames.size(), colCount);
-
-                    final List<String> colNames = new ArrayList<>(colCount);
-                    for (int i = 0; i < colCount; i++) {
-                        colNames.add(rs.getSchema().getField(i).getFieldName());
-                    }
-
-                    Assert.assertEquals(columnNames, colNames);
-
-                    // Iterate over the rest of the records to ensure that we read the entire stream. If we don't
-                    // do this, we won't consume all of the data and as a result we will not close the stream properly
-                    Record record;
-                    while ((record = rs.next()) != null) {
-                        System.out.println(record);
-                    }
-
-                    return WriteResult.of(0, Collections.emptyMap());
-                }
-
-                @Override
-                public String getMimeType() {
-                    return "text/plain";
-                }
-
-                @Override
-                public WriteResult write(Record record, OutputStream out) throws IOException {
-                    return null;
-                }
-            };
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java
new file mode 100644
index 0000000..32c3635
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java
@@ -0,0 +1,297 @@
+/*
+ * 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.processors.standard;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.standard.util.record.MockRecordParser;
+import org.apache.nifi.processors.standard.util.record.MockRecordWriter;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestQueryRecord {
+
+    static {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.processors.standard.SQLTransform", "debug");
+    }
+
+    private static final String REL_NAME = "success";
+
+    @Test
+    public void testSimple() throws InitializationException, IOException, SQLException {
+        final MockRecordParser parser = new MockRecordParser();
+        parser.addSchemaField("name", RecordFieldType.STRING);
+        parser.addSchemaField("age", RecordFieldType.INT);
+        parser.addRecord("Tom", 49);
+
+        final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
+
+        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        runner.addControllerService("parser", parser);
+        runner.enableControllerService(parser);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(REL_NAME, "select name, age from FLOWFILE WHERE name <> ''");
+        runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser");
+        runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer");
+
+        final int numIterations = 1;
+        for (int i = 0; i < numIterations; i++) {
+            runner.enqueue(new byte[0]);
+        }
+
+        runner.setThreadCount(4);
+        runner.run(2 * numIterations);
+
+        runner.assertTransferCount(REL_NAME, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(REL_NAME).get(0);
+        System.out.println(new String(out.toByteArray()));
+        out.assertContentEquals("\"name\",\"points\"\n\"Tom\",\"49\"\n");
+    }
+
+    @Test
+    public void testParseFailure() throws InitializationException, IOException, SQLException {
+        final MockRecordParser parser = new MockRecordParser();
+        parser.addSchemaField("name", RecordFieldType.STRING);
+        parser.addSchemaField("age", RecordFieldType.INT);
+        parser.addRecord("Tom", 49);
+
+        final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
+
+        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        runner.addControllerService("parser", parser);
+        runner.enableControllerService(parser);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(REL_NAME, "select name, age from FLOWFILE WHERE name <> ''");
+        runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser");
+        runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer");
+
+        final int numIterations = 1;
+        for (int i = 0; i < numIterations; i++) {
+            runner.enqueue(new byte[0]);
+        }
+
+        runner.setThreadCount(4);
+        runner.run(2 * numIterations);
+
+        runner.assertTransferCount(REL_NAME, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(REL_NAME).get(0);
+        System.out.println(new String(out.toByteArray()));
+        out.assertContentEquals("\"name\",\"points\"\n\"Tom\",\"49\"\n");
+    }
+
+
+    @Test
+    public void testTransformCalc() throws InitializationException, IOException, SQLException {
+        final MockRecordParser parser = new MockRecordParser();
+        parser.addSchemaField("ID", RecordFieldType.INT);
+        parser.addSchemaField("AMOUNT1", RecordFieldType.FLOAT);
+        parser.addSchemaField("AMOUNT2", RecordFieldType.FLOAT);
+        parser.addSchemaField("AMOUNT3", RecordFieldType.FLOAT);
+
+        parser.addRecord("008", 10.05F, 15.45F, 89.99F);
+        parser.addRecord("100", 20.25F, 25.25F, 45.25F);
+        parser.addRecord("105", 20.05F, 25.05F, 45.05F);
+        parser.addRecord("200", 34.05F, 25.05F, 75.05F);
+
+        final MockRecordWriter writer = new MockRecordWriter("\"NAME\",\"POINTS\"");
+
+        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        runner.addControllerService("parser", parser);
+        runner.enableControllerService(parser);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(REL_NAME, "select ID, AMOUNT1+AMOUNT2+AMOUNT3 as TOTAL from FLOWFILE where ID=100");
+        runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser");
+        runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer");
+
+        runner.enqueue(new byte[0]);
+        runner.run();
+
+        runner.assertTransferCount(REL_NAME, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(REL_NAME).get(0);
+
+        out.assertContentEquals("\"NAME\",\"POINTS\"\n\"100\",\"90.75\"\n");
+    }
+
+    @Test
+    public void testHandlingWithInvalidSchema() throws InitializationException {
+        final MockRecordParser parser = new MockRecordParser();
+        parser.addSchemaField("name", RecordFieldType.STRING);
+        parser.addSchemaField("favorite_color", RecordFieldType.STRING);
+        parser.addSchemaField("address", RecordFieldType.STRING);
+        parser.addRecord("Tom", "blue", null);
+        parser.addRecord("Jerry", "red", null);
+
+        final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
+
+        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        runner.enforceReadStreamsClosed(false);
+        runner.addControllerService("parser", parser);
+        runner.enableControllerService(parser);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(QueryRecord.INCLUDE_ZERO_RECORD_FLOWFILES, "false");
+        runner.setProperty("rel1", "select * from FLOWFILE where address IS NOT NULL");
+        runner.setProperty("rel2", "select name, CAST(favorite_color AS DOUBLE) AS num from FLOWFILE");
+        runner.setProperty("rel3", "select * from FLOWFILE where address IS NOT NULL");
+        runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser");
+        runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer");
+
+        runner.enqueue("");
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(QueryRecord.REL_FAILURE, 1);
+    }
+
+    @Test
+    public void testAggregateFunction() throws InitializationException, IOException {
+        final MockRecordParser parser = new MockRecordParser();
+        parser.addSchemaField("name", RecordFieldType.STRING);
+        parser.addSchemaField("points", RecordFieldType.INT);
+        parser.addRecord("Tom", 1);
+        parser.addRecord("Jerry", 2);
+        parser.addRecord("Tom", 99);
+
+        final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
+
+        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        runner.addControllerService("parser", parser);
+        runner.enableControllerService(parser);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(REL_NAME, "select name, sum(points) as points from FLOWFILE GROUP BY name");
+        runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser");
+        runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer");
+
+        runner.enqueue("");
+        runner.run();
+
+        runner.assertTransferCount(REL_NAME, 1);
+        final MockFlowFile flowFileOut = runner.getFlowFilesForRelationship(ExecuteProcess.REL_SUCCESS).get(0);
+        flowFileOut.assertContentEquals("\"name\",\"points\"\n\"Tom\",\"100\"\n\"Jerry\",\"2\"\n");
+    }
+
+    @Test
+    public void testColumnNames() throws InitializationException, IOException {
+        final MockRecordParser parser = new MockRecordParser();
+        parser.addSchemaField("name", RecordFieldType.STRING);
+        parser.addSchemaField("points", RecordFieldType.INT);
+        parser.addSchemaField("greeting", RecordFieldType.STRING);
+        parser.addRecord("Tom", 1, "Hello");
+        parser.addRecord("Jerry", 2, "Hi");
+        parser.addRecord("Tom", 99, "Howdy");
+
+        final List<String> colNames = new ArrayList<>();
+        colNames.add("name");
+        colNames.add("points");
+        colNames.add("greeting");
+        colNames.add("FAV_GREETING");
+        final ResultSetValidatingRecordWriter writer = new ResultSetValidatingRecordWriter(colNames);
+
+        final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class);
+        runner.addControllerService("parser", parser);
+        runner.enableControllerService(parser);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(REL_NAME, "select *, greeting AS FAV_GREETING from FLOWFILE");
+        runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser");
+        runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer");
+
+        runner.enqueue("");
+        runner.run();
+
+        runner.assertTransferCount(REL_NAME, 1);
+    }
+
+
+    private static class ResultSetValidatingRecordWriter extends AbstractControllerService implements RecordSetWriterFactory {
+        private final List<String> columnNames;
+
+        public ResultSetValidatingRecordWriter(final List<String> colNames) {
+            this.columnNames = new ArrayList<>(colNames);
+        }
+
+        @Override
+        public RecordSetWriter createWriter(ComponentLog logger, FlowFile flowFile, InputStream in) {
+            return new RecordSetWriter() {
+                @Override
+                public WriteResult write(final RecordSet rs, final OutputStream out) throws IOException {
+                    final int colCount = rs.getSchema().getFieldCount();
+                    Assert.assertEquals(columnNames.size(), colCount);
+
+                    final List<String> colNames = new ArrayList<>(colCount);
+                    for (int i = 0; i < colCount; i++) {
+                        colNames.add(rs.getSchema().getField(i).getFieldName());
+                    }
+
+                    Assert.assertEquals(columnNames, colNames);
+
+                    // Iterate over the rest of the records to ensure that we read the entire stream. If we don't
+                    // do this, we won't consume all of the data and as a result we will not close the stream properly
+                    Record record;
+                    while ((record = rs.next()) != null) {
+                        System.out.println(record);
+                    }
+
+                    return WriteResult.of(0, Collections.emptyMap());
+                }
+
+                @Override
+                public String getMimeType() {
+                    return "text/plain";
+                }
+
+                @Override
+                public WriteResult write(Record record, OutputStream out) throws IOException {
+                    return null;
+                }
+            };
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java
index 1a39b82..fcf0d10 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java
@@ -28,9 +28,10 @@ import java.util.Map;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
+import org.apache.nifi.serialization.RecordReaderFactory;
 import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.MapRecord;
 import org.apache.nifi.serialization.record.Record;
@@ -38,7 +39,7 @@ import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
 
-public class MockRecordParser extends AbstractControllerService implements RowRecordReaderFactory {
+public class MockRecordParser extends AbstractControllerService implements RecordReaderFactory {
     private final List<Object[]> records = new ArrayList<>();
     private final List<RecordField> fields = new ArrayList<>();
     private final int failAfterN;
@@ -61,7 +62,7 @@ public class MockRecordParser extends AbstractControllerService implements RowRe
     }
 
     @Override
-    public RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws IOException {
+    public RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws IOException, SchemaNotFoundException {
         final Iterator<Object[]> itr = records.iterator();
 
         return new RecordReader() {
@@ -99,9 +100,4 @@ public class MockRecordParser extends AbstractControllerService implements RowRe
             }
         };
     }
-
-    @Override
-    public RecordSchema getSchema(FlowFile flowFile) throws MalformedRecordException, IOException {
-        return null;
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java
index 0a57b29..1dbfd04 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java
@@ -18,10 +18,12 @@
 package org.apache.nifi.processors.standard.util.record;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.Collections;
 
 import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
@@ -49,7 +51,7 @@ public class MockRecordWriter extends AbstractControllerService implements Recor
     }
 
     @Override
-    public RecordSetWriter createWriter(final ComponentLog logger) {
+    public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream in) {
         return new RecordSetWriter() {
             @Override
             public WriteResult write(final RecordSet rs, final OutputStream out) throws IOException {
@@ -69,9 +71,11 @@ public class MockRecordWriter extends AbstractControllerService implements Recor
                         final String val = record.getAsString(fieldName);
                         if (quoteValues) {
                             out.write("\"".getBytes());
-                            out.write(val.getBytes());
+                            if (val != null) {
+                                out.write(val.getBytes());
+                            }
                             out.write("\"".getBytes());
-                        } else {
+                        } else if (val != null) {
                             out.write(val.getBytes());
                         }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaField.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaField.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaField.java
new file mode 100644
index 0000000..2fe06f4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaField.java
@@ -0,0 +1,37 @@
+/*
+ * 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.access;
+
+public enum SchemaField {
+    SCHEMA_TEXT("Schema Text"),
+    SCHEMA_TEXT_FORMAT("Schema Text Format"),
+    SCHEMA_NAME("Schema Name"),
+    SCHEMA_IDENTIFIER("Schema Identifier"),
+    SCHEMA_VERSION("Schema Version");
+
+    private final String description;
+
+    private SchemaField(final String description) {
+        this.description = description;
+    }
+
+    @Override
+    public String toString() {
+        return description;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaNotFoundException.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaNotFoundException.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaNotFoundException.java
new file mode 100644
index 0000000..9a064ff
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaNotFoundException.java
@@ -0,0 +1,32 @@
+/*
+ * 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.access;
+
+public class SchemaNotFoundException extends Exception {
+    public SchemaNotFoundException(final String message) {
+        super(message);
+    }
+
+    public SchemaNotFoundException(final String message, final Throwable cause) {
+        super(cause);
+    }
+
+    public SchemaNotFoundException(final Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7d7268e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+
+/**
+ * <p>
+ * A Controller Service that is responsible for creating a {@link RecordReader}.
+ * </p>
+ */
+public interface RecordReaderFactory extends ControllerService {
+
+    RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException;
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java
index 2286f3f..e23ad20 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java
@@ -17,14 +17,47 @@
 
 package org.apache.nifi.serialization;
 
+import java.io.IOException;
+import java.io.InputStream;
+
 import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 
 /**
  * <p>
- * A Controller Service that is responsible for creating a {@link RecordSetWriter}.
+ * A Controller Service that is responsible for creating a {@link RecordSetWriter}. The writer is created
+ * based on a FlowFile and an InputStream for that FlowFile, but it is important to note that this the FlowFile passed
+ * to the {@link #createWriter(ComponentLog, FlowFile, InputStream)} may not be the FlowFile that the Writer will writer to.
+ * Rather, it is the FlowFile and InputStream from which the Writer's Schema should be determined. This is done because most
+ * Processors that make use of Record Writers also make use of Record Readers and the schema for the output is often determined
+ * by either reading the schema from the content of the input FlowFile or from referencing attributes of the
+ * input FlowFile.
+ * </p>
+ *
+ * <p>
+ * PLEASE NOTE: This interface is still considered 'unstable' and may change in a non-backward-compatible
+ * manner between minor or incremental releases of NiFi.
  * </p>
  */
 public interface RecordSetWriterFactory extends ControllerService {
-    RecordSetWriter createWriter(ComponentLog logger);
+
+    /**
+     * <p>
+     * Creates a new RecordSetWriter that is capable of writing record contents to an OutputStream. Note that the
+     * FlowFile and InputStream that are given may well be different than the FlowFile that the writer is intended
+     * to write to. The given FlowFile and InputStream are intended to be used for determining the schema that should
+     * be used when writing records.
+     * </p>
+     *
+     * @param logger the logger to use when logging information. This is passed in, rather than using the logger of the Controller Service
+     *            because it allows messages to be logged for the component that is calling this Controller Service.
+     * @param schemaFlowFile the FlowFile from which the schema should be determined.
+     * @param schemaFlowFileContent the contents of the FlowFile from which to determine the schema
+     * @return a RecordSetWriter that can write record sets to an OutputStream
+     * @throws SchemaNotFoundException if unable to find the schema
+     * @throws IOException if unable to read from the given InputStream
+     */
+    RecordSetWriter createWriter(ComponentLog logger, FlowFile schemaFlowFile, InputStream schemaFlowFileContent) throws SchemaNotFoundException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RowRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RowRecordReaderFactory.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RowRecordReaderFactory.java
deleted file mode 100644
index fbd8a21..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RowRecordReaderFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.nifi.serialization;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.nifi.controller.ControllerService;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.serialization.record.RecordSchema;
-
-/**
- * <p>
- * A Controller Service that is responsible for creating a {@link RecordReader}.
- * </p>
- */
-public interface RowRecordReaderFactory extends ControllerService {
-
-    RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws MalformedRecordException, IOException;
-
-    RecordSchema getSchema(FlowFile flowFile) throws MalformedRecordException, IOException;
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
index 246e0af..017aef1 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
@@ -29,21 +29,69 @@ import java.util.stream.Collectors;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
 
 public class SimpleRecordSchema implements RecordSchema {
     private final List<RecordField> fields;
     private final Map<String, Integer> fieldIndices;
+    private final boolean textAvailable;
+    private final String text;
+    private final String schemaFormat;
+    private final SchemaIdentifier schemaIdentifier;
 
     public SimpleRecordSchema(final List<RecordField> fields) {
+        this(fields, createText(fields), null, false, SchemaIdentifier.EMPTY);
+    }
+
+    public SimpleRecordSchema(final List<RecordField> fields, final SchemaIdentifier id) {
+        this(fields, createText(fields), null, false, id);
+    }
+
+    public SimpleRecordSchema(final List<RecordField> fields, final String text, final String schemaFormat, final SchemaIdentifier id) {
+        this(fields, text, schemaFormat, true, id);
+    }
+
+    private SimpleRecordSchema(final List<RecordField> fields, final String text, final String schemaFormat, final boolean textAvailable, final SchemaIdentifier id) {
+        this.text = text;
+        this.schemaFormat = schemaFormat;
+        this.schemaIdentifier = id;
+        this.textAvailable = textAvailable;
         this.fields = Collections.unmodifiableList(new ArrayList<>(fields));
         this.fieldIndices = new HashMap<>(fields.size());
 
         int index = 0;
         for (final RecordField field : fields) {
-            fieldIndices.put(field.getFieldName(), index++);
+            Integer previousValue = fieldIndices.put(field.getFieldName(), index);
+            if (previousValue != null) {
+                throw new IllegalArgumentException("Two fields are given with the same name (or alias) of '" + field.getFieldName() + "'");
+            }
+
+            for (final String alias : field.getAliases()) {
+                previousValue = fieldIndices.put(alias, index);
+                if (previousValue != null) {
+                    throw new IllegalArgumentException("Two fields are given with the same name (or alias) of '" + field.getFieldName() + "'");
+                }
+            }
+
+            index++;
+        }
+    }
+
+    @Override
+    public Optional<String> getSchemaText() {
+        if (textAvailable) {
+            return Optional.ofNullable(text);
+        } else {
+            return Optional.empty();
         }
     }
 
+
+    @Override
+    public Optional<String> getSchemaFormat() {
+        return Optional.ofNullable(schemaFormat);
+    }
+
     @Override
     public List<RecordField> getFields() {
         return fields;
@@ -77,6 +125,16 @@ public class SimpleRecordSchema implements RecordSchema {
         return idx.isPresent() ? Optional.of(fields.get(idx.getAsInt()).getDataType()) : Optional.empty();
     }
 
+    @Override
+    public Optional<RecordField> getField(final String fieldName) {
+        final OptionalInt indexOption = getFieldIndex(fieldName);
+        if (indexOption.isPresent()) {
+            return Optional.of(fields.get(indexOption.getAsInt()));
+        }
+
+        return Optional.empty();
+    }
+
     private OptionalInt getFieldIndex(final String fieldName) {
         final Integer index = fieldIndices.get(fieldName);
         return index == null ? OptionalInt.empty() : OptionalInt.of(index);
@@ -103,8 +161,7 @@ public class SimpleRecordSchema implements RecordSchema {
         return 143 + 3 * fields.hashCode();
     }
 
-    @Override
-    public String toString() {
+    private static String createText(final List<RecordField> fields) {
         final StringBuilder sb = new StringBuilder("[");
 
         for (int i = 0; i < fields.size(); i++) {
@@ -123,4 +180,14 @@ public class SimpleRecordSchema implements RecordSchema {
         sb.append("]");
         return sb.toString();
     }
+
+    @Override
+    public String toString() {
+        return text;
+    }
+
+    @Override
+    public SchemaIdentifier getIdentifier() {
+        return schemaIdentifier;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/DataType.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/DataType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/DataType.java
index b72c107..6ed4bd6 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/DataType.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/DataType.java
@@ -17,6 +17,8 @@
 
 package org.apache.nifi.serialization.record;
 
+import java.util.Objects;
+
 public class DataType {
     private final RecordFieldType fieldType;
     private final String format;
@@ -36,7 +38,7 @@ public class DataType {
 
     @Override
     public int hashCode() {
-        return 31 + 41 * fieldType.hashCode() + 41 * (format == null ? 0 : format.hashCode());
+        return 31 + 41 * getFieldType().hashCode() + 41 * (getFormat() == null ? 0 : getFormat().hashCode());
     }
 
     @Override
@@ -52,15 +54,15 @@ public class DataType {
         }
 
         final DataType other = (DataType) obj;
-        return fieldType.equals(other.fieldType) && ((format == null && other.format == null) || (format != null && format.equals(other.format)));
+        return getFieldType().equals(other.getFieldType()) && Objects.equals(getFormat(), other.getFormat());
     }
 
     @Override
     public String toString() {
-        if (format == null) {
-            return fieldType.toString();
+        if (getFormat() == null) {
+            return getFieldType().toString();
         } else {
-            return fieldType.toString() + ":" + format;
+            return getFieldType().toString() + ":" + getFormat();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
index 0bbb534..56cf909 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
@@ -42,15 +42,86 @@ public class MapRecord implements Record {
     public Object[] getValues() {
         final Object[] values = new Object[schema.getFieldCount()];
         int i = 0;
-        for (final String fieldName : schema.getFieldNames()) {
-            values[i++] = getValue(fieldName);
+        for (final RecordField recordField : schema.getFields()) {
+            values[i++] = getValue(recordField);
         }
         return values;
     }
 
     @Override
     public Object getValue(final String fieldName) {
-        return values.get(fieldName);
+        final Optional<RecordField> fieldOption = schema.getField(fieldName);
+        if (fieldOption.isPresent()) {
+            return getValue(fieldOption.get());
+        }
+
+        return null;
+    }
+
+    @Override
+    public Object getValue(final RecordField field) {
+        Object explicitValue = getExplicitValue(field);
+        if (explicitValue != null) {
+            return explicitValue;
+        }
+
+        final Optional<RecordField> resolvedField = resolveField(field);
+        final boolean resolvedFieldDifferent = resolvedField.isPresent() && !resolvedField.get().equals(field);
+        if (resolvedFieldDifferent) {
+            explicitValue = getExplicitValue(resolvedField.get());
+            if (explicitValue != null) {
+                return explicitValue;
+            }
+        }
+
+        Object defaultValue = field.getDefaultValue();
+        if (defaultValue != null) {
+            return defaultValue;
+        }
+
+        if (resolvedFieldDifferent) {
+            return resolvedField.get().getDefaultValue();
+        }
+
+        return null;
+    }
+
+    private Optional<RecordField> resolveField(final RecordField field) {
+        Optional<RecordField> resolved = schema.getField(field.getFieldName());
+        if (resolved.isPresent()) {
+            return resolved;
+        }
+
+        for (final String alias : field.getAliases()) {
+            resolved = schema.getField(alias);
+            if (resolved.isPresent()) {
+                return resolved;
+            }
+        }
+
+        return Optional.empty();
+    }
+
+    private Object getExplicitValue(final RecordField field) {
+        final String canonicalFieldName = field.getFieldName();
+
+        // We use containsKey here instead of just calling get() and checking for a null value
+        // because if the true field name is set to null, we want to return null, rather than
+        // what the alias points to. Likewise for a specific alias, since aliases are defined
+        // in a List with a specific ordering.
+        Object value = values.get(canonicalFieldName);
+        if (value != null) {
+            return value;
+        }
+
+        for (final String alias : field.getAliases()) {
+            value = values.get(alias);
+            if (value != null) {
+                return value;
+            }
+        }
+
+        return null;
     }
 
     @Override
@@ -68,6 +139,11 @@ public class MapRecord implements Record {
         return convertToString(getValue(fieldName), format);
     }
 
+    @Override
+    public String getAsString(final RecordField field, final String format) {
+        return convertToString(getValue(field), format);
+    }
+
     private String getFormat(final String optionalFormat, final RecordFieldType fieldType) {
         return (optionalFormat == null) ? fieldType.getDefaultFormat() : optionalFormat;
     }
@@ -85,42 +161,42 @@ public class MapRecord implements Record {
 
     @Override
     public Long getAsLong(final String fieldName) {
-        return DataTypeUtils.toLong(getValue(fieldName));
+        return DataTypeUtils.toLong(getValue(fieldName), fieldName);
     }
 
     @Override
     public Integer getAsInt(final String fieldName) {
-        return DataTypeUtils.toInteger(getValue(fieldName));
+        return DataTypeUtils.toInteger(getValue(fieldName), fieldName);
     }
 
     @Override
     public Double getAsDouble(final String fieldName) {
-        return DataTypeUtils.toDouble(getValue(fieldName));
+        return DataTypeUtils.toDouble(getValue(fieldName), fieldName);
     }
 
     @Override
     public Float getAsFloat(final String fieldName) {
-        return DataTypeUtils.toFloat(getValue(fieldName));
+        return DataTypeUtils.toFloat(getValue(fieldName), fieldName);
     }
 
     @Override
     public Record getAsRecord(String fieldName, final RecordSchema schema) {
-        return DataTypeUtils.toRecord(getValue(fieldName), schema);
+        return DataTypeUtils.toRecord(getValue(fieldName), schema, fieldName);
     }
 
     @Override
     public Boolean getAsBoolean(final String fieldName) {
-        return DataTypeUtils.toBoolean(getValue(fieldName));
+        return DataTypeUtils.toBoolean(getValue(fieldName), fieldName);
     }
 
     @Override
     public Date getAsDate(final String fieldName, final String format) {
-        return DataTypeUtils.toDate(getValue(fieldName), format);
+        return DataTypeUtils.toDate(getValue(fieldName), format, fieldName);
     }
 
     @Override
     public Object[] getAsArray(final String fieldName) {
-        return DataTypeUtils.toArray(getValue(fieldName));
+        return DataTypeUtils.toArray(getValue(fieldName), fieldName);
     }
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/Record.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/Record.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/Record.java
index e1d52e9..5e5e7ba 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/Record.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/Record.java
@@ -38,10 +38,14 @@ public interface Record {
 
     Object getValue(String fieldName);
 
+    Object getValue(RecordField field);
+
     String getAsString(String fieldName);
 
     String getAsString(String fieldName, String format);
 
+    String getAsString(RecordField field, String format);
+
     Long getAsLong(String fieldName);
 
     Integer getAsInt(String fieldName);

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordField.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordField.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordField.java
index 135ae66..fe3d8e5 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordField.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordField.java
@@ -17,29 +17,66 @@
 
 package org.apache.nifi.serialization.record;
 
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
 public class RecordField {
     private final String fieldName;
     private final DataType dataType;
+    private final Set<String> aliases;
+    private final Object defaultValue;
 
     public RecordField(final String fieldName, final DataType dataType) {
-        this.fieldName = fieldName;
-        this.dataType = dataType;
+        this(fieldName, dataType, null, Collections.emptySet());
+    }
+
+    public RecordField(final String fieldName, final DataType dataType, final Object defaultValue) {
+        this(fieldName, dataType, defaultValue, Collections.emptySet());
+    }
+
+    public RecordField(final String fieldName, final DataType dataType, final Set<String> aliases) {
+        this(fieldName, dataType, null, aliases);
+    }
+
+    public RecordField(final String fieldName, final DataType dataType, final Object defaultValue, final Set<String> aliases) {
+        if (defaultValue != null && !DataTypeUtils.isCompatibleDataType(defaultValue, dataType)) {
+            throw new IllegalArgumentException("Cannot set the default value for field [" + fieldName + "] to [" + defaultValue
+                + "] because that is not a valid value for Data Type [" + dataType + "]");
+        }
+
+        this.fieldName = Objects.requireNonNull(fieldName);
+        this.dataType = Objects.requireNonNull(dataType);
+        this.aliases = Collections.unmodifiableSet(Objects.requireNonNull(aliases));
+        this.defaultValue = defaultValue;
     }
 
     public String getFieldName() {
         return fieldName;
     }
 
+    public Set<String> getAliases() {
+        return aliases;
+    }
+
     public DataType getDataType() {
         return dataType;
     }
 
+    public Object getDefaultValue() {
+        return defaultValue;
+    }
+
     @Override
     public int hashCode() {
         final int prime = 31;
         int result = 1;
-        result = prime * result + ((dataType == null) ? 0 : dataType.hashCode());
-        result = prime * result + ((fieldName == null) ? 0 : fieldName.hashCode());
+        result = prime * result + dataType.hashCode();
+        result = prime * result + fieldName.hashCode();
+        result = prime * result + aliases.hashCode();
+        result = prime * result + ((defaultValue == null) ? 0 : defaultValue.hashCode());
         return result;
     }
 
@@ -57,7 +94,7 @@ public class RecordField {
         }
 
         RecordField other = (RecordField) obj;
-        return dataType.equals(other.getDataType()) && fieldName.equals(other.getFieldName());
+        return dataType.equals(other.getDataType()) && fieldName.equals(other.getFieldName()) && aliases.equals(other.getAliases()) && Objects.equals(defaultValue, other.defaultValue);
     }
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
index cc83a41..785b8d2 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 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;
 
 public enum RecordFieldType {
@@ -149,7 +150,7 @@ public enum RecordFieldType {
 
     /**
      * <p>
-     * An array field type. Records should be updated using an {@code Object[]} value for this field. Note that we are explicitly indicating that
+     * An array field type. Fields of this type use a {@code Object[]} value. Note that we are explicitly indicating that
      * Object[] should be used here and not primitive array types. For instance, setting a value of {@code int[]} is not allowed. The DataType for
      * this field should be created using the {@link #getArrayDataType(DataType)} method:
      * </p>
@@ -173,7 +174,34 @@ public enum RecordFieldType {
      * </pre>
      * </code>
      */
-    ARRAY("array", null, new ArrayDataType(null));
+    ARRAY("array", null, new ArrayDataType(null)),
+
+    /**
+     * <p>
+     * A record field type. Fields of this type use a {@code Map<String, Object>} value. A Map DataType should be
+     * created by providing the {@link DataType} for the values:
+     * </p>
+     *
+     * <code>
+     * final DataType recordType = RecordFieldType.MAP.getRecordDataType( RecordFieldType.STRING.getDataType() );
+     * </code>
+     *
+     * <p>
+     * A field of type MAP should always have a {@link MapDataType}, so the following idiom is acceptable for use:
+     * </p>
+     *
+     * <code>
+     * <pre>
+     * final DataType dataType = ...;
+     * if (dataType.getFieldType() == RecordFieldType.MAP) {
+     *     final MapDataType mapDataType = (MapDataType) dataType;
+     *     final DataType valueType = mapDataType.getValueType();
+     *     ...
+     * }
+     * </pre>
+     * </code>
+     */
+    MAP("map", null, new MapDataType(null));
 
 
     private static final Map<String, RecordFieldType> SIMPLE_NAME_MAP = new HashMap<String, RecordFieldType>();
@@ -235,11 +263,11 @@ public enum RecordFieldType {
     }
 
     /**
-     * Returns a Data Type that represents a "RECORD" or "ARRAY" type with the given schema.
+     * Returns a Data Type that represents an "ARRAY" type with the given element type.
      *
      * @param elementType the type of the arrays in the element
-     * @return a DataType that represents a Record or Array with the given schema, or <code>null</code> if this RecordFieldType
-     *         is not the RECORD or ARRAY type.
+     * @return a DataType that represents an Array with the given element type, or <code>null</code> if this RecordFieldType
+     *         is not the ARRAY type.
      */
     public DataType getArrayDataType(final DataType elementType) {
         if (this != ARRAY) {
@@ -287,6 +315,21 @@ public enum RecordFieldType {
         return new ChoiceDataType(list);
     }
 
+    /**
+     * Returns a Data Type that represents a "MAP" type with the given value type.
+     *
+     * @param valueDataType the type of the values in the map
+     * @return a DataType that represents a Map with the given value type, or <code>null</code> if this RecordFieldType
+     *         is not the MAP type.
+     */
+    public DataType getMapDataType(final DataType valueDataType) {
+        if (this != MAP) {
+            return null;
+        }
+
+        return new MapDataType(valueDataType);
+    }
+
 
     public static RecordFieldType of(final String typeString) {
       return SIMPLE_NAME_MAP.get(typeString);

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java
index 115fb51..367f2b0 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java
@@ -55,4 +55,25 @@ public interface RecordSchema {
      *         <code>null</code> if the schema does not contain a field with the given name
      */
     Optional<DataType> getDataType(String fieldName);
+
+    /**
+     * @return the textual representation of the schema, if one is available
+     */
+    Optional<String> getSchemaText();
+
+    /**
+     * @return the format of the schema text, if schema text is present
+     */
+    Optional<String> getSchemaFormat();
+
+    /**
+     * @param fieldName the name of the field
+     * @return an Optional RecordField for the field with the given name
+     */
+    Optional<RecordField> getField(String fieldName);
+
+    /**
+     * @return the SchemaIdentifier, which provides various attributes for identifying a schema
+     */
+    SchemaIdentifier getIdentifier();
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java
index be064ab..b6daab7 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java
@@ -157,6 +157,20 @@ public class ResultSetRecordSet implements RecordSet, Closeable {
             case Types.LONGVARBINARY:
             case Types.VARBINARY:
                 return RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType());
+            case Types.OTHER:
+                // If we have no records to inspect, we can't really know its schema so we simply use the default data type.
+                if (rs.isAfterLast()) {
+                    return RecordFieldType.RECORD.getDataType();
+                }
+
+                final Object obj = rs.getObject(columnIndex);
+                if (obj == null || !(obj instanceof Record)) {
+                    return RecordFieldType.RECORD.getDataType();
+                }
+
+                final Record record = (Record) obj;
+                final RecordSchema recordSchema = record.getSchema();
+                return RecordFieldType.RECORD.getRecordDataType(recordSchema);
             default:
                 return getFieldType(sqlType).getDataType();
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/SchemaIdentifier.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/SchemaIdentifier.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/SchemaIdentifier.java
new file mode 100644
index 0000000..b711952
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/SchemaIdentifier.java
@@ -0,0 +1,96 @@
+/*
+ * 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.record;
+
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.OptionalLong;
+
+public interface SchemaIdentifier {
+
+    /**
+     * @return the name of the schema, if one has been defined.
+     */
+    Optional<String> getName();
+
+    /**
+     * @return the identifier of the schema, if one has been defined.
+     */
+    OptionalLong getIdentifier();
+
+    /**
+     * @return the version of the schema, if one has been defined.
+     */
+    OptionalInt getVersion();
+
+
+    public static SchemaIdentifier EMPTY = new SchemaIdentifier() {
+        @Override
+        public Optional<String> getName() {
+            return Optional.empty();
+        }
+
+        @Override
+        public OptionalLong getIdentifier() {
+            return OptionalLong.empty();
+        }
+
+        @Override
+        public OptionalInt getVersion() {
+            return OptionalInt.empty();
+        }
+    };
+
+    public static SchemaIdentifier ofName(final String name) {
+        return new SchemaIdentifier() {
+            @Override
+            public Optional<String> getName() {
+                return Optional.ofNullable(name);
+            }
+
+            @Override
+            public OptionalLong getIdentifier() {
+                return OptionalLong.empty();
+            }
+
+            @Override
+            public OptionalInt getVersion() {
+                return OptionalInt.empty();
+            }
+        };
+    }
+
+    public static SchemaIdentifier of(final String name, final long identifier, final int version) {
+        return new SchemaIdentifier() {
+            @Override
+            public Optional<String> getName() {
+                return Optional.ofNullable(name);
+            }
+
+            @Override
+            public OptionalLong getIdentifier() {
+                return OptionalLong.of(identifier);
+            }
+
+            @Override
+            public OptionalInt getVersion() {
+                return OptionalInt.of(version);
+            }
+        };
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
index f507f23..0c21239 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
@@ -52,7 +52,7 @@ public class ArrayDataType extends DataType {
         if (obj == null) {
             return false;
         }
-        if (!(obj instanceof RecordDataType)) {
+        if (!(obj instanceof ArrayDataType)) {
             return false;
         }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java
index b74cdcc..038b147 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java
@@ -53,7 +53,7 @@ public class ChoiceDataType extends DataType {
         if (obj == null) {
             return false;
         }
-        if (!(obj instanceof RecordDataType)) {
+        if (!(obj instanceof ChoiceDataType)) {
             return false;
         }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
new file mode 100644
index 0000000..a85fb5e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
@@ -0,0 +1,67 @@
+/*
+ * 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.record.type;
+
+import java.util.Objects;
+
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+public class MapDataType extends DataType {
+    private final DataType valueType;
+
+    public MapDataType(final DataType elementType) {
+        super(RecordFieldType.MAP, null);
+        this.valueType = elementType;
+    }
+
+    public DataType getValueType() {
+        return valueType;
+    }
+
+    @Override
+    public RecordFieldType getFieldType() {
+        return RecordFieldType.MAP;
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 + 41 * getFieldType().hashCode() + 41 * (valueType == null ? 0 : valueType.hashCode());
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (obj == this) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (!(obj instanceof MapDataType)) {
+            return false;
+        }
+
+        final MapDataType other = (MapDataType) obj;
+        return getValueType().equals(other.getValueType()) && Objects.equals(valueType, other.valueType);
+    }
+
+    @Override
+    public String toString() {
+        return "MAP[" + valueType + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java
index f24d036..006d34c 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java
@@ -60,4 +60,9 @@ public class RecordDataType extends DataType {
         final RecordDataType other = (RecordDataType) obj;
         return getFieldType().equals(other.getFieldType()) && Objects.equals(childSchema, other.childSchema);
     }
+
+    @Override
+    public String toString() {
+        return RecordFieldType.RECORD.toString();
+    }
 }


[2/7] nifi git commit: NIFI-3682: This closes #1682. Add Schema Access Strategy and Schema Write Strategy Record Readers and Writers; bug fixes.

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java
new file mode 100644
index 0000000..4eec14e
--- /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/access/HortonworksAttributeSchemaReferenceStrategy.java
@@ -0,0 +1,116 @@
+/*
+ * 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.access;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public class HortonworksAttributeSchemaReferenceStrategy implements SchemaAccessStrategy {
+    private final Set<SchemaField> schemaFields;
+
+    public static final String SCHEMA_ID_ATTRIBUTE = "schema.identifier";
+    public static final String SCHEMA_VERSION_ATTRIBUTE = "schema.version";
+    public static final String SCHEMA_PROTOCOL_VERSION_ATTRIBUTE = "schema.protocol.version";
+
+    private final SchemaRegistry schemaRegistry;
+
+
+    public HortonworksAttributeSchemaReferenceStrategy(final SchemaRegistry schemaRegistry) {
+        this.schemaRegistry = schemaRegistry;
+
+        schemaFields = new HashSet<>();
+        schemaFields.add(SchemaField.SCHEMA_IDENTIFIER);
+        schemaFields.add(SchemaField.SCHEMA_VERSION);
+        schemaFields.addAll(schemaRegistry == null ? Collections.emptySet() : schemaRegistry.getSuppliedSchemaFields());
+    }
+
+    @Override
+    public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException, IOException {
+        final String schemaIdentifier = flowFile.getAttribute(SCHEMA_ID_ATTRIBUTE);
+        final String schemaVersion = flowFile.getAttribute(SCHEMA_VERSION_ATTRIBUTE);
+        final String schemaProtocol = flowFile.getAttribute(SCHEMA_PROTOCOL_VERSION_ATTRIBUTE);
+        if (schemaIdentifier == null || schemaVersion == null || schemaProtocol == null) {
+            throw new SchemaNotFoundException("Could not determine Schema for " + flowFile + " because it is missing one of the following three required attributes: "
+                + SCHEMA_ID_ATTRIBUTE + ", " + SCHEMA_VERSION_ATTRIBUTE + ", " + SCHEMA_PROTOCOL_VERSION_ATTRIBUTE);
+        }
+
+        if (!isNumber(schemaProtocol)) {
+            throw new SchemaNotFoundException("Could not determine Schema for " + flowFile + " because the " + SCHEMA_PROTOCOL_VERSION_ATTRIBUTE + " has a value of '"
+                + schemaProtocol + "', which is not a valid Protocol Version number");
+        }
+
+        final int protocol = Integer.parseInt(schemaProtocol);
+        if (protocol != 1) {
+            throw new SchemaNotFoundException("Could not determine Schema for " + flowFile + " because the " + SCHEMA_PROTOCOL_VERSION_ATTRIBUTE + " has a value of '"
+                + schemaProtocol + "', which is not a valid Protocol Version number. Expected Protocol Version to be 1.");
+        }
+
+        if (!isNumber(schemaIdentifier)) {
+            throw new SchemaNotFoundException("Could not determine Schema for " + flowFile + " because the " + SCHEMA_ID_ATTRIBUTE + " has a value of '"
+                + schemaProtocol + "', which is not a valid Schema Identifier number");
+        }
+
+        if (!isNumber(schemaVersion)) {
+            throw new SchemaNotFoundException("Could not determine Schema for " + flowFile + " because the " + SCHEMA_VERSION_ATTRIBUTE + " has a value of '"
+                + schemaProtocol + "', which is not a valid Schema Version number");
+        }
+
+        final long schemaId = Long.parseLong(schemaIdentifier);
+        final int version = Integer.parseInt(schemaVersion);
+
+        final RecordSchema schema = schemaRegistry.retrieveSchema(schemaId, version);
+        if (schema == null) {
+            throw new SchemaNotFoundException("Could not find a Schema in the Schema Registry with Schema Identifier '" + schemaId + "' and Version '" + version + "'");
+        }
+
+        return schema;
+    }
+
+    private static boolean isNumber(final String value) {
+        if (value == null) {
+            return false;
+        }
+
+        final String trimmed = value.trim();
+        if (value.isEmpty()) {
+            return false;
+        }
+
+        for (int i = 0; i < trimmed.length(); i++) {
+            final char c = value.charAt(i);
+            if (c > '9' || c < '0') {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    @Override
+    public Set<SchemaField> getSuppliedSchemaFields() {
+        return schemaFields;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java
new file mode 100644
index 0000000..f492ec4
--- /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/access/HortonworksAttributeSchemaReferenceWriter.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.access;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+
+public class HortonworksAttributeSchemaReferenceWriter implements SchemaAccessWriter {
+    private static final Set<SchemaField> requiredSchemaFields = EnumSet.of(SchemaField.SCHEMA_IDENTIFIER, SchemaField.SCHEMA_VERSION);
+    private static final int LATEST_PROTOCOL_VERSION = 1;
+
+    @Override
+    public void writeHeader(RecordSchema schema, OutputStream out) throws IOException {
+    }
+
+    @Override
+    public Map<String, String> getAttributes(final RecordSchema schema) {
+        final Map<String, String> attributes = new HashMap<>(4);
+        final SchemaIdentifier id = schema.getIdentifier();
+
+        final long schemaId = id.getIdentifier().getAsLong();
+        final int schemaVersion = id.getVersion().getAsInt();
+
+        attributes.put(HortonworksAttributeSchemaReferenceStrategy.SCHEMA_ID_ATTRIBUTE, String.valueOf(schemaId));
+        attributes.put(HortonworksAttributeSchemaReferenceStrategy.SCHEMA_VERSION_ATTRIBUTE, String.valueOf(schemaVersion));
+        attributes.put(HortonworksAttributeSchemaReferenceStrategy.SCHEMA_PROTOCOL_VERSION_ATTRIBUTE, String.valueOf(LATEST_PROTOCOL_VERSION));
+
+        return attributes;
+    }
+
+    @Override
+    public void validateSchema(final RecordSchema schema) throws SchemaNotFoundException {
+        final SchemaIdentifier id = schema.getIdentifier();
+        if (!id.getIdentifier().isPresent()) {
+            throw new SchemaNotFoundException("Cannot write Schema Reference as Attributes because it does not contain a Schema Identifier");
+        }
+        if (!id.getVersion().isPresent()) {
+            throw new SchemaNotFoundException("Cannot write Schema Reference as Attributes because it does not contain a Schema Version");
+        }
+    }
+
+    @Override
+    public Set<SchemaField> getRequiredSchemaFields() {
+        return requiredSchemaFields;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java
new file mode 100644
index 0000000..081e97c
--- /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/access/HortonworksEncodedSchemaReferenceStrategy.java
@@ -0,0 +1,77 @@
+/*
+ * 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.access;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.stream.io.StreamUtils;
+
+public class HortonworksEncodedSchemaReferenceStrategy implements SchemaAccessStrategy {
+    private static final int LATEST_PROTOCOL_VERSION = 1;
+
+    private final Set<SchemaField> schemaFields;
+    private final SchemaRegistry schemaRegistry;
+
+    public HortonworksEncodedSchemaReferenceStrategy(final SchemaRegistry schemaRegistry) {
+        this.schemaRegistry = schemaRegistry;
+
+        schemaFields = new HashSet<>();
+        schemaFields.add(SchemaField.SCHEMA_IDENTIFIER);
+        schemaFields.add(SchemaField.SCHEMA_VERSION);
+        schemaFields.addAll(schemaRegistry == null ? Collections.emptySet() : schemaRegistry.getSuppliedSchemaFields());
+    }
+
+    @Override
+    public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException, IOException {
+        final byte[] buffer = new byte[13];
+        try {
+            StreamUtils.fillBuffer(contentStream, buffer);
+        } catch (final IOException ioe) {
+            throw new SchemaNotFoundException("Could not read first 13 bytes from stream", ioe);
+        }
+
+        // This encoding follows the pattern that is provided for serializing data by the Hortonworks Schema Registry serializer
+        // as it is provided at:
+        // https://github.com/hortonworks/registry/blob/master/schema-registry/serdes/src/main/java/com/hortonworks/registries/schemaregistry/serdes/avro/AvroSnapshotSerializer.java
+        final ByteBuffer bb = ByteBuffer.wrap(buffer);
+        final int protocolVersion = bb.get();
+        if (protocolVersion != 1) {
+            throw new SchemaNotFoundException("Schema Encoding appears to be of an incompatible version. The latest known Protocol is Version "
+                + LATEST_PROTOCOL_VERSION + " but the data was encoded with version " + protocolVersion);
+        }
+
+        final long schemaId = bb.getLong();
+        final int schemaVersion = bb.getInt();
+
+        return schemaRegistry.retrieveSchema(schemaId, schemaVersion);
+    }
+
+    @Override
+    public Set<SchemaField> getSuppliedSchemaFields() {
+        return schemaFields;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java
new file mode 100644
index 0000000..bf6a9ea
--- /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/access/HortonworksEncodedSchemaReferenceWriter.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.schema.access;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.OptionalInt;
+import java.util.OptionalLong;
+import java.util.Set;
+
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+
+public class HortonworksEncodedSchemaReferenceWriter implements SchemaAccessWriter {
+    private static final Set<SchemaField> requiredSchemaFields = EnumSet.of(SchemaField.SCHEMA_IDENTIFIER, SchemaField.SCHEMA_VERSION);
+    private static final int LATEST_PROTOCOL_VERSION = 1;
+
+    @Override
+    public void writeHeader(final RecordSchema schema, final OutputStream out) throws IOException {
+        final SchemaIdentifier identifier = schema.getIdentifier();
+        final long id = identifier.getIdentifier().getAsLong();
+        final int version = identifier.getVersion().getAsInt();
+
+        // This decoding follows the pattern that is provided for serializing data by the Hortonworks Schema Registry serializer
+        // as it is provided at:
+        // https://github.com/hortonworks/registry/blob/master/schema-registry/serdes/src/main/java/com/hortonworks/registries/schemaregistry/serdes/avro/AvroSnapshotSerializer.java
+        final ByteBuffer bb = ByteBuffer.allocate(13);
+        bb.put((byte) LATEST_PROTOCOL_VERSION);
+        bb.putLong(id);
+        bb.putInt(version);
+
+        out.write(bb.array());
+    }
+
+    @Override
+    public Map<String, String> getAttributes(final RecordSchema schema) {
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public void validateSchema(RecordSchema schema) throws SchemaNotFoundException {
+        final SchemaIdentifier identifier = schema.getIdentifier();
+        final OptionalLong identifierOption = identifier.getIdentifier();
+        if (!identifierOption.isPresent()) {
+            throw new SchemaNotFoundException("Cannot write Encoded Schema Reference because the Schema Identifier is not known");
+        }
+
+        final OptionalInt versionOption = identifier.getVersion();
+        if (!versionOption.isPresent()) {
+            throw new SchemaNotFoundException("Cannot write Encoded Schema Reference because the Schema Version is not known");
+        }
+    }
+
+    @Override
+    public Set<SchemaField> getRequiredSchemaFields() {
+        return requiredSchemaFields;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java
new file mode 100644
index 0000000..6635e3d
--- /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/access/SchemaAccessStrategy.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.schema.access;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Set;
+
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public interface SchemaAccessStrategy {
+    /**
+     * Returns the schema for the given FlowFile using the supplied stream of content and configuration
+     *
+     * @param flowFile flowfile
+     * @param contentStream content of flowfile
+     * @param context configuration
+     * @return the RecordSchema for the FlowFile
+     */
+    RecordSchema getSchema(FlowFile flowFile, InputStream contentStream, ConfigurationContext context) throws SchemaNotFoundException, IOException;
+
+    /**
+     * @return the set of all Schema Fields that are supplied by the RecordSchema that is returned from {@link #getSchema(FlowFile, InputStream, ConfigurationContext)}.
+     */
+    Set<SchemaField> getSuppliedSchemaFields();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java
new file mode 100644
index 0000000..30a995c
--- /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/access/SchemaAccessWriter.java
@@ -0,0 +1,63 @@
+/*
+ * 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.access;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public interface SchemaAccessWriter {
+
+    /**
+     * Writes the given Record Schema to the given OutputStream as header information, if appropriate,
+     * or returns without writing anything if the implementation does not need to write information to
+     * the contents of the FlowFile
+     *
+     * @param schema the schema to write
+     * @param out the OutputStream to write to
+     * @throws IOException if unable to write to the given stream
+     */
+    void writeHeader(RecordSchema schema, OutputStream out) throws IOException;
+
+    /**
+     * Returns a Map of String to String that represent the attributes that should be added to the FlowFile, or
+     * an empty map if no attributes should be added.
+     *
+     * @return a Map of attributes to add to the FlowFile.
+     */
+    Map<String, String> getAttributes(RecordSchema schema);
+
+    /**
+     * Ensures that the given schema can be written by this SchemaAccessWriter or throws SchemaNotFoundException if
+     * the schema does not contain sufficient information to be written
+     *
+     * @param schema the schema to validate
+     * @throws SchemaNotFoundException if the schema does not contain sufficient information to be written
+     */
+    void validateSchema(RecordSchema schema) throws SchemaNotFoundException;
+
+    /**
+     * Specifies the set of SchemaField's that are required in order to use this Schema Access Writer
+     *
+     * @return the set of SchemaField's that are required in order to use this Schema Access Writer
+     */
+    Set<SchemaField> getRequiredSchemaFields();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java
new file mode 100644
index 0000000..54a248d
--- /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/access/SchemaNameAsAttribute.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.access;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+
+public class SchemaNameAsAttribute implements SchemaAccessWriter {
+    private static final Set<SchemaField> schemaFields = EnumSet.of(SchemaField.SCHEMA_NAME);
+    private static final String SCHEMA_NAME_ATTRIBUTE = "schema.name";
+
+    @Override
+    public void writeHeader(final RecordSchema schema, final OutputStream out) throws IOException {
+    }
+
+    @Override
+    public Map<String, String> getAttributes(final RecordSchema schema) {
+        final SchemaIdentifier identifier = schema.getIdentifier();
+        final Optional<String> nameOption = identifier.getName();
+        if (nameOption.isPresent()) {
+            return Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, nameOption.get());
+        }
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public void validateSchema(final RecordSchema schema) throws SchemaNotFoundException {
+        final SchemaIdentifier schemaId = schema.getIdentifier();
+        if (!schemaId.getName().isPresent()) {
+            throw new SchemaNotFoundException("Cannot write Schema Name As Attribute because the Schema Name is not known");
+        }
+    }
+
+    @Override
+    public Set<SchemaField> getRequiredSchemaFields() {
+        return schemaFields;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java
new file mode 100644
index 0000000..bc21c1d
--- /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/access/SchemaNamePropertyStrategy.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.access;
+
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public class SchemaNamePropertyStrategy implements SchemaAccessStrategy {
+    private final Set<SchemaField> schemaFields;
+
+    private final SchemaRegistry schemaRegistry;
+    private final PropertyValue schemaNamePropertyValue;
+
+    public SchemaNamePropertyStrategy(final SchemaRegistry schemaRegistry, final PropertyValue schemaNamePropertyValue) {
+        this.schemaRegistry = schemaRegistry;
+        this.schemaNamePropertyValue = schemaNamePropertyValue;
+
+        schemaFields = new HashSet<>();
+        schemaFields.add(SchemaField.SCHEMA_NAME);
+        schemaFields.addAll(schemaRegistry == null ? Collections.emptySet() : schemaRegistry.getSuppliedSchemaFields());
+    }
+
+    @Override
+    public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException {
+        final String schemaName = schemaNamePropertyValue.evaluateAttributeExpressions(flowFile).getValue();
+        if (schemaName.trim().isEmpty()) {
+            throw new SchemaNotFoundException("FlowFile did not contain appropriate attributes to determine Schema Name.");
+        }
+
+        try {
+            final RecordSchema recordSchema = schemaRegistry.retrieveSchema(schemaName);
+            if (recordSchema == null) {
+                throw new SchemaNotFoundException("Could not find a schema with name '" + schemaName + "' in the configured Schema Registry");
+            }
+
+            return recordSchema;
+        } catch (final Exception e) {
+            throw new SchemaNotFoundException("Could not retrieve schema with name '" + schemaName + "' from the configured Schema Registry", e);
+        }
+    }
+
+    @Override
+    public Set<SchemaField> getSuppliedSchemaFields() {
+        return schemaFields;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java
new file mode 100644
index 0000000..f39bdca
--- /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/access/SchemaTextAsAttribute.java
@@ -0,0 +1,60 @@
+/*
+ * 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.access;
+
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public class SchemaTextAsAttribute implements SchemaAccessWriter {
+    private static final Set<SchemaField> schemaFields = EnumSet.of(SchemaField.SCHEMA_TEXT, SchemaField.SCHEMA_TEXT_FORMAT);
+
+    @Override
+    public void writeHeader(final RecordSchema schema, final OutputStream out) {
+    }
+
+    @Override
+    public Map<String, String> getAttributes(final RecordSchema schema) {
+        final Optional<String> textFormatOption = schema.getSchemaFormat();
+        final Optional<String> textOption = schema.getSchemaText();
+        return Collections.singletonMap(textFormatOption.get() + ".schema", textOption.get());
+    }
+
+    @Override
+    public void validateSchema(final RecordSchema schema) throws SchemaNotFoundException {
+        final Optional<String> textFormatOption = schema.getSchemaFormat();
+        if (!textFormatOption.isPresent()) {
+            throw new SchemaNotFoundException("Cannot write Schema Text as Attribute because the Schema's Text Format is not present");
+        }
+
+        final Optional<String> textOption = schema.getSchemaText();
+        if (!textOption.isPresent()) {
+            throw new SchemaNotFoundException("Cannot write Schema Text as Attribute because the Schema's Text is not present");
+        }
+    }
+
+    @Override
+    public Set<SchemaField> getRequiredSchemaFields() {
+        return schemaFields;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java
index 5545090..2260c2e 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java
@@ -17,15 +17,14 @@
 
 package org.apache.nifi.serialization;
 
-import java.util.Arrays;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 
-public abstract class DateTimeTextRecordSetWriter extends AbstractControllerService {
+public abstract class DateTimeTextRecordSetWriter extends SchemaRegistryRecordSetWriter {
 
     private volatile String dateFormat;
     private volatile String timeFormat;
@@ -33,7 +32,11 @@ public abstract class DateTimeTextRecordSetWriter extends AbstractControllerServ
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return Arrays.asList(DateTimeUtils.DATE_FORMAT, DateTimeUtils.TIME_FORMAT, DateTimeUtils.TIMESTAMP_FORMAT);
+        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(DateTimeUtils.DATE_FORMAT);
+        properties.add(DateTimeUtils.TIME_FORMAT);
+        properties.add(DateTimeUtils.TIMESTAMP_FORMAT);
+        return properties;
     }
 
     @OnEnabled

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java
deleted file mode 100644
index ee25e64..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.nifi.serialization;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.nifi.annotation.lifecycle.OnEnabled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.PropertyValue;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.controller.AbstractControllerService;
-import org.apache.nifi.controller.ConfigurationContext;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.schemaregistry.services.SchemaRegistry;
-import org.apache.nifi.serialization.record.RecordSchema;
-
-public abstract class SchemaRegistryRecordReader extends AbstractControllerService {
-
-    protected static final PropertyDescriptor REQUIRED_SCHEMA_REGISTRY = new PropertyDescriptor.Builder()
-        .name("Schema Registry")
-        .description("Specifies the Controller Service to use for the Schema Registry")
-        .identifiesControllerService(SchemaRegistry.class)
-        .required(true)
-        .build();
-
-    protected static final PropertyDescriptor OPTIONAL_SCHEMA_REGISTRY = new PropertyDescriptor.Builder()
-        .fromPropertyDescriptor(REQUIRED_SCHEMA_REGISTRY)
-        .required(false)
-        .build();
-
-    protected static final PropertyDescriptor REQUIRED_SCHEMA_NAME = new PropertyDescriptor.Builder()
-        .name("Schema Name")
-        .description("Name of the Schema that is stored in the Schema Registry")
-        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(true)
-        .required(true)
-        .build();
-
-    protected static final PropertyDescriptor OPTIONAL_SCHEMA_NAME = new PropertyDescriptor.Builder()
-        .fromPropertyDescriptor(REQUIRED_SCHEMA_NAME)
-        .required(false)
-        .build();
-
-
-    private volatile SchemaRegistry schemaRegistry;
-    private volatile PropertyValue schemaName;
-
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        final List<PropertyDescriptor> properties = new ArrayList<>(2);
-        if (isSchemaRequired()) {
-            properties.add(REQUIRED_SCHEMA_REGISTRY);
-            properties.add(REQUIRED_SCHEMA_NAME);
-        } else {
-            properties.add(OPTIONAL_SCHEMA_REGISTRY);
-            properties.add(OPTIONAL_SCHEMA_NAME);
-        }
-
-        return properties;
-    }
-
-    @OnEnabled
-    public void storeRegistryValues(final ConfigurationContext context) {
-        schemaRegistry = context.getProperty(REQUIRED_SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class);
-        schemaName = context.getProperty(REQUIRED_SCHEMA_NAME);
-    }
-
-    public RecordSchema getSchema(final FlowFile flowFile) {
-        final String evaluatedSchemaName = schemaName.evaluateAttributeExpressions(flowFile).getValue();
-        final RecordSchema schema = schemaRegistry.retrieveSchema(evaluatedSchemaName);
-        return schema;
-    }
-
-    @Override
-    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        if (validationContext.getProperty(OPTIONAL_SCHEMA_REGISTRY).isSet() && !validationContext.getProperty(OPTIONAL_SCHEMA_NAME).isSet()) {
-            return Collections.singleton(new ValidationResult.Builder()
-                .subject("Schema Registry")
-                .explanation("If the Schema Registry is configured, the Schema name must also be configured")
-                .valid(false)
-                .build());
-        }
-
-        return Collections.emptyList();
-    }
-
-    protected boolean isSchemaRequired() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/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
new file mode 100644
index 0000000..c9daded
--- /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/SchemaRegistryRecordSetWriter.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+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.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.schema.access.HortonworksAttributeSchemaReferenceWriter;
+import org.apache.nifi.schema.access.HortonworksEncodedSchemaReferenceWriter;
+import org.apache.nifi.schema.access.SchemaAccessWriter;
+import org.apache.nifi.schema.access.SchemaField;
+import org.apache.nifi.schema.access.SchemaNameAsAttribute;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schema.access.SchemaTextAsAttribute;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryService {
+
+    static final AllowableValue SCHEMA_NAME_ATTRIBUTE = new AllowableValue("schema-name", "Set 'schema.name' Attribute",
+        "The FlowFile will be given an attribute named 'schema.name' and this attribute will indicate the name of the schema in the Schema Registry. Note that if"
+            + "the schema for a record is not obtained from a Schema Registry, then no attribute will be added.");
+    static final AllowableValue AVRO_SCHEMA_ATTRIBUTE = new AllowableValue("full-schema-attribute", "Set 'avro.schema' Attribute",
+        "The FlowFile will be given an attribute named 'avro.schema' and this attribute will contain the Avro Schema that describes the records in the FlowFile. "
+            + "The contents of the FlowFile need not be Avro, but the text of the schema will be used.");
+    static final AllowableValue HWX_CONTENT_ENCODED_SCHEMA = new AllowableValue("hwx-content-encoded-schema", "HWX Content-Encoded Schema Reference",
+        "The content of the FlowFile will contain a reference to a schema in the Schema Registry service. The reference is encoded as a single byte indicating the 'protocol version', "
+            + "followed by 8 bytes indicating the schema identifier, and finally 4 bytes indicating the schema version, as per the Hortonworks Schema Registry serializers and deserializers, "
+            + "as found at https://github.com/hortonworks/registry. "
+            + "This will be prepended to each FlowFile. Note that "
+            + "if the schema for a record does not contain the necessary identifier and version, an Exception will be thrown when attempting to write the data.");
+    static final AllowableValue HWX_SCHEMA_REF_ATTRIBUTES = new AllowableValue("hwx-schema-ref-attributes", "HWX Schema Reference Attributes",
+        "The FlowFile will be given a set of 3 attributes to describe the schema: 'schema.identifier', 'schema.version', and 'schema.protocol.version'. Note that if "
+            + "the schema for a record does not contain the necessary identifier and version, an Exception will be thrown when attempting to write the data.");
+
+    protected static final PropertyDescriptor SCHEMA_WRITE_STRATEGY = new PropertyDescriptor.Builder()
+        .name("Schema Write Strategy")
+        .description("Specifies how the schema for a Record should be added to the data.")
+        .allowableValues(SCHEMA_NAME_ATTRIBUTE, AVRO_SCHEMA_ATTRIBUTE, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA)
+        .defaultValue(AVRO_SCHEMA_ATTRIBUTE.getValue())
+        .required(true)
+        .build();
+
+
+    private volatile ConfigurationContext configurationContext;
+    private volatile SchemaAccessWriter schemaAccessWriter;
+
+    private final List<AllowableValue> strategyList = Collections.unmodifiableList(Arrays.asList(SCHEMA_NAME_ATTRIBUTE, AVRO_SCHEMA_ATTRIBUTE, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA));
+
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+
+        final AllowableValue[] strategies = getSchemaWriteStrategyValues().toArray(new AllowableValue[0]);
+        properties.add(new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(SCHEMA_WRITE_STRATEGY)
+            .defaultValue(getDefaultSchemaWriteStrategy().getValue())
+            .allowableValues(strategies)
+            .build());
+        properties.addAll(super.getSupportedPropertyDescriptors());
+
+        return properties;
+    }
+
+    protected AllowableValue getDefaultSchemaWriteStrategy() {
+        return AVRO_SCHEMA_ATTRIBUTE;
+    }
+
+    protected PropertyDescriptor getSchemaWriteStrategyDescriptor() {
+        return getPropertyDescriptor(SCHEMA_WRITE_STRATEGY.getName());
+    }
+
+    @OnEnabled
+    public void storeSchemaWriteStrategy(final ConfigurationContext context) {
+        this.configurationContext = context;
+
+        final String writerValue = context.getProperty(getSchemaWriteStrategyDescriptor()).getValue();
+        this.schemaAccessWriter = getSchemaWriteStrategy(writerValue);
+    }
+
+    @Override
+    protected ConfigurationContext getConfigurationContext() {
+        return configurationContext;
+    }
+
+    protected SchemaAccessWriter getSchemaAccessWriter(final RecordSchema schema) throws SchemaNotFoundException {
+        schemaAccessWriter.validateSchema(schema);
+        return schemaAccessWriter;
+    }
+
+    protected List<AllowableValue> getSchemaWriteStrategyValues() {
+        return strategyList;
+    }
+
+    protected SchemaAccessWriter getSchemaWriteStrategy(final String allowableValue) {
+        if (allowableValue.equalsIgnoreCase(SCHEMA_NAME_ATTRIBUTE.getValue())) {
+            return new SchemaNameAsAttribute();
+        } else if (allowableValue.equalsIgnoreCase(AVRO_SCHEMA_ATTRIBUTE.getValue())) {
+            return new SchemaTextAsAttribute();
+        } else if (allowableValue.equalsIgnoreCase(HWX_CONTENT_ENCODED_SCHEMA.getValue())) {
+            return new HortonworksEncodedSchemaReferenceWriter();
+        } else if (allowableValue.equalsIgnoreCase(HWX_SCHEMA_REF_ATTRIBUTES.getValue())) {
+            return new HortonworksAttributeSchemaReferenceWriter();
+        }
+
+        return null;
+    }
+
+    protected Set<SchemaField> getRequiredSchemaFields(final ValidationContext validationContext) {
+        final String writeStrategyValue = validationContext.getProperty(getSchemaWriteStrategyDescriptor()).getValue();
+        final SchemaAccessWriter writer = getSchemaWriteStrategy(writeStrategyValue);
+        final Set<SchemaField> requiredFields = writer.getRequiredSchemaFields();
+        return requiredFields;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
+
+        final Set<SchemaField> suppliedFields = getSuppliedSchemaFields(validationContext);
+        final Set<SchemaField> requiredFields = getRequiredSchemaFields(validationContext);
+
+        final Set<SchemaField> missingFields = new HashSet<>(requiredFields);
+        missingFields.removeAll(suppliedFields);
+
+        if (!missingFields.isEmpty()) {
+            results.add(new ValidationResult.Builder()
+                .subject("Schema Access Strategy")
+                .valid(false)
+                .explanation("The configured Schema Write Strategy requires the " + missingFields.iterator().next()
+                    + " but the configured Schema Access Strategy does not provide this information in conjunction with the selected Schema Registry. "
+                    + "This Schema Access Strategy, as configured, cannot be used in conjunction with this Schema Write Strategy.")
+                .build());
+        }
+
+        return results;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java
new file mode 100644
index 0000000..0988935
--- /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/SchemaRegistryService.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.avro.AvroSchemaValidator;
+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.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.AvroSchemaTextStrategy;
+import org.apache.nifi.schema.access.HortonworksAttributeSchemaReferenceStrategy;
+import org.apache.nifi.schema.access.HortonworksEncodedSchemaReferenceStrategy;
+import org.apache.nifi.schema.access.SchemaAccessStrategy;
+import org.apache.nifi.schema.access.SchemaField;
+import org.apache.nifi.schema.access.SchemaNamePropertyStrategy;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public abstract class SchemaRegistryService extends AbstractControllerService {
+
+    static final AllowableValue SCHEMA_NAME_PROPERTY = new AllowableValue("schema-name", "Use 'Schema Name' Property",
+        "The name of the Schema to use is specified by the 'Schema Name' Property. The value of this property is used to lookup the Schema in the configured Schema Registry service.");
+    static final AllowableValue SCHEMA_TEXT_PROPERTY = new AllowableValue("schema-text-property", "Use 'Schema Text' Property",
+        "The text of the Schema itself is specified by the 'Schema Text' Property. The value of this property must be a valid Avro Schema. "
+            + "If Expression Language is used, the value of the 'Schema Text' property must be valid after substituting the expressions.");
+    static final AllowableValue HWX_CONTENT_ENCODED_SCHEMA = new AllowableValue("hwx-content-encoded-schema", "HWX Content-Encoded Schema Reference",
+        "The content of the FlowFile contains a reference to a schema in the Schema Registry service. The reference is encoded as a single byte indicating the 'protocol version', "
+            + "followed by 8 bytes indicating the schema identifier, and finally 4 bytes indicating the schema version, as per the Hortonworks Schema Registry serializers and deserializers, "
+            + "found at https://github.com/hortonworks/registry");
+    static final AllowableValue HWX_SCHEMA_REF_ATTRIBUTES = new AllowableValue("hwx-schema-ref-attributes", "HWX Schema Reference Attributes",
+        "The FlowFile contains 3 Attributes that will be used to lookup a Schema from the configured Schema Registry: 'schema.identifier', 'schema.version', and 'schema.protocol.version'");
+
+    protected static final PropertyDescriptor SCHEMA_REGISTRY = new PropertyDescriptor.Builder()
+        .name("Schema Registry")
+        .description("Specifies the Controller Service to use for the Schema Registry")
+        .identifiesControllerService(SchemaRegistry.class)
+        .required(false)
+        .build();
+
+    protected static final PropertyDescriptor SCHEMA_ACCESS_STRATEGY = new PropertyDescriptor.Builder()
+        .name("Schema Access Strategy")
+        .description("Specifies how to obtain the schema that is to be used for interpreting the data.")
+        .allowableValues(SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA)
+        .defaultValue(SCHEMA_NAME_PROPERTY.getValue())
+        .required(true)
+        .build();
+
+    static final PropertyDescriptor SCHEMA_NAME = new PropertyDescriptor.Builder()
+        .name("Schema Name")
+        .description("Specifies the name of the schema to lookup in the Schema Registry property")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .defaultValue("${schema.name}")
+        .required(false)
+        .build();
+
+    static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor.Builder()
+        .name("schema-text")
+        .displayName("Schema Text")
+        .description("The text of an Avro-formatted Schema")
+        .addValidator(new AvroSchemaValidator())
+        .expressionLanguageSupported(true)
+        .defaultValue("${avro.schema}")
+        .required(false)
+        .build();
+
+
+    private volatile ConfigurationContext configurationContext;
+    private volatile SchemaAccessStrategy schemaAccessStrategy;
+
+    private final List<AllowableValue> strategyList = Collections.unmodifiableList(Arrays.asList(SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA));
+
+    private PropertyDescriptor getSchemaAcessStrategyDescriptor() {
+        return getPropertyDescriptor(SCHEMA_ACCESS_STRATEGY.getName());
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>(2);
+
+        final AllowableValue[] strategies = getSchemaAccessStrategyValues().toArray(new AllowableValue[0]);
+        properties.add(new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(SCHEMA_ACCESS_STRATEGY)
+            .allowableValues(strategies)
+            .defaultValue(getDefaultSchemaAccessStrategy().getValue())
+            .build());
+
+        properties.add(SCHEMA_REGISTRY);
+        properties.add(SCHEMA_NAME);
+        properties.add(SCHEMA_TEXT);
+
+        return properties;
+    }
+
+    protected AllowableValue getDefaultSchemaAccessStrategy() {
+        return SCHEMA_NAME_PROPERTY;
+    }
+
+    @OnEnabled
+    public void storeSchemaAccessStrategy(final ConfigurationContext context) {
+        this.configurationContext = context;
+
+        final SchemaRegistry schemaRegistry = context.getProperty(SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class);
+
+        final PropertyDescriptor descriptor = getSchemaAcessStrategyDescriptor();
+        final String schemaAccess = context.getProperty(descriptor).getValue();
+        this.schemaAccessStrategy = getSchemaAccessStrategy(schemaAccess, schemaRegistry);
+    }
+
+    protected ConfigurationContext getConfigurationContext() {
+        return configurationContext;
+    }
+
+    protected SchemaAccessStrategy getSchemaAccessStrategy() {
+        return schemaAccessStrategy;
+    }
+
+    public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream) throws SchemaNotFoundException, IOException {
+        return getSchemaAccessStrategy().getSchema(flowFile, contentStream, configurationContext);
+    }
+
+    private String getSchemaAccessStrategyName(final String schemaAccessValue) {
+        for (final AllowableValue allowableValue : getSchemaAcessStrategyDescriptor().getAllowableValues()) {
+            if (allowableValue.getValue().equalsIgnoreCase(schemaAccessValue)) {
+                return allowableValue.getDisplayName();
+            }
+        }
+
+        return null;
+    }
+
+    private boolean isSchemaRegistryRequired(final String schemaAccessValue) {
+        return HWX_CONTENT_ENCODED_SCHEMA.getValue().equalsIgnoreCase(schemaAccessValue) || SCHEMA_NAME_PROPERTY.getValue().equalsIgnoreCase(schemaAccessValue)
+            || HWX_SCHEMA_REF_ATTRIBUTES.getValue().equalsIgnoreCase(schemaAccessValue);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final String schemaAccessStrategy = validationContext.getProperty(getSchemaAcessStrategyDescriptor()).getValue();
+        if (isSchemaRegistryRequired(schemaAccessStrategy)) {
+            final boolean registrySet = validationContext.getProperty(SCHEMA_REGISTRY).isSet();
+            if (!registrySet) {
+                final String schemaAccessStrategyName = getSchemaAccessStrategyName(schemaAccessStrategy);
+
+                return Collections.singleton(new ValidationResult.Builder()
+                    .subject("Schema Registry")
+                    .explanation("The '" + schemaAccessStrategyName + "' Schema Access Strategy requires that the Schema Registry property be set.")
+                    .valid(false)
+                    .build());
+            }
+        }
+
+        return Collections.emptyList();
+    }
+
+    protected List<AllowableValue> getSchemaAccessStrategyValues() {
+        return strategyList;
+    }
+
+    protected Set<SchemaField> getSuppliedSchemaFields(final ValidationContext validationContext) {
+        final String accessStrategyValue = validationContext.getProperty(getSchemaAcessStrategyDescriptor()).getValue();
+        final SchemaRegistry schemaRegistry = validationContext.getProperty(SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class);
+        final SchemaAccessStrategy accessStrategy = getSchemaAccessStrategy(accessStrategyValue, schemaRegistry, validationContext);
+
+        final Set<SchemaField> suppliedFields = accessStrategy.getSuppliedSchemaFields();
+        return suppliedFields;
+    }
+
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry) {
+        if (allowableValue.equalsIgnoreCase(SCHEMA_NAME_PROPERTY.getValue())) {
+            return new SchemaNamePropertyStrategy(schemaRegistry, getConfigurationContext().getProperty(SCHEMA_NAME));
+        } else if (allowableValue.equalsIgnoreCase(SCHEMA_TEXT_PROPERTY.getValue())) {
+            return new AvroSchemaTextStrategy(getConfigurationContext().getProperty(SCHEMA_TEXT));
+        } else if (allowableValue.equalsIgnoreCase(HWX_CONTENT_ENCODED_SCHEMA.getValue())) {
+            return new HortonworksEncodedSchemaReferenceStrategy(schemaRegistry);
+        } else if (allowableValue.equalsIgnoreCase(HWX_SCHEMA_REF_ATTRIBUTES.getValue())) {
+            return new HortonworksAttributeSchemaReferenceStrategy(schemaRegistry);
+        }
+
+        return null;
+    }
+
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry, final ValidationContext context) {
+        if (allowableValue.equalsIgnoreCase(SCHEMA_NAME_PROPERTY.getValue())) {
+            return new SchemaNamePropertyStrategy(schemaRegistry, context.getProperty(SCHEMA_NAME));
+        } else if (allowableValue.equalsIgnoreCase(SCHEMA_TEXT_PROPERTY.getValue())) {
+            return new AvroSchemaTextStrategy(context.getProperty(SCHEMA_TEXT));
+        } else if (allowableValue.equalsIgnoreCase(HWX_CONTENT_ENCODED_SCHEMA.getValue())) {
+            return new HortonworksEncodedSchemaReferenceStrategy(schemaRegistry);
+        } else if (allowableValue.equalsIgnoreCase(HWX_SCHEMA_REF_ATTRIBUTES.getValue())) {
+            return new HortonworksAttributeSchemaReferenceStrategy(schemaRegistry);
+        }
+
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java
index 598a8c4..cb69444 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,6 +17,7 @@
 
 package org.apache.nifi.text;
 
+import java.io.InputStream;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.List;
@@ -28,6 +29,7 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.serialization.RecordSetWriter;
@@ -35,13 +37,12 @@ import org.apache.nifi.serialization.RecordSetWriterFactory;
 
 @Tags({"text", "freeform", "expression", "language", "el", "record", "recordset", "resultset", "writer", "serialize"})
 @CapabilityDescription("Writes the contents of a RecordSet as free-form text. The configured "
-    + "text is able to make use of the Expression Language to reference each of the columns that are available "
+    + "text is able to make use of the Expression Language to reference each of the fields that are available "
     + "in a Record. Each record in the RecordSet will be separated by a single newline character.")
 public class FreeFormTextRecordSetWriter extends AbstractControllerService implements RecordSetWriterFactory {
     static final PropertyDescriptor TEXT = new PropertyDescriptor.Builder()
         .name("Text")
-        .description("The text to use when writing the results. This property will evaluate the Expression Language using any of the columns available to the Result Set. For example, if the "
-            + "following SQL Query is used: \"SELECT Name, COUNT(*) AS Count\" then the Expression can reference \"Name\" and \"Count\", such as \"${Name:toUpper()} ${Count:minus(1)}\"")
+        .description("The text to use when writing the results. This property will evaluate the Expression Language using any of the fields available in a Record.")
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .expressionLanguageSupported(true)
         .required(true)
@@ -73,7 +74,7 @@ public class FreeFormTextRecordSetWriter extends AbstractControllerService imple
     }
 
     @Override
-    public RecordSetWriter createWriter(final ComponentLog logger) {
+    public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream in) {
         return new FreeFormTextWriter(textValue, characterSet);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextWriter.java
index 781f41f..7fdc7a4 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextWriter.java
@@ -20,8 +20,10 @@ package org.apache.nifi.text;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.charset.Charset;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.nifi.components.PropertyValue;
@@ -29,6 +31,7 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.WriteResult;
 import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 
@@ -48,7 +51,7 @@ public class FreeFormTextWriter implements RecordSetWriter {
 
         try {
             final RecordSchema schema = recordSet.getSchema();
-            final String[] colNames = getColumnNames(schema);
+            final List<String> colNames = getColumnNames(schema);
 
             Record record;
             while ((record = recordSet.next()) != null) {
@@ -62,11 +65,13 @@ public class FreeFormTextWriter implements RecordSetWriter {
         return WriteResult.of(count, Collections.emptyMap());
     }
 
-    private String[] getColumnNames(final RecordSchema schema) {
-        final int numCols = schema.getFieldCount();
-        final String[] columnNames = new String[numCols];
-        for (int i = 0; i < numCols; i++) {
-            columnNames[i] = schema.getField(i).getFieldName();
+    private List<String> getColumnNames(final RecordSchema schema) {
+        final List<String> columnNames = new ArrayList<>();
+        for (final RecordField field : schema.getFields()) {
+            columnNames.add(field.getFieldName());
+            for (final String alias : field.getAliases()) {
+                columnNames.add(alias);
+            }
         }
 
         return columnNames;
@@ -78,11 +83,11 @@ public class FreeFormTextWriter implements RecordSetWriter {
         return WriteResult.of(1, Collections.emptyMap());
     }
 
-    private void write(final Record record, final OutputStream out, final String[] columnNames) throws IOException {
-        final int numCols = columnNames.length;
+    private void write(final Record record, final OutputStream out, final List<String> columnNames) throws IOException {
+        final int numCols = columnNames.size();
         final Map<String, String> values = new HashMap<>(numCols);
         for (int i = 0; i < numCols; i++) {
-            final String columnName = columnNames[i];
+            final String columnName = columnNames.get(i);
             final String columnValue = record.getAsString(columnName);
             values.put(columnName, columnValue);
         }


[6/7] nifi git commit: NIFI-3682: This closes #1682. Add Schema Access Strategy and Schema Write Strategy Record Readers and Writers; bug fixes.

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..5982908
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java
@@ -0,0 +1,587 @@
+/*
+ * 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.processors.standard;
+
+import java.io.BufferedInputStream;
+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 org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+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;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.queryrecord.FlowFileTable;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.ResultSetRecordSet;
+import org.apache.nifi.util.StopWatch;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"sql", "query", "calcite", "route", "record", "transform", "select", "update", "modify", "etl", "filter", "record", "csv", "json", "logs", "text", "avro", "aggregate"})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@CapabilityDescription("Evaluates one or more SQL queries against the contents of a FlowFile. The result of the "
+    + "SQL query then becomes the content of the output FlowFile. This can be used, for example, "
+    + "for field-specific filtering, transformation, and row-level filtering. "
+    + "Columns can be renamed, simple calculations and aggregations performed, etc. "
+    + "The Processor is configured with a Record Reader Controller Service and a Record Writer service so as to allow flexibility in incoming and outgoing data formats. "
+    + "The Processor must be configured with at least one user-defined property. The name of the Property "
+    + "is the Relationship to route data to, and the value of the Property is a SQL SELECT statement that is used to specify how input data should be transformed/filtered. "
+    + "The SQL statement must be valid ANSI SQL and is powered by Apache Calcite. "
+    + "If the transformation fails, the original FlowFile is routed to the 'failure' relationship. Otherwise, the data selected will be routed to the associated "
+    + "relationship. See the Processor Usage documentation for more information.")
+@DynamicRelationship(name="<Property Name>", description="Each user-defined property defines a new Relationship for this Processor.")
+@DynamicProperty(name = "The name of the relationship to route data to", value="A SQL SELECT statement that is used to determine what data should be routed to this "
+        + "relationship.", supportsExpressionLanguage=true, description="Each user-defined property specifies a SQL SELECT statement to run over the data, with the data "
+        + "that is selected being routed to the relationship whose name is the property name")
+public class QueryRecord extends AbstractProcessor {
+    static final PropertyDescriptor RECORD_READER_FACTORY = new PropertyDescriptor.Builder()
+        .name("record-reader")
+        .displayName("Record Reader")
+        .description("Specifies the Controller Service to use for parsing incoming data and determining the data's schema")
+        .identifiesControllerService(RecordReaderFactory.class)
+        .required(true)
+        .build();
+    static final PropertyDescriptor RECORD_WRITER_FACTORY = new PropertyDescriptor.Builder()
+        .name("record-writer")
+        .displayName("Record Writer")
+        .description("Specifies the Controller Service to use for writing results to a FlowFile")
+        .identifiesControllerService(RecordSetWriterFactory.class)
+        .required(true)
+        .build();
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+        .name("include-zero-record-flowfiles")
+        .displayName("Include Zero Record FlowFiles")
+        .description("When running the SQL statement against an incoming FlowFile, if the result has no data, "
+            + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+        .expressionLanguageSupported(false)
+        .allowableValues("true", "false")
+        .defaultValue("true")
+        .required(true)
+        .build();
+    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.")
+        .expressionLanguageSupported(false)
+        .allowableValues("true", "false")
+        .defaultValue("true")
+        .required(true)
+        .build();
+
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+        .name("original")
+        .description("The original FlowFile is routed to this relationship")
+        .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("If a FlowFile fails processing for any reason (for example, the SQL "
+            + "statement contains columns not present in input data), the original FlowFile it will "
+            + "be routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> properties;
+    private final Set<Relationship> relationships = Collections.synchronizedSet(new HashSet<>());
+
+    private final Map<String, BlockingQueue<CachedStatement>> statementQueues = new HashMap<>();
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        try {
+            DriverManager.registerDriver(new org.apache.calcite.jdbc.Driver());
+        } catch (final SQLException e) {
+            throw new ProcessException("Failed to load Calcite JDBC Driver", e);
+        }
+
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(RECORD_READER_FACTORY);
+        properties.add(RECORD_WRITER_FACTORY);
+        properties.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        properties.add(CACHE_SCHEMA);
+        this.properties = Collections.unmodifiableList(properties);
+
+        relationships.add(REL_FAILURE);
+        relationships.add(REL_ORIGINAL);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (!descriptor.isDynamic()) {
+            return;
+        }
+
+        final Relationship relationship = new Relationship.Builder()
+            .name(descriptor.getName())
+            .description("User-defined relationship that specifies where data that matches the specified SQL query should be routed")
+            .build();
+
+        if (newValue == null) {
+            relationships.remove(relationship);
+        } else {
+            relationships.add(relationship);
+        }
+    }
+
+    @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)
+            .description("SQL select statement specifies how data should be filtered/transformed. "
+                + "SQL SELECT should select from the FLOWFILE table")
+            .required(false)
+            .dynamic(true)
+            .expressionLanguageSupported(true)
+            .addValidator(new SqlValidator())
+            .build();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        final FlowFile original = session.get();
+        if (original == null) {
+            return;
+        }
+
+        final StopWatch stopWatch = new StopWatch(true);
+
+        final RecordSetWriterFactory resultSetWriterFactory = context.getProperty(RECORD_WRITER_FACTORY)
+            .asControllerService(RecordSetWriterFactory.class);
+        final RecordReaderFactory recordParserFactory = context.getProperty(RECORD_READER_FACTORY)
+            .asControllerService(RecordReaderFactory.class);
+
+        final Map<FlowFile, Relationship> transformedFlowFiles = new HashMap<>();
+        final Set<FlowFile> createdFlowFiles = new HashSet<>();
+
+        int recordsRead = 0;
+
+        try {
+            final RecordSetWriter resultSetWriter;
+            try (final InputStream rawIn = session.read(original);
+                final InputStream in = new BufferedInputStream(rawIn)) {
+                resultSetWriter = resultSetWriterFactory.createWriter(getLogger(), original, in);
+            }
+
+            for (final PropertyDescriptor descriptor : context.getProperties().keySet()) {
+                if (!descriptor.isDynamic()) {
+                    continue;
+                }
+
+                final Relationship relationship = new Relationship.Builder().name(descriptor.getName()).build();
+
+                // We have to fork a child because we may need to read the input FlowFile more than once,
+                // and we cannot call session.read() on the original FlowFile while we are within a write
+                // callback for the original FlowFile.
+                FlowFile transformed = session.create(original);
+                boolean flowFileRemoved = false;
+
+                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, recordParserFactory);
+                    } else {
+                        queryResult = query(session, original, sql, context, recordParserFactory);
+                    }
+
+                    try {
+                        final ResultSet rs = queryResult.getResultSet();
+                        transformed = session.write(transformed, new OutputStreamCallback() {
+                            @Override
+                            public void process(final OutputStream out) throws IOException {
+                                try {
+                                    final ResultSetRecordSet recordSet = new ResultSetRecordSet(rs);
+                                    writeResultRef.set(resultSetWriter.write(recordSet, out));
+                                } catch (final Exception e) {
+                                    throw new IOException(e);
+                                }
+                            }
+                        });
+                    } finally {
+                        closeQuietly(queryResult);
+                    }
+
+                    recordsRead = Math.max(recordsRead, queryResult.getRecordsRead());
+                    final WriteResult result = writeResultRef.get();
+                    if (result.getRecordCount() == 0 && !context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean()) {
+                        session.remove(transformed);
+                        flowFileRemoved = true;
+                        transformedFlowFiles.remove(transformed);
+                        getLogger().info("Transformed {} but the result contained no data so will not pass on a FlowFile", new Object[] {original});
+                    } else {
+                        final Map<String, String> attributesToAdd = new HashMap<>();
+                        if (result.getAttributes() != null) {
+                            attributesToAdd.putAll(result.getAttributes());
+                        }
+
+                        attributesToAdd.put(CoreAttributes.MIME_TYPE.key(), resultSetWriter.getMimeType());
+                        attributesToAdd.put("record.count", String.valueOf(result.getRecordCount()));
+                        transformed = session.putAllAttributes(transformed, attributesToAdd);
+                        transformedFlowFiles.put(transformed, relationship);
+
+                        session.adjustCounter("Records Written", result.getRecordCount(), false);
+                    }
+                } finally {
+                    // Ensure that we have the FlowFile in the set in case we throw any Exception
+                    if (!flowFileRemoved) {
+                        createdFlowFiles.add(transformed);
+                    }
+                }
+            }
+
+            final long elapsedMillis = stopWatch.getElapsed(TimeUnit.MILLISECONDS);
+            if (transformedFlowFiles.size() > 0) {
+                session.getProvenanceReporter().fork(original, transformedFlowFiles.keySet(), elapsedMillis);
+
+                for (final Map.Entry<FlowFile, Relationship> entry : transformedFlowFiles.entrySet()) {
+                    final FlowFile transformed = entry.getKey();
+                    final Relationship relationship = entry.getValue();
+
+                    session.getProvenanceReporter().route(transformed, relationship);
+                    session.transfer(transformed, relationship);
+                }
+            }
+
+            getLogger().info("Successfully queried {} in {} millis", new Object[] {original, elapsedMillis});
+            session.transfer(original, REL_ORIGINAL);
+        } catch (final SQLException e) {
+            getLogger().error("Unable to query {} due to {}", new Object[] {original, e.getCause() == null ? e : e.getCause()});
+            session.remove(createdFlowFiles);
+            session.transfer(original, REL_FAILURE);
+        } catch (final Exception e) {
+            getLogger().error("Unable to query {} due to {}", new Object[] {original, e});
+            session.remove(createdFlowFiles);
+            session.transfer(original, REL_FAILURE);
+        }
+
+        session.adjustCounter("Records Read", recordsRead, false);
+    }
+
+    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);
+        }
+
+        final CachedStatement cachedStmt = statementQueue.poll();
+        if (cachedStmt != null) {
+            return cachedStmt;
+        }
+
+        return buildCachedStatement(sql, connectionSupplier, session, flowFile, recordReaderFactory);
+    }
+
+    private CachedStatement buildCachedStatement(final String sql, final Supplier<CalciteConnection> connectionSupplier, final ProcessSession session,
+        final FlowFile flowFile, final RecordReaderFactory recordReaderFactory) throws SQLException {
+
+        final CalciteConnection connection = connectionSupplier.get();
+        final SchemaPlus rootSchema = connection.getRootSchema();
+
+        final FlowFileTable<?, ?> flowFileTable = new FlowFileTable<>(session, flowFile, 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());
+            }
+        }
+
+        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);
+        }
+    }
+
+    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());
+
+            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 CachedStatement cachedStatement = getStatement(sql, connectionSupplier, session, flowFile, recordParserFactory);
+        final PreparedStatement stmt = cachedStatement.getStatement();
+        final FlowFileTable<?, ?> table = cachedStatement.getTable();
+        table.setFlowFile(session, flowFile);
+
+        final ResultSet rs = stmt.executeQuery();
+
+        return new QueryResult() {
+            @Override
+            public void close() throws IOException {
+                final BlockingQueue<CachedStatement> statementQueue = statementQueues.get(sql);
+                if (statementQueue == null || !statementQueue.offer(cachedStatement)) {
+                    try {
+                        cachedStatement.getConnection().close();
+                    } catch (SQLException e) {
+                        throw new IOException("Failed to close statement", e);
+                    }
+                }
+            }
+
+            @Override
+            public ResultSet getResultSet() {
+                return rs;
+            }
+
+            @Override
+            public int getRecordsRead() {
+                return table.getRecordsRead();
+            }
+
+        };
+    }
+
+    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.JAVA.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();
+            resultSet = statement.executeQuery(sql);
+
+            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) {
+            return;
+        }
+
+        for (final AutoCloseable closeable : closeables) {
+            if (closeable == null) {
+                continue;
+            }
+
+            try {
+                closeable.close();
+            } catch (final Exception e) {
+                getLogger().warn("Failed to close SQL resource", e);
+            }
+        }
+    }
+
+    private static class SqlValidator implements Validator {
+        @Override
+        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+            if (context.isExpressionLanguagePresent(input)) {
+                return new ValidationResult.Builder()
+                    .input(input)
+                    .subject(subject)
+                    .valid(true)
+                    .explanation("Expression Language Present")
+                    .build();
+            }
+
+            final String substituted = context.newPropertyValue(input).evaluateAttributeExpressions().getValue();
+            final SqlParser parser = SqlParser.create(substituted);
+            try {
+                parser.parseStmt();
+                return new ValidationResult.Builder()
+                    .subject(subject)
+                    .input(input)
+                    .valid(true)
+                    .build();
+            } catch (final Exception e) {
+                return new ValidationResult.Builder()
+                    .subject(subject)
+                    .input(input)
+                    .valid(false)
+                    .explanation("Not a valid SQL Statement: " + e.getMessage())
+                    .build();
+            }
+        }
+    }
+
+    private static interface QueryResult extends Closeable {
+        ResultSet getResultSet();
+
+        int getRecordsRead();
+    }
+
+    private static class CachedStatement {
+        private final FlowFileTable<?, ?> table;
+        private final PreparedStatement statement;
+        private 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() {
+            return table;
+        }
+
+        public PreparedStatement getStatement() {
+            return statement;
+        }
+
+        public Connection getConnection() {
+            return connection;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileEnumerator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileEnumerator.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileEnumerator.java
deleted file mode 100644
index 7daa002..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileEnumerator.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.nifi.queryflowfile;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.calcite.linq4j.Enumerator;
-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.MalformedRecordException;
-import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
-import org.apache.nifi.serialization.record.Record;
-
-public class FlowFileEnumerator<InternalType> implements Enumerator<Object> {
-    private final ProcessSession session;
-    private final FlowFile flowFile;
-    private final ComponentLog logger;
-    private final RowRecordReaderFactory recordParserFactory;
-    private final int[] fields;
-
-    private InputStream rawIn;
-    private Object currentRow;
-    private RecordReader recordParser;
-
-    public FlowFileEnumerator(final ProcessSession session, final FlowFile flowFile, final ComponentLog logger, final RowRecordReaderFactory parserFactory, final int[] fields) {
-        this.session = session;
-        this.flowFile = flowFile;
-        this.recordParserFactory = parserFactory;
-        this.logger = logger;
-        this.fields = fields;
-        reset();
-    }
-
-    @Override
-    public Object current() {
-        return currentRow;
-    }
-
-    @Override
-    public boolean moveNext() {
-        currentRow = null;
-        while (currentRow == null) {
-            try {
-                currentRow = filterColumns(recordParser.nextRecord());
-                break;
-            } catch (final IOException e) {
-                logger.error("Failed to read next record in stream for " + flowFile + ". Assuming end of stream.", e);
-                currentRow = null;
-                break;
-            } catch (final MalformedRecordException mre) {
-                logger.error("Failed to parse record in stream for " + flowFile + ". Will skip record and continue reading", mre);
-            }
-        }
-
-        if (currentRow == null) {
-            // If we are out of data, close the InputStream. We do this because
-            // Calcite does not necessarily call our close() method.
-            close();
-        }
-        return (currentRow != null);
-    }
-
-    private Object filterColumns(final Record record) {
-        if (record == null) {
-            return null;
-        }
-
-        final Object[] row = record.getValues();
-
-        // If we want no fields or if the row is null, just return null
-        if (fields == null || row == null) {
-            return row;
-        }
-
-        // If we want only a single field, then Calcite is going to expect us to return
-        // the actual value, NOT a 1-element array of values.
-        if (fields.length == 1) {
-            final int desiredCellIndex = fields[0];
-            return row[desiredCellIndex];
-        }
-
-        // Create a new Object array that contains only the desired fields.
-        final Object[] filtered = new Object[fields.length];
-        for (int i = 0; i < fields.length; i++) {
-            final int indexToKeep = fields[i];
-            filtered[i] = row[indexToKeep];
-        }
-
-        return filtered;
-    }
-
-    @Override
-    public void reset() {
-        if (rawIn != null) {
-            try {
-                rawIn.close();
-            } catch (final Exception e) {
-                logger.warn("Could not close FlowFile's input due to " + e, e);
-            }
-        }
-
-        rawIn = session.read(flowFile);
-
-        try {
-            recordParser = recordParserFactory.createRecordReader(flowFile, rawIn, logger);
-        } catch (final MalformedRecordException | IOException e) {
-            throw new ProcessException("Failed to reset stream", e);
-        }
-    }
-
-    @Override
-    public void close() {
-        if (recordParser != null) {
-            try {
-                recordParser.close();
-            } catch (final Exception e) {
-                logger.warn("Failed to close decorated source for " + flowFile, e);
-            }
-        }
-
-        try {
-            rawIn.close();
-        } catch (final Exception e) {
-            logger.warn("Failed to close InputStream for " + flowFile, e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileProjectTableScanRule.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileProjectTableScanRule.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileProjectTableScanRule.java
deleted file mode 100644
index c5179c9..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileProjectTableScanRule.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.queryflowfile;
-
-import java.util.List;
-
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * Planner rule that projects from a {@link FlowFileTableScan} scan just the columns
- * needed to satisfy a projection. If the projection's expressions are trivial,
- * the projection is removed.
- */
-public class FlowFileProjectTableScanRule extends RelOptRule {
-    public static final FlowFileProjectTableScanRule INSTANCE = new FlowFileProjectTableScanRule();
-
-    private FlowFileProjectTableScanRule() {
-        super(
-            operand(LogicalProject.class,
-                operand(FlowFileTableScan.class, none())),
-            "FlowFileProjectTableScanRule");
-    }
-
-    @Override
-    public void onMatch(RelOptRuleCall call) {
-        final LogicalProject project = call.rel(0);
-        final FlowFileTableScan scan = call.rel(1);
-        final int[] fields = getProjectFields(project.getProjects());
-
-        if (fields == null) {
-            // Project contains expressions more complex than just field references.
-            return;
-        }
-
-        call.transformTo(
-            new FlowFileTableScan(
-                scan.getCluster(),
-                scan.getTable(),
-                scan.flowFileTable,
-                fields));
-    }
-
-    private int[] getProjectFields(List<RexNode> exps) {
-        final int[] fields = new int[exps.size()];
-
-        for (int i = 0; i < exps.size(); i++) {
-            final RexNode exp = exps.get(i);
-
-            if (exp instanceof RexInputRef) {
-                fields[i] = ((RexInputRef) exp).getIndex();
-            } else {
-                return null; // not a simple projection
-            }
-        }
-
-        return fields;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTable.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTable.java
deleted file mode 100644
index 27f0c42..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTable.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.queryflowfile;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.lang.reflect.Type;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.linq4j.AbstractEnumerable;
-import org.apache.calcite.linq4j.Enumerable;
-import org.apache.calcite.linq4j.Enumerator;
-import org.apache.calcite.linq4j.QueryProvider;
-import org.apache.calcite.linq4j.Queryable;
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.schema.QueryableTable;
-import org.apache.calcite.schema.Schema.TableType;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Schemas;
-import org.apache.calcite.schema.TranslatableTable;
-import org.apache.calcite.schema.impl.AbstractTable;
-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.MalformedRecordException;
-import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
-import org.apache.nifi.serialization.record.DataType;
-import org.apache.nifi.serialization.record.RecordField;
-import org.apache.nifi.serialization.record.RecordSchema;
-
-
-public class FlowFileTable<S, E> extends AbstractTable implements QueryableTable, TranslatableTable {
-
-    private final RowRecordReaderFactory recordParserFactory;
-    private final ComponentLog logger;
-
-    private RecordSchema recordSchema;
-    private RelDataType relDataType = null;
-
-    private volatile ProcessSession session;
-    private volatile FlowFile flowFile;
-
-    /**
-     * Creates a FlowFile table.
-     */
-    public FlowFileTable(final ProcessSession session, final FlowFile flowFile, final RowRecordReaderFactory recordParserFactory, final ComponentLog logger) {
-        this.session = session;
-        this.flowFile = flowFile;
-        this.recordParserFactory = recordParserFactory;
-        this.logger = logger;
-    }
-
-    public void setFlowFile(final ProcessSession session, final FlowFile flowFile) {
-        this.session = session;
-        this.flowFile = flowFile;
-    }
-
-
-    @Override
-    public String toString() {
-        return "FlowFileTable";
-    }
-
-    /**
-     * Returns an enumerable over a given projection of the fields.
-     *
-     * <p>
-     * Called from generated code.
-     */
-    public Enumerable<Object> project(final int[] fields) {
-        return new AbstractEnumerable<Object>() {
-            @Override
-            @SuppressWarnings({"unchecked", "rawtypes"})
-            public Enumerator<Object> enumerator() {
-                return new FlowFileEnumerator(session, flowFile, logger, recordParserFactory, fields);
-            }
-        };
-    }
-
-    @Override
-    @SuppressWarnings("rawtypes")
-    public Expression getExpression(final SchemaPlus schema, final String tableName, final Class clazz) {
-        return Schemas.tableExpression(schema, getElementType(), tableName, clazz);
-    }
-
-    @Override
-    public Type getElementType() {
-        return Object[].class;
-    }
-
-    @Override
-    public <T> Queryable<T> asQueryable(final QueryProvider queryProvider, final SchemaPlus schema, final String tableName) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public RelNode toRel(final RelOptTable.ToRelContext context, final RelOptTable relOptTable) {
-        // Request all fields.
-        final int fieldCount = relOptTable.getRowType().getFieldCount();
-        final int[] fields = new int[fieldCount];
-        for (int i = 0; i < fieldCount; i++) {
-            fields[i] = i;
-        }
-
-        return new FlowFileTableScan(context.getCluster(), relOptTable, this, fields);
-    }
-
-    @Override
-    public RelDataType getRowType(final RelDataTypeFactory typeFactory) {
-        if (relDataType != null) {
-            return relDataType;
-        }
-
-        RecordSchema schema;
-        try (final InputStream in = session.read(flowFile)) {
-            final RecordReader recordParser = recordParserFactory.createRecordReader(flowFile, in, logger);
-            schema = recordParser.getSchema();
-        } catch (final MalformedRecordException | IOException 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()) {
-            names.add(field.getFieldName());
-            types.add(getRelDataType(field.getDataType(), javaTypeFactory));
-        }
-
-        logger.debug("Found Schema: {}", new Object[] {schema});
-
-        if (recordSchema == null) {
-            recordSchema = schema;
-        }
-
-        relDataType = typeFactory.createStructType(Pair.zip(names, types));
-        return relDataType;
-    }
-
-    private RelDataType getRelDataType(final DataType fieldType, final JavaTypeFactory typeFactory) {
-        switch (fieldType.getFieldType()) {
-            case BOOLEAN:
-                return typeFactory.createJavaType(boolean.class);
-            case BYTE:
-                return typeFactory.createJavaType(byte.class);
-            case CHAR:
-                return typeFactory.createJavaType(char.class);
-            case DATE:
-                return typeFactory.createJavaType(java.sql.Date.class);
-            case DOUBLE:
-                return typeFactory.createJavaType(double.class);
-            case FLOAT:
-                return typeFactory.createJavaType(float.class);
-            case INT:
-                return typeFactory.createJavaType(int.class);
-            case SHORT:
-                return typeFactory.createJavaType(short.class);
-            case TIME:
-                return typeFactory.createJavaType(java.sql.Time.class);
-            case TIMESTAMP:
-                return typeFactory.createJavaType(java.sql.Timestamp.class);
-            case LONG:
-                return typeFactory.createJavaType(long.class);
-            case STRING:
-                return typeFactory.createJavaType(String.class);
-            case ARRAY:
-                return typeFactory.createJavaType(Object[].class);
-            case RECORD:
-                return typeFactory.createJavaType(Object.class);
-        }
-
-        throw new IllegalArgumentException("Unknown Record Field Type: " + fieldType);
-    }
-
-    @Override
-    public TableType getJdbcTableType() {
-        return TableType.TEMPORARY_TABLE;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTableScan.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTableScan.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTableScan.java
deleted file mode 100644
index ad3a1c3..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTableScan.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.queryflowfile;
-
-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;
-import org.apache.calcite.adapter.enumerable.PhysType;
-import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
-import org.apache.calcite.linq4j.tree.Blocks;
-import org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.calcite.linq4j.tree.Primitive;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeField;
-
-/**
- * Relational expression representing a scan of a FlowFile.
- *
- * <p>
- * Like any table scan, it serves as a leaf node of a query tree.
- * </p>
- */
-public class FlowFileTableScan extends TableScan implements EnumerableRel {
-    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;
-        this.fields = fields;
-    }
-
-    @Override
-    public RelNode copy(final RelTraitSet traitSet, final List<RelNode> inputs) {
-        return new FlowFileTableScan(getCluster(), table, flowFileTable, fields);
-    }
-
-    @Override
-    public RelWriter explainTerms(final RelWriter pw) {
-        return super.explainTerms(pw).item("fields", Primitive.asList(fields));
-    }
-
-    @Override
-    public RelDataType deriveRowType() {
-        final List<RelDataTypeField> fieldList = table.getRowType().getFieldList();
-        final RelDataTypeFactory.FieldInfoBuilder builder = getCluster().getTypeFactory().builder();
-        for (int field : fields) {
-            builder.add(fieldList.get(field));
-        }
-        return builder.build();
-    }
-
-    @Override
-    public void register(RelOptPlanner planner) {
-        planner.addRule(FlowFileProjectTableScanRule.INSTANCE);
-    }
-
-    @Override
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-        PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), getRowType(), pref.preferArray());
-
-        return implementor.result(physType, Blocks.toBlock(
-            Expressions.call(table.getExpression(FlowFileTable.class), "project", Expressions.constant(fields))));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..e7b2c26
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java
@@ -0,0 +1,157 @@
+/*
+ * 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.queryrecord;
+
+import java.io.InputStream;
+
+import org.apache.calcite.linq4j.Enumerator;
+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.Record;
+
+public class FlowFileEnumerator<InternalType> implements Enumerator<Object> {
+    private final ProcessSession session;
+    private final FlowFile flowFile;
+    private final ComponentLog logger;
+    private final RecordReaderFactory recordParserFactory;
+    private final int[] fields;
+
+    private InputStream rawIn;
+    private Object currentRow;
+    private RecordReader recordParser;
+    private int recordsRead = 0;
+
+    public FlowFileEnumerator(final ProcessSession session, final FlowFile flowFile, final ComponentLog logger, final RecordReaderFactory parserFactory, final int[] fields) {
+        this.session = session;
+        this.flowFile = flowFile;
+        this.recordParserFactory = parserFactory;
+        this.logger = logger;
+        this.fields = fields;
+        reset();
+    }
+
+    @Override
+    public Object current() {
+        return currentRow;
+    }
+
+    @Override
+    public boolean moveNext() {
+        currentRow = null;
+        while (currentRow == null) {
+            try {
+                currentRow = filterColumns(recordParser.nextRecord());
+                break;
+            } catch (final Exception e) {
+                throw new ProcessException("Failed to read next record in stream for " + flowFile, e);
+            }
+        }
+
+        if (currentRow == null) {
+            // If we are out of data, close the InputStream. We do this because
+            // Calcite does not necessarily call our close() method.
+            close();
+            try {
+                onFinish();
+            } catch (final Exception e) {
+                logger.error("Failed to perform tasks when enumerator was finished", e);
+            }
+
+            return false;
+        }
+
+        recordsRead++;
+        return true;
+    }
+
+    protected int getRecordsRead() {
+        return recordsRead;
+    }
+
+    protected void onFinish() {
+    }
+
+    private Object filterColumns(final Record record) {
+        if (record == null) {
+            return null;
+        }
+
+        final Object[] row = record.getValues();
+
+        // If we want no fields or if the row is null, just return null
+        if (fields == null || row == null) {
+            return row;
+        }
+
+        // If we want only a single field, then Calcite is going to expect us to return
+        // the actual value, NOT a 1-element array of values.
+        if (fields.length == 1) {
+            final int desiredCellIndex = fields[0];
+            return row[desiredCellIndex];
+        }
+
+        // Create a new Object array that contains only the desired fields.
+        final Object[] filtered = new Object[fields.length];
+        for (int i = 0; i < fields.length; i++) {
+            final int indexToKeep = fields[i];
+            filtered[i] = row[indexToKeep];
+        }
+
+        return filtered;
+    }
+
+    @Override
+    public void reset() {
+        if (rawIn != null) {
+            try {
+                rawIn.close();
+            } catch (final Exception e) {
+                logger.warn("Could not close FlowFile's input due to " + e, e);
+            }
+        }
+
+        rawIn = session.read(flowFile);
+
+        try {
+            recordParser = recordParserFactory.createRecordReader(flowFile, rawIn, logger);
+        } catch (final Exception e) {
+            throw new ProcessException("Failed to reset stream", e);
+        }
+    }
+
+    @Override
+    public void close() {
+        if (recordParser != null) {
+            try {
+                recordParser.close();
+            } catch (final Exception e) {
+                logger.warn("Failed to close decorated source for " + flowFile, e);
+            }
+        }
+
+        try {
+            rawIn.close();
+        } catch (final Exception e) {
+            logger.warn("Failed to close InputStream for " + flowFile, e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileProjectTableScanRule.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileProjectTableScanRule.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileProjectTableScanRule.java
new file mode 100644
index 0000000..a44a5d8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileProjectTableScanRule.java
@@ -0,0 +1,76 @@
+/*
+ * 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.queryrecord;
+
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * Planner rule that projects from a {@link FlowFileTableScan} scan just the columns
+ * needed to satisfy a projection. If the projection's expressions are trivial,
+ * the projection is removed.
+ */
+public class FlowFileProjectTableScanRule extends RelOptRule {
+    public static final FlowFileProjectTableScanRule INSTANCE = new FlowFileProjectTableScanRule();
+
+    private FlowFileProjectTableScanRule() {
+        super(
+            operand(LogicalProject.class,
+                operand(FlowFileTableScan.class, none())),
+            "FlowFileProjectTableScanRule");
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        final LogicalProject project = call.rel(0);
+        final FlowFileTableScan scan = call.rel(1);
+        final int[] fields = getProjectFields(project.getProjects());
+
+        if (fields == null) {
+            // Project contains expressions more complex than just field references.
+            return;
+        }
+
+        call.transformTo(
+            new FlowFileTableScan(
+                scan.getCluster(),
+                scan.getTable(),
+                scan.flowFileTable,
+                fields));
+    }
+
+    private int[] getProjectFields(List<RexNode> exps) {
+        final int[] fields = new int[exps.size()];
+
+        for (int i = 0; i < exps.size(); i++) {
+            final RexNode exp = exps.get(i);
+
+            if (exp instanceof RexInputRef) {
+                fields[i] = ((RexInputRef) exp).getIndex();
+            } else {
+                return null; // not a simple projection
+            }
+        }
+
+        return fields;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..bd15dc2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java
@@ -0,0 +1,218 @@
+/*
+ * 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.queryrecord;
+
+import java.io.InputStream;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.Schema.TableType;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractTable;
+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.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+
+public class FlowFileTable<S, E> extends AbstractTable implements QueryableTable, TranslatableTable {
+
+    private final RecordReaderFactory recordParserFactory;
+    private final ComponentLog logger;
+
+    private RecordSchema recordSchema;
+    private RelDataType relDataType = null;
+
+    private volatile ProcessSession session;
+    private volatile FlowFile flowFile;
+    private volatile int maxRecordsRead;
+
+    /**
+     * Creates a FlowFile table.
+     */
+    public FlowFileTable(final ProcessSession session, final FlowFile flowFile, final RecordReaderFactory recordParserFactory, final ComponentLog logger) {
+        this.session = session;
+        this.flowFile = flowFile;
+        this.recordParserFactory = recordParserFactory;
+        this.logger = logger;
+    }
+
+    public void setFlowFile(final ProcessSession session, final FlowFile flowFile) {
+        this.session = session;
+        this.flowFile = flowFile;
+        this.maxRecordsRead = 0;
+    }
+
+
+    @Override
+    public String toString() {
+        return "FlowFileTable";
+    }
+
+    /**
+     * Returns an enumerable over a given projection of the fields.
+     *
+     * <p>
+     * Called from generated code.
+     */
+    public Enumerable<Object> project(final int[] fields) {
+        return new AbstractEnumerable<Object>() {
+            @Override
+            @SuppressWarnings({"unchecked", "rawtypes"})
+            public Enumerator<Object> enumerator() {
+                return new FlowFileEnumerator(session, flowFile, logger, recordParserFactory, fields) {
+                    @Override
+                    protected void onFinish() {
+                        final int recordCount = getRecordsRead();
+                        if (recordCount > maxRecordsRead) {
+                            maxRecordsRead = recordCount;
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    public int getRecordsRead() {
+        return maxRecordsRead;
+    }
+
+    @Override
+    @SuppressWarnings("rawtypes")
+    public Expression getExpression(final SchemaPlus schema, final String tableName, final Class clazz) {
+        return Schemas.tableExpression(schema, getElementType(), tableName, clazz);
+    }
+
+    @Override
+    public Type getElementType() {
+        return Object[].class;
+    }
+
+    @Override
+    public <T> Queryable<T> asQueryable(final QueryProvider queryProvider, final SchemaPlus schema, final String tableName) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public RelNode toRel(final RelOptTable.ToRelContext context, final RelOptTable relOptTable) {
+        // Request all fields.
+        final int fieldCount = relOptTable.getRowType().getFieldCount();
+        final int[] fields = new int[fieldCount];
+        for (int i = 0; i < fieldCount; i++) {
+            fields[i] = i;
+        }
+
+        return new FlowFileTableScan(context.getCluster(), relOptTable, this, fields);
+    }
+
+    @Override
+    public RelDataType getRowType(final RelDataTypeFactory typeFactory) {
+        if (relDataType != null) {
+            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()) {
+            names.add(field.getFieldName());
+            types.add(getRelDataType(field.getDataType(), javaTypeFactory));
+        }
+
+        logger.debug("Found Schema: {}", new Object[] {schema});
+
+        if (recordSchema == null) {
+            recordSchema = schema;
+        }
+
+        relDataType = typeFactory.createStructType(Pair.zip(names, types));
+        return relDataType;
+    }
+
+    private RelDataType getRelDataType(final DataType fieldType, final JavaTypeFactory typeFactory) {
+        switch (fieldType.getFieldType()) {
+            case BOOLEAN:
+                return typeFactory.createJavaType(boolean.class);
+            case BYTE:
+                return typeFactory.createJavaType(byte.class);
+            case CHAR:
+                return typeFactory.createJavaType(char.class);
+            case DATE:
+                return typeFactory.createJavaType(java.sql.Date.class);
+            case DOUBLE:
+                return typeFactory.createJavaType(double.class);
+            case FLOAT:
+                return typeFactory.createJavaType(float.class);
+            case INT:
+                return typeFactory.createJavaType(int.class);
+            case SHORT:
+                return typeFactory.createJavaType(short.class);
+            case TIME:
+                return typeFactory.createJavaType(java.sql.Time.class);
+            case TIMESTAMP:
+                return typeFactory.createJavaType(java.sql.Timestamp.class);
+            case LONG:
+                return typeFactory.createJavaType(long.class);
+            case STRING:
+                return typeFactory.createJavaType(String.class);
+            case ARRAY:
+                return typeFactory.createJavaType(Object[].class);
+            case RECORD:
+                return typeFactory.createJavaType(Object.class);
+            case MAP:
+                return typeFactory.createJavaType(HashMap.class);
+        }
+
+        throw new IllegalArgumentException("Unknown Record Field Type: " + fieldType);
+    }
+
+    @Override
+    public TableType getJdbcTableType() {
+        return TableType.TEMPORARY_TABLE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTableScan.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..afca202
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTableScan.java
@@ -0,0 +1,91 @@
+/*
+ * 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.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;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+
+/**
+ * Relational expression representing a scan of a FlowFile.
+ *
+ * <p>
+ * Like any table scan, it serves as a leaf node of a query tree.
+ * </p>
+ */
+public class FlowFileTableScan extends TableScan implements EnumerableRel {
+    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;
+        this.fields = fields;
+    }
+
+    @Override
+    public RelNode copy(final RelTraitSet traitSet, final List<RelNode> inputs) {
+        return new FlowFileTableScan(getCluster(), table, flowFileTable, fields);
+    }
+
+    @Override
+    public RelWriter explainTerms(final RelWriter pw) {
+        return super.explainTerms(pw).item("fields", Primitive.asList(fields));
+    }
+
+    @Override
+    public RelDataType deriveRowType() {
+        final List<RelDataTypeField> fieldList = table.getRowType().getFieldList();
+        final RelDataTypeFactory.FieldInfoBuilder builder = getCluster().getTypeFactory().builder();
+        for (int field : fields) {
+            builder.add(fieldList.get(field));
+        }
+        return builder.build();
+    }
+
+    @Override
+    public void register(RelOptPlanner planner) {
+        planner.addRule(FlowFileProjectTableScanRule.INSTANCE);
+    }
+
+    @Override
+    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+        PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), getRowType(), pref.preferArray());
+
+        return implementor.result(physType, Blocks.toBlock(
+            Expressions.call(table.getExpression(FlowFileTable.class), "project", Expressions.constant(fields))));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index 3891ee6..d85e663 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -76,7 +76,7 @@ org.apache.nifi.processors.standard.PutSyslog
 org.apache.nifi.processors.standard.PutTCP
 org.apache.nifi.processors.standard.PutUDP
 org.apache.nifi.processors.standard.QueryDatabaseTable
-org.apache.nifi.processors.standard.QueryFlowFile
+org.apache.nifi.processors.standard.QueryRecord
 org.apache.nifi.processors.standard.ReplaceText
 org.apache.nifi.processors.standard.RouteText
 org.apache.nifi.processors.standard.ReplaceTextWithMapping

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryFlowFile/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryFlowFile/additionalDetails.html b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryFlowFile/additionalDetails.html
deleted file mode 100644
index 0dffc0d..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryFlowFile/additionalDetails.html
+++ /dev/null
@@ -1,48 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-    <!--
-      Licensed to the Apache Software Foundation (ASF) under one or more
-      contributor license agreements.  See the NOTICE file distributed with
-      this work for additional information regarding copyright ownership.
-      The ASF licenses this file to You under the Apache License, Version 2.0
-      (the "License"); you may not use this file except in compliance with
-      the License.  You may obtain a copy of the License at
-          http://www.apache.org/licenses/LICENSE-2.0
-      Unless required by applicable law or agreed to in writing, software
-      distributed under the License is distributed on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-      See the License for the specific language governing permissions and
-      limitations under the License.
-    -->
-    <head>
-        <meta charset="utf-8" />
-        <title>QueryFlowFile</title>
-
-        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
-    </head>
-
-    <body>
-    	<p>
-    		QueryFlowFile provides users a tremendous amount of power by leveraging an extremely well-known
-    		syntax (SQL) to route, filter, transform, and query data as it traverses the system. In order to
-    		provide the Processor with the maximum amount of flexibility, it is configured with a Controller
-    		Service that is responsible for reading and parsing the incoming FlowFiles and a Controller Service
-    		that is responsible for writing the results out. By using this paradigm, users are not forced to
-    		convert their data from one format to another just to query it, and then transform the data back
-    		into the form that they want. Rather, the appropriate Controller Service can easily be configured
-    		and put to use for the appropriate data format. 
-    	</p>
-    	
-    	<p>
-    		Rather than providing a single "SQL SELECT Statement" type of Property, this Processor makes use
-    		of user-defined properties. Each user-defined property that is added to the Processor has a name
-    		that becomes a new Relationship for the Processor and a corresponding SQL query that will be evaluated
-    		against each FlowFile. This allows multiple SQL queries to be run against each FlowFile.
-    	</p>
-    	
-    	<p>
-			The SQL syntax that is supported by this Processor is ANSI SQL and is powered by Apache Calcite. Please
-			note that identifiers are quoted using double-quotes, and column names/labels are case-insensitive.
-    	</p>
-	</body>
-</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryRecord/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryRecord/additionalDetails.html b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryRecord/additionalDetails.html
new file mode 100644
index 0000000..93bbe2a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryRecord/additionalDetails.html
@@ -0,0 +1,48 @@
+<!DOCTYPE html>
+<html lang="en">
+    <!--
+      Licensed to the Apache Software Foundation (ASF) under one or more
+      contributor license agreements.  See the NOTICE file distributed with
+      this work for additional information regarding copyright ownership.
+      The ASF licenses this file to You under the Apache License, Version 2.0
+      (the "License"); you may not use this file except in compliance with
+      the License.  You may obtain a copy of the License at
+          http://www.apache.org/licenses/LICENSE-2.0
+      Unless required by applicable law or agreed to in writing, software
+      distributed under the License is distributed on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+      See the License for the specific language governing permissions and
+      limitations under the License.
+    -->
+    <head>
+        <meta charset="utf-8" />
+        <title>QueryRecord</title>
+
+        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
+    </head>
+
+    <body>
+    	<p>
+    		QueryRecord provides users a tremendous amount of power by leveraging an extremely well-known
+    		syntax (SQL) to route, filter, transform, and query data as it traverses the system. In order to
+    		provide the Processor with the maximum amount of flexibility, it is configured with a Controller
+    		Service that is responsible for reading and parsing the incoming FlowFiles and a Controller Service
+    		that is responsible for writing the results out. By using this paradigm, users are not forced to
+    		convert their data from one format to another just to query it, and then transform the data back
+    		into the form that they want. Rather, the appropriate Controller Service can easily be configured
+    		and put to use for the appropriate data format. 
+    	</p>
+    	
+    	<p>
+    		Rather than providing a single "SQL SELECT Statement" type of Property, this Processor makes use
+    		of user-defined properties. Each user-defined property that is added to the Processor has a name
+    		that becomes a new Relationship for the Processor and a corresponding SQL query that will be evaluated
+    		against each FlowFile. This allows multiple SQL queries to be run against each FlowFile.
+    	</p>
+    	
+    	<p>
+			The SQL syntax that is supported by this Processor is ANSI SQL and is powered by Apache Calcite. Please
+			note that identifiers are quoted using double-quotes, and column names/labels are case-insensitive.
+    	</p>
+	</body>
+</html>
\ No newline at end of file


[7/7] nifi git commit: NIFI-3682: This closes #1682. Add Schema Access Strategy and Schema Write Strategy Record Readers and Writers; bug fixes.

Posted by jo...@apache.org.
NIFI-3682: This closes #1682. Add Schema Access Strategy and Schema Write Strategy Record Readers and Writers; bug fixes.

Signed-off-by: joewitt <jo...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/50ea1083
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/50ea1083
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/50ea1083

Branch: refs/heads/master
Commit: 50ea1083ec416c5d6e70c4e7bb5b2abde8f6266c
Parents: 57ccf97
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Apr 19 12:39:35 2017 -0400
Committer: joewitt <jo...@apache.org>
Committed: Mon Apr 24 17:02:45 2017 -0400

----------------------------------------------------------------------
 .../apache/nifi/components/PropertyValue.java   |   9 +
 .../expression/language/EmptyPreparedQuery.java |   5 +
 .../language/InvalidPreparedQuery.java          |   5 +
 .../expression/language/PreparedQuery.java      |   2 +
 .../language/StandardPreparedQuery.java         |   5 +
 .../language/StandardPropertyValue.java         |   5 +
 .../apache/nifi/util/MockProcessSession.java    |  12 +-
 .../org/apache/nifi/util/MockPropertyValue.java |  13 +
 .../apache/nifi/util/MockSessionFactory.java    |   6 +-
 .../nifi/util/StandardProcessorTestRunner.java  |  13 +-
 .../java/org/apache/nifi/util/TestRunner.java   |  16 +
 .../scheduling/ConnectableProcessContext.java   |   5 +
 .../src/main/resources/conf/logback.xml         |   2 +
 .../services/AvroSchemaRegistry.java            |  86 +--
 .../services/TestAvroSchemaRegistry.java        |  19 +-
 .../nifi/processors/standard/ConvertRecord.java |  20 +-
 .../nifi/processors/standard/QueryFlowFile.java | 554 -----------------
 .../nifi/processors/standard/QueryRecord.java   | 587 +++++++++++++++++++
 .../nifi/queryflowfile/FlowFileEnumerator.java  | 146 -----
 .../FlowFileProjectTableScanRule.java           |  76 ---
 .../nifi/queryflowfile/FlowFileTable.java       | 203 -------
 .../nifi/queryflowfile/FlowFileTableScan.java   |  91 ---
 .../nifi/queryrecord/FlowFileEnumerator.java    | 157 +++++
 .../FlowFileProjectTableScanRule.java           |  76 +++
 .../apache/nifi/queryrecord/FlowFileTable.java  | 218 +++++++
 .../nifi/queryrecord/FlowFileTableScan.java     |  91 +++
 .../org.apache.nifi.processor.Processor         |   2 +-
 .../additionalDetails.html                      |  48 --
 .../additionalDetails.html                      |  48 ++
 .../processors/standard/TestQueryFlowFile.java  | 265 ---------
 .../processors/standard/TestQueryRecord.java    | 297 ++++++++++
 .../standard/util/record/MockRecordParser.java  |  12 +-
 .../standard/util/record/MockRecordWriter.java  |  10 +-
 .../apache/nifi/schema/access/SchemaField.java  |  37 ++
 .../schema/access/SchemaNotFoundException.java  |  32 +
 .../nifi/serialization/RecordReaderFactory.java |  37 ++
 .../serialization/RecordSetWriterFactory.java   |  37 +-
 .../serialization/RowRecordReaderFactory.java   |  38 --
 .../nifi/serialization/SimpleRecordSchema.java  |  73 ++-
 .../nifi/serialization/record/DataType.java     |  12 +-
 .../nifi/serialization/record/MapRecord.java    |  98 +++-
 .../nifi/serialization/record/Record.java       |   4 +
 .../nifi/serialization/record/RecordField.java  |  47 +-
 .../serialization/record/RecordFieldType.java   |  53 +-
 .../nifi/serialization/record/RecordSchema.java |  21 +
 .../record/ResultSetRecordSet.java              |  14 +
 .../serialization/record/SchemaIdentifier.java  |  96 +++
 .../record/type/ArrayDataType.java              |   2 +-
 .../record/type/ChoiceDataType.java             |   2 +-
 .../serialization/record/type/MapDataType.java  |  67 +++
 .../record/type/RecordDataType.java             |   5 +
 .../record/util/DataTypeUtils.java              | 170 ++++--
 .../serialization/TestSimpleRecordSchema.java   |  79 +++
 .../serialization/record/TestMapRecord.java     | 188 ++++++
 .../nifi-record-serialization-services/pom.xml  |   7 +-
 .../java/org/apache/nifi/avro/AvroReader.java   |  38 +-
 .../nifi/avro/AvroReaderWithEmbeddedSchema.java |  62 ++
 .../nifi/avro/AvroReaderWithExplicitSchema.java |  75 +++
 .../org/apache/nifi/avro/AvroRecordReader.java  |  56 +-
 .../apache/nifi/avro/AvroRecordSetWriter.java   |  81 ++-
 .../apache/nifi/avro/AvroSchemaValidator.java   |   9 +
 .../java/org/apache/nifi/avro/AvroTypeUtil.java |  38 +-
 .../org/apache/nifi/avro/WriteAvroResult.java   |  76 +--
 .../avro/WriteAvroResultWithExternalSchema.java |  75 +++
 .../nifi/avro/WriteAvroResultWithSchema.java    |  62 ++
 .../nifi/csv/CSVHeaderSchemaStrategy.java       |  67 +++
 .../java/org/apache/nifi/csv/CSVReader.java     |  49 +-
 .../org/apache/nifi/csv/CSVRecordReader.java    |  32 +-
 .../org/apache/nifi/csv/CSVRecordSetWriter.java |  14 +-
 .../main/java/org/apache/nifi/csv/CSVUtils.java |  27 +-
 .../org/apache/nifi/csv/WriteCSVResult.java     |  40 +-
 .../java/org/apache/nifi/grok/GrokReader.java   | 126 +++-
 .../org/apache/nifi/grok/GrokRecordReader.java  | 125 +---
 .../nifi/json/AbstractJsonRowRecordReader.java  |  57 --
 .../org/apache/nifi/json/JsonPathReader.java    |  11 +-
 .../nifi/json/JsonPathRowRecordReader.java      |  19 +-
 .../apache/nifi/json/JsonRecordSetWriter.java   |  10 +-
 .../org/apache/nifi/json/JsonTreeReader.java    |  11 +-
 .../nifi/json/JsonTreeRowRecordReader.java      |  81 ++-
 .../org/apache/nifi/json/WriteJsonResult.java   |  77 ++-
 .../schema/access/AvroSchemaTextStrategy.java   |  64 ++
 ...onworksAttributeSchemaReferenceStrategy.java | 116 ++++
 ...rtonworksAttributeSchemaReferenceWriter.java |  69 +++
 ...rtonworksEncodedSchemaReferenceStrategy.java |  77 +++
 ...HortonworksEncodedSchemaReferenceWriter.java |  78 +++
 .../schema/access/SchemaAccessStrategy.java     |  43 ++
 .../nifi/schema/access/SchemaAccessWriter.java  |  63 ++
 .../schema/access/SchemaNameAsAttribute.java    |  62 ++
 .../access/SchemaNamePropertyStrategy.java      |  69 +++
 .../schema/access/SchemaTextAsAttribute.java    |  60 ++
 .../DateTimeTextRecordSetWriter.java            |  11 +-
 .../SchemaRegistryRecordReader.java             | 110 ----
 .../SchemaRegistryRecordSetWriter.java          | 164 ++++++
 .../serialization/SchemaRegistryService.java    | 227 +++++++
 .../nifi/text/FreeFormTextRecordSetWriter.java  |   9 +-
 .../apache/nifi/text/FreeFormTextWriter.java    |  23 +-
 .../avro/TestAvroReaderWithEmbeddedSchema.java  | 290 +++++++++
 .../apache/nifi/avro/TestAvroRecordReader.java  | 296 ----------
 .../apache/nifi/avro/TestWriteAvroResult.java   |  40 +-
 .../avro/TestWriteAvroResultWithSchema.java     |  46 ++
 .../avro/TestWriteAvroResultWithoutSchema.java  |  56 ++
 .../nifi/csv/TestCSVHeaderSchemaStrategy.java   |  69 +++
 .../org/apache/nifi/csv/TestWriteCSVResult.java |   7 +-
 .../apache/nifi/grok/TestGrokRecordReader.java  |  10 +-
 .../nifi/json/TestJsonTreeRowRecordReader.java  |  23 +
 .../apache/nifi/json/TestWriteJsonResult.java   |  14 +-
 .../test/resources/json/output/dataTypes.json   |   6 +-
 .../single-bank-account-wrong-field-type.json   |  13 +
 .../schemaregistry/services/SchemaRegistry.java |  66 ++-
 109 files changed, 5284 insertions(+), 2438 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-api/src/main/java/org/apache/nifi/components/PropertyValue.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/components/PropertyValue.java b/nifi-api/src/main/java/org/apache/nifi/components/PropertyValue.java
index efe76ee..05f262f 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/PropertyValue.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/PropertyValue.java
@@ -324,4 +324,13 @@ public interface PropertyValue {
      * Exception to be thrown
      */
     PropertyValue evaluateAttributeExpressions(FlowFile flowFile, AttributeValueDecorator decorator) throws ProcessException;
+
+    /**
+     * <p>
+     * Indicates whether the value of the property uses Expression Language.
+     * </p>
+     *
+     * @return <code>true</code> if the property value makes use of the Expression Language, <code>false</code> otherwise.
+     */
+    boolean isExpressionLanguagePresent();
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
index 4037531..a435b08 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
@@ -38,4 +38,9 @@ public class EmptyPreparedQuery implements PreparedQuery {
     public String evaluateExpressions(Map<String, String> attributes, AttributeValueDecorator decorator, Map<String, String> stateVariables) throws ProcessException {
         return value;
     }
+
+    @Override
+    public boolean isExpressionLanguagePresent() {
+        return false;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
index 1033c71..ce0dec3 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
@@ -47,4 +47,9 @@ public class InvalidPreparedQuery implements PreparedQuery {
     public String evaluateExpressions( Map<String, String> valueLookup, AttributeValueDecorator decorator, Map<String, String> stateVariables) throws ProcessException {
         throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
     }
+
+    @Override
+    public boolean isExpressionLanguagePresent() {
+        return false;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
index 53f7296..5552cac 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
@@ -26,4 +26,6 @@ public interface PreparedQuery {
     String evaluateExpressions(Map<String, String> valueLookup, AttributeValueDecorator decorator) throws ProcessException;
 
     String evaluateExpressions(final Map<String, String> valueLookup, final AttributeValueDecorator decorator, final Map<String, String> stateVariables) throws ProcessException;
+
+    boolean isExpressionLanguagePresent();
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
index 39cfb25..9f12c92 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
@@ -59,4 +59,9 @@ public class StandardPreparedQuery implements PreparedQuery {
             throws ProcessException {
         return evaluateExpressions(valMap, decorator, null);
     }
+
+    @Override
+    public boolean isExpressionLanguagePresent() {
+        return !trees.isEmpty();
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java
index 94c1c50..ac370bd 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java
@@ -195,4 +195,9 @@ public class StandardPropertyValue implements PropertyValue {
     public boolean isSet() {
         return rawValue != null;
     }
+
+    @Override
+    public boolean isExpressionLanguagePresent() {
+        return preparedQuery.isExpressionLanguagePresent();
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
----------------------------------------------------------------------
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 faf6e42..f05b9b3 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
@@ -72,6 +72,7 @@ public class MockProcessSession implements ProcessSession {
     private final Map<String, Long> counterMap = new HashMap<>();
     private final Set<FlowFile> recursionSet = new HashSet<>();
     private final MockProvenanceReporter provenanceReporter;
+    private final boolean enforceReadStreamsClosed;
 
     // A List of InputStreams that have been created by calls to {@link #read(FlowFile)} and have not yet been closed.
     private final Map<FlowFile, InputStream> openInputStreams = new HashMap<>();
@@ -83,7 +84,12 @@ public class MockProcessSession implements ProcessSession {
     private static final AtomicLong enqueuedIndex = new AtomicLong(0L);
 
     public MockProcessSession(final SharedSessionState sharedState, final Processor processor) {
+        this(sharedState, processor, true);
+    }
+
+    public MockProcessSession(final SharedSessionState sharedState, final Processor processor, final boolean enforceReadStreamsClosed) {
         this.processor = processor;
+        this.enforceReadStreamsClosed = enforceReadStreamsClosed;
         this.sharedState = sharedState;
         this.processorQueue = sharedState.getFlowFileQueue();
         provenanceReporter = new MockProvenanceReporter(this, sharedState, processor.getIdentifier(), processor.getClass().getSimpleName());
@@ -218,8 +224,10 @@ public class MockProcessSession implements ProcessSession {
                 }
             }
 
-            throw new FlowFileHandlingException("Cannot commit session because the following Input Streams were created via "
-                + "calls to ProcessSession.read(FlowFile) and never closed: " + openStreamCopy);
+            if (enforceReadStreamsClosed) {
+                throw new FlowFileHandlingException("Cannot commit session because the following Input Streams were created via "
+                    + "calls to ProcessSession.read(FlowFile) and never closed: " + openStreamCopy);
+            }
         }
 
         committed = true;

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
index b6752a7..c55ad23 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
@@ -16,10 +16,13 @@
  */
 package org.apache.nifi.util;
 
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
+import org.apache.nifi.attribute.expression.language.Query.Range;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.controller.ControllerService;
@@ -219,4 +222,14 @@ public class MockPropertyValue implements PropertyValue {
     public String toString() {
         return getValue();
     }
+
+    @Override
+    public boolean isExpressionLanguagePresent() {
+        if (!expectExpressions) {
+            return false;
+        }
+
+        final List<Range> elRanges = Query.extractExpressionRanges(rawValue);
+        return (elRanges != null && !elRanges.isEmpty());
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-mock/src/main/java/org/apache/nifi/util/MockSessionFactory.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockSessionFactory.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockSessionFactory.java
index 49b8796..010cc97 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockSessionFactory.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockSessionFactory.java
@@ -29,15 +29,17 @@ public class MockSessionFactory implements ProcessSessionFactory {
     private final Processor processor;
     private final SharedSessionState sharedState;
     private final Set<MockProcessSession> createdSessions = new CopyOnWriteArraySet<>();
+    private final boolean enforceReadStreamsClosed;
 
-    MockSessionFactory(final SharedSessionState sharedState, final Processor processor) {
+    MockSessionFactory(final SharedSessionState sharedState, final Processor processor, final boolean enforceReadStreamsClosed) {
         this.sharedState = sharedState;
         this.processor = processor;
+        this.enforceReadStreamsClosed = enforceReadStreamsClosed;
     }
 
     @Override
     public ProcessSession createSession() {
-        final MockProcessSession session = new MockProcessSession(sharedState, processor);
+        final MockProcessSession session = new MockProcessSession(sharedState, processor, enforceReadStreamsClosed);
         createdSessions.add(session);
         return session;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
index 451eea9..6c35643 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
@@ -78,7 +78,6 @@ public class StandardProcessorTestRunner implements TestRunner {
     private final Processor processor;
     private final MockProcessContext context;
     private final MockFlowFileQueue flowFileQueue;
-    private final MockSessionFactory sessionFactory;
     private final SharedSessionState sharedState;
     private final AtomicLong idGenerator;
     private final boolean triggerSerially;
@@ -87,17 +86,19 @@ public class StandardProcessorTestRunner implements TestRunner {
     private final MockVariableRegistry variableRegistry;
 
     private int numThreads = 1;
+    private MockSessionFactory sessionFactory;
     private final AtomicInteger invocations = new AtomicInteger(0);
 
     private final Map<String, MockComponentLog> controllerServiceLoggers = new HashMap<>();
     private final MockComponentLog logger;
+    private boolean enforceReadStreamsClosed = true;
 
     StandardProcessorTestRunner(final Processor processor) {
         this.processor = processor;
         this.idGenerator = new AtomicLong(0L);
         this.sharedState = new SharedSessionState(processor, idGenerator);
         this.flowFileQueue = sharedState.getFlowFileQueue();
-        this.sessionFactory = new MockSessionFactory(sharedState, processor);
+        this.sessionFactory = new MockSessionFactory(sharedState, processor, enforceReadStreamsClosed);
         this.processorStateManager = new MockStateManager(processor);
         this.variableRegistry = new MockVariableRegistry();
         this.context = new MockProcessContext(processor, processorStateManager, variableRegistry);
@@ -118,6 +119,12 @@ public class StandardProcessorTestRunner implements TestRunner {
     }
 
     @Override
+    public void enforceReadStreamsClosed(final boolean enforce) {
+        enforceReadStreamsClosed = enforce;
+        this.sessionFactory = new MockSessionFactory(sharedState, processor, enforceReadStreamsClosed);
+    }
+
+    @Override
     public void setValidateExpressionUsage(final boolean validate) {
         context.setValidateExpressionUsage(validate);
     }
@@ -412,7 +419,7 @@ public class StandardProcessorTestRunner implements TestRunner {
 
     @Override
     public MockFlowFile enqueue(final InputStream data, final Map<String, String> attributes) {
-        final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator), processor);
+        final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator), processor, enforceReadStreamsClosed);
         MockFlowFile flowFile = session.create();
         flowFile = session.importFrom(data, flowFile);
         flowFile = session.putAllAttributes(flowFile, attributes);

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
index 1c014c3..9a1a10d 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
@@ -30,6 +30,7 @@ import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.queue.QueueSize;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
@@ -949,4 +950,19 @@ public interface TestRunner {
      * @param predicate conditions
      */
     void assertAllConditionsMet(final Relationship relationship, Predicate<MockFlowFile> predicate);
+
+    /**
+     * By default, if {@link ProcessSession#read(FlowFile)} is called, the InputStream that is returned MUST be closed by
+     * the processor under test or calls to {@link ProcessSession#commit()} will throw an Exception. This method allows
+     * the developer to indicate explicitly that they do or do not want this functionality. The ProcessSession that is used
+     * in the framework when running NiFi does not enforce this, as the framework itself tracks the InputStreams that it returns
+     * and ensures that they are properly closed on session commit or rollback. However, it is considered a best practice for
+     * Processors to close the streams themselves whenever they are no longer needed. There may be cases, however, where this
+     * is not feasible or easy and this method provides developers the ability to indicate that by disabling enforcement so that
+     * the framework will handle this.
+     *
+     * @param enforce <code>true</code> if calls to session.commit() should fail if the read streams are not properly closed.
+     */
+    void enforceReadStreamsClosed(boolean enforce);
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
index 7c4ce77..0d755b0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
@@ -166,6 +166,11 @@ public class ConnectableProcessContext implements ProcessContext {
                     throws ProcessException {
                 return null;
             }
+
+            @Override
+            public boolean isExpressionLanguagePresent() {
+                return false;
+            }
         };
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
index f357c41..f2da200 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
@@ -96,6 +96,8 @@
     <logger name="org.apache.zookeeper.ZooKeeper" level="ERROR" />
     <logger name="org.apache.zookeeper.server.PrepRequestProcessor" level="ERROR" />
 
+    <logger name="org.apache.calcite.runtime.CalciteException" level="OFF" />
+
     <logger name="org.apache.curator.framework.recipes.leader.LeaderSelector" level="OFF" />
     <logger name="org.apache.curator.ConnectionState" level="OFF" />
     

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java
index 13b1d5d..8fcb016 100644
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java
+++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java
@@ -16,10 +16,13 @@
  */
 package org.apache.nifi.schemaregistry.services;
 
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.stream.Collectors;
 
 import org.apache.avro.LogicalType;
@@ -34,18 +37,21 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.schema.access.SchemaField;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 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 org.apache.nifi.serialization.record.SchemaIdentifier;
 
 @Tags({"schema", "registry", "avro", "json", "csv"})
 @CapabilityDescription("Provides a service for registering and accessing schemas. You can register a schema "
     + "as a dynamic property where 'name' represents the schema name and 'value' represents the textual "
     + "representation of the actual schema following the syntax and semantics of Avro's Schema format.")
 public class AvroSchemaRegistry extends AbstractControllerService implements SchemaRegistry {
-
+    private static final Set<SchemaField> schemaFields = EnumSet.of(SchemaField.SCHEMA_NAME, SchemaField.SCHEMA_TEXT, SchemaField.SCHEMA_TEXT_FORMAT);
     private final Map<String, String> schemaNameToSchemaMap;
 
     private static final String LOGICAL_TYPE_DATE = "date";
@@ -54,39 +60,51 @@ public class AvroSchemaRegistry extends AbstractControllerService implements Sch
     private static final String LOGICAL_TYPE_TIMESTAMP_MILLIS = "timestamp-millis";
     private static final String LOGICAL_TYPE_TIMESTAMP_MICROS = "timestamp-micros";
 
-
     public AvroSchemaRegistry() {
         this.schemaNameToSchemaMap = new HashMap<>();
     }
 
-    @OnEnabled
-    public void enable(ConfigurationContext configuratiponContext) throws InitializationException {
-        this.schemaNameToSchemaMap.putAll(configuratiponContext.getProperties().entrySet().stream()
-            .filter(propEntry -> propEntry.getKey().isDynamic())
-            .collect(Collectors.toMap(propEntry -> propEntry.getKey().getName(), propEntry -> propEntry.getValue())));
+    @Override
+    public String retrieveSchemaText(final String schemaName) throws SchemaNotFoundException {
+        final String schemaText = schemaNameToSchemaMap.get(schemaName);
+        if (schemaText == null) {
+            throw new SchemaNotFoundException("Unable to find schema with name '" + schemaName + "'");
+        }
+
+        return schemaText;
     }
 
     @Override
-    public String retrieveSchemaText(String schemaName) {
-        if (!this.schemaNameToSchemaMap.containsKey(schemaName)) {
-            throw new IllegalArgumentException("Failed to find schema; Name: '" + schemaName + ".");
-        } else {
-            return this.schemaNameToSchemaMap.get(schemaName);
-        }
+    public RecordSchema retrieveSchema(final String schemaName) throws SchemaNotFoundException {
+        final String schemaText = retrieveSchemaText(schemaName);
+        final Schema schema = new Schema.Parser().parse(schemaText);
+        return createRecordSchema(schema, schemaText, schemaName);
     }
 
     @Override
-    public String retrieveSchemaText(String schemaName, Map<String, String> attributes) {
-        throw new UnsupportedOperationException("This version of schema registry does not "
-            + "support this operation, since schemas are only identofied by name.");
+    public RecordSchema retrieveSchema(long schemaId, int version) throws IOException, SchemaNotFoundException {
+        throw new SchemaNotFoundException("This Schema Registry does not support schema lookup by identifier and version - only by name.");
     }
 
     @Override
+    public String retrieveSchemaText(long schemaId, int version) throws IOException, SchemaNotFoundException {
+        throw new SchemaNotFoundException("This Schema Registry does not support schema lookup by identifier and version - only by name.");
+    }
+
     @OnDisabled
     public void close() throws Exception {
-        this.schemaNameToSchemaMap.clear();
+        schemaNameToSchemaMap.clear();
     }
 
+
+    @OnEnabled
+    public void enable(final ConfigurationContext configurationContext) throws InitializationException {
+        this.schemaNameToSchemaMap.putAll(configurationContext.getProperties().entrySet().stream()
+            .filter(propEntry -> propEntry.getKey().isDynamic())
+            .collect(Collectors.toMap(propEntry -> propEntry.getKey().getName(), propEntry -> propEntry.getValue())));
+    }
+
+
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
@@ -99,28 +117,24 @@ public class AvroSchemaRegistry extends AbstractControllerService implements Sch
     }
 
 
-    @Override
-    public RecordSchema retrieveSchema(String schemaName) {
-        final String schemaText = this.retrieveSchemaText(schemaName);
-        final Schema schema = new Schema.Parser().parse(schemaText);
-        return createRecordSchema(schema);
-    }
-
     /**
      * Converts an Avro Schema to a RecordSchema
      *
      * @param avroSchema the Avro Schema to convert
+     * @param text the textual representation of the schema
+     * @param schemaName the name of the schema
      * @return the Corresponding Record Schema
      */
-    private RecordSchema createRecordSchema(final Schema avroSchema) {
+    private RecordSchema createRecordSchema(final Schema avroSchema, final String text, final String schemaName) {
         final List<RecordField> recordFields = new ArrayList<>(avroSchema.getFields().size());
         for (final Field field : avroSchema.getFields()) {
             final String fieldName = field.name();
             final DataType dataType = determineDataType(field.schema());
-            recordFields.add(new RecordField(fieldName, dataType));
+
+            recordFields.add(new RecordField(fieldName, dataType, field.defaultVal(), field.aliases()));
         }
 
-        final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
+        final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, text, "avro", SchemaIdentifier.ofName(schemaName));
         return recordSchema;
     }
 
@@ -175,15 +189,19 @@ public class AvroSchemaRegistry extends AbstractControllerService implements Sch
                     final String fieldName = field.name();
                     final Schema fieldSchema = field.schema();
                     final DataType fieldType = determineDataType(fieldSchema);
-                    recordFields.add(new RecordField(fieldName, fieldType));
+
+                    recordFields.add(new RecordField(fieldName, fieldType, field.defaultVal(), field.aliases()));
                 }
 
-                final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
+                final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, avroSchema.toString(), "avro", SchemaIdentifier.EMPTY);
                 return RecordFieldType.RECORD.getRecordDataType(recordSchema);
             }
             case NULL:
+                return RecordFieldType.STRING.getDataType();
             case MAP:
-                return RecordFieldType.RECORD.getDataType();
+                final Schema valueSchema = avroSchema.getValueType();
+                final DataType valueType = determineDataType(valueSchema);
+                return RecordFieldType.MAP.getMapDataType(valueType);
             case UNION: {
                 final List<Schema> nonNullSubSchemas = avroSchema.getTypes().stream()
                     .filter(s -> s.getType() != Type.NULL)
@@ -206,12 +224,8 @@ public class AvroSchemaRegistry extends AbstractControllerService implements Sch
         return null;
     }
 
-    /*
-     * For this implementation 'attributes' argument is ignored since the underlying storage mechanisms
-     * is based strictly on key/value pairs. In other implementation additional attributes may play a role (e.g., version id,)
-     */
     @Override
-    public RecordSchema retrieveSchema(String schemaName, Map<String, String> attributes) {
-        return this.retrieveSchema(schemaName);
+    public Set<SchemaField> getSuppliedSchemaFields() {
+        return schemaFields;
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/TestAvroSchemaRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/TestAvroSchemaRegistry.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/TestAvroSchemaRegistry.java
index 929aab9..a63097a 100644
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/TestAvroSchemaRegistry.java
+++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/TestAvroSchemaRegistry.java
@@ -17,7 +17,6 @@
 package org.apache.nifi.schemaregistry.services;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -27,9 +26,11 @@ import java.util.Map;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class TestAvroSchemaRegistry {
@@ -55,17 +56,17 @@ public class TestAvroSchemaRegistry {
         properties.put(fooSchema, fooSchemaText);
         properties.put(barSchema, "");
         when(configContext.getProperties()).thenReturn(properties);
-        SchemaRegistry delegate = new AvroSchemaRegistry();
-        ((AvroSchemaRegistry) delegate).enable(configContext);
+        AvroSchemaRegistry delegate = new AvroSchemaRegistry();
+        delegate.enable(configContext);
 
         String locatedSchemaText = delegate.retrieveSchemaText(schemaName);
         assertEquals(fooSchemaText, locatedSchemaText);
         try {
-            locatedSchemaText = delegate.retrieveSchemaText("barSchema");
-            fail();
-        } catch (Exception e) {
-            // ignore
+            delegate.retrieveSchemaText("barSchema");
+            Assert.fail("Expected a SchemaNotFoundException to be thrown but it was not");
+        } catch (final SchemaNotFoundException expected) {
         }
+
         delegate.close();
     }
 
@@ -91,8 +92,8 @@ public class TestAvroSchemaRegistry {
         properties.put(fooSchema, fooSchemaText);
         properties.put(barSchema, "");
         when(configContext.getProperties()).thenReturn(properties);
-        SchemaRegistry delegate = new AvroSchemaRegistry();
-        ((AvroSchemaRegistry) delegate).enable(configContext);
+        AvroSchemaRegistry delegate = new AvroSchemaRegistry();
+        delegate.enable(configContext);
 
         RecordSchema locatedSchema = delegate.retrieveSchema(schemaName);
         List<RecordField> recordFields = locatedSchema.getFields();

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertRecord.java
----------------------------------------------------------------------
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 9a505a2..2b2caa4 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
@@ -17,6 +17,7 @@
 
 package org.apache.nifi.processors.standard;
 
+import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -46,11 +47,12 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
 import org.apache.nifi.serialization.WriteResult;
 
 @EventDriven
@@ -74,7 +76,7 @@ public class ConvertRecord extends AbstractProcessor {
         .name("record-reader")
         .displayName("Record Reader")
         .description("Specifies the Controller Service to use for reading incoming data")
-        .identifiesControllerService(RowRecordReaderFactory.class)
+        .identifiesControllerService(RecordReaderFactory.class)
         .required(true)
         .build();
     static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
@@ -118,9 +120,17 @@ public class ConvertRecord extends AbstractProcessor {
             return;
         }
 
-        final RowRecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RowRecordReaderFactory.class);
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
         final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
-        final RecordSetWriter writer = writerFactory.createWriter(getLogger());
+        final RecordSetWriter writer;
+        try (final InputStream rawIn = session.read(flowFile);
+            final InputStream in = new BufferedInputStream(rawIn)) {
+            writer = writerFactory.createWriter(getLogger(), flowFile, in);
+        } catch (final Exception e) {
+            getLogger().error("Failed to convert records for {}; will route to failure", new Object[] {flowFile, e});
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
 
         final AtomicReference<WriteResult> writeResultRef = new AtomicReference<>();
 
@@ -134,7 +144,7 @@ public class ConvertRecord extends AbstractProcessor {
                         final WriteResult writeResult = writer.write(reader.createRecordSet(), out);
                         writeResultRef.set(writeResult);
 
-                    } catch (final MalformedRecordException e) {
+                    } catch (final SchemaNotFoundException | MalformedRecordException e) {
                         throw new ProcessException("Could not parse incoming data", e);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryFlowFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryFlowFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryFlowFile.java
deleted file mode 100644
index 83a3d4b..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryFlowFile.java
+++ /dev/null
@@ -1,554 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.standard;
-
-import java.io.Closeable;
-import java.io.IOException;
-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 org.apache.calcite.config.CalciteConnectionProperty;
-import org.apache.calcite.config.Lex;
-import org.apache.calcite.jdbc.CalciteConnection;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.nifi.annotation.behavior.DynamicProperty;
-import org.apache.nifi.annotation.behavior.DynamicRelationship;
-import org.apache.nifi.annotation.behavior.EventDriven;
-import org.apache.nifi.annotation.behavior.InputRequirement;
-import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
-import org.apache.nifi.annotation.behavior.SideEffectFree;
-import org.apache.nifi.annotation.behavior.SupportsBatching;
-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;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.processor.AbstractProcessor;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.ProcessorInitializationContext;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.OutputStreamCallback;
-import org.apache.nifi.queryflowfile.FlowFileTable;
-import org.apache.nifi.serialization.RecordSetWriter;
-import org.apache.nifi.serialization.RecordSetWriterFactory;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
-import org.apache.nifi.serialization.WriteResult;
-import org.apache.nifi.serialization.record.ResultSetRecordSet;
-import org.apache.nifi.util.StopWatch;
-
-@EventDriven
-@SideEffectFree
-@SupportsBatching
-@Tags({"sql", "query", "calcite", "route", "record", "transform", "select", "update", "modify", "etl", "filter", "record", "csv", "json", "logs", "text", "avro", "aggregate"})
-@InputRequirement(Requirement.INPUT_REQUIRED)
-@CapabilityDescription("Evaluates one or more SQL queries against the contents of a FlowFile. The result of the "
-    + "SQL query then becomes the content of the output FlowFile. This can be used, for example, "
-    + "for field-specific filtering, transformation, and row-level filtering. "
-    + "Columns can be renamed, simple calculations and aggregations performed, etc. "
-    + "The Processor is configured with a Record Reader Controller Service and a Record Writer service so as to allow flexibility in incoming and outgoing data formats. "
-    + "The Processor must be configured with at least one user-defined property. The name of the Property "
-    + "is the Relationship to route data to, and the value of the Property is a SQL SELECT statement that is used to specify how input data should be transformed/filtered. "
-    + "The SQL statement must be valid ANSI SQL and is powered by Apache Calcite. "
-    + "If the transformation fails, the original FlowFile is routed to the 'failure' relationship. Otherwise, the data selected will be routed to the associated "
-    + "relationship. See the Processor Usage documentation for more information.")
-@DynamicRelationship(name="<Property Name>", description="Each user-defined property defines a new Relationship for this Processor.")
-@DynamicProperty(name = "The name of the relationship to route data to", value="A SQL SELECT statement that is used to determine what data should be routed to this "
-        + "relationship.", supportsExpressionLanguage=true, description="Each user-defined property specifies a SQL SELECT statement to run over the data, with the data "
-        + "that is selected being routed to the relationship whose name is the property name")
-public class QueryFlowFile extends AbstractProcessor {
-    static final PropertyDescriptor RECORD_READER_FACTORY = new PropertyDescriptor.Builder()
-        .name("record-reader")
-        .displayName("Record Reader")
-        .description("Specifies the Controller Service to use for parsing incoming data and determining the data's schema")
-        .identifiesControllerService(RowRecordReaderFactory.class)
-        .required(true)
-        .build();
-    static final PropertyDescriptor RECORD_WRITER_FACTORY = new PropertyDescriptor.Builder()
-        .name("record-writer")
-        .displayName("Record Writer")
-        .description("Specifies the Controller Service to use for writing results to a FlowFile")
-        .identifiesControllerService(RecordSetWriterFactory.class)
-        .required(true)
-        .build();
-    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
-        .name("include-zero-record-flowfiles")
-        .displayName("Include Zero Record FlowFiles")
-        .description("When running the SQL statement against an incoming FlowFile, if the result has no data, "
-            + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
-        .expressionLanguageSupported(false)
-        .allowableValues("true", "false")
-        .defaultValue("true")
-        .required(true)
-        .build();
-    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.")
-        .expressionLanguageSupported(false)
-        .allowableValues("true", "false")
-        .defaultValue("true")
-        .required(true)
-        .build();
-
-    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
-        .name("original")
-        .description("The original FlowFile is routed to this relationship")
-        .build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder()
-        .name("failure")
-        .description("If a FlowFile fails processing for any reason (for example, the SQL "
-            + "statement contains columns not present in input data), the original FlowFile it will "
-            + "be routed to this relationship")
-        .build();
-
-    private List<PropertyDescriptor> properties;
-    private final Set<Relationship> relationships = Collections.synchronizedSet(new HashSet<>());
-
-    private final Map<String, BlockingQueue<CachedStatement>> statementQueues = new HashMap<>();
-
-    @Override
-    protected void init(final ProcessorInitializationContext context) {
-        try {
-            DriverManager.registerDriver(new org.apache.calcite.jdbc.Driver());
-        } catch (final SQLException e) {
-            throw new ProcessException("Failed to load Calcite JDBC Driver", e);
-        }
-
-        final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(RECORD_READER_FACTORY);
-        properties.add(RECORD_WRITER_FACTORY);
-        properties.add(INCLUDE_ZERO_RECORD_FLOWFILES);
-        properties.add(CACHE_SCHEMA);
-        this.properties = Collections.unmodifiableList(properties);
-
-        relationships.add(REL_FAILURE);
-        relationships.add(REL_ORIGINAL);
-    }
-
-    @Override
-    public Set<Relationship> getRelationships() {
-        return relationships;
-    }
-
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return properties;
-    }
-
-    @Override
-    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
-        if (!descriptor.isDynamic()) {
-            return;
-        }
-
-        final Relationship relationship = new Relationship.Builder()
-            .name(descriptor.getName())
-            .description("User-defined relationship that specifies where data that matches the specified SQL query should be routed")
-            .build();
-
-        if (newValue == null) {
-            relationships.remove(relationship);
-        } else {
-            relationships.add(relationship);
-        }
-    }
-
-    @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)
-            .description("SQL select statement specifies how data should be filtered/transformed. "
-                + "SQL SELECT should select from the FLOWFILE table")
-            .required(false)
-            .dynamic(true)
-            .expressionLanguageSupported(true)
-            .addValidator(new SqlValidator())
-            .build();
-    }
-
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSession session) {
-        final FlowFile original = session.get();
-        if (original == null) {
-            return;
-        }
-
-        final StopWatch stopWatch = new StopWatch(true);
-
-        final RecordSetWriterFactory resultSetWriterFactory = context.getProperty(RECORD_WRITER_FACTORY)
-            .asControllerService(RecordSetWriterFactory.class);
-        final RowRecordReaderFactory recordParserFactory = context.getProperty(RECORD_READER_FACTORY)
-            .asControllerService(RowRecordReaderFactory.class);
-
-        final RecordSetWriter resultSetWriter = resultSetWriterFactory.createWriter(getLogger());
-        final Map<FlowFile, Relationship> transformedFlowFiles = new HashMap<>();
-        final Set<FlowFile> createdFlowFiles = new HashSet<>();
-
-        try {
-            for (final PropertyDescriptor descriptor : context.getProperties().keySet()) {
-                if (!descriptor.isDynamic()) {
-                    continue;
-                }
-
-                final Relationship relationship = new Relationship.Builder().name(descriptor.getName()).build();
-
-                // We have to fork a child because we may need to read the input FlowFile more than once,
-                // and we cannot call session.read() on the original FlowFile while we are within a write
-                // callback for the original FlowFile.
-                FlowFile transformed = session.create(original);
-
-                // Ensure that we have the FlowFile in the map in case we throw any Exception
-                createdFlowFiles.add(transformed);
-
-                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, recordParserFactory);
-                } else {
-                    queryResult = query(session, original, sql, context, recordParserFactory);
-                }
-
-                try {
-                    final ResultSet rs = queryResult.getResultSet();
-                    transformed = session.write(transformed, new OutputStreamCallback() {
-                        @Override
-                        public void process(final OutputStream out) throws IOException {
-                            try {
-                                final ResultSetRecordSet recordSet = new ResultSetRecordSet(rs);
-                                writeResultRef.set(resultSetWriter.write(recordSet, out));
-                            } catch (final Exception e) {
-                                throw new IOException(e);
-                            }
-                        }
-                    });
-                } finally {
-                    closeQuietly(queryResult);
-                }
-
-                final WriteResult result = writeResultRef.get();
-                if (result.getRecordCount() == 0 && !context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean()) {
-                    session.remove(transformed);
-                    transformedFlowFiles.remove(transformed);
-                    getLogger().info("Transformed {} but the result contained no data so will not pass on a FlowFile", new Object[] {original});
-                } else {
-                    final Map<String, String> attributesToAdd = new HashMap<>();
-                    if (result.getAttributes() != null) {
-                        attributesToAdd.putAll(result.getAttributes());
-                    }
-
-                    attributesToAdd.put(CoreAttributes.MIME_TYPE.key(), resultSetWriter.getMimeType());
-                    attributesToAdd.put("record.count", String.valueOf(result.getRecordCount()));
-                    transformed = session.putAllAttributes(transformed, attributesToAdd);
-                    transformedFlowFiles.put(transformed, relationship);
-                }
-            }
-
-            final long elapsedMillis = stopWatch.getElapsed(TimeUnit.MILLISECONDS);
-            if (transformedFlowFiles.size() > 0) {
-                session.getProvenanceReporter().fork(original, transformedFlowFiles.keySet(), elapsedMillis);
-
-                for (final Map.Entry<FlowFile, Relationship> entry : transformedFlowFiles.entrySet()) {
-                    final FlowFile transformed = entry.getKey();
-                    final Relationship relationship = entry.getValue();
-
-                    session.getProvenanceReporter().route(transformed, relationship);
-                    session.transfer(transformed, relationship);
-                }
-            }
-
-            getLogger().info("Successfully transformed {} in {} millis", new Object[] {original, elapsedMillis});
-            session.transfer(original, REL_ORIGINAL);
-        } catch (ProcessException e) {
-            getLogger().error("Unable to transform {} due to {}", new Object[] {original, e});
-            session.remove(createdFlowFiles);
-            session.transfer(original, REL_FAILURE);
-        } catch (final SQLException e) {
-            getLogger().error("Unable to transform {} due to {}", new Object[] {original, e.getCause() == null ? e : e.getCause()});
-            session.remove(createdFlowFiles);
-            session.transfer(original, REL_FAILURE);
-        }
-    }
-
-    private synchronized CachedStatement getStatement(final String sql, final Supplier<CalciteConnection> connectionSupplier, final ProcessSession session,
-        final FlowFile flowFile, final RowRecordReaderFactory recordReaderFactory) throws SQLException {
-
-        final BlockingQueue<CachedStatement> statementQueue = statementQueues.get(sql);
-        if (statementQueue == null) {
-            return buildCachedStatement(sql, connectionSupplier, session, flowFile, recordReaderFactory);
-        }
-
-        final CachedStatement cachedStmt = statementQueue.poll();
-        if (cachedStmt != null) {
-            return cachedStmt;
-        }
-
-        return buildCachedStatement(sql, connectionSupplier, session, flowFile, recordReaderFactory);
-    }
-
-    private CachedStatement buildCachedStatement(final String sql, final Supplier<CalciteConnection> connectionSupplier, final ProcessSession session,
-        final FlowFile flowFile, final RowRecordReaderFactory recordReaderFactory) throws SQLException {
-
-        final CalciteConnection connection = connectionSupplier.get();
-        final SchemaPlus rootSchema = connection.getRootSchema();
-
-        final FlowFileTable<?, ?> flowFileTable = new FlowFileTable<>(session, flowFile, 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());
-            }
-        }
-
-        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);
-        }
-    }
-
-    protected QueryResult queryWithCache(final ProcessSession session, final FlowFile flowFile, final String sql, final ProcessContext context,
-        final RowRecordReaderFactory recordParserFactory) throws SQLException {
-
-        final Supplier<CalciteConnection> connectionSupplier = () -> {
-            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);
-            }
-        };
-
-        final CachedStatement cachedStatement = getStatement(sql, connectionSupplier, session, flowFile, recordParserFactory);
-        final PreparedStatement stmt = cachedStatement.getStatement();
-        final FlowFileTable<?, ?> table = cachedStatement.getTable();
-        table.setFlowFile(session, flowFile);
-
-        final ResultSet rs = stmt.executeQuery();
-
-        return new QueryResult() {
-            @Override
-            public void close() throws IOException {
-                final BlockingQueue<CachedStatement> statementQueue = statementQueues.get(sql);
-                if (statementQueue == null || !statementQueue.offer(cachedStatement)) {
-                    try {
-                        cachedStatement.getConnection().close();
-                    } catch (SQLException e) {
-                        throw new IOException("Failed to close statement", e);
-                    }
-                }
-            }
-
-            @Override
-            public ResultSet getResultSet() {
-                return rs;
-            }
-        };
-    }
-
-    protected QueryResult query(final ProcessSession session, final FlowFile flowFile, final String sql, final ProcessContext context,
-        final RowRecordReaderFactory recordParserFactory) throws SQLException {
-
-        final Properties properties = new Properties();
-        properties.put(CalciteConnectionProperty.LEX.camelName(), Lex.JAVA.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();
-            resultSet = statement.executeQuery(sql);
-
-            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;
-                }
-            };
-        } catch (final Exception e) {
-            closeQuietly(resultSet, statement, connection);
-            throw e;
-        }
-    }
-
-    private void closeQuietly(final AutoCloseable... closeables) {
-        if (closeables == null) {
-            return;
-        }
-
-        for (final AutoCloseable closeable : closeables) {
-            if (closeable == null) {
-                continue;
-            }
-
-            try {
-                closeable.close();
-            } catch (final Exception e) {
-                getLogger().warn("Failed to close SQL resource", e);
-            }
-        }
-    }
-
-    private static class SqlValidator implements Validator {
-        @Override
-        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-            if (context.isExpressionLanguagePresent(input)) {
-                return new ValidationResult.Builder()
-                    .input(input)
-                    .subject(subject)
-                    .valid(true)
-                    .explanation("Expression Language Present")
-                    .build();
-            }
-
-            final String substituted = context.newPropertyValue(input).evaluateAttributeExpressions().getValue();
-            final SqlParser parser = SqlParser.create(substituted);
-            try {
-                parser.parseStmt();
-                return new ValidationResult.Builder()
-                    .subject(subject)
-                    .input(input)
-                    .valid(true)
-                    .build();
-            } catch (final Exception e) {
-                return new ValidationResult.Builder()
-                    .subject(subject)
-                    .input(input)
-                    .valid(false)
-                    .explanation("Not a valid SQL Statement: " + e.getMessage())
-                    .build();
-            }
-        }
-    }
-
-    private static interface QueryResult extends Closeable {
-        ResultSet getResultSet();
-    }
-
-    private static class CachedStatement {
-        private final FlowFileTable<?, ?> table;
-        private final PreparedStatement statement;
-        private 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() {
-            return table;
-        }
-
-        public PreparedStatement getStatement() {
-            return statement;
-        }
-
-        public Connection getConnection() {
-            return connection;
-        }
-    }
-}


[4/7] nifi git commit: NIFI-3682: This closes #1682. Add Schema Access Strategy and Schema Write Strategy Record Readers and Writers; bug fixes.

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
index 1cdefb8..05b3157 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
@@ -42,44 +42,46 @@ public class DataTypeUtils {
 
     private static final TimeZone gmt = TimeZone.getTimeZone("gmt");
 
-    public static Object convertType(final Object value, final DataType dataType) {
-        return convertType(value, dataType, RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
+    public static Object convertType(final Object value, final DataType dataType, final String fieldName) {
+        return convertType(value, dataType, RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat(), fieldName);
     }
 
-    public static Object convertType(final Object value, final DataType dataType, final String dateFormat, final String timeFormat, final String timestampFormat) {
+    public static Object convertType(final Object value, final DataType dataType, final String dateFormat, final String timeFormat, final String timestampFormat, final String fieldName) {
         switch (dataType.getFieldType()) {
             case BIGINT:
-                return toBigInt(value);
+                return toBigInt(value, fieldName);
             case BOOLEAN:
-                return toBoolean(value);
+                return toBoolean(value, fieldName);
             case BYTE:
-                return toByte(value);
+                return toByte(value, fieldName);
             case CHAR:
-                return toCharacter(value);
+                return toCharacter(value, fieldName);
             case DATE:
-                return toDate(value, dateFormat);
+                return toDate(value, dateFormat, fieldName);
             case DOUBLE:
-                return toDouble(value);
+                return toDouble(value, fieldName);
             case FLOAT:
-                return toFloat(value);
+                return toFloat(value, fieldName);
             case INT:
-                return toInteger(value);
+                return toInteger(value, fieldName);
             case LONG:
-                return toLong(value);
+                return toLong(value, fieldName);
             case SHORT:
-                return toShort(value);
+                return toShort(value, fieldName);
             case STRING:
                 return toString(value, dateFormat, timeFormat, timestampFormat);
             case TIME:
-                return toTime(value, timeFormat);
+                return toTime(value, timeFormat, fieldName);
             case TIMESTAMP:
-                return toTimestamp(value, timestampFormat);
+                return toTimestamp(value, timestampFormat, fieldName);
             case ARRAY:
-                return toArray(value);
+                return toArray(value, fieldName);
+            case MAP:
+                return toMap(value, fieldName);
             case RECORD:
                 final RecordDataType recordType = (RecordDataType) dataType;
                 final RecordSchema childSchema = recordType.getChildSchema();
-                return toRecord(value, childSchema);
+                return toRecord(value, childSchema, fieldName);
             case CHOICE: {
                 if (value == null) {
                     return null;
@@ -89,10 +91,10 @@ public class DataTypeUtils {
                 final DataType chosenDataType = chooseDataType(value, choiceDataType);
                 if (chosenDataType == null) {
                     throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass()
-                        + " to any of the following available Sub-Types for a Choice: " + choiceDataType.getPossibleSubTypes());
+                        + " for field " + fieldName + " to any of the following available Sub-Types for a Choice: " + choiceDataType.getPossibleSubTypes());
                 }
 
-                return convertType(value, chosenDataType);
+                return convertType(value, chosenDataType, fieldName);
             }
         }
 
@@ -132,6 +134,8 @@ public class DataTypeUtils {
                 return isTimestampTypeCompatible(value, dataType.getFormat());
             case STRING:
                 return isStringTypeCompatible(value);
+            case MAP:
+                return isMapTypeCompatible(value);
             case CHOICE: {
                 final DataType chosenDataType = chooseDataType(value, (ChoiceDataType) dataType);
                 return chosenDataType != null;
@@ -151,7 +155,7 @@ public class DataTypeUtils {
         return null;
     }
 
-    public static Record toRecord(final Object value, final RecordSchema recordSchema) {
+    public static Record toRecord(final Object value, final RecordSchema recordSchema, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -163,7 +167,7 @@ public class DataTypeUtils {
         if (value instanceof Map) {
             if (recordSchema == null) {
                 throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass()
-                    + " to Record because the value is a Map but no Record Schema was provided");
+                    + " to Record for field " + fieldName + " because the value is a Map but no Record Schema was provided");
             }
 
             final Map<?, ?> map = (Map<?, ?>) value;
@@ -182,21 +186,21 @@ public class DataTypeUtils {
                 }
 
                 final Object rawValue = entry.getValue();
-                final Object coercedValue = convertType(rawValue, desiredTypeOption.get());
+                final Object coercedValue = convertType(rawValue, desiredTypeOption.get(), fieldName);
                 coercedValues.put(key, coercedValue);
             }
 
             return new MapRecord(recordSchema, coercedValues);
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Record");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Record for field " + fieldName);
     }
 
     public static boolean isRecordTypeCompatible(final Object value) {
         return value != null && value instanceof Record;
     }
 
-    public static Object[] toArray(final Object value) {
+    public static Object[] toArray(final Object value, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -205,13 +209,70 @@ public class DataTypeUtils {
             return (Object[]) value;
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Object Array");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Object Array for field " + fieldName);
     }
 
     public static boolean isArrayTypeCompatible(final Object value) {
         return value != null && value instanceof Object[];
     }
 
+    @SuppressWarnings("unchecked")
+    public static Map<String, Object> toMap(final Object value, final String fieldName) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Map) {
+            final Map<?, ?> original = (Map<?, ?>) value;
+
+            boolean keysAreStrings = true;
+            for (final Object key : original.keySet()) {
+                if (!(key instanceof String)) {
+                    keysAreStrings = false;
+                }
+            }
+
+            if (keysAreStrings) {
+                return (Map<String, Object>) value;
+            }
+
+            final Map<String, Object> transformed = new HashMap<>();
+            for (final Map.Entry<?, ?> entry : original.entrySet()) {
+                final Object key = entry.getKey();
+                if (key == null) {
+                    transformed.put(null, entry.getValue());
+                } else {
+                    transformed.put(key.toString(), entry.getValue());
+                }
+            }
+
+            return transformed;
+        }
+
+        if (value instanceof Record) {
+            final Record record = (Record) value;
+            final RecordSchema recordSchema = record.getSchema();
+            if (recordSchema == null) {
+                throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type Record to Map for field " + fieldName
+                    + " because Record does not have an associated Schema");
+            }
+
+            final Map<String, Object> map = new HashMap<>();
+            for (final String recordFieldName : recordSchema.getFieldNames()) {
+                map.put(recordFieldName, record.getValue(recordFieldName));
+            }
+
+            return map;
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Map for field " + fieldName);
+    }
+
+    public static boolean isMapTypeCompatible(final Object value) {
+        return value != null && value instanceof Map;
+    }
+
+
     public static String toString(final Object value, final String dateFormat, final String timeFormat, final String timestampFormat) {
         if (value == null) {
             return null;
@@ -238,10 +299,10 @@ public class DataTypeUtils {
     }
 
     public static boolean isStringTypeCompatible(final Object value) {
-        return value != null && (value instanceof String || value instanceof java.util.Date);
+        return value != null;
     }
 
-    public static java.sql.Date toDate(final Object value, final String format) {
+    public static java.sql.Date toDate(final Object value, final String format, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -261,11 +322,11 @@ public class DataTypeUtils {
                 return new Date(utilDate.getTime());
             } catch (final ParseException e) {
                 throw new IllegalTypeConversionException("Could not convert value [" + value
-                    + "] of type java.lang.String to Date because the value is not in the expected date format: " + format);
+                    + "] of type java.lang.String to Date because the value is not in the expected date format: " + format + " for field " + fieldName);
             }
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Date");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Date for field " + fieldName);
     }
 
     public static boolean isDateTypeCompatible(final Object value, final String format) {
@@ -289,7 +350,7 @@ public class DataTypeUtils {
         return false;
     }
 
-    public static Time toTime(final Object value, final String format) {
+    public static Time toTime(final Object value, final String format, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -309,11 +370,11 @@ public class DataTypeUtils {
                 return new Time(utilDate.getTime());
             } catch (final ParseException e) {
                 throw new IllegalTypeConversionException("Could not convert value [" + value
-                    + "] of type java.lang.String to Time because the value is not in the expected date format: " + format);
+                    + "] of type java.lang.String to Time for field " + fieldName + " because the value is not in the expected date format: " + format);
             }
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Time");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Time for field " + fieldName);
     }
 
     private static DateFormat getDateFormat(final String format) {
@@ -326,7 +387,7 @@ public class DataTypeUtils {
         return isDateTypeCompatible(value, format);
     }
 
-    public static Timestamp toTimestamp(final Object value, final String format) {
+    public static Timestamp toTimestamp(final Object value, final String format, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -346,11 +407,11 @@ public class DataTypeUtils {
                 return new Timestamp(utilDate.getTime());
             } catch (final ParseException e) {
                 throw new IllegalTypeConversionException("Could not convert value [" + value
-                    + "] of type java.lang.String to Timestamp because the value is not in the expected date format: " + format);
+                    + "] of type java.lang.String to Timestamp for field " + fieldName + " because the value is not in the expected date format: " + format);
             }
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Timestamp");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Timestamp for field " + fieldName);
     }
 
     public static boolean isTimestampTypeCompatible(final Object value, final String format) {
@@ -358,7 +419,7 @@ public class DataTypeUtils {
     }
 
 
-    public static BigInteger toBigInt(final Object value) {
+    public static BigInteger toBigInt(final Object value, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -370,14 +431,14 @@ public class DataTypeUtils {
             return BigInteger.valueOf((Long) value);
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to BigInteger");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to BigInteger for field " + fieldName);
     }
 
     public static boolean isBigIntTypeCompatible(final Object value) {
         return value == null && (value instanceof BigInteger || value instanceof Long);
     }
 
-    public static Boolean toBoolean(final Object value) {
+    public static Boolean toBoolean(final Object value, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -394,7 +455,7 @@ public class DataTypeUtils {
             }
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Boolean");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Boolean for field " + fieldName);
     }
 
     public static boolean isBooleanTypeCompatible(final Object value) {
@@ -411,7 +472,7 @@ public class DataTypeUtils {
         return false;
     }
 
-    public static Double toDouble(final Object value) {
+    public static Double toDouble(final Object value, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -424,7 +485,7 @@ public class DataTypeUtils {
             return Double.parseDouble((String) value);
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Double");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Double for field " + fieldName);
     }
 
     public static boolean isDoubleTypeCompatible(final Object value) {
@@ -452,7 +513,7 @@ public class DataTypeUtils {
         return false;
     }
 
-    public static Float toFloat(final Object value) {
+    public static Float toFloat(final Object value, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -465,14 +526,14 @@ public class DataTypeUtils {
             return Float.parseFloat((String) value);
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Float");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Float for field " + fieldName);
     }
 
     public static boolean isFloatTypeCompatible(final Object value) {
         return isNumberTypeCompatible(value, s -> Float.parseFloat(s));
     }
 
-    public static Long toLong(final Object value) {
+    public static Long toLong(final Object value, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -489,7 +550,7 @@ public class DataTypeUtils {
             return ((java.util.Date) value).getTime();
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Long");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Long for field " + fieldName);
     }
 
     public static boolean isLongTypeCompatible(final Object value) {
@@ -518,7 +579,7 @@ public class DataTypeUtils {
     }
 
 
-    public static Integer toInteger(final Object value) {
+    public static Integer toInteger(final Object value, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -531,7 +592,7 @@ public class DataTypeUtils {
             return Integer.parseInt((String) value);
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Integer");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Integer for field " + fieldName);
     }
 
     public static boolean isIntegerTypeCompatible(final Object value) {
@@ -539,7 +600,7 @@ public class DataTypeUtils {
     }
 
 
-    public static Short toShort(final Object value) {
+    public static Short toShort(final Object value, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -552,14 +613,14 @@ public class DataTypeUtils {
             return Short.parseShort((String) value);
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Short");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Short for field " + fieldName);
     }
 
     public static boolean isShortTypeCompatible(final Object value) {
         return isNumberTypeCompatible(value, s -> Short.parseShort(s));
     }
 
-    public static Byte toByte(final Object value) {
+    public static Byte toByte(final Object value, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -572,7 +633,7 @@ public class DataTypeUtils {
             return Byte.parseByte((String) value);
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Byte");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Byte for field " + fieldName);
     }
 
     public static boolean isByteTypeCompatible(final Object value) {
@@ -580,7 +641,7 @@ public class DataTypeUtils {
     }
 
 
-    public static Character toCharacter(final Object value) {
+    public static Character toCharacter(final Object value, final String fieldName) {
         if (value == null) {
             return null;
         }
@@ -592,13 +653,14 @@ public class DataTypeUtils {
         if (value instanceof CharSequence) {
             final CharSequence charSeq = (CharSequence) value;
             if (charSeq.length() == 0) {
-                throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Character because it has a length of 0");
+                throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass()
+                    + " to Character because it has a length of 0 for field " + fieldName);
             }
 
             return charSeq.charAt(0);
         }
 
-        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Character");
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Character for field " + fieldName);
     }
 
     public static boolean isCharacterTypeCompatible(final Object value) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java
new file mode 100644
index 0000000..5a61275
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestSimpleRecordSchema {
+
+    @Test
+    public void testPreventsTwoFieldsWithSameAlias() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType(), null, set("foo", "bar")));
+        fields.add(new RecordField("goodbye", RecordFieldType.STRING.getDataType(), null, set("baz", "bar")));
+
+        try {
+            new SimpleRecordSchema(fields);
+            Assert.fail("Was able to create two fields with same alias");
+        } catch (final IllegalArgumentException expected) {
+        }
+    }
+
+    @Test
+    public void testPreventsTwoFieldsWithSameName() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType(), null, set("foo", "bar")));
+        fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType()));
+
+        try {
+            new SimpleRecordSchema(fields);
+            Assert.fail("Was able to create two fields with same name");
+        } catch (final IllegalArgumentException expected) {
+        }
+    }
+
+    @Test
+    public void testPreventsTwoFieldsWithConflictingNamesAliases() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType(), null, set("foo", "bar")));
+        fields.add(new RecordField("bar", RecordFieldType.STRING.getDataType()));
+
+        try {
+            new SimpleRecordSchema(fields);
+            Assert.fail("Was able to create two fields with conflicting names/aliases");
+        } catch (final IllegalArgumentException expected) {
+        }
+    }
+
+    private Set<String> set(final String... values) {
+        final Set<String> set = new HashSet<>();
+        for (final String value : values) {
+            set.add(value);
+        }
+        return set;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/record/TestMapRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/record/TestMapRecord.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/record/TestMapRecord.java
new file mode 100644
index 0000000..82e20a6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/record/TestMapRecord.java
@@ -0,0 +1,188 @@
+/*
+ * 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.record;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+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 org.apache.nifi.serialization.SimpleRecordSchema;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestMapRecord {
+
+    @Test
+    public void testDefaultValue() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("noDefault", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("defaultOfHello", RecordFieldType.STRING.getDataType(), "hello"));
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+        final Map<String, Object> values = new HashMap<>();
+        final Record record = new MapRecord(schema, values);
+
+        assertNull(record.getValue("noDefault"));
+        assertEquals("hello", record.getValue("defaultOfHello"));
+    }
+
+    @Test
+    public void testDefaultValueInGivenField() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("noDefault", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("defaultOfHello", RecordFieldType.STRING.getDataType(), "hello"));
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+        final Map<String, Object> values = new HashMap<>();
+        final Record record = new MapRecord(schema, values);
+
+        assertNull(record.getValue("noDefault"));
+        assertEquals("hello", record.getValue("defaultOfHello"));
+
+        final RecordField newField = new RecordField("noDefault", RecordFieldType.STRING.getDataType(), "new");
+        assertEquals("new", record.getValue(newField));
+    }
+
+    @Test
+    public void testIllegalDefaultValue() {
+        new RecordField("hello", RecordFieldType.STRING.getDataType(), 84);
+        new RecordField("hello", RecordFieldType.STRING.getDataType(), (Object) null);
+        new RecordField("hello", RecordFieldType.INT.getDataType(), 84);
+        new RecordField("hello", RecordFieldType.INT.getDataType(), (Object) null);
+
+        try {
+            new RecordField("hello", RecordFieldType.INT.getDataType(), "foo");
+            Assert.fail("Was able to set a default value of \"foo\" for INT type");
+        } catch (final IllegalArgumentException expected) {
+            // expected
+        }
+    }
+
+    private Set<String> set(final String... values) {
+        final Set<String> set = new HashSet<>();
+        for (final String value : values) {
+            set.add(value);
+        }
+        return set;
+    }
+
+    @Test
+    public void testAliasOneValue() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), null, set("bar", "baz")));
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+        final Map<String, Object> values = new HashMap<>();
+        values.put("bar", 1);
+
+        final Record record = new MapRecord(schema, values);
+        assertEquals(1, record.getValue("foo"));
+        assertEquals(1, record.getValue("bar"));
+        assertEquals(1, record.getValue("baz"));
+    }
+
+    @Test
+    public void testAliasConflictingValues() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), null, set("bar", "baz")));
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+        final Map<String, Object> values = new HashMap<>();
+        values.put("bar", 1);
+        values.put("foo", null);
+
+        final Record record = new MapRecord(schema, values);
+        assertEquals(1, record.getValue("foo"));
+        assertEquals(1, record.getValue("bar"));
+        assertEquals(1, record.getValue("baz"));
+    }
+
+    @Test
+    public void testAliasConflictingAliasValues() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), null, set("bar", "baz")));
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+        final Map<String, Object> values = new HashMap<>();
+        values.put("baz", 1);
+        values.put("bar", 33);
+
+        final Record record = new MapRecord(schema, values);
+        assertEquals(33, record.getValue("foo"));
+        assertEquals(33, record.getValue("bar"));
+        assertEquals(33, record.getValue("baz"));
+    }
+
+    @Test
+    public void testAliasInGivenField() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), null, set("bar", "baz")));
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+        final Map<String, Object> values = new HashMap<>();
+        values.put("bar", 33);
+
+        final Record record = new MapRecord(schema, values);
+        assertEquals(33, record.getValue("foo"));
+        assertEquals(33, record.getValue("bar"));
+        assertEquals(33, record.getValue("baz"));
+
+        final RecordField noAlias = new RecordField("hello", RecordFieldType.STRING.getDataType());
+        assertNull(record.getValue(noAlias));
+
+        final RecordField withAlias = new RecordField("hello", RecordFieldType.STRING.getDataType(), null, set("baz"));
+        assertEquals(33, record.getValue(withAlias));
+        assertEquals("33", record.getAsString(withAlias, withAlias.getDataType().getFormat()));
+    }
+
+
+    @Test
+    public void testDefaultValueWithAliasValue() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), "hello", set("bar", "baz")));
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+        final Map<String, Object> values = new HashMap<>();
+        values.put("baz", 1);
+        values.put("bar", 33);
+
+        final Record record = new MapRecord(schema, values);
+        assertEquals(33, record.getValue("foo"));
+        assertEquals(33, record.getValue("bar"));
+        assertEquals(33, record.getValue("baz"));
+    }
+
+    @Test
+    public void testDefaultValueWithAliasesDefined() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), "hello", set("bar", "baz")));
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+        final Map<String, Object> values = new HashMap<>();
+        final Record record = new MapRecord(schema, values);
+        assertEquals("hello", record.getValue("foo"));
+        assertEquals("hello", record.getValue("bar"));
+        assertEquals("hello", record.getValue("baz"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml
index d86a8c5..16479f1 100644
--- 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
@@ -70,7 +70,11 @@
         <dependency>
             <groupId>org.apache.avro</groupId>
             <artifactId>avro</artifactId>
-            <version>1.8.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <scope>test</scope>
         </dependency>
     </dependencies>
 
@@ -96,6 +100,7 @@
                         <exclude>src/test/resources/json/json-with-unicode.json</exclude>
                         <exclude>src/test/resources/json/primitive-type-array.json</exclude>
                         <exclude>src/test/resources/json/single-bank-account.json</exclude>
+                        <exclude>src/test/resources/json/single-bank-account-wrong-field-type.json</exclude>
                         <exclude>src/test/resources/json/single-element-nested-array.json</exclude>
                         <exclude>src/test/resources/json/single-element-nested.json</exclude>
                         <exclude>src/test/resources/json/output/dataTypes.json</exclude>

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 f92816f..f5b4373 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
@@ -19,31 +19,47 @@ package org.apache.nifi.avro;
 
 import java.io.IOException;
 import java.io.InputStream;
+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.controller.AbstractControllerService;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
-import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
 
 @Tags({"avro", "parse", "record", "row", "reader", "delimited", "comma", "separated", "values"})
-@CapabilityDescription("Parses Avro data and returns each Avro record as an separate Record object. The Avro data must contain "
-    + "the schema itself.")
-public class AvroReader extends AbstractControllerService implements RowRecordReaderFactory {
+@CapabilityDescription("Parses Avro data and returns each Avro record as an separate Record object. The Avro data may contain the schema itself, "
+    + "or the schema can be externalized and accessed by one of the methods offered by the 'Schema Access Strategy' property.")
+public class AvroReader extends SchemaRegistryService implements RecordReaderFactory {
+    private final AllowableValue EMBEDDED_AVRO_SCHEMA = new AllowableValue("embedded-avro-schema",
+        "Use Embedded Avro Schema", "The FlowFile has the Avro Schema embedded within the content, and this schema will be used.");
+
 
     @Override
-    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws MalformedRecordException, IOException {
-        return new AvroRecordReader(in);
+    protected List<AllowableValue> getSchemaAccessStrategyValues() {
+        final List<AllowableValue> allowableValues = new ArrayList<>(super.getSchemaAccessStrategyValues());
+        allowableValues.add(EMBEDDED_AVRO_SCHEMA);
+        return allowableValues;
     }
 
     @Override
-    public RecordSchema getSchema(final FlowFile flowFile) throws MalformedRecordException, IOException {
-        // TODO: Need to support retrieving schema from registry instead of requiring that it be in the Avro file.
-        return null;
+    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException {
+        final String schemaAccessStrategy = getConfigurationContext().getProperty(SCHEMA_ACCESS_STRATEGY).getValue();
+        if (EMBEDDED_AVRO_SCHEMA.getValue().equals(schemaAccessStrategy)) {
+            return new AvroReaderWithEmbeddedSchema(in);
+        } else {
+            return new AvroReaderWithExplicitSchema(in, getSchema(flowFile, in));
+        }
     }
 
+    @Override
+    protected AllowableValue getDefaultSchemaAccessStrategy() {
+        return EMBEDDED_AVRO_SCHEMA;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithEmbeddedSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithEmbeddedSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithEmbeddedSchema.java
new file mode 100644
index 0000000..aa61e4c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithEmbeddedSchema.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.avro;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public class AvroReaderWithEmbeddedSchema extends AvroRecordReader {
+    private final DataFileStream<GenericRecord> dataFileStream;
+    private final InputStream in;
+    private final Schema avroSchema;
+    private final RecordSchema recordSchema;
+
+    public AvroReaderWithEmbeddedSchema(final InputStream in) throws IOException {
+        this.in = in;
+        dataFileStream = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>());
+        this.avroSchema = dataFileStream.getSchema();
+        recordSchema = AvroTypeUtil.createSchema(avroSchema);
+    }
+
+    @Override
+    public void close() throws IOException {
+        dataFileStream.close();
+        in.close();
+    }
+
+    @Override
+    protected GenericRecord nextAvroRecord() {
+        if (!dataFileStream.hasNext()) {
+            return null;
+        }
+
+        return dataFileStream.next();
+    }
+
+    @Override
+    public RecordSchema getSchema() throws MalformedRecordException {
+        return recordSchema;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java
new file mode 100644
index 0000000..104214c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java
@@ -0,0 +1,75 @@
+/*
+ * 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.avro;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public class AvroReaderWithExplicitSchema extends AvroRecordReader {
+    private final InputStream in;
+    private final Schema avroSchema;
+    private final RecordSchema recordSchema;
+    private final DatumReader<GenericRecord> datumReader;
+    private final BinaryDecoder decoder;
+    private GenericRecord genericRecord;
+
+    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema) throws IOException, SchemaNotFoundException {
+        this.in = in;
+        this.recordSchema = recordSchema;
+
+        this.avroSchema = AvroTypeUtil.extractAvroSchema(recordSchema);
+        datumReader = new GenericDatumReader<GenericRecord>(avroSchema);
+        decoder = DecoderFactory.get().binaryDecoder(in, null);
+    }
+
+    @Override
+    public void close() throws IOException {
+        in.close();
+    }
+
+    @Override
+    protected GenericRecord nextAvroRecord() throws IOException {
+        if (decoder.isEnd()) {
+            return null;
+        }
+
+        try {
+            genericRecord = datumReader.read(genericRecord, decoder);
+        } catch (final EOFException eof) {
+            return null;
+        }
+
+        return genericRecord;
+    }
+
+    @Override
+    public RecordSchema getSchema() throws MalformedRecordException {
+        return recordSchema;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java
index d725cbf..621ec74 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java
@@ -18,7 +18,6 @@
 package org.apache.nifi.avro;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -30,11 +29,8 @@ import org.apache.avro.LogicalType;
 import org.apache.avro.LogicalTypes;
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
-import org.apache.avro.file.DataFileStream;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericData.Array;
-import org.apache.avro.generic.GenericData.StringType;
-import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.generic.GenericFixed;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.util.Utf8;
@@ -48,38 +44,19 @@ import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.util.DataTypeUtils;
 
-public class AvroRecordReader implements RecordReader {
-    private final InputStream in;
-    private final Schema avroSchema;
-    private final DataFileStream<GenericRecord> dataFileStream;
-    private RecordSchema recordSchema;
+public abstract class AvroRecordReader implements RecordReader {
 
 
-    public AvroRecordReader(final InputStream in) throws IOException, MalformedRecordException {
-        this.in = in;
+    protected abstract GenericRecord nextAvroRecord() throws IOException;
 
-        dataFileStream = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>());
-        this.avroSchema = dataFileStream.getSchema();
-        GenericData.setStringType(this.avroSchema, StringType.String);
-    }
-
-    @Override
-    public void close() throws IOException {
-        dataFileStream.close();
-        in.close();
-    }
 
     @Override
     public Record nextRecord() throws IOException, MalformedRecordException {
-        if (!dataFileStream.hasNext()) {
+        GenericRecord record = nextAvroRecord();
+        if (record == null) {
             return null;
         }
 
-        GenericRecord record = null;
-        while (record == null && dataFileStream.hasNext()) {
-            record = dataFileStream.next();
-        }
-
         final RecordSchema schema = getSchema();
         final Map<String, Object> values = convertAvroRecordToMap(record, schema);
         return new MapRecord(schema, values);
@@ -89,9 +66,18 @@ public class AvroRecordReader implements RecordReader {
     private Map<String, Object> convertAvroRecordToMap(final GenericRecord avroRecord, final RecordSchema recordSchema) {
         final Map<String, Object> values = new HashMap<>(recordSchema.getFieldCount());
 
-        for (final String fieldName : recordSchema.getFieldNames()) {
-            final Object value = avroRecord.get(fieldName);
+        for (final RecordField recordField : recordSchema.getFields()) {
+            Object value = avroRecord.get(recordField.getFieldName());
+            if (value == null) {
+                for (final String alias : recordField.getAliases()) {
+                    value = avroRecord.get(alias);
+                    if (value != null) {
+                        break;
+                    }
+                }
+            }
 
+            final String fieldName = recordField.getFieldName();
             final Field avroField = avroRecord.getSchema().getField(fieldName);
             if (avroField == null) {
                 values.put(fieldName, null);
@@ -101,8 +87,8 @@ public class AvroRecordReader implements RecordReader {
             final Schema fieldSchema = avroField.schema();
             final Object rawValue = normalizeValue(value, fieldSchema);
 
-            final DataType desiredType = recordSchema.getDataType(fieldName).get();
-            final Object coercedValue = DataTypeUtils.convertType(rawValue, desiredType);
+            final DataType desiredType = recordField.getDataType();
+            final Object coercedValue = DataTypeUtils.convertType(rawValue, desiredType, fieldName);
 
             values.put(fieldName, coercedValue);
         }
@@ -215,13 +201,5 @@ public class AvroRecordReader implements RecordReader {
     }
 
 
-    @Override
-    public RecordSchema getSchema() throws MalformedRecordException {
-        if (recordSchema != null) {
-            return recordSchema;
-        }
 
-        recordSchema = AvroTypeUtil.createSchema(avroSchema);
-        return recordSchema;
-    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java
index 03d766c..381e978 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java
@@ -17,48 +17,93 @@
 
 package org.apache.nifi.avro;
 
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.avro.Schema;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.AbstractControllerService;
-import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.schema.access.SchemaField;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.SchemaRegistryRecordSetWriter;
+import org.apache.nifi.serialization.record.RecordSchema;
 
 @Tags({"avro", "result", "set", "writer", "serializer", "record", "recordset", "row"})
 @CapabilityDescription("Writes the contents of a RecordSet in Binary Avro format.")
-public class AvroRecordSetWriter extends AbstractControllerService implements RecordSetWriterFactory {
-    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
-        .name("Avro Schema")
-        .description("The Avro Schema to use when writing out the Result Set")
-        .addValidator(new AvroSchemaValidator())
-        .expressionLanguageSupported(false)
-        .required(true)
+public class AvroRecordSetWriter extends SchemaRegistryRecordSetWriter implements RecordSetWriterFactory {
+    private static final Set<SchemaField> requiredSchemaFields = EnumSet.of(SchemaField.SCHEMA_TEXT, SchemaField.SCHEMA_TEXT_FORMAT);
+
+    static final AllowableValue AVRO_EMBEDDED = new AllowableValue("avro-embedded", "Embed Avro Schema",
+        "The FlowFile will have the Avro schema embedded into the content, as is typical with Avro");
+
+    protected static final PropertyDescriptor SCHEMA_REGISTRY = new PropertyDescriptor.Builder()
+        .name("Schema Registry")
+        .description("Specifies the Controller Service to use for the Schema Registry")
+        .identifiesControllerService(SchemaRegistry.class)
+        .required(false)
         .build();
 
-    private volatile Schema schema;
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(SCHEMA);
+        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(SCHEMA_ACCESS_STRATEGY);
+        properties.add(SCHEMA_REGISTRY);
         return properties;
     }
 
-    @OnEnabled
-    public void storePropertyValues(final ConfigurationContext context) {
-        schema = new Schema.Parser().parse(context.getProperty(SCHEMA).getValue());
+
+    @Override
+    public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream in) throws IOException {
+        final String strategyValue = getConfigurationContext().getProperty(SCHEMA_WRITE_STRATEGY).getValue();
+
+        try {
+            final RecordSchema recordSchema = getSchema(flowFile, in);
+            final Schema avroSchema = AvroTypeUtil.extractAvroSchema(recordSchema);
+
+            if (AVRO_EMBEDDED.getValue().equals(strategyValue)) {
+                return new WriteAvroResultWithSchema(avroSchema);
+            } else {
+                return new WriteAvroResultWithExternalSchema(avroSchema, recordSchema, getSchemaAccessWriter(recordSchema));
+            }
+        } catch (final SchemaNotFoundException e) {
+            throw new ProcessException("Could not determine the Avro Schema to use for writing the content", e);
+        }
+    }
+
+    @Override
+    protected List<AllowableValue> getSchemaWriteStrategyValues() {
+        final List<AllowableValue> allowableValues = new ArrayList<>();
+        allowableValues.add(AVRO_EMBEDDED);
+        allowableValues.addAll(super.getSchemaWriteStrategyValues());
+        return allowableValues;
     }
 
     @Override
-    public RecordSetWriter createWriter(final ComponentLog logger) {
-        return new WriteAvroResult(schema);
+    protected AllowableValue getDefaultSchemaWriteStrategy() {
+        return AVRO_EMBEDDED;
     }
 
+    @Override
+    protected Set<SchemaField> getRequiredSchemaFields(final ValidationContext validationContext) {
+        final String writeStrategyValue = validationContext.getProperty(SCHEMA_WRITE_STRATEGY).getValue();
+        if (writeStrategyValue.equalsIgnoreCase(AVRO_EMBEDDED.getValue())) {
+            return requiredSchemaFields;
+        }
+
+        return super.getRequiredSchemaFields(validationContext);
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroSchemaValidator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroSchemaValidator.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroSchemaValidator.java
index 7151348..4449afc 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroSchemaValidator.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroSchemaValidator.java
@@ -26,6 +26,15 @@ public class AvroSchemaValidator implements Validator {
 
     @Override
     public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+        if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input)) {
+            return new ValidationResult.Builder()
+                .input(input)
+                .subject(subject)
+                .valid(true)
+                .explanation("Expression Language is present")
+                .build();
+        }
+
         try {
             new Schema.Parser().parse(input);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
index 1810c83..b65026a 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
@@ -20,6 +20,7 @@ package org.apache.nifi.avro;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Optional;
 import java.util.stream.Collectors;
 
 import org.apache.avro.LogicalType;
@@ -27,14 +28,37 @@ import org.apache.avro.LogicalTypes;
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
 import org.apache.avro.Schema.Type;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 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 org.apache.nifi.serialization.record.SchemaIdentifier;
 import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
 
 public class AvroTypeUtil {
+    public static final String AVRO_SCHEMA_FORMAT = "avro";
+
+    public static Schema extractAvroSchema(final RecordSchema recordSchema) throws SchemaNotFoundException {
+        final Optional<String> schemaFormatOption = recordSchema.getSchemaFormat();
+        if (!schemaFormatOption.isPresent()) {
+            throw new SchemaNotFoundException("No Schema Format was present in the RecordSchema");
+        }
+
+        final String schemaFormat = schemaFormatOption.get();
+        if (!schemaFormat.equals(AVRO_SCHEMA_FORMAT)) {
+            throw new SchemaNotFoundException("Schema provided is not in Avro format");
+        }
+
+        final Optional<String> textOption = recordSchema.getSchemaText();
+        if (!textOption.isPresent()) {
+            throw new SchemaNotFoundException("No Schema text was present in the RecordSchema");
+        }
+
+        final String text = textOption.get();
+        return new Schema.Parser().parse(text);
+    }
 
     public static DataType determineDataType(final Schema avroSchema) {
         final Type avroType = avroSchema.getType();
@@ -93,15 +117,18 @@ public class AvroTypeUtil {
                     final String fieldName = field.name();
                     final Schema fieldSchema = field.schema();
                     final DataType fieldType = determineDataType(fieldSchema);
-                    recordFields.add(new RecordField(fieldName, fieldType));
+                    recordFields.add(new RecordField(fieldName, fieldType, field.defaultVal(), field.aliases()));
                 }
 
-                final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
+                final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, avroSchema.toString(), AVRO_SCHEMA_FORMAT, SchemaIdentifier.EMPTY);
                 return RecordFieldType.RECORD.getRecordDataType(recordSchema);
             }
             case NULL:
+                return RecordFieldType.STRING.getDataType();
             case MAP:
-                return RecordFieldType.RECORD.getDataType();
+                final Schema valueSchema = avroSchema.getValueType();
+                final DataType valueType = determineDataType(valueSchema);
+                return RecordFieldType.MAP.getMapDataType(valueType);
             case UNION: {
                 final List<Schema> nonNullSubSchemas = avroSchema.getTypes().stream()
                     .filter(s -> s.getType() != Type.NULL)
@@ -129,10 +156,11 @@ public class AvroTypeUtil {
         for (final Field field : avroSchema.getFields()) {
             final String fieldName = field.name();
             final DataType dataType = AvroTypeUtil.determineDataType(field.schema());
-            recordFields.add(new RecordField(fieldName, dataType));
+
+            recordFields.add(new RecordField(fieldName, dataType, field.defaultVal(), field.aliases()));
         }
 
-        final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
+        final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, avroSchema.toString(), AVRO_SCHEMA_FORMAT, SchemaIdentifier.EMPTY);
         return recordSchema;
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java
index b512b82..55f796a 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java
@@ -42,61 +42,44 @@ import org.apache.avro.io.DatumWriter;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.WriteResult;
 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.apache.nifi.serialization.record.util.DataTypeUtils;
 import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
 
-public class WriteAvroResult implements RecordSetWriter {
+public abstract class WriteAvroResult implements RecordSetWriter {
     private final Schema schema;
 
     public WriteAvroResult(final Schema schema) {
         this.schema = schema;
     }
 
-    @Override
-    public WriteResult write(final RecordSet rs, final OutputStream outStream) throws IOException {
-        Record record = rs.next();
-        if (record == null) {
-            return WriteResult.of(0, Collections.emptyMap());
-        }
-
-        int nrOfRows = 0;
-        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
-        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
-            dataFileWriter.create(schema, outStream);
-
-            do {
-                final GenericRecord rec = createAvroRecord(record, schema);
-                dataFileWriter.append(rec);
-                nrOfRows++;
-            } while ((record = rs.next()) != null);
-        }
-
-        return WriteResult.of(nrOfRows, Collections.emptyMap());
+    protected Schema getSchema() {
+        return schema;
     }
 
-    private GenericRecord createAvroRecord(final Record record, final Schema avroSchema) throws IOException {
+    protected GenericRecord createAvroRecord(final Record record, final Schema avroSchema) throws IOException {
         final GenericRecord rec = new GenericData.Record(avroSchema);
         final RecordSchema recordSchema = record.getSchema();
 
-        for (final String fieldName : recordSchema.getFieldNames()) {
-            final Object rawValue = record.getValue(fieldName);
+        for (final RecordField recordField : recordSchema.getFields()) {
+            final Object rawValue = record.getValue(recordField);
+            final String fieldName = recordField.getFieldName();
 
             final Field field = avroSchema.getField(fieldName);
             if (field == null) {
                 continue;
             }
 
-            final Object converted = convertToAvroObject(rawValue, field.schema());
+            final Object converted = convertToAvroObject(rawValue, field.schema(), fieldName);
             rec.put(fieldName, converted);
         }
 
         return rec;
     }
 
-    private Object convertToAvroObject(final Object rawValue, final Schema fieldSchema) throws IOException {
+    protected Object convertToAvroObject(final Object rawValue, final Schema fieldSchema, final String fieldName) throws IOException {
         if (rawValue == null) {
             return null;
         }
@@ -105,43 +88,43 @@ public class WriteAvroResult implements RecordSetWriter {
             case INT: {
                 final LogicalType logicalType = fieldSchema.getLogicalType();
                 if (logicalType == null) {
-                    return DataTypeUtils.toInteger(rawValue);
+                    return DataTypeUtils.toInteger(rawValue, fieldName);
                 }
 
                 if (LogicalTypes.date().getName().equals(logicalType.getName())) {
-                    final long longValue = DataTypeUtils.toLong(rawValue);
+                    final long longValue = DataTypeUtils.toLong(rawValue, fieldName);
                     final Date date = new Date(longValue);
                     final Duration duration = Duration.between(new Date(0L).toInstant(), date.toInstant());
                     final long days = duration.toDays();
                     return (int) days;
                 } else if (LogicalTypes.timeMillis().getName().equals(logicalType.getName())) {
-                    final long longValue = DataTypeUtils.toLong(rawValue);
+                    final long longValue = DataTypeUtils.toLong(rawValue, fieldName);
                     final Date date = new Date(longValue);
                     final Duration duration = Duration.between(date.toInstant().truncatedTo(ChronoUnit.DAYS), date.toInstant());
                     final long millisSinceMidnight = duration.toMillis();
                     return (int) millisSinceMidnight;
                 }
 
-                return DataTypeUtils.toInteger(rawValue);
+                return DataTypeUtils.toInteger(rawValue, fieldName);
             }
             case LONG: {
                 final LogicalType logicalType = fieldSchema.getLogicalType();
                 if (logicalType == null) {
-                    return DataTypeUtils.toLong(rawValue);
+                    return DataTypeUtils.toLong(rawValue, fieldName);
                 }
 
                 if (LogicalTypes.timeMicros().getName().equals(logicalType.getName())) {
-                    final long longValue = DataTypeUtils.toLong(rawValue);
+                    final long longValue = DataTypeUtils.toLong(rawValue, fieldName);
                     final Date date = new Date(longValue);
                     final Duration duration = Duration.between(date.toInstant().truncatedTo(ChronoUnit.DAYS), date.toInstant());
                     return duration.toMillis() * 1000L;
                 } else if (LogicalTypes.timestampMillis().getName().equals(logicalType.getName())) {
-                    return DataTypeUtils.toLong(rawValue);
+                    return DataTypeUtils.toLong(rawValue, fieldName);
                 } else if (LogicalTypes.timestampMicros().getName().equals(logicalType.getName())) {
-                    return DataTypeUtils.toLong(rawValue) * 1000L;
+                    return DataTypeUtils.toLong(rawValue, fieldName) * 1000L;
                 }
 
-                return DataTypeUtils.toLong(rawValue);
+                return DataTypeUtils.toLong(rawValue, fieldName);
             }
             case BYTES:
             case FIXED:
@@ -157,10 +140,10 @@ public class WriteAvroResult implements RecordSetWriter {
                 if (rawValue instanceof Record) {
                     final Record recordValue = (Record) rawValue;
                     final Map<String, Object> map = new HashMap<>();
-                    for (final String recordFieldName : recordValue.getSchema().getFieldNames()) {
-                        final Object v = recordValue.getValue(recordFieldName);
+                    for (final RecordField recordField : recordValue.getSchema().getFields()) {
+                        final Object v = recordValue.getValue(recordField);
                         if (v != null) {
-                            map.put(recordFieldName, v);
+                            map.put(recordField.getFieldName(), v);
                         }
                     }
 
@@ -172,15 +155,16 @@ public class WriteAvroResult implements RecordSetWriter {
                 final GenericData.Record avroRecord = new GenericData.Record(fieldSchema);
 
                 final Record record = (Record) rawValue;
-                for (final String recordFieldName : record.getSchema().getFieldNames()) {
-                    final Object recordFieldValue = record.getValue(recordFieldName);
+                for (final RecordField recordField : record.getSchema().getFields()) {
+                    final Object recordFieldValue = record.getValue(recordField);
+                    final String recordFieldName = recordField.getFieldName();
 
                     final Field field = fieldSchema.getField(recordFieldName);
                     if (field == null) {
                         continue;
                     }
 
-                    final Object converted = convertToAvroObject(recordFieldValue, field.schema());
+                    final Object converted = convertToAvroObject(recordFieldValue, field.schema(), fieldName);
                     avroRecord.put(recordFieldName, converted);
                 }
                 return avroRecord;
@@ -188,16 +172,16 @@ public class WriteAvroResult implements RecordSetWriter {
                 final Object[] objectArray = (Object[]) rawValue;
                 final List<Object> list = new ArrayList<>(objectArray.length);
                 for (final Object o : objectArray) {
-                    final Object converted = convertToAvroObject(o, fieldSchema.getElementType());
+                    final Object converted = convertToAvroObject(o, fieldSchema.getElementType(), fieldName);
                     list.add(converted);
                 }
                 return list;
             case BOOLEAN:
-                return DataTypeUtils.toBoolean(rawValue);
+                return DataTypeUtils.toBoolean(rawValue, fieldName);
             case DOUBLE:
-                return DataTypeUtils.toDouble(rawValue);
+                return DataTypeUtils.toDouble(rawValue, fieldName);
             case FLOAT:
-                return DataTypeUtils.toFloat(rawValue);
+                return DataTypeUtils.toFloat(rawValue, fieldName);
             case NULL:
                 return null;
             case ENUM:

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithExternalSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithExternalSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithExternalSchema.java
new file mode 100644
index 0000000..74306e4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithExternalSchema.java
@@ -0,0 +1,75 @@
+/*
+ * 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.avro;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.nifi.schema.access.SchemaAccessWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+public class WriteAvroResultWithExternalSchema extends WriteAvroResult {
+    private final SchemaAccessWriter schemaAccessWriter;
+    private final RecordSchema recordSchema;
+
+    public WriteAvroResultWithExternalSchema(final Schema avroSchema, final RecordSchema recordSchema, final SchemaAccessWriter schemaAccessWriter) {
+        super(avroSchema);
+        this.recordSchema = recordSchema;
+        this.schemaAccessWriter = schemaAccessWriter;
+    }
+
+    @Override
+    public WriteResult write(final RecordSet rs, final OutputStream outStream) throws IOException {
+        Record record = rs.next();
+        if (record == null) {
+            return WriteResult.of(0, Collections.emptyMap());
+        }
+
+        int nrOfRows = 0;
+        final Schema schema = getSchema();
+        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+
+        final BufferedOutputStream bufferedOut = new BufferedOutputStream(outStream);
+        schemaAccessWriter.writeHeader(recordSchema, bufferedOut);
+
+        final BinaryEncoder encoder = EncoderFactory.get().blockingBinaryEncoder(bufferedOut, null);
+
+        do {
+            final GenericRecord rec = createAvroRecord(record, schema);
+
+            datumWriter.write(rec, encoder);
+            encoder.flush();
+            nrOfRows++;
+        } while ((record = rs.next()) != null);
+
+        bufferedOut.flush();
+
+        return WriteResult.of(nrOfRows, schemaAccessWriter.getAttributes(recordSchema));
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithSchema.java
new file mode 100644
index 0000000..dca8aac
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithSchema.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.avro;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSet;
+
+public class WriteAvroResultWithSchema extends WriteAvroResult {
+
+    public WriteAvroResultWithSchema(final Schema schema) {
+        super(schema);
+    }
+
+    @Override
+    public WriteResult write(final RecordSet rs, final OutputStream outStream) throws IOException {
+        Record record = rs.next();
+        if (record == null) {
+            return WriteResult.of(0, Collections.emptyMap());
+        }
+
+        int nrOfRows = 0;
+        final Schema schema = getSchema();
+        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+
+        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
+            dataFileWriter.create(schema, outStream);
+
+            do {
+                final GenericRecord rec = createAvroRecord(record, schema);
+                dataFileWriter.append(rec);
+                nrOfRows++;
+            } while ((record = rs.next()) != null);
+        }
+
+        return WriteResult.of(nrOfRows, Collections.emptyMap());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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
new file mode 100644
index 0000000..71093de
--- /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/CSVHeaderSchemaStrategy.java
@@ -0,0 +1,67 @@
+/*
+ * 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 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.Set;
+
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.io.input.BOMInputStream;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
+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.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public class CSVHeaderSchemaStrategy implements SchemaAccessStrategy {
+    private static final Set<SchemaField> schemaFields = EnumSet.noneOf(SchemaField.class);
+
+    @Override
+    public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException {
+        try {
+            final CSVFormat csvFormat = CSVUtils.createCSVFormat(context).withFirstRecordAsHeader();
+            try (final Reader reader = new InputStreamReader(new BOMInputStream(contentStream));
+                final CSVParser csvParser = new CSVParser(reader, csvFormat)) {
+
+                final List<RecordField> fields = new ArrayList<>();
+                for (final String columnName : csvParser.getHeaderMap().keySet()) {
+                    fields.add(new RecordField(columnName, RecordFieldType.STRING.getDataType()));
+                }
+
+                return new SimpleRecordSchema(fields);
+            }
+        } catch (final Exception e) {
+            throw new SchemaNotFoundException("Failed to read Header line from CSV", e);
+        }
+    }
+
+    @Override
+    public Set<SchemaField> getSuppliedSchemaFields() {
+        return schemaFields;
+    }
+}


[3/7] nifi git commit: NIFI-3682: This closes #1682. Add Schema Access Strategy and Schema Write Strategy Record Readers and Writers; bug fixes.

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 6b06ebf..fb34f8f 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,6 +17,7 @@
 
 package org.apache.nifi.csv;
 
+import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
@@ -26,21 +27,31 @@ import org.apache.commons.csv.CSVFormat;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaAccessStrategy;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
 import org.apache.nifi.serialization.DateTimeUtils;
 import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
-import org.apache.nifi.serialization.SchemaRegistryRecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
 import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.stream.io.NonCloseableInputStream;
 
 @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 "
     + "the values. See Controller Service's Usage for further documentation.")
-public class CSVReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory {
+public class CSVReader extends SchemaRegistryService implements RecordReaderFactory {
+
+    private final AllowableValue headerDerivedAllowableValue = new AllowableValue("csv-header-derived", "Use String Fields From Header",
+        "The first non-comment line of the CSV file is a header line that contains the names of the columns. The schema will be derived by using the "
+            + "column names in the header and assuming that all columns are of type String.");
+    private final SchemaAccessStrategy headerDerivedSchemaStrategy = new CSVHeaderSchemaStrategy();
 
     private volatile CSVFormat csvFormat;
     private volatile String dateFormat;
@@ -56,6 +67,7 @@ public class CSVReader extends SchemaRegistryRecordReader implements RowRecordRe
         properties.add(DateTimeUtils.TIMESTAMP_FORMAT);
         properties.add(CSVUtils.CSV_FORMAT);
         properties.add(CSVUtils.VALUE_SEPARATOR);
+        properties.add(CSVUtils.SKIP_HEADER_LINE);
         properties.add(CSVUtils.QUOTE_CHAR);
         properties.add(CSVUtils.ESCAPE_CHAR);
         properties.add(CSVUtils.COMMENT_MARKER);
@@ -73,9 +85,34 @@ public class CSVReader extends SchemaRegistryRecordReader implements RowRecordRe
     }
 
     @Override
-    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException {
-        final RecordSchema schema = getSchema(flowFile);
-        return new CSVRecordReader(in, logger, schema, csvFormat, dateFormat, timeFormat, timestampFormat);
+    public RecordReader createRecordReader(final FlowFile flowFile, 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(flowFile, new NonCloseableInputStream(bufferedIn));
+        bufferedIn.reset();
+
+        return new CSVRecordReader(bufferedIn, logger, schema, csvFormat, dateFormat, timeFormat, timestampFormat);
+    }
+
+    @Override
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry) {
+        if (allowableValue.equalsIgnoreCase(headerDerivedAllowableValue.getValue())) {
+            return headerDerivedSchemaStrategy;
+        }
+
+        return super.getSchemaAccessStrategy(allowableValue, schemaRegistry);
     }
 
+    @Override
+    protected List<AllowableValue> getSchemaAccessStrategyValues() {
+        final List<AllowableValue> allowableValues = new ArrayList<>(super.getSchemaAccessStrategyValues());
+        allowableValues.add(headerDerivedAllowableValue);
+        return allowableValues;
+    }
+
+    @Override
+    protected AllowableValue getDefaultSchemaAccessStrategy() {
+        return headerDerivedAllowableValue;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java
index d02768c..241d604 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java
@@ -34,6 +34,7 @@ import org.apache.nifi.serialization.RecordReader;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.MapRecord;
 import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.util.DataTypeUtils;
 
@@ -48,13 +49,14 @@ public class CSVRecordReader implements RecordReader {
     public CSVRecordReader(final InputStream in, final ComponentLog logger, final RecordSchema schema, final CSVFormat csvFormat,
         final String dateFormat, final String timeFormat, final String timestampFormat) throws IOException {
 
-        final Reader reader = new InputStreamReader(new BOMInputStream(in));
-        csvParser = new CSVParser(reader, csvFormat);
-
         this.schema = schema;
         this.dateFormat = dateFormat;
         this.timeFormat = timeFormat;
         this.timestampFormat = timestampFormat;
+
+        final Reader reader = new InputStreamReader(new BOMInputStream(in));
+        final CSVFormat withHeader = csvFormat.withHeader(schema.getFieldNames().toArray(new String[0]));
+        csvParser = new CSVParser(reader, withHeader);
     }
 
     @Override
@@ -64,15 +66,27 @@ public class CSVRecordReader implements RecordReader {
         for (final CSVRecord csvRecord : csvParser) {
             final Map<String, Object> rowValues = new HashMap<>(schema.getFieldCount());
 
-            for (final String fieldName : schema.getFieldNames()) {
-                final String rawValue = csvRecord.get(fieldName);
+            for (final RecordField recordField : schema.getFields()) {
+                String rawValue = csvRecord.get(recordField.getFieldName());
+                if (rawValue == null) {
+                    for (final String alias : recordField.getAliases()) {
+                        rawValue = csvRecord.get(alias);
+                        if (rawValue != null) {
+                            break;
+                        }
+                    }
+                }
+
+                final String fieldName = recordField.getFieldName();
                 if (rawValue == null) {
                     rowValues.put(fieldName, null);
                     continue;
                 }
 
-                final Object converted = convert(rawValue, schema.getDataType(fieldName).orElse(null));
-                rowValues.put(fieldName, converted);
+                final Object converted = convert(rawValue, recordField.getDataType(), fieldName);
+                if (converted != null) {
+                    rowValues.put(fieldName, converted);
+                }
             }
 
             return new MapRecord(schema, rowValues);
@@ -86,7 +100,7 @@ public class CSVRecordReader implements RecordReader {
         return schema;
     }
 
-    protected Object convert(final String value, final DataType dataType) {
+    protected Object convert(final String value, final DataType dataType, final String fieldName) {
         if (dataType == null || value == null) {
             return value;
         }
@@ -97,7 +111,7 @@ public class CSVRecordReader implements RecordReader {
             return null;
         }
 
-        return DataTypeUtils.convertType(trimmed, dataType, dateFormat, timeFormat, timestampFormat);
+        return DataTypeUtils.convertType(trimmed, dataType, dateFormat, timeFormat, timestampFormat, fieldName);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java
index 6a7b758..95c86e7 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java
@@ -17,6 +17,8 @@
 
 package org.apache.nifi.csv;
 
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -26,10 +28,13 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.DateTimeTextRecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordSchema;
 
 @Tags({"csv", "result", "set", "recordset", "record", "writer", "serializer", "row", "tsv", "tab", "separated", "delimited"})
 @CapabilityDescription("Writes the contents of a RecordSet as CSV data. The first line written "
@@ -37,12 +42,14 @@ import org.apache.nifi.serialization.RecordSetWriterFactory;
 public class CSVRecordSetWriter extends DateTimeTextRecordSetWriter implements RecordSetWriterFactory {
 
     private volatile CSVFormat csvFormat;
+    private volatile boolean includeHeader;
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
         properties.add(CSVUtils.CSV_FORMAT);
         properties.add(CSVUtils.VALUE_SEPARATOR);
+        properties.add(CSVUtils.INCLUDE_HEADER_LINE);
         properties.add(CSVUtils.QUOTE_CHAR);
         properties.add(CSVUtils.ESCAPE_CHAR);
         properties.add(CSVUtils.COMMENT_MARKER);
@@ -57,11 +64,12 @@ public class CSVRecordSetWriter extends DateTimeTextRecordSetWriter implements R
     @OnEnabled
     public void storeCsvFormat(final ConfigurationContext context) {
         this.csvFormat = CSVUtils.createCSVFormat(context);
+        this.includeHeader = context.getProperty(CSVUtils.INCLUDE_HEADER_LINE).asBoolean();
     }
 
     @Override
-    public RecordSetWriter createWriter(final ComponentLog logger) {
-        return new WriteCSVResult(csvFormat, getDateFormat(), getTimeFormat(), getTimestampFormat());
+    public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream in) throws SchemaNotFoundException, IOException {
+        final RecordSchema schema = getSchema(flowFile, in);
+        return new WriteCSVResult(csvFormat, schema, getSchemaAccessWriter(schema), getDateFormat(), getTimeFormat(), getTimestampFormat(), includeHeader);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java
index e23b6e1..1048d21 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java
@@ -61,6 +61,18 @@ public class CSVUtils {
         .defaultValue("\"")
         .required(true)
         .build();
+    static final PropertyDescriptor SKIP_HEADER_LINE = new PropertyDescriptor.Builder()
+        .name("Skip Header Line")
+        .description("Specifies whether or not the first line of CSV should be considered a Header and skipped. If the Schema Access Strategy "
+            + "indicates that the columns must be defined in the header, then this property will be ignored, since the header must always be "
+            + "present and won't be processed as a Record. Otherwise, this property should be 'true' if the first non-comment line of CSV "
+            + "contains header information that needs to be ignored.")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(false)
+        .allowableValues("true", "false")
+        .defaultValue("true")
+        .required(true)
+        .build();
     static final PropertyDescriptor COMMENT_MARKER = new PropertyDescriptor.Builder()
         .name("Comment Marker")
         .description("The character that is used to denote the start of a comment. Any line that begins with this comment will be ignored.")
@@ -124,7 +136,13 @@ public class CSVUtils {
         .defaultValue("\\n")
         .required(true)
         .build();
-
+    static final PropertyDescriptor INCLUDE_HEADER_LINE = new PropertyDescriptor.Builder()
+        .name("Include Header Line")
+        .description("Specifies whether or not the CSV column names should be written out as the first line.")
+        .allowableValues("true", "false")
+        .defaultValue("true")
+        .required(true)
+        .build();
 
     static CSVFormat createCSVFormat(final ConfigurationContext context) {
         final String formatName = context.getProperty(CSV_FORMAT).getValue();
@@ -156,8 +174,11 @@ public class CSVUtils {
         final char valueSeparator = getChar(context, VALUE_SEPARATOR);
         CSVFormat format = CSVFormat.newFormat(valueSeparator)
             .withAllowMissingColumnNames()
-            .withIgnoreEmptyLines()
-            .withFirstRecordAsHeader();
+            .withIgnoreEmptyLines();
+
+        if (context.getProperty(SKIP_HEADER_LINE).asBoolean()) {
+            format = format.withFirstRecordAsHeader();
+        }
 
         format = format.withQuote(getChar(context, QUOTE_CHAR));
         format = format.withEscape(getChar(context, ESCAPE_CHAR));

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java
index e0eb813..7c53ace 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java
@@ -21,38 +21,41 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.util.Collections;
-import java.util.Optional;
 
 import org.apache.commons.csv.CSVFormat;
 import org.apache.commons.csv.CSVPrinter;
+import org.apache.nifi.schema.access.SchemaAccessWriter;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.WriteResult;
 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 org.apache.nifi.serialization.record.RecordSet;
 import org.apache.nifi.stream.io.NonCloseableOutputStream;
 
 public class WriteCSVResult implements RecordSetWriter {
     private final CSVFormat csvFormat;
+    private final RecordSchema recordSchema;
+    private final SchemaAccessWriter schemaWriter;
     private final String dateFormat;
     private final String timeFormat;
     private final String timestampFormat;
+    private final boolean includeHeaderLine;
 
-    public WriteCSVResult(final CSVFormat csvFormat, final String dateFormat, final String timeFormat, final String timestampFormat) {
+    public WriteCSVResult(final CSVFormat csvFormat, final RecordSchema recordSchema, final SchemaAccessWriter schemaWriter,
+        final String dateFormat, final String timeFormat, final String timestampFormat, final boolean includeHeaderLine) {
         this.csvFormat = csvFormat;
+        this.recordSchema = recordSchema;
+        this.schemaWriter = schemaWriter;
         this.dateFormat = dateFormat;
         this.timeFormat = timeFormat;
         this.timestampFormat = timestampFormat;
+        this.includeHeaderLine = includeHeaderLine;
     }
 
-    private String getFormat(final Record record, final String fieldName) {
-        final Optional<DataType> dataTypeOption = record.getSchema().getDataType(fieldName);
-        if (!dataTypeOption.isPresent()) {
-            return null;
-        }
-
-        final DataType dataType = dataTypeOption.get();
+    private String getFormat(final Record record, final RecordField field) {
+        final DataType dataType = field.getDataType();
         switch (dataType.getFieldType()) {
             case DATE:
                 return dateFormat == null ? dataType.getFormat() : dateFormat;
@@ -69,9 +72,10 @@ public class WriteCSVResult implements RecordSetWriter {
     public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws IOException {
         int count = 0;
 
-        final RecordSchema schema = rs.getSchema();
-        final String[] columnNames = schema.getFieldNames().toArray(new String[0]);
-        final CSVFormat formatWithHeader = csvFormat.withHeader(columnNames);
+        final String[] columnNames = recordSchema.getFieldNames().toArray(new String[0]);
+        final CSVFormat formatWithHeader = csvFormat.withHeader(columnNames).withSkipHeaderRecord(!includeHeaderLine);
+
+        schemaWriter.writeHeader(recordSchema, rawOut);
 
         try (final OutputStream nonCloseable = new NonCloseableOutputStream(rawOut);
             final OutputStreamWriter streamWriter = new OutputStreamWriter(nonCloseable);
@@ -80,10 +84,10 @@ public class WriteCSVResult implements RecordSetWriter {
             try {
                 Record record;
                 while ((record = rs.next()) != null) {
-                    final Object[] colVals = new Object[schema.getFieldCount()];
+                    final Object[] colVals = new Object[recordSchema.getFieldCount()];
                     int i = 0;
-                    for (final String fieldName : schema.getFieldNames()) {
-                        colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName));
+                    for (final RecordField recordField : recordSchema.getFields()) {
+                        colVals[i++] = record.getAsString(recordField, getFormat(record, recordField));
                     }
 
                     printer.printRecord(colVals);
@@ -94,7 +98,7 @@ public class WriteCSVResult implements RecordSetWriter {
             }
         }
 
-        return WriteResult.of(count, Collections.emptyMap());
+        return WriteResult.of(count, schemaWriter.getAttributes(recordSchema));
     }
 
     @Override
@@ -108,8 +112,8 @@ public class WriteCSVResult implements RecordSetWriter {
                 final RecordSchema schema = record.getSchema();
                 final Object[] colVals = new Object[schema.getFieldCount()];
                 int i = 0;
-                for (final String fieldName : schema.getFieldNames()) {
-                    colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName));
+                for (final RecordField recordField : schema.getFields()) {
+                    colVals[i++] = record.getAsString(recordField, getFormat(record, recordField));
                 }
 
                 printer.printRecord(colVals);

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 f444b8a..778c738 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
@@ -22,39 +22,62 @@ 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 java.util.regex.Matcher;
 
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.util.StandardValidators;
+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.schemaregistry.services.SchemaRegistry;
 import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
-import org.apache.nifi.serialization.SchemaRegistryRecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+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 io.thekraken.grok.api.Grok;
+import io.thekraken.grok.api.GrokUtils;
 import io.thekraken.grok.api.exception.GrokException;
 
 @Tags({"grok", "logs", "logfiles", "parse", "unstructured", "text", "record", "reader", "regex", "pattern", "logstash"})
 @CapabilityDescription("Provides a mechanism for reading unstructured text data, such as log files, and structuring the data "
     + "so that it can be processed. The service is configured using Grok patterns. "
     + "The service reads from a stream of data and splits each message that it finds into a separate Record, each containing the fields that are configured. "
-    + "If a line in the input does not match the expected message pattern, the line of text is considered to be part of the previous "
-    + "message, with the exception of stack traces. A stack trace that is found at the end of a log message is considered to be part "
-    + "of the previous message but is added to the 'STACK_TRACE' field of the Record. If a record has no stack trace, it will have a NULL value "
-    + "for the STACK_TRACE field. All fields that are parsed are considered to be of type String by default. If there is need to change the type of a field, "
-    + "this can be accomplished by configuring the Schema Registry to use and adding the appropriate schema.")
-public class GrokReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory {
+    + "If a line in the input does not match the expected message pattern, the line of text is either considered to be part of the previous "
+    + "message or is skipped, depending on the configuration, with the exception of stack traces. A stack trace that is found at the end of "
+    + "a log message is considered to be part of the previous message but is added to the 'stackTrace' field of the Record. If a record has "
+    + "no stack trace, it will have a NULL value for the stackTrace field (assuming that the schema does in fact include a stackTrace field of type String).")
+public class GrokReader extends SchemaRegistryService implements RecordReaderFactory {
     private volatile Grok grok;
-    private volatile boolean useSchemaRegistry;
+    private volatile boolean appendUnmatchedLine;
+    private volatile RecordSchema recordSchema;
 
     private static final String DEFAULT_PATTERN_NAME = "/default-grok-patterns.txt";
 
+    static final AllowableValue APPEND_TO_PREVIOUS_MESSAGE = new AllowableValue("append-to-previous-message", "Append to Previous Message",
+        "The line of text that does not match the Grok Expression will be appended to the last field of the prior message.");
+    static final AllowableValue SKIP_LINE = new AllowableValue("skip-line", "Skip Line",
+        "The line of text that does not match the Grok Expression will be skipped.");
+
+    static final AllowableValue STRING_FIELDS_FROM_GROK_EXPRESSION = new AllowableValue("string-fields-from-grok-expression", "Use String Fields From Grok Expression",
+        "The schema will be derived by using the field names present in the Grok Expression. All fields will be assumed to be of type String. Additionally, a field will be included "
+            + "with a name of 'stackTrace' and a type of String.");
+
     static final PropertyDescriptor PATTERN_FILE = new PropertyDescriptor.Builder()
         .name("Grok Pattern File")
         .description("Path to a file that contains Grok Patterns to use for parsing logs. If not specified, a built-in default Pattern file "
@@ -73,11 +96,22 @@ public class GrokReader extends SchemaRegistryRecordReader implements RowRecordR
         .required(true)
         .build();
 
+    static final PropertyDescriptor NO_MATCH_BEHAVIOR = new PropertyDescriptor.Builder()
+        .name("no-match-behavior")
+        .displayName("No Match Behavior")
+        .description("If a line of text is encountered and it does not match the given Grok Expression, and it is not part of a stack trace, "
+            + "this property specifies how the text should be processed.")
+        .allowableValues(APPEND_TO_PREVIOUS_MESSAGE, SKIP_LINE)
+        .defaultValue(APPEND_TO_PREVIOUS_MESSAGE.getValue())
+        .required(true)
+        .build();
+
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
         properties.add(PATTERN_FILE);
         properties.add(GROK_EXPRESSION);
+        properties.add(NO_MATCH_BEHAVIOR);
         return properties;
     }
 
@@ -95,17 +129,79 @@ public class GrokReader extends SchemaRegistryRecordReader implements RowRecordR
         }
 
         grok.compile(context.getProperty(GROK_EXPRESSION).getValue());
-        useSchemaRegistry = context.getProperty(OPTIONAL_SCHEMA_NAME).isSet() && context.getProperty(OPTIONAL_SCHEMA_REGISTRY).isSet();
+
+        appendUnmatchedLine = context.getProperty(NO_MATCH_BEHAVIOR).getValue().equalsIgnoreCase(APPEND_TO_PREVIOUS_MESSAGE.getValue());
+
+        this.recordSchema = createRecordSchema(grok);
+    }
+
+    static RecordSchema createRecordSchema(final Grok grok) {
+        final List<RecordField> fields = new ArrayList<>();
+
+        String grokExpression = grok.getOriginalGrokPattern();
+        while (grokExpression.length() > 0) {
+            final Matcher matcher = GrokUtils.GROK_PATTERN.matcher(grokExpression);
+            if (matcher.find()) {
+                final Map<String, String> namedGroups = GrokUtils.namedGroups(matcher, grokExpression);
+                final String fieldName = namedGroups.get("subname");
+
+                DataType dataType = RecordFieldType.STRING.getDataType();
+                final RecordField recordField = new RecordField(fieldName, dataType);
+                fields.add(recordField);
+
+                if (grokExpression.length() > matcher.end() + 1) {
+                    grokExpression = grokExpression.substring(matcher.end() + 1);
+                } else {
+                    break;
+                }
+            }
+        }
+
+        fields.add(new RecordField(GrokRecordReader.STACK_TRACE_COLUMN_NAME, RecordFieldType.STRING.getDataType()));
+
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+        return schema;
     }
 
+
     @Override
-    protected boolean isSchemaRequired() {
-        return false;
+    protected List<AllowableValue> getSchemaAccessStrategyValues() {
+        final List<AllowableValue> allowableValues = new ArrayList<>();
+        allowableValues.add(STRING_FIELDS_FROM_GROK_EXPRESSION);
+        allowableValues.addAll(super.getSchemaAccessStrategyValues());
+        return allowableValues;
     }
 
     @Override
-    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException {
-        final RecordSchema schema = useSchemaRegistry ? getSchema(flowFile) : null;
-        return new GrokRecordReader(in, grok, schema);
+    protected AllowableValue getDefaultSchemaAccessStrategy() {
+        return STRING_FIELDS_FROM_GROK_EXPRESSION;
+    }
+
+    @Override
+    protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry) {
+        if (allowableValue.equalsIgnoreCase(STRING_FIELDS_FROM_GROK_EXPRESSION.getValue())) {
+            return new SchemaAccessStrategy() {
+                private final Set<SchemaField> schemaFields = EnumSet.noneOf(SchemaField.class);
+
+                @Override
+                public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException {
+                    return recordSchema;
+                }
+
+                @Override
+                public Set<SchemaField> getSuppliedSchemaFields() {
+                    return schemaFields;
+                }
+            };
+        } else {
+            return super.getSchemaAccessStrategy(allowableValue, schemaRegistry);
+        }
+    }
+
+
+    @Override
+    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException {
+        final RecordSchema schema = getSchema(flowFile, in);
+        return new GrokRecordReader(in, grok, schema, appendUnmatchedLine);
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java
index 458dbd8..5859f6f 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java
@@ -21,40 +21,34 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.text.ParseException;
-import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.TimeZone;
-import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.commons.lang3.time.FastDateFormat;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.MapRecord;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
 
 import io.thekraken.grok.api.Grok;
-import io.thekraken.grok.api.GrokUtils;
 import io.thekraken.grok.api.Match;
 
 public class GrokRecordReader implements RecordReader {
     private final BufferedReader reader;
     private final Grok grok;
+    private final boolean append;
     private RecordSchema schema;
 
     private String nextLine;
 
-    static final String STACK_TRACE_COLUMN_NAME = "STACK_TRACE";
+    static final String STACK_TRACE_COLUMN_NAME = "stackTrace";
     private static final Pattern STACK_TRACE_PATTERN = Pattern.compile(
         "^\\s*(?:(?:    |\\t)+at )|"
             + "(?:(?:    |\\t)+\\[CIRCULAR REFERENCE\\:)|"
@@ -62,21 +56,11 @@ public class GrokRecordReader implements RecordReader {
             + "(?:Suppressed\\: )|"
             + "(?:\\s+... \\d+ (?:more|common frames? omitted)$)");
 
-    private static final FastDateFormat TIME_FORMAT_DATE;
-    private static final FastDateFormat TIME_FORMAT_TIME;
-    private static final FastDateFormat TIME_FORMAT_TIMESTAMP;
-
-    static {
-        final TimeZone gmt = TimeZone.getTimeZone("GMT");
-        TIME_FORMAT_DATE = FastDateFormat.getInstance("yyyy-MM-dd", gmt);
-        TIME_FORMAT_TIME = FastDateFormat.getInstance("HH:mm:ss", gmt);
-        TIME_FORMAT_TIMESTAMP = FastDateFormat.getInstance("yyyy-MM-dd HH:mm:ss", gmt);
-    }
-
-    public GrokRecordReader(final InputStream in, final Grok grok, final RecordSchema schema) {
+    public GrokRecordReader(final InputStream in, final Grok grok, final RecordSchema schema, final boolean append) {
         this.reader = new BufferedReader(new InputStreamReader(in));
         this.grok = grok;
         this.schema = schema;
+        this.append = append;
     }
 
     @Override
@@ -115,7 +99,7 @@ public class GrokRecordReader implements RecordReader {
                 if (isStartOfStackTrace(nextLine)) {
                     stackTrace = readStackTrace(nextLine);
                     break;
-                } else {
+                } else if (append) {
                     toAppend.append("\n").append(nextLine);
                 }
             } else {
@@ -128,20 +112,34 @@ public class GrokRecordReader implements RecordReader {
             final List<DataType> fieldTypes = schema.getDataTypes();
             final Map<String, Object> values = new HashMap<>(fieldTypes.size());
 
-            for (final String fieldName : schema.getFieldNames()) {
-                final Object value = valueMap.get(fieldName);
+            for (final RecordField field : schema.getFields()) {
+                Object value = valueMap.get(field.getFieldName());
+                if (value == null) {
+                    for (final String alias : field.getAliases()) {
+                        value = valueMap.get(alias);
+                        if (value != null) {
+                            break;
+                        }
+                    }
+                }
+
+                final String fieldName = field.getFieldName();
                 if (value == null) {
                     values.put(fieldName, null);
                     continue;
                 }
 
-                final DataType fieldType = schema.getDataType(fieldName).orElse(null);
-                final Object converted = convert(fieldType, value.toString());
+                final DataType fieldType = field.getDataType();
+                final Object converted = convert(fieldType, value.toString(), fieldName);
                 values.put(fieldName, converted);
             }
 
-            final String lastFieldBeforeStackTrace = schema.getFieldNames().get(schema.getFieldCount() - 2);
-            if (toAppend.length() > 0) {
+            if (append && toAppend.length() > 0) {
+                final String lastFieldName = schema.getField(schema.getFieldCount() - 1).getFieldName();
+
+                final int fieldIndex = STACK_TRACE_COLUMN_NAME.equals(lastFieldName) ? schema.getFieldCount() - 2 : schema.getFieldCount() - 1;
+                final String lastFieldBeforeStackTrace = schema.getFieldNames().get(fieldIndex);
+
                 final Object existingValue = values.get(lastFieldBeforeStackTrace);
                 final String updatedValue = existingValue == null ? toAppend.toString() : existingValue + toAppend.toString();
                 values.put(lastFieldBeforeStackTrace, updatedValue);
@@ -205,7 +203,7 @@ public class GrokRecordReader implements RecordReader {
     }
 
 
-    protected Object convert(final DataType fieldType, final String string) {
+    protected Object convert(final DataType fieldType, final String string, final String fieldName) {
         if (fieldType == null) {
             return string;
         }
@@ -220,79 +218,12 @@ public class GrokRecordReader implements RecordReader {
             return null;
         }
 
-        switch (fieldType.getFieldType()) {
-            case BOOLEAN:
-                return Boolean.parseBoolean(string);
-            case BYTE:
-                return Byte.parseByte(string);
-            case SHORT:
-                return Short.parseShort(string);
-            case INT:
-                return Integer.parseInt(string);
-            case LONG:
-                return Long.parseLong(string);
-            case FLOAT:
-                return Float.parseFloat(string);
-            case DOUBLE:
-                return Double.parseDouble(string);
-            case DATE:
-                try {
-                    Date date = TIME_FORMAT_DATE.parse(string);
-                    return new java.sql.Date(date.getTime());
-                } catch (ParseException e) {
-                    return null;
-                }
-            case TIME:
-                try {
-                    Date date = TIME_FORMAT_TIME.parse(string);
-                    return new java.sql.Time(date.getTime());
-                } catch (ParseException e) {
-                    return null;
-                }
-            case TIMESTAMP:
-                try {
-                    Date date = TIME_FORMAT_TIMESTAMP.parse(string);
-                    return new java.sql.Timestamp(date.getTime());
-                } catch (ParseException e) {
-                    return null;
-                }
-            case STRING:
-            default:
-                return string;
-        }
+        return DataTypeUtils.convertType(string, fieldType, fieldName);
     }
 
 
     @Override
     public RecordSchema getSchema() {
-        if (schema != null) {
-            return schema;
-        }
-
-        final List<RecordField> fields = new ArrayList<>();
-
-        String grokExpression = grok.getOriginalGrokPattern();
-        while (grokExpression.length() > 0) {
-            final Matcher matcher = GrokUtils.GROK_PATTERN.matcher(grokExpression);
-            if (matcher.find()) {
-                final Map<String, String> namedGroups = GrokUtils.namedGroups(matcher, grokExpression);
-                final String fieldName = namedGroups.get("subname");
-
-                DataType dataType = RecordFieldType.STRING.getDataType();
-                final RecordField recordField = new RecordField(fieldName, dataType);
-                fields.add(recordField);
-
-                if (grokExpression.length() > matcher.end() + 1) {
-                    grokExpression = grokExpression.substring(matcher.end() + 1);
-                } else {
-                    break;
-                }
-            }
-        }
-
-        fields.add(new RecordField(STACK_TRACE_COLUMN_NAME, RecordFieldType.STRING.getDataType()));
-
-        schema = new SimpleRecordSchema(fields);
         return schema;
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 ad04912..c5c5fb0 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
@@ -19,20 +19,12 @@ package org.apache.nifi.json;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-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;
-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.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonNode;
@@ -98,55 +90,6 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
         }
     }
 
-    protected DataType determineFieldType(final JsonNode node) {
-        if (node.isDouble()) {
-            return RecordFieldType.DOUBLE.getDataType();
-        }
-        if (node.isBoolean()) {
-            return RecordFieldType.BOOLEAN.getDataType();
-        }
-        if (node.isFloatingPointNumber()) {
-            return RecordFieldType.FLOAT.getDataType();
-        }
-        if (node.isBigInteger()) {
-            return RecordFieldType.BIGINT.getDataType();
-        }
-        if (node.isBigDecimal()) {
-            return RecordFieldType.DOUBLE.getDataType();
-        }
-        if (node.isLong()) {
-            return RecordFieldType.LONG.getDataType();
-        }
-        if (node.isInt()) {
-            return RecordFieldType.INT.getDataType();
-        }
-        if (node.isTextual()) {
-            return RecordFieldType.STRING.getDataType();
-        }
-        if (node.isArray()) {
-            return RecordFieldType.ARRAY.getDataType();
-        }
-
-        final RecordSchema childSchema = determineSchema(node);
-        return RecordFieldType.RECORD.getRecordDataType(childSchema);
-    }
-
-    protected RecordSchema determineSchema(final JsonNode jsonNode) {
-        final List<RecordField> recordFields = new ArrayList<>();
-
-        final Iterator<Map.Entry<String, JsonNode>> itr = jsonNode.getFields();
-        while (itr.hasNext()) {
-            final Map.Entry<String, JsonNode> entry = itr.next();
-            final String elementName = entry.getKey();
-            final JsonNode node = entry.getValue();
-
-            DataType dataType = determineFieldType(node);
-            recordFields.add(new RecordField(elementName, dataType));
-        }
-
-        return new SimpleRecordSchema(recordFields);
-    }
-
     protected Object getRawNodeValue(final JsonNode fieldNode) throws IOException {
         if (fieldNode == null || !fieldNode.isValueNode()) {
             return null;

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 467ecf8..2d11a9b 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
@@ -36,11 +36,12 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.DateTimeUtils;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
-import org.apache.nifi.serialization.SchemaRegistryRecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
 import org.apache.nifi.serialization.record.RecordSchema;
 
 import com.jayway.jsonpath.JsonPath;
@@ -56,7 +57,7 @@ import com.jayway.jsonpath.JsonPath;
         + "field whose name is the same as the property name.",
     description="User-defined properties identifiy how to extract specific fields from a JSON object in order to create a Record",
     supportsExpressionLanguage=false)
-public class JsonPathReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory {
+public class JsonPathReader extends SchemaRegistryService implements RecordReaderFactory {
 
     private volatile String dateFormat;
     private volatile String timeFormat;
@@ -127,8 +128,8 @@ public class JsonPathReader extends SchemaRegistryRecordReader implements RowRec
     }
 
     @Override
-    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
-        final RecordSchema schema = getSchema(flowFile);
+    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException {
+        final RecordSchema schema = getSchema(flowFile, in);
         return new JsonPathRowRecordReader(jsonPaths, schema, in, logger, dateFormat, timeFormat, timestampFormat);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 a0f3c32..8675e0e 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
@@ -30,6 +30,7 @@ import org.apache.nifi.serialization.MalformedRecordException;
 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;
@@ -106,7 +107,10 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
                 value = null;
             }
 
-            value = convert(value, desiredType);
+            final Optional<RecordField> field = schema.getField(fieldName);
+            final Object defaultValue = field.isPresent() ? field.get().getDefaultValue() : null;
+
+            value = convert(value, desiredType, fieldName, defaultValue);
             values.put(fieldName, value);
         }
 
@@ -115,9 +119,9 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
 
 
     @SuppressWarnings("unchecked")
-    protected Object convert(final Object value, final DataType dataType) {
+    protected Object convert(final Object value, final DataType dataType, final String fieldName, final Object defaultValue) {
         if (value == null) {
-            return null;
+            return defaultValue;
         }
 
         if (value instanceof List) {
@@ -131,7 +135,7 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
             final Object[] coercedValues = new Object[list.size()];
             int i = 0;
             for (final Object rawValue : list) {
-                coercedValues[i++] = DataTypeUtils.convertType(rawValue, arrayType.getElementType(), dateFormat, timeFormat, timestampFormat);
+                coercedValues[i++] = convert(rawValue, arrayType.getElementType(), fieldName, null);
             }
             return coercedValues;
         }
@@ -147,14 +151,17 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
                 final String key = entry.getKey();
                 final Optional<DataType> desiredTypeOption = childSchema.getDataType(key);
                 if (desiredTypeOption.isPresent()) {
-                    final Object coercedValue = DataTypeUtils.convertType(entry.getValue(), desiredTypeOption.get(), dateFormat, timeFormat, timestampFormat);
+                    final Optional<RecordField> field = childSchema.getField(key);
+                    final Object defaultFieldValue = field.isPresent() ? field.get().getDefaultValue() : null;
+
+                    final Object coercedValue = convert(entry.getValue(), desiredTypeOption.get(), fieldName + "." + key, defaultFieldValue);
                     coercedValues.put(key, coercedValue);
                 }
             }
 
             return new MapRecord(childSchema, coercedValues);
         } else {
-            return DataTypeUtils.convertType(value, dataType, dateFormat, timeFormat, timestampFormat);
+            return DataTypeUtils.convertType(value, dataType, dateFormat, timeFormat, timestampFormat, fieldName);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
index d09f135..e6b5c02 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
@@ -17,6 +17,8 @@
 
 package org.apache.nifi.json;
 
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -25,10 +27,13 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.DateTimeTextRecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordSchema;
 
 @Tags({"json", "resultset", "writer", "serialize", "record", "recordset", "row"})
 @CapabilityDescription("Writes the results of a RecordSet as a JSON Array. Even if the RecordSet "
@@ -59,8 +64,9 @@ public class JsonRecordSetWriter extends DateTimeTextRecordSetWriter implements
     }
 
     @Override
-    public RecordSetWriter createWriter(final ComponentLog logger) {
-        return new WriteJsonResult(logger, prettyPrint, getDateFormat(), getTimeFormat(), getTimestampFormat());
+    public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream flowFileContent) throws SchemaNotFoundException, IOException {
+        final RecordSchema schema = getSchema(flowFile, flowFileContent);
+        return new WriteJsonResult(logger, schema, getSchemaAccessWriter(schema), prettyPrint, getDateFormat(), getTimeFormat(), getTimestampFormat());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 1abb1f4..1dd9834 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
@@ -30,11 +30,12 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.DateTimeUtils;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
-import org.apache.nifi.serialization.SchemaRegistryRecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
 
 @Tags({"json", "tree", "record", "reader", "parser"})
 @CapabilityDescription("Parses JSON into individual Record objects. The Record that is produced will contain all top-level "
@@ -45,7 +46,7 @@ import org.apache.nifi.serialization.SchemaRegistryRecordReader;
     + "a field that is not present in the schema, that field will be skipped. "
     + "See the Usage of the Controller Service for more information and examples.")
 @SeeAlso(JsonPathReader.class)
-public class JsonTreeReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory {
+public class JsonTreeReader extends SchemaRegistryService implements RecordReaderFactory {
 
     private volatile String dateFormat;
     private volatile String timeFormat;
@@ -68,7 +69,7 @@ public class JsonTreeReader extends SchemaRegistryRecordReader implements RowRec
     }
 
     @Override
-    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
-        return new JsonTreeRowRecordReader(in, logger, getSchema(flowFile), dateFormat, timeFormat, timestampFormat);
+    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException {
+        return new JsonTreeRowRecordReader(in, logger, getSchema(flowFile, in), dateFormat, timeFormat, timestampFormat);
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 c8d07f4..301b724 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
@@ -19,17 +19,23 @@ package org.apache.nifi.json;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 
 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.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.MapDataType;
 import org.apache.nifi.serialization.record.type.RecordDataType;
 import org.apache.nifi.serialization.record.util.DataTypeUtils;
 import org.codehaus.jackson.JsonNode;
@@ -55,6 +61,10 @@ public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
 
     @Override
     protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema) throws IOException, MalformedRecordException {
+        return convertJsonNodeToRecord(jsonNode, schema, "");
+    }
+
+    private Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema, final String fieldNamePrefix) throws IOException, MalformedRecordException {
         if (jsonNode == null) {
             return null;
         }
@@ -63,10 +73,19 @@ public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
         for (int i = 0; i < schema.getFieldCount(); i++) {
             final RecordField field = schema.getField(i);
             final String fieldName = field.getFieldName();
-            final JsonNode fieldNode = jsonNode.get(fieldName);
+
+            JsonNode fieldNode = jsonNode.get(fieldName);
+            if (fieldNode == null) {
+                for (final String alias : field.getAliases()) {
+                    fieldNode = jsonNode.get(alias);
+                    if (fieldNode != null) {
+                        break;
+                    }
+                }
+            }
 
             final DataType desiredType = field.getDataType();
-            final Object value = convertField(fieldNode, fieldName, desiredType);
+            final Object value = convertField(fieldNode, fieldNamePrefix + fieldName, desiredType);
             values.put(fieldName, value);
         }
 
@@ -80,42 +99,50 @@ public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
 
         switch (desiredType.getFieldType()) {
             case BOOLEAN:
-                return DataTypeUtils.toBoolean(getRawNodeValue(fieldNode));
+                return DataTypeUtils.toBoolean(getRawNodeValue(fieldNode), fieldName);
             case BYTE:
-                return DataTypeUtils.toByte(getRawNodeValue(fieldNode));
+                return DataTypeUtils.toByte(getRawNodeValue(fieldNode), fieldName);
             case CHAR:
-                return DataTypeUtils.toCharacter(getRawNodeValue(fieldNode));
+                return DataTypeUtils.toCharacter(getRawNodeValue(fieldNode), fieldName);
             case DOUBLE:
-                return DataTypeUtils.toDouble(getRawNodeValue(fieldNode));
+                return DataTypeUtils.toDouble(getRawNodeValue(fieldNode), fieldName);
             case FLOAT:
-                return DataTypeUtils.toFloat(getRawNodeValue(fieldNode));
+                return DataTypeUtils.toFloat(getRawNodeValue(fieldNode), fieldName);
             case INT:
-                return DataTypeUtils.toInteger(getRawNodeValue(fieldNode));
+                return DataTypeUtils.toInteger(getRawNodeValue(fieldNode), fieldName);
             case LONG:
-                return DataTypeUtils.toLong(getRawNodeValue(fieldNode));
+                return DataTypeUtils.toLong(getRawNodeValue(fieldNode), fieldName);
             case SHORT:
-                return DataTypeUtils.toShort(getRawNodeValue(fieldNode));
+                return DataTypeUtils.toShort(getRawNodeValue(fieldNode), fieldName);
             case STRING:
                 return DataTypeUtils.toString(getRawNodeValue(fieldNode), dateFormat, timeFormat, timestampFormat);
             case DATE:
-                return DataTypeUtils.toDate(getRawNodeValue(fieldNode), dateFormat);
+                return DataTypeUtils.toDate(getRawNodeValue(fieldNode), dateFormat, fieldName);
             case TIME:
-                return DataTypeUtils.toTime(getRawNodeValue(fieldNode), timeFormat);
+                return DataTypeUtils.toTime(getRawNodeValue(fieldNode), timeFormat, fieldName);
             case TIMESTAMP:
-                return DataTypeUtils.toTimestamp(getRawNodeValue(fieldNode), timestampFormat);
+                return DataTypeUtils.toTimestamp(getRawNodeValue(fieldNode), timestampFormat, fieldName);
+            case MAP: {
+                final DataType valueType = ((MapDataType) desiredType).getValueType();
+
+                final Map<String, Object> map = new HashMap<>();
+                final Iterator<String> fieldNameItr = fieldNode.getFieldNames();
+                while (fieldNameItr.hasNext()) {
+                    final String childName = fieldNameItr.next();
+                    final JsonNode childNode = fieldNode.get(childName);
+                    final Object childValue = convertField(childNode, fieldName + "." + childName, valueType);
+                    map.put(childName, childValue);
+                }
+
+                return map;
+            }
             case ARRAY: {
                 final ArrayNode arrayNode = (ArrayNode) fieldNode;
                 final int numElements = arrayNode.size();
                 final Object[] arrayElements = new Object[numElements];
                 int count = 0;
                 for (final JsonNode node : arrayNode) {
-                    final DataType elementType;
-                    if (desiredType instanceof ArrayDataType) {
-                        elementType = ((ArrayDataType) desiredType).getElementType();
-                    } else {
-                        elementType = determineFieldType(node);
-                    }
-
+                    final DataType elementType = ((ArrayDataType) desiredType).getElementType();
                     final Object converted = convertField(node, fieldName, elementType);
                     arrayElements[count++] = converted;
                 }
@@ -124,14 +151,24 @@ public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
             }
             case RECORD: {
                 if (fieldNode.isObject()) {
-                    final RecordSchema childSchema;
+                    RecordSchema childSchema;
                     if (desiredType instanceof RecordDataType) {
                         childSchema = ((RecordDataType) desiredType).getChildSchema();
                     } else {
                         return null;
                     }
 
-                    return convertJsonNodeToRecord(fieldNode, childSchema);
+                    if (childSchema == null) {
+                        final List<RecordField> fields = new ArrayList<>();
+                        final Iterator<String> fieldNameItr = fieldNode.getFieldNames();
+                        while (fieldNameItr.hasNext()) {
+                            fields.add(new RecordField(fieldNameItr.next(), RecordFieldType.STRING.getDataType()));
+                        }
+
+                        childSchema = new SimpleRecordSchema(fields);
+                    }
+
+                    return convertJsonNodeToRecord(fieldNode, childSchema, fieldName + ".");
                 } else {
                     return null;
                 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 05895d8..943e1d5 100644
--- 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
@@ -22,17 +22,22 @@ import java.io.OutputStream;
 import java.math.BigInteger;
 import java.sql.SQLException;
 import java.util.Collections;
+import java.util.Map;
 
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schema.access.SchemaAccessWriter;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.WriteResult;
 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.RecordSet;
 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.apache.nifi.serialization.record.util.DataTypeUtils;
 import org.apache.nifi.stream.io.NonCloseableOutputStream;
 import org.codehaus.jackson.JsonFactory;
@@ -42,25 +47,32 @@ import org.codehaus.jackson.JsonGenerator;
 public class WriteJsonResult implements RecordSetWriter {
     private final ComponentLog logger;
     private final boolean prettyPrint;
+    private final SchemaAccessWriter schemaAccess;
+    private final RecordSchema recordSchema;
     private final JsonFactory factory = new JsonFactory();
     private final String dateFormat;
     private final String timeFormat;
     private final String timestampFormat;
 
-    public WriteJsonResult(final ComponentLog logger, final boolean prettyPrint, final String dateFormat, final String timeFormat, final String timestampFormat) {
+    public WriteJsonResult(final ComponentLog logger, final RecordSchema recordSchema, final SchemaAccessWriter schemaAccess, final boolean prettyPrint,
+        final String dateFormat, final String timeFormat, final String timestampFormat) {
+
+        this.logger = logger;
+        this.recordSchema = recordSchema;
         this.prettyPrint = prettyPrint;
+        this.schemaAccess = schemaAccess;
 
         this.dateFormat = dateFormat;
         this.timeFormat = timeFormat;
         this.timestampFormat = timestampFormat;
-
-        this.logger = logger;
     }
 
     @Override
     public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws IOException {
         int count = 0;
 
+        schemaAccess.writeHeader(recordSchema, rawOut);
+
         try (final JsonGenerator generator = factory.createJsonGenerator(new NonCloseableOutputStream(rawOut))) {
             if (prettyPrint) {
                 generator.useDefaultPrettyPrinter();
@@ -71,7 +83,7 @@ public class WriteJsonResult implements RecordSetWriter {
             Record record;
             while ((record = rs.next()) != null) {
                 count++;
-                writeRecord(record, generator, g -> g.writeStartObject(), g -> g.writeEndObject());
+                writeRecord(record, recordSchema, generator, g -> g.writeStartObject(), g -> g.writeEndObject());
             }
 
             generator.writeEndArray();
@@ -79,7 +91,7 @@ public class WriteJsonResult implements RecordSetWriter {
             throw new IOException("Failed to serialize Result Set to stream", e);
         }
 
-        return WriteResult.of(count, Collections.emptyMap());
+        return WriteResult.of(count, schemaAccess.getAttributes(recordSchema));
     }
 
     @Override
@@ -89,7 +101,7 @@ public class WriteJsonResult implements RecordSetWriter {
                 generator.useDefaultPrettyPrinter();
             }
 
-            writeRecord(record, generator, g -> g.writeStartObject(), g -> g.writeEndObject());
+            writeRecord(record, recordSchema, generator, g -> g.writeStartObject(), g -> g.writeEndObject());
         } catch (final SQLException e) {
             throw new IOException("Failed to write records to stream", e);
         }
@@ -97,24 +109,24 @@ public class WriteJsonResult implements RecordSetWriter {
         return WriteResult.of(1, Collections.emptyMap());
     }
 
-    private void writeRecord(final Record record, final JsonGenerator generator, final GeneratorTask startTask, final GeneratorTask endTask)
+    private void writeRecord(final Record record, final RecordSchema writeSchema, final JsonGenerator generator, final GeneratorTask startTask, final GeneratorTask endTask)
         throws JsonGenerationException, IOException, SQLException {
 
         try {
-            final RecordSchema schema = record.getSchema();
             startTask.apply(generator);
-            for (int i = 0; i < schema.getFieldCount(); i++) {
-                final String fieldName = schema.getField(i).getFieldName();
-                final Object value = record.getValue(fieldName);
+            for (int i = 0; i < writeSchema.getFieldCount(); i++) {
+                final RecordField field = writeSchema.getField(i);
+                final String fieldName = field.getFieldName();
+                final Object value = record.getValue(field);
                 if (value == null) {
                     generator.writeNullField(fieldName);
                     continue;
                 }
 
                 generator.writeFieldName(fieldName);
-                final DataType dataType = schema.getDataType(fieldName).get();
+                final DataType dataType = writeSchema.getDataType(fieldName).get();
 
-                writeValue(generator, value, dataType, i < schema.getFieldCount() - 1);
+                writeValue(generator, value, fieldName, dataType, i < writeSchema.getFieldCount() - 1);
             }
 
             endTask.apply(generator);
@@ -125,7 +137,8 @@ public class WriteJsonResult implements RecordSetWriter {
     }
 
 
-    private void writeValue(final JsonGenerator generator, final Object value, final DataType dataType, final boolean moreCols)
+    @SuppressWarnings("unchecked")
+    private void writeValue(final JsonGenerator generator, final Object value, final String fieldName, final DataType dataType, final boolean moreCols)
         throws JsonGenerationException, IOException, SQLException {
         if (value == null) {
             generator.writeNull();
@@ -133,7 +146,7 @@ public class WriteJsonResult implements RecordSetWriter {
         }
 
         final DataType chosenDataType = dataType.getFieldType() == RecordFieldType.CHOICE ? DataTypeUtils.chooseDataType(value, (ChoiceDataType) dataType) : dataType;
-        final Object coercedValue = DataTypeUtils.convertType(value, chosenDataType);
+        final Object coercedValue = DataTypeUtils.convertType(value, chosenDataType, fieldName);
         if (coercedValue == null) {
             generator.writeNull();
             return;
@@ -146,18 +159,18 @@ public class WriteJsonResult implements RecordSetWriter {
                 generator.writeString(DataTypeUtils.toString(coercedValue, dateFormat, timeFormat, timestampFormat));
                 break;
             case DOUBLE:
-                generator.writeNumber(DataTypeUtils.toDouble(coercedValue));
+                generator.writeNumber(DataTypeUtils.toDouble(coercedValue, fieldName));
                 break;
             case FLOAT:
-                generator.writeNumber(DataTypeUtils.toFloat(coercedValue));
+                generator.writeNumber(DataTypeUtils.toFloat(coercedValue, fieldName));
                 break;
             case LONG:
-                generator.writeNumber(DataTypeUtils.toLong(coercedValue));
+                generator.writeNumber(DataTypeUtils.toLong(coercedValue, fieldName));
                 break;
             case INT:
             case BYTE:
             case SHORT:
-                generator.writeNumber(DataTypeUtils.toInteger(coercedValue));
+                generator.writeNumber(DataTypeUtils.toInteger(coercedValue, fieldName));
                 break;
             case CHAR:
             case STRING:
@@ -182,7 +195,24 @@ public class WriteJsonResult implements RecordSetWriter {
                 break;
             case RECORD: {
                 final Record record = (Record) coercedValue;
-                writeRecord(record, generator, gen -> gen.writeStartObject(), gen -> gen.writeEndObject());
+                final RecordDataType recordDataType = (RecordDataType) chosenDataType;
+                final RecordSchema childSchema = recordDataType.getChildSchema();
+                writeRecord(record, childSchema, generator, gen -> gen.writeStartObject(), gen -> gen.writeEndObject());
+                break;
+            }
+            case MAP: {
+                final MapDataType mapDataType = (MapDataType) chosenDataType;
+                final DataType valueDataType = mapDataType.getValueType();
+                final Map<String, ?> map = (Map<String, ?>) coercedValue;
+                generator.writeStartObject();
+                int i = 0;
+                for (final Map.Entry<String, ?> entry : map.entrySet()) {
+                    final String mapKey = entry.getKey();
+                    final Object mapValue = entry.getValue();
+                    generator.writeFieldName(mapKey);
+                    writeValue(generator, mapValue, fieldName + "." + mapKey, valueDataType, ++i < map.size());
+                }
+                generator.writeEndObject();
                 break;
             }
             case ARRAY:
@@ -191,7 +221,7 @@ public class WriteJsonResult implements RecordSetWriter {
                     final Object[] values = (Object[]) coercedValue;
                     final ArrayDataType arrayDataType = (ArrayDataType) dataType;
                     final DataType elementType = arrayDataType.getElementType();
-                    writeArray(values, generator, elementType);
+                    writeArray(values, fieldName, generator, elementType);
                 } else {
                     generator.writeString(coercedValue.toString());
                 }
@@ -199,12 +229,13 @@ public class WriteJsonResult implements RecordSetWriter {
         }
     }
 
-    private void writeArray(final Object[] values, final JsonGenerator generator, final DataType elementType) throws JsonGenerationException, IOException, SQLException {
+    private void writeArray(final Object[] values, final String fieldName, final JsonGenerator generator, final DataType elementType)
+        throws JsonGenerationException, IOException, SQLException {
         generator.writeStartArray();
         for (int i = 0; i < values.length; i++) {
             final boolean moreEntries = i < values.length - 1;
             final Object element = values[i];
-            writeValue(generator, element, elementType, moreEntries);
+            writeValue(generator, element, fieldName, elementType, moreEntries);
         }
         generator.writeEndArray();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/50ea1083/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java
new file mode 100644
index 0000000..27f84e4
--- /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/access/AvroSchemaTextStrategy.java
@@ -0,0 +1,64 @@
+/*
+ * 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.access;
+
+import java.io.InputStream;
+import java.util.EnumSet;
+import java.util.Set;
+
+import org.apache.avro.Schema;
+import org.apache.nifi.avro.AvroTypeUtil;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AvroSchemaTextStrategy implements SchemaAccessStrategy {
+    private static final Set<SchemaField> schemaFields = EnumSet.of(SchemaField.SCHEMA_TEXT, SchemaField.SCHEMA_TEXT_FORMAT);
+
+    private static final Logger logger = LoggerFactory.getLogger(AvroSchemaTextStrategy.class);
+    private final PropertyValue schemaTextPropertyValue;
+
+    public AvroSchemaTextStrategy(final PropertyValue schemaTextPropertyValue) {
+        this.schemaTextPropertyValue = schemaTextPropertyValue;
+    }
+
+    @Override
+    public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException {
+        final String schemaText = schemaTextPropertyValue.evaluateAttributeExpressions(flowFile).getValue();
+        if (schemaText == null || schemaText.trim().isEmpty()) {
+            throw new SchemaNotFoundException("FlowFile did not contain appropriate attributes to determine Schema Text");
+        }
+
+        logger.debug("For {} found schema text {}", flowFile, schemaText);
+
+        try {
+            final Schema avroSchema = new Schema.Parser().parse(schemaText);
+            return AvroTypeUtil.createSchema(avroSchema);
+        } catch (final Exception e) {
+            throw new SchemaNotFoundException("Failed to create schema from the Schema Text after evaluating FlowFile Attributes", e);
+        }
+    }
+
+    @Override
+    public Set<SchemaField> getSuppliedSchemaFields() {
+        return schemaFields;
+    }
+}