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

[01/19] nifi git commit: NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Servic

Repository: nifi
Updated Branches:
  refs/heads/master 52cf9a795 -> 68c592ea4


http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..3757ab1
--- /dev/null
+++ 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
@@ -0,0 +1,190 @@
+/*
+ * 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.grok;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.record.Record;
+import org.junit.Test;
+
+import io.thekraken.grok.api.Grok;
+import io.thekraken.grok.api.exception.GrokException;
+
+public class TestGrokRecordReader {
+
+    @Test
+    public void testParseSingleLineLogMessages() throws GrokException, IOException, MalformedRecordException {
+        try (final InputStream fis = new FileInputStream(new File("src/test/resources/grok/single-line-log-messages.txt"))) {
+            final Grok grok = new Grok();
+            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, Collections.emptyMap());
+
+            final String[] logLevels = new String[] {"INFO", "WARN", "ERROR", "FATAL", "FINE"};
+            final String[] messages = new String[] {"Test Message 1", "Red", "Green", "Blue", "Yellow"};
+
+            for (int i = 0; i < logLevels.length; i++) {
+                final Object[] values = deserializer.nextRecord().getValues();
+
+                assertNotNull(values);
+                assertEquals(4, values.length); // values[] contains 4 elements: timestamp, level, message, STACK_TRACE
+                assertEquals("2016-11-08 21:24:23,029", values[0]);
+                assertEquals(logLevels[i], values[1]);
+                assertEquals(messages[i], values[2]);
+                assertNull(values[3]);
+            }
+
+            assertNull(deserializer.nextRecord());
+        }
+    }
+
+
+    @Test
+    public void testParseEmptyMessageWithStackTrace() throws GrokException, IOException, MalformedRecordException {
+        final Grok grok = new Grok();
+        grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt");
+        grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \\[%{DATA:thread}\\] %{DATA:class} %{GREEDYDATA:message}");
+
+        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, Collections.emptyMap());
+
+        final Object[] values = deserializer.nextRecord().getValues();
+
+        assertNotNull(values);
+        assertEquals(6, values.length); // values[] contains 4 elements: timestamp, level, message, STACK_TRACE
+        assertEquals("2016-08-04 13:26:32,473", values[0]);
+        assertEquals("INFO", values[1]);
+        assertEquals("Leader Election Notification Thread-1", values[2]);
+        assertEquals("o.a.n.LoggerClass", values[3]);
+        assertEquals("", values[4]);
+        assertEquals("org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces", values[5]);
+    }
+
+
+
+    @Test
+    public void testParseNiFiSampleLog() throws IOException, GrokException, MalformedRecordException {
+        try (final InputStream fis = new FileInputStream(new File("src/test/resources/grok/nifi-log-sample.log"))) {
+            final Grok grok = new Grok();
+            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, Collections.emptyMap());
+
+            final String[] logLevels = new String[] {"INFO", "INFO", "INFO", "WARN", "WARN"};
+
+            for (int i = 0; i < logLevels.length; i++) {
+                final Object[] values = deserializer.nextRecord().getValues();
+
+                assertNotNull(values);
+                assertEquals(6, values.length); // values[] contains 6 elements: timestamp, level, thread, class, message, STACK_TRACE
+                assertEquals(logLevels[i], values[1]);
+                assertNull(values[5]);
+            }
+
+            assertNull(deserializer.nextRecord());
+        }
+    }
+
+    @Test
+    public void testParseNiFiSampleMultilineWithStackTrace() throws IOException, GrokException, MalformedRecordException {
+        try (final InputStream fis = new FileInputStream(new File("src/test/resources/grok/nifi-log-sample-multiline-with-stacktrace.log"))) {
+            final Grok grok = new Grok();
+            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, Collections.emptyMap());
+
+            final String[] logLevels = new String[] {"INFO", "INFO", "ERROR", "WARN", "WARN"};
+
+            for (int i = 0; i < logLevels.length; i++) {
+                final Record record = deserializer.nextRecord();
+                final Object[] values = record.getValues();
+
+                assertNotNull(values);
+                assertEquals(6, values.length); // values[] contains 6 elements: timestamp, level, thread, class, message, STACK_TRACE
+                assertEquals(logLevels[i], values[1]);
+                if ("ERROR".equals(values[1])) {
+                    final String msg = (String) values[4];
+                    assertEquals("One\nTwo\nThree", msg);
+                    assertNotNull(values[5]);
+                } else {
+                    assertNull(values[5]);
+                }
+            }
+
+            assertNull(deserializer.nextRecord());
+        }
+    }
+
+
+    @Test
+    public void testParseStackTrace() throws GrokException, IOException, MalformedRecordException {
+        try (final InputStream fis = new FileInputStream(new File("src/test/resources/grok/error-with-stack-trace.log"))) {
+            final Grok grok = new Grok();
+            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, Collections.emptyMap());
+
+            final String[] logLevels = new String[] {"INFO", "ERROR", "INFO"};
+            final String[] messages = new String[] {"message without stack trace",
+                "Log message with stack trace",
+                "message without stack trace"};
+
+            for (int i = 0; i < logLevels.length; i++) {
+                final Object[] values = deserializer.nextRecord().getValues();
+
+                assertNotNull(values);
+                assertEquals(4, values.length); // values[] contains 4 elements: timestamp, level, message, STACK_TRACE
+                assertEquals(logLevels[i], values[1]);
+                assertEquals(messages[i], values[2]);
+
+                if (values[1].equals("ERROR")) {
+                    final String stackTrace = (String) values[3];
+                    assertNotNull(stackTrace);
+                    assertTrue(stackTrace.startsWith("org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces"));
+                    assertTrue(stackTrace.contains("        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress("
+                        + "NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]"));
+                    assertTrue(stackTrace.contains("Caused by: org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces"));
+                    assertTrue(stackTrace.contains("at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress("
+                        + "NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]"));
+                    assertTrue(stackTrace.endsWith("    ... 12 common frames omitted"));
+                }
+            }
+
+            assertNull(deserializer.nextRecord());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.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/TestJsonPathRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.java
new file mode 100644
index 0000000..fa41396
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.java
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.json;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.jayway.jsonpath.JsonPath;
+
+public class TestJsonPathRowRecordReader {
+    private final LinkedHashMap<String, JsonPath> allJsonPaths = new LinkedHashMap<>();
+
+    @Before
+    public void populateJsonPaths() {
+        allJsonPaths.clear();
+
+        allJsonPaths.put("id", JsonPath.compile("$.id"));
+        allJsonPaths.put("name", JsonPath.compile("$.name"));
+        allJsonPaths.put("balance", JsonPath.compile("$.balance"));
+        allJsonPaths.put("address", JsonPath.compile("$.address"));
+        allJsonPaths.put("city", JsonPath.compile("$.city"));
+        allJsonPaths.put("state", JsonPath.compile("$.state"));
+        allJsonPaths.put("zipCode", JsonPath.compile("$.zipCode"));
+        allJsonPaths.put("country", JsonPath.compile("$.country"));
+    }
+
+    @Test
+    public void testReadArray() throws IOException, MalformedRecordException {
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
+                RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
+
+            final Object[] secondRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", "USA"}, secondRecordValues);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testSingleJsonElement() throws IOException, MalformedRecordException {
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-bank-account.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
+                RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+
+
+    @Test
+    public void testElementWithNestedData() throws IOException, MalformedRecordException {
+        final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
+        jsonPaths.put("account", JsonPath.compile("$.account"));
+
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "account"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.STRING,
+                RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.RECORD});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            final Object[] simpleElements = Arrays.copyOfRange(firstRecordValues, 0, firstRecordValues.length - 1);
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", null, "123 My Street", "My City", "MS", "11111", "USA"}, simpleElements);
+
+            final Object lastElement = firstRecordValues[firstRecordValues.length - 1];
+            assertTrue(lastElement instanceof Record);
+            final Record record = (Record) lastElement;
+            assertEquals(42, record.getValue("id"));
+            assertEquals(4750.89D, record.getValue("balance"));
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testElementWithNestedArray() throws IOException, MalformedRecordException {
+        final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
+        jsonPaths.put("accounts", JsonPath.compile("$.accounts"));
+
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested-array.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {
+                "id", "name", "balance", "address", "city", "state", "zipCode", "country", "accounts"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.STRING,
+                RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.ARRAY});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            final Object[] nonArrayValues = Arrays.copyOfRange(firstRecordValues, 0, firstRecordValues.length - 1);
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", null, "123 My Street", "My City", "MS", "11111", "USA"}, nonArrayValues);
+
+            final Object lastRecord = firstRecordValues[firstRecordValues.length - 1];
+            assertTrue(Object[].class.isAssignableFrom(lastRecord.getClass()));
+
+            final Object[] array = (Object[]) lastRecord;
+            assertEquals(2, array.length);
+            final Object firstElement = array[0];
+            assertTrue(firstElement instanceof Map);
+
+            final Map<?, ?> firstMap = (Map<?, ?>) firstElement;
+            assertEquals(42, firstMap.get("id"));
+            assertEquals(4750.89D, firstMap.get("balance"));
+
+            final Object secondElement = array[1];
+            assertTrue(secondElement instanceof Map);
+            final Map<?, ?> secondMap = (Map<?, ?>) secondElement;
+            assertEquals(43, secondMap.get("id"));
+            assertEquals(48212.38D, secondMap.get("balance"));
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testReadArrayDifferentSchemas() throws IOException, MalformedRecordException {
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
+                RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
+
+            final Object[] secondRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", null}, secondRecordValues);
+
+            final Object[] thirdRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {3, "Jake Doe", 4751.89, "124 My Street", "My City", "MS", "11111", "USA"}, thirdRecordValues);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testReadArrayDifferentSchemasWithOverride() throws IOException, MalformedRecordException {
+        final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
+        jsonPaths.put("address2", JsonPath.compile("$.address2"));
+        final Map<String, DataType> typeOverrides = Collections.singletonMap("address2", RecordFieldType.STRING.getDataType());
+
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, typeOverrides, in, Mockito.mock(ComponentLog.class))) {
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "address2"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.DOUBLE, RecordFieldType.STRING,
+                RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA", null}, firstRecordValues);
+
+            final Object[] secondRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", null, null}, secondRecordValues);
+
+            final Object[] thirdRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {3, "Jake Doe", 4751.89, "124 My Street", "My City", "MS", "11111", "USA", "Apt. #12"}, thirdRecordValues);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testPrimitiveTypeArrays() throws IOException, MalformedRecordException {
+        final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
+        jsonPaths.put("accountIds", JsonPath.compile("$.accountIds"));
+
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/primitive-type-array.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "accountIds"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.DOUBLE, RecordFieldType.STRING,
+                RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.ARRAY});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+
+            final Object[] nonArrayValues = Arrays.copyOfRange(firstRecordValues, 0, firstRecordValues.length - 1);
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"}, nonArrayValues);
+
+            final Object lastRecord = firstRecordValues[firstRecordValues.length - 1];
+            assertNotNull(lastRecord);
+            assertTrue(Object[].class.isAssignableFrom(lastRecord.getClass()));
+
+            final Object[] array = (Object[]) lastRecord;
+            Assert.assertArrayEquals(new Object[] {1, 2, 3}, array);
+
+            assertNull(reader.nextRecord());
+            assertNull(reader.nextRecord());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..c5ee0e3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.json;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestJsonTreeRowRecordReader {
+
+    @Test
+    public void testReadArray() throws IOException, MalformedRecordException {
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
+                RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
+
+            final Object[] secondRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", "USA"}, secondRecordValues);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testSingleJsonElement() throws IOException, MalformedRecordException {
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-bank-account.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
+                RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testElementWithNestedData() throws IOException, MalformedRecordException {
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "address", "city", "state", "zipCode", "country", "account"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
+                RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.RECORD});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            final Object[] allButLast = Arrays.copyOfRange(firstRecordValues, 0, firstRecordValues.length - 1);
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", "123 My Street", "My City", "MS", "11111", "USA"}, allButLast);
+
+            final Object last = firstRecordValues[firstRecordValues.length - 1];
+            assertTrue(Record.class.isAssignableFrom(last.getClass()));
+            final Record record = (Record) last;
+            assertEquals(42, record.getValue("id"));
+            assertEquals(4750.89, record.getValue("balance"));
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testElementWithNestedArray() throws IOException, MalformedRecordException {
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested-array.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {
+                "id", "name", "address", "city", "state", "zipCode", "country", "accounts"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
+                RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.ARRAY});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            final Object[] nonArrayValues = Arrays.copyOfRange(firstRecordValues, 0, firstRecordValues.length - 1);
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", "123 My Street", "My City", "MS", "11111", "USA"}, nonArrayValues);
+
+            final Object lastRecord = firstRecordValues[firstRecordValues.length - 1];
+            assertTrue(Object[].class.isAssignableFrom(lastRecord.getClass()));
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testReadArrayDifferentSchemas() throws IOException, MalformedRecordException {
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
+                RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
+
+            final Object[] secondRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", null}, secondRecordValues);
+
+            final Object[] thirdRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {3, "Jake Doe", 4751.89, "124 My Street", "My City", "MS", "11111", "USA"}, thirdRecordValues);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testReadArrayDifferentSchemasWithOverride() throws IOException, MalformedRecordException {
+        final Map<String, DataType> overrides = new HashMap<>();
+        overrides.put("address2", RecordFieldType.STRING.getDataType());
+
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), overrides)) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "address2"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.DOUBLE, RecordFieldType.STRING,
+                RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA", null}, firstRecordValues);
+
+            final Object[] secondRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", null, null}, secondRecordValues);
+
+            final Object[] thirdRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {3, "Jake Doe", 4751.89, "124 My Street", "My City", "MS", "11111", "USA", "Apt. #12"}, thirdRecordValues);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testReadArrayDifferentSchemasWithOptionalElementOverridden() throws IOException, MalformedRecordException {
+        final Map<String, DataType> overrides = new HashMap<>();
+        overrides.put("balance", RecordFieldType.DOUBLE.getDataType());
+
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-optional-balance.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), overrides)) {
+
+            final RecordSchema schema = reader.getSchema();
+
+            final List<String> fieldNames = schema.getFieldNames();
+            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
+            assertEquals(expectedFieldNames, fieldNames);
+
+            final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.DOUBLE, RecordFieldType.STRING,
+                RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
+            assertEquals(expectedTypes, dataTypes);
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
+
+            final Object[] secondRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {2, "Jane Doe", null, "321 Your Street", "Your City", "NY", "33333", "USA"}, secondRecordValues);
+
+            final Object[] thirdRecordValues = reader.nextRecord().getValues();
+            Assert.assertArrayEquals(new Object[] {3, "Jimmy Doe", null, "321 Your Street", "Your City", "NY", "33333", "USA"}, thirdRecordValues);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+
+    @Test
+    public void testReadUnicodeCharacters() throws IOException, MalformedRecordException {
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/json-with-unicode.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+
+            final Object[] firstRecordValues = reader.nextRecord().getValues();
+
+            final Object secondValue = firstRecordValues[1];
+            assertTrue(secondValue instanceof Long);
+            assertEquals(832036744985577473L, secondValue);
+
+            final Object unicodeValue = firstRecordValues[2];
+            assertEquals("\u3061\u3083\u6ce3\u304d\u305d\u3046", unicodeValue);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..f9849ba
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.json;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+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) {
+                final List<DataType> possibleTypes = new ArrayList<>();
+                possibleTypes.add(RecordFieldType.INT.getDataType());
+                possibleTypes.add(RecordFieldType.LONG.getDataType());
+
+                fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType(possibleTypes)));
+            } else {
+                fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType()));
+            }
+        }
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+
+        final long time = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS").parse("2017/01/01 17:00:00.000").getTime();
+        final Map<String, Object> valueMap = new LinkedHashMap<>();
+        valueMap.put("string", "string");
+        valueMap.put("boolean", true);
+        valueMap.put("byte", (byte) 1);
+        valueMap.put("char", 'c');
+        valueMap.put("short", (short) 8);
+        valueMap.put("int", 9);
+        valueMap.put("bigint", BigInteger.valueOf(8L));
+        valueMap.put("long", 8L);
+        valueMap.put("float", 8.0F);
+        valueMap.put("double", 8.0D);
+        valueMap.put("date", new Date(time));
+        valueMap.put("time", new Time(time));
+        valueMap.put("timestamp", new Timestamp(time));
+        valueMap.put("record", null);
+        valueMap.put("array", null);
+        valueMap.put("choice", 48L);
+
+        final Record record = new MapRecord(schema, valueMap);
+        final RecordSet rs = RecordSet.of(schema, record);
+
+        final String output;
+        try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+            writer.write(rs, baos);
+            output = baos.toString();
+        }
+
+        final String expected = new String(Files.readAllBytes(Paths.get("src/test/resources/json/output/dataTypes.json")));
+        assertEquals(expected, output);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/extra-white-space.csv
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/extra-white-space.csv b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/extra-white-space.csv
new file mode 100644
index 0000000..4531083
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/extra-white-space.csv
@@ -0,0 +1,9 @@
+id, name, balance, address, city, state, zipCode, country
+1, John Doe, "4750.89", "123 My Street", My City, MS, 11111, USA
+
+
+
+
+     2, Jane Doe, 4820.09, 321 Your Street, Your City, NY, 33333, USA     
+
+

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/multi-bank-account.csv
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/multi-bank-account.csv b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/multi-bank-account.csv
new file mode 100644
index 0000000..9f761c6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/multi-bank-account.csv
@@ -0,0 +1,3 @@
+id, name, balance, address, city, state, zipCode, country
+1, John Doe, "4750.89", "123 My Street", My City, MS, 11111, USA
+2, Jane Doe, 4820.09, 321 Your Street, Your City, NY, 33333, USA
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/single-bank-account.csv
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/single-bank-account.csv b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/single-bank-account.csv
new file mode 100644
index 0000000..11ce6d4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/csv/single-bank-account.csv
@@ -0,0 +1,2 @@
+id, name, balance, address, city, state, zipCode, country
+1, John Doe, "4750.89", "123 My Street", My City, MS, 11111, USA
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/error-with-stack-trace.log
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/error-with-stack-trace.log b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/error-with-stack-trace.log
new file mode 100644
index 0000000..43f1b56
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/error-with-stack-trace.log
@@ -0,0 +1,25 @@
+2016-11-23 16:00:00,000 INFO message without stack trace
+2016-11-23 16:00:02,689 ERROR Log message with stack trace
+org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_45]
+	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_45]
+        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_45]
+        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_45]
+        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_45]
+        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_45]
+        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
+Caused by: org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+    ... 12 common frames omitted
+2016-11-23 16:05:00,000 INFO message without stack trace

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/nifi-log-sample-multiline-with-stacktrace.log
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/nifi-log-sample-multiline-with-stacktrace.log b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/nifi-log-sample-multiline-with-stacktrace.log
new file mode 100644
index 0000000..673908f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/nifi-log-sample-multiline-with-stacktrace.log
@@ -0,0 +1,29 @@
+2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.c.l.e.CuratorLeaderElectionManager org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 has been interrupted; no longer leader for role 'Cluster Coordinator'
+2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.c.l.e.CuratorLeaderElectionManager org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 This node is no longer leader for role 'Cluster Coordinator'
+2016-08-04 13:26:32,474 ERROR [Leader Election Notification Thread-2] o.apache.nifi.controller.FlowController One
+Two
+Three
+org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_45]
+	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_45]
+        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_45]
+        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_45]
+        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_45]
+        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_45]
+        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
+Caused by: org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
+    ... 12 common frames omitted
+2016-08-04 13:26:35,475 WARN [Curator-Framework-0] org.apache.curator.ConnectionState Connection attempt unsuccessful after 3008 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.
+2016-08-04 13:26:35,479 WARN [Curator-Framework-0] org.apache.curator.ConnectionState Connection attempt unsuccessful after 3007 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/nifi-log-sample.log
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/nifi-log-sample.log b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/nifi-log-sample.log
new file mode 100644
index 0000000..e4c7385
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/nifi-log-sample.log
@@ -0,0 +1,5 @@
+2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.c.l.e.CuratorLeaderElectionManager org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 has been interrupted; no longer leader for role 'Cluster Coordinator'
+2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.c.l.e.CuratorLeaderElectionManager org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 This node is no longer leader for role 'Cluster Coordinator'
+2016-08-04 13:26:32,474 INFO [Leader Election Notification Thread-2] o.apache.nifi.controller.FlowController This node is no longer Primary Node
+2016-08-04 13:26:35,475 WARN [Curator-Framework-0] org.apache.curator.ConnectionState Connection attempt unsuccessful after 3008 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.
+2016-08-04 13:26:35,479 WARN [Curator-Framework-0] org.apache.curator.ConnectionState Connection attempt unsuccessful after 3007 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/single-line-log-messages.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/single-line-log-messages.txt b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/single-line-log-messages.txt
new file mode 100644
index 0000000..737ad95
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/grok/single-line-log-messages.txt
@@ -0,0 +1,5 @@
+2016-11-08 21:24:23,029 INFO Test Message 1
+2016-11-08 21:24:23,029 WARN Red
+2016-11-08 21:24:23,029 ERROR Green
+2016-11-08 21:24:23,029 FATAL Blue
+2016-11-08 21:24:23,029 FINE Yellow
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array-different-schemas.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array-different-schemas.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array-different-schemas.json
new file mode 100644
index 0000000..e79a6d7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array-different-schemas.json
@@ -0,0 +1,30 @@
+[
+	{
+		"id": 1,
+		"name": "John Doe",
+		"balance": 4750.89,
+		"address": "123 My Street",
+		"city": "My City", 
+		"state": "MS",
+		"zipCode": "11111",
+		"country": "USA"
+	}, {
+		"id": 2,
+		"name": "Jane Doe",
+		"balance": 4820.09,
+		"address": "321 Your Street",
+		"city": "Your City", 
+		"state": "NY",
+		"zipCode": "33333"
+	}, {
+		"id": 3,
+		"name": "Jake Doe",
+		"balance": 4751.89,
+		"address": "124 My Street",
+		"address2": "Apt. #12",
+		"city": "My City", 
+		"state": "MS",
+		"zipCode": "11111",
+		"country": "USA"
+	}
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array-optional-balance.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array-optional-balance.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array-optional-balance.json
new file mode 100644
index 0000000..cb614f1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array-optional-balance.json
@@ -0,0 +1,29 @@
+[
+	{
+		"id": 1,
+		"name": "John Doe",
+		"balance": 4750.89,
+		"address": "123 My Street",
+		"city": "My City", 
+		"state": "MS",
+		"zipCode": "11111",
+		"country": "USA"
+	}, {
+		"id": 2,
+		"name": "Jane Doe",
+		"balance": null,
+		"address": "321 Your Street",
+		"city": "Your City", 
+		"state": "NY",
+		"zipCode": "33333",
+		"country": "USA"
+	}, {
+		"id": 3,
+		"name": "Jimmy Doe",
+		"address": "321 Your Street",
+		"city": "Your City", 
+		"state": "NY",
+		"zipCode": "33333",
+		"country": "USA"
+	}
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array.json
new file mode 100644
index 0000000..d821cc1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/bank-account-array.json
@@ -0,0 +1,21 @@
+[
+	{
+		"id": 1,
+		"name": "John Doe",
+		"balance": 4750.89,
+		"address": "123 My Street",
+		"city": "My City", 
+		"state": "MS",
+		"zipCode": "11111",
+		"country": "USA"
+	}, {
+		"id": 2,
+		"name": "Jane Doe",
+		"balance": 4820.09,
+		"address": "321 Your Street",
+		"city": "Your City", 
+		"state": "NY",
+		"zipCode": "33333",
+		"country": "USA"
+	}
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/json-with-unicode.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/json-with-unicode.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/json-with-unicode.json
new file mode 100644
index 0000000..880fabf
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/json-with-unicode.json
@@ -0,0 +1,9 @@
+{
+	"created_at":"Thu Feb 16 01:19:42 +0000 2017",
+	"id":832036744985577473,
+	"unicode":"\u3061\u3083\u6ce3\u304d\u305d\u3046",
+	"from":{
+		"id":788946702264507903,
+		"name":"john"
+	}
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..40c28dd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/output/dataTypes.json
@@ -0,0 +1,18 @@
+[ {
+  "string" : "string",
+  "boolean" : true,
+  "byte" : 1,
+  "char" : "c",
+  "short" : 8,
+  "int" : 9,
+  "bigint" : 8,
+  "long" : 8,
+  "float" : 8.0,
+  "double" : 8.0,
+  "date" : "2017-01-01",
+  "time" : "17:00:00",
+  "timestamp" : "2017-01-01 17:00:00",
+  "record" : null,
+  "choice" : 48,
+  "array" : null
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/primitive-type-array.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/primitive-type-array.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/primitive-type-array.json
new file mode 100644
index 0000000..47e3276
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/primitive-type-array.json
@@ -0,0 +1,13 @@
+[
+	{
+		"id": 1,
+		"name": "John Doe",
+		"balance": 4750.89,
+		"address": "123 My Street",
+		"city": "My City", 
+		"state": "MS",
+		"zipCode": "11111",
+		"country": "USA",
+		"accountIds": [1, 2, 3]
+	}
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-bank-account.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.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-bank-account.json
new file mode 100644
index 0000000..a8d6890
--- /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.json
@@ -0,0 +1,10 @@
+{
+	"id": 1,
+	"name": "John Doe",
+	"balance": 4750.89,
+	"address": "123 My Street",
+	"city": "My City", 
+	"state": "MS",
+	"zipCode": "11111",
+	"country": "USA"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-element-nested-array.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-element-nested-array.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-element-nested-array.json
new file mode 100644
index 0000000..0bca3a0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-element-nested-array.json
@@ -0,0 +1,16 @@
+{
+	"id": 1,
+	"name": "John Doe",
+	"address": "123 My Street",
+	"city": "My City", 
+	"state": "MS",
+	"zipCode": "11111",
+	"country": "USA",
+	"accounts": [{
+		"id": 42,
+		"balance": 4750.89
+	}, {
+		"id": 43,
+		"balance": 48212.38
+	}]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-element-nested.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-element-nested.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-element-nested.json
new file mode 100644
index 0000000..26a59e1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-element-nested.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": 4750.89
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/pom.xml
new file mode 100644
index 0000000..b1ac470
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/pom.xml
@@ -0,0 +1,30 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      Licensed to the Apache Software Foundation (ASF) under one or more
+      contributor license agreements.  See the NOTICE file distributed with
+      this work for additional information regarding copyright ownership.
+      The ASF licenses this file to You under the Apache License, Version 2.0
+      (the "License"); you may not use this file except in compliance with
+      the License.  You may obtain a copy of the License at
+          http://www.apache.org/licenses/LICENSE-2.0
+      Unless required by applicable law or agreed to in writing, software
+      distributed under the License is distributed on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+      See the License for the specific language governing permissions and
+      limitations under the License.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-standard-services</artifactId>
+        <version>1.2.0-SNAPSHOT</version>
+    </parent>
+    
+    <artifactId>nifi-record-serialization-services-bundle</artifactId>
+    <packaging>pom</packaging>
+    
+    <modules>
+        <module>nifi-record-serialization-services</module>
+        <module>nifi-record-serialization-services-nar</module>
+    </modules>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml
index c6e6128..eae3515 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml
@@ -56,5 +56,10 @@
             <artifactId>nifi-hbase-client-service-api</artifactId>
             <scope>compile</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/pom.xml
index 7e754be..3948a1b 100644
--- a/nifi-nar-bundles/nifi-standard-services/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-services/pom.xml
@@ -35,5 +35,7 @@
         <module>nifi-dbcp-service-bundle</module>
         <module>nifi-hbase-client-service-api</module>
         <module>nifi-hbase_1_1_2-client-service-bundle</module>
+        <module>nifi-record-serialization-service-api</module>
+        <module>nifi-record-serialization-services-bundle</module>
     </modules>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 449450d..057832b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -966,6 +966,11 @@ language governing permissions and limitations under the License. -->
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-record-serialization-service-api</artifactId>
+                <version>1.2.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-distributed-cache-services-nar</artifactId>
                 <version>1.2.0-SNAPSHOT</version>
                 <type>nar</type>
@@ -1293,6 +1298,12 @@ language governing permissions and limitations under the License. -->
                 <version>1.2.0-SNAPSHOT</version>
                 <type>nar</type>
             </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-record-serialization-services-nar</artifactId>
+                <version>1.2.0-SNAPSHOT</version>
+                <type>nar</type>
+            </dependency>
 	        <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-evtx-nar</artifactId>


[04/19] nifi git commit: NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Servic

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..1848020
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,77 @@
+nifi-record-serialization-services-nar
+Copyright 2014-2017 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+===========================================
+Apache Software License v2
+===========================================
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) Apache Commons Lang
+    The following NOTICE information applies:
+      Apache Commons Lang
+      Copyright 2001-2015 The Apache Software Foundation
+
+      This product includes software from the Spring Framework,
+      under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+  (ASLv2) Grok
+    The following NOTICE information applies:
+      Grok
+      Copyright 2014 Anthony Corbacho, and contributors.
+
+  (ASLv2) Groovy (org.codehaus.groovy:groovy:jar:2.4.5 - http://www.groovy-lang.org)
+    The following NOTICE information applies:
+      Groovy Language
+         Copyright 2003-2015 The respective authors and developers
+         Developers and Contributors are listed in the project POM file
+         and Gradle build file
+
+         This product includes software developed by
+         The Groovy community (http://groovy.codehaus.org/).
+
+  (ASLv2) Google GSON
+    The following NOTICE information applies:
+      Copyright 2008 Google Inc.
+
+  (ASLv2) Jackson JSON processor
+    The following NOTICE information applies:
+      # Jackson JSON processor
+
+      Jackson is a high-performance, Free/Open Source JSON processing library.
+      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+      been in development since 2007.
+      It is currently developed by a community of developers, as well as supported
+      commercially by FasterXML.com.
+
+      ## Licensing
+
+      Jackson core and extension components may licensed under different licenses.
+      To find the details that apply to this artifact see the accompanying LICENSE file.
+      For more information, including possible other licensing options, contact
+      FasterXML.com (http://fasterxml.com).
+
+      ## Credits
+
+      A list of contributors may be found from CREDITS file, which is included
+      in some artifacts (usually source distributions); but is always available
+      from the source code management (SCM) system project uses.
+
+  (ASLv2) JSON-SMART
+    The following NOTICE information applies:
+      Copyright 2011 JSON-SMART authors
+
+   (ASLv2) JsonPath
+     The following NOTICE information applies:
+       Copyright 2011 JsonPath authors
+
+  (ASLv2) opencsv (net.sf.opencsv:opencsv:2.3)
+
+  (ASLv2) Apache Avro
+    The following NOTICE information applies:
+      Apache Avro
+      Copyright 2009-2013 The Apache Software Foundation
+  
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/.gitignore
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/.gitignore b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/.gitignore
new file mode 100644
index 0000000..ae3c172
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/.gitignore
@@ -0,0 +1 @@
+/bin/

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..9b2a56c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml
@@ -0,0 +1,94 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!-- Licensed to the Apache Software Foundation (ASF) under one or more 
+        contributor license agreements. See the NOTICE file distributed with this 
+        work for additional information regarding copyright ownership. The ASF licenses 
+        this file to You under the Apache License, Version 2.0 (the "License"); you 
+        may not use this file except in compliance with the License. You may obtain 
+        a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless 
+        required by applicable law or agreed to in writing, software distributed 
+        under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 
+        OR CONDITIONS OF ANY KIND, either express or implied. See the License for 
+        the specific language governing permissions and limitations under the License. -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-record-serialization-services-bundle</artifactId>
+        <version>1.2.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-record-serialization-services</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.jayway.jsonpath</groupId>
+            <artifactId>json-path</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.codehaus.jackson</groupId>
+            <artifactId>jackson-mapper-asl</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>net.sf.opencsv</groupId>
+            <artifactId>opencsv</artifactId>
+            <version>2.3</version>
+        </dependency>
+        <dependency>
+            <groupId>io.thekraken</groupId>
+            <artifactId>grok</artifactId>
+            <version>0.1.5</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes combine.children="append">
+                        <exclude>src/test/resources/csv/extra-white-space.csv</exclude>
+                        <exclude>src/test/resources/csv/multi-bank-account.csv</exclude>
+                        <exclude>src/test/resources/csv/single-bank-account.csv</exclude>
+                        <exclude>src/test/resources/grok/error-with-stack-trace.log</exclude>
+                        <exclude>src/test/resources/grok/nifi-log-sample-multiline-with-stacktrace.log</exclude>
+                        <exclude>src/test/resources/grok/nifi-log-sample.log</exclude>
+                        <exclude>src/test/resources/grok/single-line-log-messages.txt</exclude>
+                        <exclude>src/test/resources/json/bank-account-array-different-schemas.json</exclude>
+                        <exclude>src/test/resources/json/bank-account-array.json</exclude>
+                        <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-element-nested-array.json</exclude>
+                        <exclude>src/test/resources/json/single-element-nested.json</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..fc0c598
--- /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/AvroReader.java
@@ -0,0 +1,40 @@
+/*
+ * 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.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RowRecordReaderFactory;
+
+@Tags({"avro", "parse", "record", "row", "reader", "delimited", "comma", "separated", "values"})
+@CapabilityDescription("Parses Avro data and returns each Avro record as an separate record.")
+public class AvroReader extends AbstractControllerService implements RowRecordReaderFactory {
+
+    @Override
+    public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws MalformedRecordException, IOException {
+        return new AvroRecordReader(in);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..e98a5ad
--- /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/AvroRecordReader.java
@@ -0,0 +1,254 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.Schema.Type;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericData.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;
+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;
+
+public class AvroRecordReader implements RecordReader {
+    private final InputStream in;
+    private final Schema avroSchema;
+    private final DataFileStream<GenericRecord> dataFileStream;
+    private RecordSchema recordSchema;
+
+    public AvroRecordReader(final InputStream in) throws IOException, MalformedRecordException {
+        this.in = in;
+
+        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()) {
+            return null;
+        }
+
+        GenericRecord record = null;
+        while (record == null && dataFileStream.hasNext()) {
+            record = dataFileStream.next();
+        }
+
+        final RecordSchema schema = getSchema();
+        final Map<String, Object> values = convertRecordToObjectArray(record, schema);
+        return new MapRecord(schema, values);
+    }
+
+
+    private Map<String, Object> convertRecordToObjectArray(final GenericRecord record, final RecordSchema schema) {
+        final Map<String, Object> values = new HashMap<>(schema.getFieldCount());
+
+        for (final String fieldName : schema.getFieldNames()) {
+            final Object value = record.get(fieldName);
+
+            final Field avroField = record.getSchema().getField(fieldName);
+            if (avroField == null) {
+                values.put(fieldName, null);
+                continue;
+            }
+
+            final Schema fieldSchema = avroField.schema();
+            final DataType dataType = schema.getDataType(fieldName).orElse(null);
+            final Object converted = convertValue(value, fieldSchema, avroField.name(), dataType);
+            values.put(fieldName, converted);
+        }
+
+        return values;
+    }
+
+
+    @Override
+    public RecordSchema getSchema() throws MalformedRecordException {
+        if (recordSchema != null) {
+            return recordSchema;
+        }
+
+        recordSchema = createSchema(avroSchema);
+        return recordSchema;
+    }
+
+    private RecordSchema createSchema(final Schema avroSchema) {
+        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));
+        }
+
+        final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
+        return recordSchema;
+    }
+
+    private Object convertValue(final Object value, final Schema avroSchema, final String fieldName, final DataType desiredType) {
+        if (value == null) {
+            return null;
+        }
+
+        switch (avroSchema.getType()) {
+            case UNION:
+                if (value instanceof GenericData.Record) {
+                    final GenericData.Record record = (GenericData.Record) value;
+                    return convertValue(value, record.getSchema(), fieldName, desiredType);
+                }
+                break;
+            case RECORD:
+                final GenericData.Record record = (GenericData.Record) value;
+                final Schema recordSchema = record.getSchema();
+                final List<Field> recordFields = recordSchema.getFields();
+                final Map<String, Object> values = new HashMap<>(recordFields.size());
+                for (final Field field : recordFields) {
+                    final DataType desiredFieldType = determineDataType(field.schema());
+                    final Object avroFieldValue = record.get(field.name());
+                    final Object fieldValue = convertValue(avroFieldValue, field.schema(), field.name(), desiredFieldType);
+                    values.put(field.name(), fieldValue);
+                }
+                final RecordSchema childSchema = createSchema(recordSchema);
+                return new MapRecord(childSchema, values);
+            case BYTES:
+                final ByteBuffer bb = (ByteBuffer) value;
+                return bb.array();
+            case FIXED:
+                final GenericFixed fixed = (GenericFixed) value;
+                return fixed.bytes();
+            case ENUM:
+                return value.toString();
+            case NULL:
+                return null;
+            case STRING:
+                return value.toString();
+            case ARRAY:
+                final Array<?> array = (Array<?>) value;
+                final Object[] valueArray = new Object[array.size()];
+                for (int i = 0; i < array.size(); i++) {
+                    final Schema elementSchema = avroSchema.getElementType();
+                    valueArray[i] = convertValue(array.get(i), elementSchema, fieldName, determineDataType(elementSchema));
+                }
+                return valueArray;
+            case MAP:
+                final Map<?, ?> avroMap = (Map<?, ?>) value;
+                final Map<String, Object> map = new HashMap<>(avroMap.size());
+                for (final Map.Entry<?, ?> entry : avroMap.entrySet()) {
+                    Object obj = entry.getValue();
+                    if (obj instanceof Utf8 || obj instanceof CharSequence) {
+                        obj = obj.toString();
+                    }
+
+                    map.put(entry.getKey().toString(), obj);
+                }
+                return map;
+        }
+
+        return value;
+    }
+
+
+    private DataType determineDataType(final Schema avroSchema) {
+        final Type avroType = avroSchema.getType();
+
+        switch (avroType) {
+            case ARRAY:
+            case BYTES:
+            case FIXED:
+                return RecordFieldType.ARRAY.getDataType();
+            case BOOLEAN:
+                return RecordFieldType.BOOLEAN.getDataType();
+            case DOUBLE:
+                return RecordFieldType.DOUBLE.getDataType();
+            case ENUM:
+            case STRING:
+                return RecordFieldType.STRING.getDataType();
+            case FLOAT:
+                return RecordFieldType.FLOAT.getDataType();
+            case INT:
+                return RecordFieldType.INT.getDataType();
+            case LONG:
+                return RecordFieldType.LONG.getDataType();
+            case RECORD: {
+                final List<Field> avroFields = avroSchema.getFields();
+                final List<RecordField> recordFields = new ArrayList<>(avroFields.size());
+
+                for (final Field field : avroFields) {
+                    final String fieldName = field.name();
+                    final Schema fieldSchema = field.schema();
+                    final DataType fieldType = determineDataType(fieldSchema);
+                    recordFields.add(new RecordField(fieldName, fieldType));
+                }
+
+                final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
+                return RecordFieldType.RECORD.getDataType(recordSchema);
+            }
+            case NULL:
+            case MAP:
+                return RecordFieldType.RECORD.getDataType();
+            case UNION: {
+                final List<Schema> nonNullSubSchemas = avroSchema.getTypes().stream()
+                    .filter(s -> s.getType() != Type.NULL)
+                    .collect(Collectors.toList());
+
+                if (nonNullSubSchemas.size() == 1) {
+                    return determineDataType(nonNullSubSchemas.get(0));
+                }
+
+                final List<DataType> possibleChildTypes = new ArrayList<>(nonNullSubSchemas.size());
+                for (final Schema subSchema : nonNullSubSchemas) {
+                    final DataType childDataType = determineDataType(subSchema);
+                    possibleChildTypes.add(childDataType);
+                }
+
+                return RecordFieldType.CHOICE.getDataType(possibleChildTypes);
+            }
+        }
+
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..d56c716
--- /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/AvroRecordSetWriter.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.avro;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.AbstractRecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+
+@Tags({"avro", "result", "set", "writer", "serializer", "record", "row"})
+@CapabilityDescription("Writes the contents of a Database ResultSet in Binary Avro format. The data types in the Result Set must match those "
+    + "specified by the Avro Schema. No type coercion will occur, with the exception of Date, Time, and Timestamps fields because Avro does not provide "
+    + "support for these types specifically. As a result, they will be converted to String fields using the configured formats. In addition, the label"
+    + "of the column must be a valid Avro field name.")
+public class AvroRecordSetWriter extends AbstractRecordSetWriter 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)
+        .build();
+
+    private volatile Schema schema;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(SCHEMA);
+        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) {
+        return new WriteAvroResult(schema, getDateFormat(), getTimeFormat(), getTimestampFormat());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..7151348
--- /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/AvroSchemaValidator.java
@@ -0,0 +1,45 @@
+/*
+ * 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 org.apache.avro.Schema;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+
+public class AvroSchemaValidator implements Validator {
+
+    @Override
+    public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+        try {
+            new Schema.Parser().parse(input);
+
+            return new ValidationResult.Builder()
+                .valid(true)
+                .build();
+        } catch (final Exception e) {
+            return new ValidationResult.Builder()
+                .input(input)
+                .subject(subject)
+                .valid(false)
+                .explanation("Not a valid Avro Schema: " + e.getMessage())
+                .build();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..d75d86d
--- /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/WriteAvroResult.java
@@ -0,0 +1,286 @@
+/*
+ * 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.io.OutputStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+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.RecordSetWriter;
+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 WriteAvroResult implements RecordSetWriter {
+    private final Schema schema;
+    private final DateFormat dateFormat;
+    private final DateFormat timeFormat;
+    private final DateFormat timestampFormat;
+
+    public WriteAvroResult(final Schema schema, final String dateFormat, final String timeFormat, final String timestampFormat) {
+        this.schema = schema;
+        this.dateFormat = new SimpleDateFormat(dateFormat);
+        this.timeFormat = new SimpleDateFormat(timeFormat);
+        this.timestampFormat = new SimpleDateFormat(timestampFormat);
+    }
+
+    @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());
+        }
+
+        final GenericRecord rec = new GenericData.Record(schema);
+
+        int nrOfRows = 0;
+        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
+            dataFileWriter.create(schema, outStream);
+
+            final RecordSchema recordSchema = rs.getSchema();
+
+            do {
+                for (final String fieldName : recordSchema.getFieldNames()) {
+                    final Object value = record.getValue(fieldName);
+
+                    final Field field = schema.getField(fieldName);
+                    if (field == null) {
+                        continue;
+                    }
+
+                    final Object converted;
+                    try {
+                        converted = convert(value, field.schema(), fieldName);
+                    } catch (final SQLException e) {
+                        throw new IOException("Failed to write records to stream", e);
+                    }
+
+                    rec.put(fieldName, converted);
+                }
+
+                dataFileWriter.append(rec);
+                nrOfRows++;
+            } while ((record = rs.next()) != null);
+        }
+
+        return WriteResult.of(nrOfRows, Collections.emptyMap());
+    }
+
+    @Override
+    public WriteResult write(final Record record, final OutputStream out) throws IOException {
+        final GenericRecord rec = new GenericData.Record(schema);
+
+        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
+            dataFileWriter.create(schema, out);
+            final RecordSchema recordSchema = record.getSchema();
+
+            for (final String fieldName : recordSchema.getFieldNames()) {
+                final Object value = record.getValue(fieldName);
+
+                final Field field = schema.getField(fieldName);
+                if (field == null) {
+                    continue;
+                }
+
+                final Object converted;
+                try {
+                    converted = convert(value, field.schema(), fieldName);
+                } catch (final SQLException e) {
+                    throw new IOException("Failed to write records to stream", e);
+                }
+
+                rec.put(fieldName, converted);
+            }
+
+            dataFileWriter.append(rec);
+        }
+
+        return WriteResult.of(1, Collections.emptyMap());
+    }
+
+
+    private Object convert(final Object value, final Schema schema, final String fieldName) throws SQLException, IOException {
+        if (value == null) {
+            return null;
+        }
+
+        // Need to handle CLOB and BLOB before getObject() is called, due to ResultSet's maximum portability statement
+        if (value instanceof Clob) {
+            final Clob clob = (Clob) value;
+
+            long numChars = clob.length();
+            char[] buffer = new char[(int) numChars];
+            InputStream is = clob.getAsciiStream();
+            int index = 0;
+            int c = is.read();
+            while (c > 0) {
+                buffer[index++] = (char) c;
+                c = is.read();
+            }
+
+            clob.free();
+            return new String(buffer);
+        }
+
+        if (value instanceof Blob) {
+            final Blob blob = (Blob) value;
+
+            final long numChars = blob.length();
+            final byte[] buffer = new byte[(int) numChars];
+            final InputStream is = blob.getBinaryStream();
+            int index = 0;
+            int c = is.read();
+            while (c > 0) {
+                buffer[index++] = (byte) c;
+                c = is.read();
+            }
+
+            final ByteBuffer bb = ByteBuffer.wrap(buffer);
+            blob.free();
+            return bb;
+        }
+
+        if (value instanceof byte[]) {
+            // bytes requires little bit different handling
+            return ByteBuffer.wrap((byte[]) value);
+        } else if (value instanceof Byte) {
+            // tinyint(1) type is returned by JDBC driver as java.sql.Types.TINYINT
+            // But value is returned by JDBC as java.lang.Byte
+            // (at least H2 JDBC works this way)
+            // direct put to avro record results:
+            // org.apache.avro.AvroRuntimeException: Unknown datum type java.lang.Byte
+            return ((Byte) value).intValue();
+        } else if (value instanceof Short) {
+            //MS SQL returns TINYINT as a Java Short, which Avro doesn't understand.
+            return ((Short) value).intValue();
+        } else if (value instanceof BigDecimal) {
+            // Avro can't handle BigDecimal as a number - it will throw an AvroRuntimeException such as: "Unknown datum type: java.math.BigDecimal: 38"
+            return value.toString();
+        } else if (value instanceof BigInteger) {
+            // Check the precision of the BIGINT. Some databases allow arbitrary precision (> 19), but Avro won't handle that.
+            // It the SQL type is BIGINT and the precision is between 0 and 19 (inclusive); if so, the BigInteger is likely a
+            // long (and the schema says it will be), so try to get its value as a long.
+            // Otherwise, Avro can't handle BigInteger as a number - it will throw an AvroRuntimeException
+            // such as: "Unknown datum type: java.math.BigInteger: 38". In this case the schema is expecting a string.
+            final BigInteger bigInt = (BigInteger) value;
+            if (bigInt.compareTo(BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
+                return value.toString();
+            } else {
+                return bigInt.longValue();
+            }
+        } else if (value instanceof Boolean) {
+            return value;
+        } else if (value instanceof Map) {
+            // TODO: Revisit how we handle a lot of these cases....
+            switch (schema.getType()) {
+                case MAP:
+                    return value;
+                case RECORD:
+                    final GenericData.Record avroRecord = new GenericData.Record(schema);
+
+                    final Record record = (Record) value;
+                    for (final String recordFieldName : record.getSchema().getFieldNames()) {
+                        final Object recordFieldValue = record.getValue(recordFieldName);
+
+                        final Field field = schema.getField(recordFieldName);
+                        if (field == null) {
+                            continue;
+                        }
+
+                        final Object converted = convert(recordFieldValue, field.schema(), recordFieldName);
+                        avroRecord.put(recordFieldName, converted);
+                    }
+                    return avroRecord;
+            }
+
+            return value.toString();
+
+        } else if (value instanceof List) {
+            return value;
+        } else if (value instanceof Object[]) {
+            final List<Object> list = new ArrayList<>();
+            for (final Object o : ((Object[]) value)) {
+                final Object converted = convert(o, schema.getElementType(), fieldName);
+                list.add(converted);
+            }
+            return list;
+        } else if (value instanceof Number) {
+            return value;
+        } else if (value instanceof java.util.Date) {
+            final java.util.Date date = (java.util.Date) value;
+            return dateFormat.format(date);
+        } else if (value instanceof java.sql.Date) {
+            final java.sql.Date sqlDate = (java.sql.Date) value;
+            final java.util.Date date = new java.util.Date(sqlDate.getTime());
+            return dateFormat.format(date);
+        } else if (value instanceof Time) {
+            final Time time = (Time) value;
+            final java.util.Date date = new java.util.Date(time.getTime());
+            return timeFormat.format(date);
+        } else if (value instanceof Timestamp) {
+            final Timestamp time = (Timestamp) value;
+            final java.util.Date date = new java.util.Date(time.getTime());
+            return timestampFormat.format(date);
+        }
+
+        // The different types that we support are numbers (int, long, double, float),
+        // as well as boolean values and Strings. Since Avro doesn't provide
+        // timestamp types, we want to convert those to Strings. So we will cast anything other
+        // than numbers or booleans to strings by using the toString() method.
+        return value.toString();
+    }
+
+
+    @Override
+    public String getMimeType() {
+        return "application/avro-binary";
+    }
+
+
+    public static String normalizeNameForAvro(String inputName) {
+        String normalizedName = inputName.replaceAll("[^A-Za-z0-9_]", "_");
+        if (Character.isDigit(normalizedName.charAt(0))) {
+            normalizedName = "_" + normalizedName;
+        }
+        return normalizedName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..eccad7d
--- /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/CSVReader.java
@@ -0,0 +1,49 @@
+/*
+ * 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.IOException;
+import java.io.InputStream;
+
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RowRecordReaderFactory;
+import org.apache.nifi.serialization.UserTypeOverrideRowReader;
+
+@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. By default, the reader will assume that all columns are of 'String' type, but this can be "
+    + "overridden by adding a user-defined Property where the key is the name of a column and the value is the "
+    + "type of the column. For example, if a Property has the name \"balance\" with a value of float, it the "
+    + "reader will attempt to coerce all values in the \"balance\" column into a floating-point number. See "
+    + "Controller Service's Usage for further documentation.")
+@DynamicProperty(name = "<name of column in CSV>", value = "<type of column values in CSV>",
+    description = "User-defined properties are used to indicate that the values of a specific column should be interpreted as a "
+    + "user-defined data type (e.g., int, double, float, date, etc.)", supportsExpressionLanguage = false)
+public class CSVReader extends UserTypeOverrideRowReader implements RowRecordReaderFactory {
+
+    @Override
+    public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException {
+        return new CSVRecordReader(in, logger, getFieldTypeOverrides());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..c2e8963
--- /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/CSVRecordReader.java
@@ -0,0 +1,216 @@
+/*
+ * 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.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+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.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 au.com.bytecode.opencsv.CSVReader;
+
+public class CSVRecordReader implements RecordReader {
+    private final ComponentLog logger;
+    private final CSVReader reader;
+    private final String[] firstLine;
+    private final Map<String, DataType> fieldTypeOverrides;
+    private RecordSchema schema;
+
+    public CSVRecordReader(final InputStream in, final ComponentLog logger, final Map<String, DataType> fieldTypeOverrides) throws IOException {
+        this.logger = logger;
+        reader = new CSVReader(new InputStreamReader(new BufferedInputStream(in)));
+        firstLine = reader.readNext();
+        this.fieldTypeOverrides = fieldTypeOverrides;
+    }
+
+    @Override
+    public Record nextRecord() throws IOException, MalformedRecordException {
+        final RecordSchema schema = getSchema();
+
+        while (true) {
+            final String[] line = reader.readNext();
+            if (line == null) {
+                return null;
+            }
+
+            final List<DataType> fieldTypes = schema.getDataTypes();
+            if (fieldTypes.size() != line.length) {
+                logger.warn("Found record with incorrect number of fields. Expected {} but found {}; skipping record", new Object[] {fieldTypes.size(), line.length});
+                continue;
+            }
+
+            try {
+                final Map<String, Object> rowValues = new HashMap<>(schema.getFieldCount());
+
+                int i = 0;
+                for (final String fieldName : schema.getFieldNames()) {
+                    if (i >= line.length) {
+                        rowValues.put(fieldName, null);
+                        continue;
+                    }
+
+                    final String rawValue = line[i++].trim();
+                    final Object converted = convert(schema.getDataType(fieldName).orElse(null), rawValue);
+                    rowValues.put(fieldName, converted);
+                }
+
+                return new MapRecord(schema, rowValues);
+            } catch (final Exception e) {
+                throw new MalformedRecordException("Found invalid CSV record", e);
+            }
+        }
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        if (schema != null) {
+            return schema;
+        }
+
+        final List<RecordField> recordFields = new ArrayList<>();
+        for (final String element : firstLine) {
+
+            final String name = element.trim();
+            final DataType dataType;
+
+            final DataType overriddenDataType = fieldTypeOverrides.get(name);
+            if (overriddenDataType != null) {
+                dataType = overriddenDataType;
+            } else {
+                dataType = RecordFieldType.STRING.getDataType();
+            }
+
+            final RecordField field = new RecordField(name, dataType);
+            recordFields.add(field);
+        }
+
+        if (recordFields.isEmpty()) {
+            recordFields.add(new RecordField("line", RecordFieldType.STRING.getDataType()));
+        }
+
+        schema = new SimpleRecordSchema(recordFields);
+        return schema;
+    }
+
+    protected Object convert(final DataType dataType, final String value) {
+        if (dataType == null) {
+            return value;
+        }
+
+        switch (dataType.getFieldType()) {
+            case BOOLEAN:
+                if (value.length() == 0) {
+                    return null;
+                }
+                return Boolean.parseBoolean(value);
+            case BYTE:
+                if (value.length() == 0) {
+                    return null;
+                }
+                return Byte.parseByte(value);
+            case SHORT:
+                if (value.length() == 0) {
+                    return null;
+                }
+                return Short.parseShort(value);
+            case INT:
+                if (value.length() == 0) {
+                    return null;
+                }
+                return Integer.parseInt(value);
+            case LONG:
+            case BIGINT:
+                if (value.length() == 0) {
+                    return null;
+                }
+                return Long.parseLong(value);
+            case FLOAT:
+                if (value.length() == 0) {
+                    return null;
+                }
+                return Float.parseFloat(value);
+            case DOUBLE:
+                if (value.length() == 0) {
+                    return null;
+                }
+                return Double.parseDouble(value);
+            case DATE:
+                if (value.length() == 0) {
+                    return null;
+                }
+                try {
+                    final Date date = new SimpleDateFormat(dataType.getFormat()).parse(value);
+                    return new java.sql.Date(date.getTime());
+                } catch (final ParseException e) {
+                    logger.warn("Found invalid value for DATE field: " + value + " does not match expected format of "
+                        + dataType.getFormat() + "; will substitute a NULL value for this field");
+                    return null;
+                }
+            case TIME:
+                if (value.length() == 0) {
+                    return null;
+                }
+                try {
+                    final Date date = new SimpleDateFormat(dataType.getFormat()).parse(value);
+                    return new java.sql.Time(date.getTime());
+                } catch (final ParseException e) {
+                    logger.warn("Found invalid value for TIME field: " + value + " does not match expected format of "
+                        + dataType.getFormat() + "; will substitute a NULL value for this field");
+                    return null;
+                }
+            case TIMESTAMP:
+                if (value.length() == 0) {
+                    return null;
+                }
+                try {
+                    final Date date = new SimpleDateFormat(dataType.getFormat()).parse(value);
+                    return new java.sql.Timestamp(date.getTime());
+                } catch (final ParseException e) {
+                    logger.warn("Found invalid value for TIMESTAMP field: " + value + " does not match expected format of "
+                        + dataType.getFormat() + "; will substitute a NULL value for this field");
+                    return null;
+                }
+            case STRING:
+            default:
+                return value;
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        reader.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..906e9c4
--- /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/CSVRecordSetWriter.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.csv;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.AbstractRecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+
+@Tags({"csv", "result", "set", "writer", "serializer", "record", "row"})
+@CapabilityDescription("Writes the contents of a Database ResultSet as CSV data. The first line written "
+    + "will be the column names. All subsequent lines will be the values corresponding to those columns.")
+public class CSVRecordSetWriter extends AbstractRecordSetWriter implements RecordSetWriterFactory {
+
+    @Override
+    public RecordSetWriter createWriter(final ComponentLog logger) {
+        return new WriteCSVResult(getDateFormat(), getTimeFormat(), getTimestampFormat());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..79c602d
--- /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/WriteCSVResult.java
@@ -0,0 +1,127 @@
+/*
+ * 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.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Collections;
+import java.util.Optional;
+
+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.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.stream.io.NonCloseableOutputStream;
+
+import au.com.bytecode.opencsv.CSVWriter;
+
+public class WriteCSVResult implements RecordSetWriter {
+    private final String dateFormat;
+    private final String timeFormat;
+    private final String timestampFormat;
+
+    public WriteCSVResult(final String dateFormat, final String timeFormat, final String timestampFormat) {
+        this.dateFormat = dateFormat;
+        this.timeFormat = timeFormat;
+        this.timestampFormat = timestampFormat;
+    }
+
+    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();
+        switch (dataType.getFieldType()) {
+            case DATE:
+                return dateFormat == null ? dataType.getFormat() : dateFormat;
+            case TIME:
+                return timeFormat == null ? dataType.getFormat() : timeFormat;
+            case TIMESTAMP:
+                return timestampFormat == null ? dataType.getFormat() : timestampFormat;
+        }
+
+        return dataType.getFormat();
+    }
+
+    @Override
+    public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws IOException {
+        int count = 0;
+        try (final OutputStream nonCloseable = new NonCloseableOutputStream(rawOut);
+            final OutputStreamWriter streamWriter = new OutputStreamWriter(nonCloseable);
+            final CSVWriter writer = new CSVWriter(streamWriter)) {
+
+            try {
+                final RecordSchema schema = rs.getSchema();
+                final String[] columnNames = schema.getFieldNames().toArray(new String[0]);
+                writer.writeNext(columnNames);
+
+                Record record;
+                while ((record = rs.next()) != null) {
+                    final String[] colVals = new String[schema.getFieldCount()];
+                    int i = 0;
+                    for (final String fieldName : schema.getFieldNames()) {
+                        colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName));
+                    }
+
+                    writer.writeNext(colVals);
+                    count++;
+                }
+            } catch (final Exception e) {
+                throw new IOException("Failed to serialize results", e);
+            }
+        }
+
+        return WriteResult.of(count, Collections.emptyMap());
+    }
+
+    @Override
+    public WriteResult write(final Record record, final OutputStream rawOut) throws IOException {
+        try (final OutputStream nonCloseable = new NonCloseableOutputStream(rawOut);
+            final OutputStreamWriter streamWriter = new OutputStreamWriter(nonCloseable);
+            final CSVWriter writer = new CSVWriter(streamWriter)) {
+
+            try {
+                final RecordSchema schema = record.getSchema();
+                final String[] columnNames = schema.getFieldNames().toArray(new String[0]);
+                writer.writeNext(columnNames);
+
+                final String[] colVals = new String[schema.getFieldCount()];
+                int i = 0;
+                for (final String fieldName : schema.getFieldNames()) {
+                    colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName));
+                }
+
+                writer.writeNext(colVals);
+            } catch (final Exception e) {
+                throw new IOException("Failed to serialize results", e);
+            }
+        }
+
+        return WriteResult.of(1, Collections.emptyMap());
+    }
+
+    @Override
+    public String getMimeType() {
+        return "text/csv";
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokExpressionValidator.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/GrokExpressionValidator.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokExpressionValidator.java
new file mode 100644
index 0000000..dd9c4e0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokExpressionValidator.java
@@ -0,0 +1,48 @@
+/*
+ * 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.grok;
+
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+
+import io.thekraken.grok.api.Grok;
+
+public class GrokExpressionValidator implements Validator {
+
+    @Override
+    public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+        try {
+            new Grok().compile(input);
+        } catch (final Exception e) {
+            return new ValidationResult.Builder()
+                .input(input)
+                .subject(subject)
+                .valid(false)
+                .explanation("Invalid Grok pattern: " + e.getMessage())
+                .build();
+        }
+
+        return new ValidationResult.Builder()
+            .input(input)
+            .subject(subject)
+            .valid(true)
+            .build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..f72d5d5
--- /dev/null
+++ 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
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.grok;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RowRecordReaderFactory;
+import org.apache.nifi.serialization.UserTypeOverrideRowReader;
+
+import io.thekraken.grok.api.Grok;
+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.")
+public class GrokReader extends UserTypeOverrideRowReader implements RowRecordReaderFactory {
+    private volatile Grok grok;
+
+    private static final String DEFAULT_PATTERN_NAME = "/default-grok-patterns.txt";
+
+    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 "
+            + "will be used. If specified, all patterns in the given pattern file will override the default patterns. See the Controller Service's "
+            + "Additional Details for a list of pre-defined patterns.")
+        .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .required(false)
+        .build();
+    static final PropertyDescriptor GROK_EXPRESSION = new PropertyDescriptor.Builder()
+        .name("Grok Expression")
+        .description("Specifies the format of a log line in Grok format. This allows the Record Reader to understand how to parse each log line. "
+            + "If a line in the log file does not match this pattern, the line will be assumed to belong to the previous log message.")
+        .addValidator(new GrokExpressionValidator())
+        .required(true)
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(PATTERN_FILE);
+        properties.add(GROK_EXPRESSION);
+        return properties;
+    }
+
+    @OnEnabled
+    public void preCompile(final ConfigurationContext context) throws GrokException, IOException {
+        grok = new Grok();
+
+        try (final InputStream in = getClass().getResourceAsStream(DEFAULT_PATTERN_NAME);
+            final Reader reader = new InputStreamReader(in)) {
+            grok.addPatternFromReader(reader);
+        }
+
+        if (context.getProperty(PATTERN_FILE).isSet()) {
+            grok.addPatternFromFile(context.getProperty(PATTERN_FILE).getValue());
+        }
+
+        grok.compile(context.getProperty(GROK_EXPRESSION).getValue());
+    }
+
+    @Override
+    public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException {
+        return new GrokRecordReader(in, grok, getFieldTypeOverrides());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..bdf12f9
--- /dev/null
+++ 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
@@ -0,0 +1,323 @@
+/*
+ * 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.grok;
+
+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 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 Map<String, DataType> fieldTypeOverrides;
+
+    private String nextLine;
+    private RecordSchema schema;
+
+    static final String STACK_TRACE_COLUMN_NAME = "STACK_TRACE";
+    private static final Pattern STACK_TRACE_PATTERN = Pattern.compile(
+        "^\\s*(?:(?:    |\\t)+at )|"
+            + "(?:(?:    |\\t)+\\[CIRCULAR REFERENCE\\:)|"
+            + "(?:Caused by\\: )|"
+            + "(?: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 Map<String, DataType> fieldTypeOverrides) {
+        this.reader = new BufferedReader(new InputStreamReader(in));
+        this.grok = grok;
+        this.fieldTypeOverrides = fieldTypeOverrides;
+    }
+
+    @Override
+    public void close() throws IOException {
+        reader.close();
+    }
+
+    @Override
+    public Record nextRecord() throws IOException, MalformedRecordException {
+        final String line = nextLine == null ? reader.readLine() : nextLine;
+        nextLine = null; // ensure that we don't process nextLine again
+        if (line == null) {
+            return null;
+        }
+
+        final RecordSchema schema = getSchema();
+
+        final Match match = grok.match(line);
+        match.captures();
+        final Map<String, Object> valueMap = match.toMap();
+        if (valueMap.isEmpty()) {   // We were unable to match the pattern so return an empty Object array.
+            return new MapRecord(schema, Collections.emptyMap());
+        }
+
+        // Read the next line to see if it matches the pattern (in which case we will simply leave it for
+        // the next call to nextRecord()) or we will attach it to the previously read record.
+        String stackTrace = null;
+        final StringBuilder toAppend = new StringBuilder();
+        while ((nextLine = reader.readLine()) != null) {
+            final Match nextLineMatch = grok.match(nextLine);
+            nextLineMatch.captures();
+            final Map<String, Object> nextValueMap = nextLineMatch.toMap();
+            if (nextValueMap.isEmpty()) {
+                // next line did not match. Check if it indicates a Stack Trace. If so, read until
+                // the stack trace ends. Otherwise, append the next line to the last field in the record.
+                if (isStartOfStackTrace(nextLine)) {
+                    stackTrace = readStackTrace(nextLine);
+                    break;
+                } else {
+                    toAppend.append("\n").append(nextLine);
+                }
+            } else {
+                // The next line matched our pattern.
+                break;
+            }
+        }
+
+        try {
+            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);
+                if (value == null) {
+                    values.put(fieldName, null);
+                    continue;
+                }
+
+                final DataType fieldType = schema.getDataType(fieldName).orElse(null);
+                final Object converted = convert(fieldType, value.toString());
+                values.put(fieldName, converted);
+            }
+
+            final String lastFieldBeforeStackTrace = schema.getFieldNames().get(schema.getFieldCount() - 2);
+            if (toAppend.length() > 0) {
+                final Object existingValue = values.get(lastFieldBeforeStackTrace);
+                final String updatedValue = existingValue == null ? toAppend.toString() : existingValue + toAppend.toString();
+                values.put(lastFieldBeforeStackTrace, updatedValue);
+            }
+
+            values.put(STACK_TRACE_COLUMN_NAME, stackTrace);
+
+            return new MapRecord(schema, values);
+        } catch (final Exception e) {
+            throw new MalformedRecordException("Found invalid log record and will skip it. Record: " + line, e);
+        }
+    }
+
+
+    private boolean isStartOfStackTrace(final String line) {
+        if (line == null) {
+            return false;
+        }
+
+        // Stack Traces are generally of the form:
+        // java.lang.IllegalArgumentException: My message
+        //   at org.apache.nifi.MyClass.myMethod(MyClass.java:48)
+        //   at java.lang.Thread.run(Thread.java:745) [na:1.8.0_60]
+        // Caused by: java.net.SocketTimeoutException: null
+        //   ... 13 common frames omitted
+
+        int index = line.indexOf("Exception: ");
+        if (index < 0) {
+            index = line.indexOf("Error: ");
+        }
+
+        if (index < 0) {
+            return false;
+        }
+
+        if (line.indexOf(" ") < index) {
+            return false;
+        }
+
+        return true;
+    }
+
+    private String readStackTrace(final String firstLine) throws IOException {
+        final StringBuilder sb = new StringBuilder(firstLine);
+
+        String line;
+        while ((line = reader.readLine()) != null) {
+            if (isLineInStackTrace(line)) {
+                sb.append("\n").append(line);
+            } else {
+                nextLine = line;
+                break;
+            }
+        }
+
+        return sb.toString();
+    }
+
+    private boolean isLineInStackTrace(final String line) {
+        return STACK_TRACE_PATTERN.matcher(line).find();
+    }
+
+
+    protected Object convert(final DataType fieldType, final String string) {
+        if (fieldType == null) {
+            return string;
+        }
+        switch (fieldType.getFieldType()) {
+            case BOOLEAN:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Boolean.parseBoolean(string);
+            case BYTE:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Byte.parseByte(string);
+            case SHORT:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Short.parseShort(string);
+            case INT:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Integer.parseInt(string);
+            case LONG:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Long.parseLong(string);
+            case FLOAT:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Float.parseFloat(string);
+            case DOUBLE:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Double.parseDouble(string);
+            case DATE:
+                if (string.length() == 0) {
+                    return null;
+                }
+                try {
+                    Date date = TIME_FORMAT_DATE.parse(string);
+                    return new java.sql.Date(date.getTime());
+                } catch (ParseException e) {
+                    return null;
+                }
+            case TIME:
+                if (string.length() == 0) {
+                    return null;
+                }
+                try {
+                    Date date = TIME_FORMAT_TIME.parse(string);
+                    return new java.sql.Time(date.getTime());
+                } catch (ParseException e) {
+                    return null;
+                }
+            case TIMESTAMP:
+                if (string.length() == 0) {
+                    return null;
+                }
+                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;
+        }
+    }
+
+
+    @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 = fieldTypeOverrides.get(fieldName);
+                if (dataType == null) {
+                    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;
+    }
+
+}


[11/19] nifi git commit: NIFI-1280 Create FilterCSVColumns Processor.

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500.csv
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500.csv b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500.csv
new file mode 100644
index 0000000..61ce4bd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500.csv
@@ -0,0 +1 @@
+FIRST_NAME:string,LAST_NAME,COMPANY_NAME,ADDRESS,CITY,COUNTY,STATE,zip,phone1,phone2,email,web
"James","Butt","Benton, John B Jr","6649 N Blue Gum St","New Orleans","Orleans","LA",70116,"504-621-8927","504-845-1427","jbutt@gmail.com","http://www.bentonjohnbjr.com"
"Josephine","Darakjy","Chanay, Jeffrey A Esq","4 B Blue Ridge Blvd","Brighton","Livingston","MI",48116,"810-292-9388","810-374-9840","josephine_darakjy@darakjy.org","http://www.chanayjeffreyaesq.com"
"Art","Venere","Chemel, James L Cpa","8 W Cerritos Ave #54","Bridgeport","Gloucester","NJ","08014","856-636-8749","856-264-4130","art@venere.org","http://www.chemeljameslcpa.com"
"Lenna","Paprocki","Feltz Printing Service","639 Main St","Anchorage","Anchorage","AK",99501,"907-385-4412","907-921-2010","lpaprocki@hotmail.com","http://www.feltzprintingservice.com"
"Donette","Foller","Printing Dimensions","34 Center St","Hamilton","Butler","OH",45011,"513-570-1893","513-549-4561","donette.foller@cox.net","http://www.printingdimens
 ions.com"
"Simona","Morasca","Chapman, Ross E Esq","3 Mcauley Dr","Ashland","Ashland","OH",44805,"419-503-2484","419-800-6759","simona@morasca.com","http://www.chapmanrosseesq.com"
"Mitsue","Tollner","Morlong Associates","7 Eads St","Chicago","Cook","IL",60632,"773-573-6914","773-924-8565","mitsue_tollner@yahoo.com","http://www.morlongassociates.com"
"Leota","Dilliard","Commercial Press","7 W Jackson Blvd","San Jose","Santa Clara","CA",95111,"408-752-3500","408-813-1105","leota@hotmail.com","http://www.commercialpress.com"
"Sage","Wieser","Truhlar And Truhlar Attys","5 Boston Ave #88","Sioux Falls","Minnehaha","SD",57105,"605-414-2147","605-794-4895","sage_wieser@cox.net","http://www.truhlarandtruhlarattys.com"
"Kris","Marrier","King, Christopher A Esq","228 Runamuck Pl #2808","Baltimore","Baltimore City","MD",21224,"410-655-8723","410-804-4694","kris@gmail.com","http://www.kingchristopheraesq.com"
"Minna","Amigon","Dorl, James J Esq","2371 Jerrold Ave","Kulpsville","Montgomery","PA
 ",19443,"215-874-1229","215-422-8694","minna_amigon@yahoo.com","http://www.dorljamesjesq.com"
"Abel","Maclead","Rangoni Of Florence","37275 St  Rt 17m M","Middle Island","Suffolk","NY",11953,"631-335-3414","631-677-3675","amaclead@gmail.com","http://www.rangoniofflorence.com"
"Kiley","Caldarera","Feiner Bros","25 E 75th St #69","Los Angeles","Los Angeles","CA",90034,"310-498-5651","310-254-3084","kiley.caldarera@aol.com","http://www.feinerbros.com"
"Graciela","Ruta","Buckley Miller & Wright","98 Connecticut Ave Nw","Chagrin Falls","Geauga","OH",44023,"440-780-8425","440-579-7763","gruta@cox.net","http://www.buckleymillerwright.com"
"Cammy","Albares","Rousseaux, Michael Esq","56 E Morehead St","Laredo","Webb","TX",78045,"956-537-6195","956-841-7216","calbares@gmail.com","http://www.rousseauxmichaelesq.com"
"Mattie","Poquette","Century Communications","73 State Road 434 E","Phoenix","Maricopa","AZ",85013,"602-277-4385","602-953-6360","mattie@aol.com","http://www.centurycommunications.
 com"
"Meaghan","Garufi","Bolton, Wilbur Esq","69734 E Carrillo St","Mc Minnville","Warren","TN",37110,"931-313-9635","931-235-7959","meaghan@hotmail.com","http://www.boltonwilburesq.com"
"Gladys","Rim","T M Byxbee Company Pc","322 New Horizon Blvd","Milwaukee","Milwaukee","WI",53207,"414-661-9598","414-377-2880","gladys.rim@rim.org","http://www.tmbyxbeecompanypc.com"
"Yuki","Whobrey","Farmers Insurance Group","1 State Route 27","Taylor","Wayne","MI",48180,"313-288-7937","313-341-4470","yuki_whobrey@aol.com","http://www.farmersinsurancegroup.com"
"Fletcher","Flosi","Post Box Services Plus","394 Manchester Blvd","Rockford","Winnebago","IL",61109,"815-828-2147","815-426-5657","fletcher.flosi@yahoo.com","http://www.postboxservicesplus.com"
"Bette","Nicka","Sport En Art","6 S 33rd St","Aston","Delaware","PA",19014,"610-545-3615","610-492-4643","bette_nicka@cox.net","http://www.sportenart.com"
"Veronika","Inouye","C 4 Network Inc","6 Greenleaf Ave","San Jose","Santa Clara","CA",95111,"408
 -540-1785","408-813-4592","vinouye@aol.com","http://www.cnetworkinc.com"
"Willard","Kolmetz","Ingalls, Donald R Esq","618 W Yakima Ave","Irving","Dallas","TX",75062,"972-303-9197","972-896-4882","willard@hotmail.com","http://www.ingallsdonaldresq.com"
"Maryann","Royster","Franklin, Peter L Esq","74 S Westgate St","Albany","Albany","NY",12204,"518-966-7987","518-448-8982","mroyster@royster.com","http://www.franklinpeterlesq.com"
"Alisha","Slusarski","Wtlz Power 107 Fm","3273 State St","Middlesex","Middlesex","NJ","08846","732-658-3154","732-635-3453","alisha@slusarski.com","http://www.wtlzpowerfm.com"
"Allene","Iturbide","Ledecky, David Esq","1 Central Ave","Stevens Point","Portage","WI",54481,"715-662-6764","715-530-9863","allene_iturbide@cox.net","http://www.ledeckydavidesq.com"
"Chanel","Caudy","Professional Image Inc","86 Nw 66th St #8673","Shawnee","Johnson","KS",66218,"913-388-2079","913-899-1103","chanel.caudy@caudy.org","http://www.professionalimageinc.com"
"Ezekiel","Chui","
 Sider, Donald C Esq","2 Cedar Ave #84","Easton","Talbot","MD",21601,"410-669-1642","410-235-8738","ezekiel@chui.com","http://www.siderdonaldcesq.com"
"Willow","Kusko","U Pull It","90991 Thorburn Ave","New York","New York","NY",10011,"212-582-4976","212-934-5167","wkusko@yahoo.com","http://www.upullit.com"
"Bernardo","Figeroa","Clark, Richard Cpa","386 9th Ave N","Conroe","Montgomery","TX",77301,"936-336-3951","936-597-3614","bfigeroa@aol.com","http://www.clarkrichardcpa.com"
"Ammie","Corrio","Moskowitz, Barry S","74874 Atlantic Ave","Columbus","Franklin","OH",43215,"614-801-9788","614-648-3265","ammie@corrio.com","http://www.moskowitzbarrys.com"
"Francine","Vocelka","Cascade Realty Advisors Inc","366 South Dr","Las Cruces","Dona Ana","NM",88011,"505-977-3911","505-335-5293","francine_vocelka@vocelka.com","http://www.cascaderealtyadvisorsinc.com"
"Ernie","Stenseth","Knwz Newsradio","45 E Liberty St","Ridgefield Park","Bergen","NJ","07660","201-709-6245","201-387-9093","ernie_stenseth
 @aol.com","http://www.knwznewsradio.com"
"Albina","Glick","Giampetro, Anthony D","4 Ralph Ct","Dunellen","Middlesex","NJ","08812","732-924-7882","732-782-6701","albina@glick.com","http://www.giampetroanthonyd.com"
"Alishia","Sergi","Milford Enterprises Inc","2742 Distribution Way","New York","New York","NY",10025,"212-860-1579","212-753-2740","asergi@gmail.com","http://www.milfordenterprisesinc.com"
"Solange","Shinko","Mosocco, Ronald A","426 Wolf St","Metairie","Jefferson","LA",70002,"504-979-9175","504-265-8174","solange@shinko.com","http://www.mosoccoronalda.com"
"Jose","Stockham","Tri State Refueler Co","128 Bransten Rd","New York","New York","NY",10011,"212-675-8570","212-569-4233","jose@yahoo.com","http://www.tristaterefuelerco.com"
"Rozella","Ostrosky","Parkway Company","17 Morena Blvd","Camarillo","Ventura","CA",93012,"805-832-6163","805-609-1531","rozella.ostrosky@ostrosky.com","http://www.parkwaycompany.com"
"Valentine","Gillian","Fbs Business Finance","775 W 17th St","San
  Antonio","Bexar","TX",78204,"210-812-9597","210-300-6244","valentine_gillian@gmail.com","http://www.fbsbusinessfinance.com"
"Kati","Rulapaugh","Eder Assocs Consltng Engrs Pc","6980 Dorsett Rd","Abilene","Dickinson","KS",67410,"785-463-7829","785-219-7724","kati.rulapaugh@hotmail.com","http://www.ederassocsconsltngengrspc.com"
"Youlanda","Schemmer","Tri M Tool Inc","2881 Lewis Rd","Prineville","Crook","OR",97754,"541-548-8197","541-993-2611","youlanda@aol.com","http://www.trimtoolinc.com"
"Dyan","Oldroyd","International Eyelets Inc","7219 Woodfield Rd","Overland Park","Johnson","KS",66204,"913-413-4604","913-645-8918","doldroyd@aol.com","http://www.internationaleyeletsinc.com"
"Roxane","Campain","Rapid Trading Intl","1048 Main St","Fairbanks","Fairbanks North Star","AK",99708,"907-231-4722","907-335-6568","roxane@hotmail.com","http://www.rapidtradingintl.com"
"Lavera","Perin","Abc Enterprises Inc","678 3rd Ave","Miami","Miami-Dade","FL",33196,"305-606-7291","305-995-2078","lperin@pe
 rin.org","http://www.abcenterprisesinc.com"
"Erick","Ferencz","Cindy Turner Associates","20 S Babcock St","Fairbanks","Fairbanks North Star","AK",99712,"907-741-1044","907-227-6777","erick.ferencz@aol.com","http://www.cindyturnerassociates.com"
"Fatima","Saylors","Stanton, James D Esq","2 Lighthouse Ave","Hopkins","Hennepin","MN",55343,"952-768-2416","952-479-2375","fsaylors@saylors.org","http://www.stantonjamesdesq.com"
"Jina","Briddick","Grace Pastries Inc","38938 Park Blvd","Boston","Suffolk","MA","02128","617-399-5124","617-997-5771","jina_briddick@briddick.com","http://www.gracepastriesinc.com"
"Kanisha","Waycott","Schroer, Gene E Esq","5 Tomahawk Dr","Los Angeles","Los Angeles","CA",90006,"323-453-2780","323-315-7314","kanisha_waycott@yahoo.com","http://www.schroergeneeesq.com"
"Emerson","Bowley","Knights Inn","762 S Main St","Madison","Dane","WI",53711,"608-336-7444","608-658-7940","emerson.bowley@bowley.org","http://www.knightsinn.com"
"Blair","Malet","Bollinger Mach Shp & S
 hipyard","209 Decker Dr","Philadelphia","Philadelphia","PA",19132,"215-907-9111","215-794-4519","bmalet@yahoo.com","http://www.bollingermachshpshipyard.com"
"Brock","Bolognia","Orinda News","4486 W O St #1","New York","New York","NY",10003,"212-402-9216","212-617-5063","bbolognia@yahoo.com","http://www.orindanews.com"
"Lorrie","Nestle","Ballard Spahr Andrews","39 S 7th St","Tullahoma","Coffee","TN",37388,"931-875-6644","931-303-6041","lnestle@hotmail.com","http://www.ballardspahrandrews.com"
"Sabra","Uyetake","Lowy Limousine Service","98839 Hawthorne Blvd #6101","Columbia","Richland","SC",29201,"803-925-5213","803-681-3678","sabra@uyetake.org","http://www.lowylimousineservice.com"
"Marjory","Mastella","Vicon Corporation","71 San Mateo Ave","Wayne","Delaware","PA",19087,"610-814-5533","610-379-7125","mmastella@mastella.com","http://www.viconcorporation.com"
"Karl","Klonowski","Rossi, Michael M","76 Brooks St #9","Flemington","Hunterdon","NJ","08822","908-877-6135","908-470-4661","kar
 l_klonowski@yahoo.com","http://www.rossimichaelm.com"
"Tonette","Wenner","Northwest Publishing","4545 Courthouse Rd","Westbury","Nassau","NY",11590,"516-968-6051","516-333-4861","twenner@aol.com","http://www.northwestpublishing.com"
"Amber","Monarrez","Branford Wire & Mfg Co","14288 Foster Ave #4121","Jenkintown","Montgomery","PA",19046,"215-934-8655","215-329-6386","amber_monarrez@monarrez.org","http://www.branfordwiremfgco.com"
"Shenika","Seewald","East Coast Marketing","4 Otis St","Van Nuys","Los Angeles","CA",91405,"818-423-4007","818-749-8650","shenika@gmail.com","http://www.eastcoastmarketing.com"
"Delmy","Ahle","Wye Technologies Inc","65895 S 16th St","Providence","Providence","RI","02909","401-458-2547","401-559-8961","delmy.ahle@hotmail.com","http://www.wyetechnologiesinc.com"
"Deeanna","Juhas","Healy, George W Iv","14302 Pennsylvania Ave","Huntingdon Valley","Montgomery","PA",19006,"215-211-9589","215-417-9563","deeanna_juhas@gmail.com","http://www.healygeorgewiv.com"
"Blo
 ndell","Pugh","Alpenlite Inc","201 Hawk Ct","Providence","Providence","RI","02904","401-960-8259","401-300-8122","bpugh@aol.com","http://www.alpenliteinc.com"
"Jamal","Vanausdal","Hubbard, Bruce Esq","53075 Sw 152nd Ter #615","Monroe Township","Middlesex","NJ","08831","732-234-1546","732-904-2931","jamal@vanausdal.org","http://www.hubbardbruceesq.com"
"Cecily","Hollack","Arthur A Oliver & Son Inc","59 N Groesbeck Hwy","Austin","Travis","TX",78731,"512-486-3817","512-861-3814","cecily@hollack.org","http://www.arthuraoliversoninc.com"
"Carmelina","Lindall","George Jessop Carter Jewelers","2664 Lewis Rd","Littleton","Douglas","CO",80126,"303-724-7371","303-874-5160","carmelina_lindall@lindall.com","http://www.georgejessopcarterjewelers.com"
"Maurine","Yglesias","Schultz, Thomas C Md","59 Shady Ln #53","Milwaukee","Milwaukee","WI",53214,"414-748-1374","414-573-7719","maurine_yglesias@yglesias.com","http://www.schultzthomascmd.com"
"Tawna","Buvens","H H H Enterprises Inc","3305 Nabell Av
 e #679","New York","New York","NY",10009,"212-674-9610","212-462-9157","tawna@gmail.com","http://www.hhhenterprisesinc.com"
"Penney","Weight","Hawaiian King Hotel","18 Fountain St","Anchorage","Anchorage","AK",99515,"907-797-9628","907-873-2882","penney_weight@aol.com","http://www.hawaiiankinghotel.com"
"Elly","Morocco","Killion Industries","7 W 32nd St","Erie","Erie","PA",16502,"814-393-5571","814-420-3553","elly_morocco@gmail.com","http://www.killionindustries.com"
"Ilene","Eroman","Robinson, William J Esq","2853 S Central Expy","Glen Burnie","Anne Arundel","MD",21061,"410-914-9018","410-937-4543","ilene.eroman@hotmail.com","http://www.robinsonwilliamjesq.com"
"Vallie","Mondella","Private Properties","74 W College St","Boise","Ada","ID",83707,"208-862-5339","208-737-8439","vmondella@mondella.com","http://www.privateproperties.com"
"Kallie","Blackwood","Rowley Schlimgen Inc","701 S Harrison Rd","San Francisco","San Francisco","CA",94104,"415-315-2761","415-604-7609","kallie.blackwo
 od@gmail.com","http://www.rowleyschlimgeninc.com"
"Johnetta","Abdallah","Forging Specialties","1088 Pinehurst St","Chapel Hill","Orange","NC",27514,"919-225-9345","919-715-3791","johnetta_abdallah@aol.com","http://www.forgingspecialties.com"
"Bobbye","Rhym","Smits, Patricia Garity","30 W 80th St #1995","San Carlos","San Mateo","CA",94070,"650-528-5783","650-811-9032","brhym@rhym.com","http://www.smitspatriciagarity.com"
"Micaela","Rhymes","H Lee Leonard Attorney At Law","20932 Hedley St","Concord","Contra Costa","CA",94520,"925-647-3298","925-522-7798","micaela_rhymes@gmail.com","http://www.hleeleonardattorneyatlaw.com"
"Tamar","Hoogland","A K Construction Co","2737 Pistorio Rd #9230","London","Madison","OH",43140,"740-343-8575","740-526-5410","tamar@hotmail.com","http://www.akconstructionco.com"
"Moon","Parlato","Ambelang, Jessica M Md","74989 Brandon St","Wellsville","Allegany","NY",14895,"585-866-8313","585-498-4278","moon@yahoo.com","http://www.ambelangjessicammd.com"
"Laurel","
 Reitler","Q A Service","6 Kains Ave","Baltimore","Baltimore City","MD",21215,"410-520-4832","410-957-6903","laurel_reitler@reitler.com","http://www.qaservice.com"
"Delisa","Crupi","Wood & Whitacre Contractors","47565 W Grand Ave","Newark","Essex","NJ","07105","973-354-2040","973-847-9611","delisa.crupi@crupi.com","http://www.woodwhitacrecontractors.com"
"Viva","Toelkes","Mark Iv Press Ltd","4284 Dorigo Ln","Chicago","Cook","IL",60647,"773-446-5569","773-352-3437","viva.toelkes@gmail.com","http://www.markivpressltd.com"
"Elza","Lipke","Museum Of Science & Industry","6794 Lake Dr E","Newark","Essex","NJ","07104","973-927-3447","973-796-3667","elza@yahoo.com","http://www.museumofscienceindustry.com"
"Devorah","Chickering","Garrison Ind","31 Douglas Blvd #950","Clovis","Curry","NM",88101,"505-975-8559","505-950-1763","devorah@hotmail.com","http://www.garrisonind.com"
"Timothy","Mulqueen","Saronix Nymph Products","44 W 4th St","Staten Island","Richmond","NY",10309,"718-332-6527","718-654
 -7063","timothy_mulqueen@mulqueen.org","http://www.saronixnymphproducts.com"
"Arlette","Honeywell","Smc Inc","11279 Loytan St","Jacksonville","Duval","FL",32254,"904-775-4480","904-514-9918","ahoneywell@honeywell.com","http://www.smcinc.com"
"Dominque","Dickerson","E A I Electronic Assocs Inc","69 Marquette Ave","Hayward","Alameda","CA",94545,"510-993-3758","510-901-7640","dominque.dickerson@dickerson.org","http://www.eaielectronicassocsinc.com"
"Lettie","Isenhower","Conte, Christopher A Esq","70 W Main St","Beachwood","Cuyahoga","OH",44122,"216-657-7668","216-733-8494","lettie_isenhower@yahoo.com","http://www.contechristopheraesq.com"
"Myra","Munns","Anker Law Office","461 Prospect Pl #316","Euless","Tarrant","TX",76040,"817-914-7518","817-451-3518","mmunns@cox.net","http://www.ankerlawoffice.com"
"Stephaine","Barfield","Beutelschies & Company","47154 Whipple Ave Nw","Gardena","Los Angeles","CA",90247,"310-774-7643","310-968-1219","stephaine@barfield.com","http://www.beutelschiesco
 mpany.com"
"Lai","Gato","Fligg, Kenneth I Jr","37 Alabama Ave","Evanston","Cook","IL",60201,"847-728-7286","847-957-4614","lai.gato@gato.org","http://www.fliggkennethijr.com"
"Stephen","Emigh","Sharp, J Daniel Esq","3777 E Richmond St #900","Akron","Summit","OH",44302,"330-537-5358","330-700-2312","stephen_emigh@hotmail.com","http://www.sharpjdanielesq.com"
"Tyra","Shields","Assink, Anne H Esq","3 Fort Worth Ave","Philadelphia","Philadelphia","PA",19106,"215-255-1641","215-228-8264","tshields@gmail.com","http://www.assinkannehesq.com"
"Tammara","Wardrip","Jewel My Shop Inc","4800 Black Horse Pike","Burlingame","San Mateo","CA",94010,"650-803-1936","650-216-5075","twardrip@cox.net","http://www.jewelmyshopinc.com"
"Cory","Gibes","Chinese Translation Resources","83649 W Belmont Ave","San Gabriel","Los Angeles","CA",91776,"626-572-1096","626-696-2777","cory.gibes@gmail.com","http://www.chinesetranslationresources.com"
"Danica","Bruschke","Stevens, Charles T","840 15th Ave","Waco","McLen
 nan","TX",76708,"254-782-8569","254-205-1422","danica_bruschke@gmail.com","http://www.stevenscharlest.com"
"Wilda","Giguere","Mclaughlin, Luther W Cpa","1747 Calle Amanecer #2","Anchorage","Anchorage","AK",99501,"907-870-5536","907-914-9482","wilda@cox.net","http://www.mclaughlinlutherwcpa.com"
"Elvera","Benimadho","Tree Musketeers","99385 Charity St #840","San Jose","Santa Clara","CA",95110,"408-703-8505","408-440-8447","elvera.benimadho@cox.net","http://www.treemusketeers.com"
"Carma","Vanheusen","Springfield Div Oh Edison Co","68556 Central Hwy","San Leandro","Alameda","CA",94577,"510-503-7169","510-452-4835","carma@cox.net","http://www.springfielddivohedisonco.com"
"Malinda","Hochard","Logan Memorial Hospital","55 Riverside Ave","Indianapolis","Marion","IN",46202,"317-722-5066","317-472-2412","malinda.hochard@yahoo.com","http://www.loganmemorialhospital.com"
"Natalie","Fern","Kelly, Charles G Esq","7140 University Ave","Rock Springs","Sweetwater","WY",82901,"307-704-8713","307-2
 79-3793","natalie.fern@hotmail.com","http://www.kellycharlesgesq.com"
"Lisha","Centini","Industrial Paper Shredders Inc","64 5th Ave #1153","Mc Lean","Fairfax","VA",22102,"703-235-3937","703-475-7568","lisha@centini.org","http://www.industrialpapershreddersinc.com"
"Arlene","Klusman","Beck Horizon Builders","3 Secor Rd","New Orleans","Orleans","LA",70112,"504-710-5840","504-946-1807","arlene_klusman@gmail.com","http://www.beckhorizonbuilders.com"
"Alease","Buemi","Porto Cayo At Hawks Cay","4 Webbs Chapel Rd","Boulder","Boulder","CO",80303,"303-301-4946","303-521-9860","alease@buemi.com","http://www.portocayoathawkscay.com"
"Louisa","Cronauer","Pacific Grove Museum Ntrl Hist","524 Louisiana Ave Nw","San Leandro","Alameda","CA",94577,"510-828-7047","510-472-7758","louisa@cronauer.com","http://www.pacificgrovemuseumntrlhist.com"
"Angella","Cetta","Bender & Hatley Pc","185 Blackstone Bldge","Honolulu","Honolulu","HI",96817,"808-892-7943","808-475-2310","angella.cetta@hotmail.com","http:
 //www.benderhatleypc.com"
"Cyndy","Goldammer","Di Cristina J & Son","170 Wyoming Ave","Burnsville","Dakota","MN",55337,"952-334-9408","952-938-9457","cgoldammer@cox.net","http://www.dicristinajson.com"
"Rosio","Cork","Green Goddess","4 10th St W","High Point","Guilford","NC",27263,"336-243-5659","336-497-4407","rosio.cork@gmail.com","http://www.greengoddess.com"
"Celeste","Korando","American Arts & Graphics","7 W Pinhook Rd","Lynbrook","Nassau","NY",11563,"516-509-2347","516-365-7266","ckorando@hotmail.com","http://www.americanartsgraphics.com"
"Twana","Felger","Opryland Hotel","1 Commerce Way","Portland","Washington","OR",97224,"503-939-3153","503-909-7167","twana.felger@felger.org","http://www.oprylandhotel.com"
"Estrella","Samu","Marking Devices Pubg Co","64 Lakeview Ave","Beloit","Rock","WI",53511,"608-976-7199","608-942-8836","estrella@aol.com","http://www.markingdevicespubgco.com"
"Donte","Kines","W Tc Industries Inc","3 Aspen St","Worcester","Worcester","MA","01602","508-429-
 8576","508-843-1426","dkines@hotmail.com","http://www.wtcindustriesinc.com"
"Tiffiny","Steffensmeier","Whitehall Robbins Labs Divsn","32860 Sierra Rd","Miami","Miami-Dade","FL",33133,"305-385-9695","305-304-6573","tiffiny_steffensmeier@cox.net","http://www.whitehallrobbinslabsdivsn.com"
"Edna","Miceli","Sampler","555 Main St","Erie","Erie","PA",16502,"814-460-2655","814-299-2877","emiceli@miceli.org","http://www.sampler.com"
"Sue","Kownacki","Juno Chefs Incorporated","2 Se 3rd Ave","Mesquite","Dallas","TX",75149,"972-666-3413","972-742-4000","sue@aol.com","http://www.junochefsincorporated.com"
"Jesusa","Shin","Carroccio, A Thomas Esq","2239 Shawnee Mission Pky","Tullahoma","Coffee","TN",37388,"931-273-8709","931-739-1551","jshin@shin.com","http://www.carroccioathomasesq.com"
"Rolland","Francescon","Stanley, Richard L Esq","2726 Charcot Ave","Paterson","Passaic","NJ","07501","973-649-2922","973-284-4048","rolland@cox.net","http://www.stanleyrichardlesq.com"
"Pamella","Schmierer","K C
 s Cstm Mouldings Windows","5161 Dorsett Rd","Homestead","Miami-Dade","FL",33030,"305-420-8970","305-575-8481","pamella.schmierer@schmierer.org","http://www.kcscstmmouldingswindows.com"
"Glory","Kulzer","Comfort Inn","55892 Jacksonville Rd","Owings Mills","Baltimore","MD",21117,"410-224-9462","410-916-8015","gkulzer@kulzer.org","http://www.comfortinn.com"
"Shawna","Palaspas","Windsor, James L Esq","5 N Cleveland Massillon Rd","Thousand Oaks","Ventura","CA",91362,"805-275-3566","805-638-6617","shawna_palaspas@palaspas.org","http://www.windsorjameslesq.com"
"Brandon","Callaro","Jackson Shields Yeiser","7 Benton Dr","Honolulu","Honolulu","HI",96819,"808-215-6832","808-240-5168","brandon_callaro@hotmail.com","http://www.jacksonshieldsyeiser.com"
"Scarlet","Cartan","Box, J Calvin Esq","9390 S Howell Ave","Albany","Dougherty","GA",31701,"229-735-3378","229-365-9658","scarlet.cartan@yahoo.com","http://www.boxjcalvinesq.com"
"Oretha","Menter","Custom Engineering Inc","8 County Center Dr #647
 ","Boston","Suffolk","MA","02210","617-418-5043","617-697-6024","oretha_menter@yahoo.com","http://www.customengineeringinc.com"
"Ty","Smith","Bresler Eitel Framg Gllry Ltd","4646 Kaahumanu St","Hackensack","Bergen","NJ","07601","201-672-1553","201-995-3149","tsmith@aol.com","http://www.breslereitelframggllryltd.com"
"Xuan","Rochin","Carol, Drake Sparks Esq","2 Monroe St","San Mateo","San Mateo","CA",94403,"650-933-5072","650-247-2625","xuan@gmail.com","http://www.caroldrakesparksesq.com"
"Lindsey","Dilello","Biltmore Investors Bank","52777 Leaders Heights Rd","Ontario","San Bernardino","CA",91761,"909-639-9887","909-589-1693","lindsey.dilello@hotmail.com","http://www.biltmoreinvestorsbank.com"
"Devora","Perez","Desco Equipment Corp","72868 Blackington Ave","Oakland","Alameda","CA",94606,"510-955-3016","510-755-9274","devora_perez@perez.org","http://www.descoequipmentcorp.com"
"Herman","Demesa","Merlin Electric Co","9 Norristown Rd","Troy","Rensselaer","NY",12180,"518-497-2940","518-
 931-7852","hdemesa@cox.net","http://www.merlinelectricco.com"
"Rory","Papasergi","Bailey Cntl Co Div Babcock","83 County Road 437 #8581","Clarks Summit","Lackawanna","PA",18411,"570-867-7489","570-469-8401","rpapasergi@cox.net","http://www.baileycntlcodivbabcock.com"
"Talia","Riopelle","Ford Brothers Wholesale Inc","1 N Harlem Ave #9","Orange","Essex","NJ","07050","973-245-2133","973-818-9788","talia_riopelle@aol.com","http://www.fordbrotherswholesaleinc.com"
"Van","Shire","Cambridge Inn","90131 J St","Pittstown","Hunterdon","NJ","08867","908-409-2890","908-448-1209","van.shire@shire.com","http://www.cambridgeinn.com"
"Lucina","Lary","Matricciani, Albert J Jr","8597 W National Ave","Cocoa","Brevard","FL",32922,"321-749-4981","321-632-4668","lucina_lary@cox.net","http://www.matriccianialbertjjr.com"
"Bok","Isaacs","Nelson Hawaiian Ltd","6 Gilson St","Bronx","Bronx","NY",10468,"718-809-3762","718-478-8568","bok.isaacs@aol.com","http://www.nelsonhawaiianltd.com"
"Rolande","Spickerman",
 "Neland Travel Agency","65 W Maple Ave","Pearl City","Honolulu","HI",96782,"808-315-3077","808-526-5863","rolande.spickerman@spickerman.com","http://www.nelandtravelagency.com"
"Howard","Paulas","Asendorf, J Alan Esq","866 34th Ave","Denver","Denver","CO",80231,"303-623-4241","303-692-3118","hpaulas@gmail.com","http://www.asendorfjalanesq.com"
"Kimbery","Madarang","Silberman, Arthur L Esq","798 Lund Farm Way","Rockaway","Morris","NJ","07866","973-310-1634","973-225-6259","kimbery_madarang@cox.net","http://www.silbermanarthurlesq.com"
"Thurman","Manno","Honey Bee Breeding Genetics &","9387 Charcot Ave","Absecon","Atlantic","NJ","08201","609-524-3586","609-234-8376","thurman.manno@yahoo.com","http://www.honeybeebreedinggenetics.com"
"Becky","Mirafuentes","Wells Kravitz Schnitzer","30553 Washington Rd","Plainfield","Union","NJ","07062","908-877-8409","908-426-8272","becky.mirafuentes@mirafuentes.com","http://www.wellskravitzschnitzer.com"
"Beatriz","Corrington","Prohab Rehabilitation S
 ervs","481 W Lemon St","Middleboro","Plymouth","MA","02346","508-584-4279","508-315-3867","beatriz@yahoo.com","http://www.prohabrehabilitationservs.com"
"Marti","Maybury","Eldridge, Kristin K Esq","4 Warehouse Point Rd #7","Chicago","Cook","IL",60638,"773-775-4522","773-539-1058","marti.maybury@yahoo.com","http://www.eldridgekristinkesq.com"
"Nieves","Gotter","Vlahos, John J Esq","4940 Pulaski Park Dr","Portland","Multnomah","OR",97202,"503-527-5274","503-455-3094","nieves_gotter@gmail.com","http://www.vlahosjohnjesq.com"
"Leatha","Hagele","Ninas Indian Grs & Videos","627 Walford Ave","Dallas","Dallas","TX",75227,"214-339-1809","214-225-5850","lhagele@cox.net","http://www.ninasindiangrsvideos.com"
"Valentin","Klimek","Schmid, Gayanne K Esq","137 Pioneer Way","Chicago","Cook","IL",60604,"312-303-5453","312-512-2338","vklimek@klimek.org","http://www.schmidgayannekesq.com"
"Melissa","Wiklund","Moapa Valley Federal Credit Un","61 13 Stoneridge #835","Findlay","Hancock","OH",45840,"419-9
 39-3613","419-254-4591","melissa@cox.net","http://www.moapavalleyfederalcreditun.com"
"Sheridan","Zane","Kentucky Tennessee Clay Co","2409 Alabama Rd","Riverside","Riverside","CA",92501,"951-645-3605","951-248-6822","sheridan.zane@zane.com","http://www.kentuckytennesseeclayco.com"
"Bulah","Padilla","Admiral Party Rentals & Sales","8927 Vandever Ave","Waco","McLennan","TX",76707,"254-463-4368","254-816-8417","bulah_padilla@hotmail.com","http://www.admiralpartyrentalssales.com"
"Audra","Kohnert","Nelson, Karolyn King Esq","134 Lewis Rd","Nashville","Davidson","TN",37211,"615-406-7854","615-448-9249","audra@kohnert.com","http://www.nelsonkarolynkingesq.com"
"Daren","Weirather","Panasystems","9 N College Ave #3","Milwaukee","Milwaukee","WI",53216,"414-959-2540","414-838-3151","dweirather@aol.com","http://www.panasystems.com"
"Fernanda","Jillson","Shank, Edward L Esq","60480 Old Us Highway 51","Preston","Caroline","MD",21655,"410-387-5260","410-724-6472","fjillson@aol.com","http://www.sh
 ankedwardlesq.com"
"Gearldine","Gellinger","Megibow & Edwards","4 Bloomfield Ave","Irving","Dallas","TX",75061,"972-934-6914","972-821-7118","gearldine_gellinger@gellinger.com","http://www.megibowedwards.com"
"Chau","Kitzman","Benoff, Edward Esq","429 Tiger Ln","Beverly Hills","Los Angeles","CA",90212,"310-560-8022","310-969-7230","chau@gmail.com","http://www.benoffedwardesq.com"
"Theola","Frey","Woodbridge Free Public Library","54169 N Main St","Massapequa","Nassau","NY",11758,"516-948-5768","516-357-3362","theola_frey@frey.com","http://www.woodbridgefreepubliclibrary.com"
"Cheryl","Haroldson","New York Life John Thune","92 Main St","Atlantic City","Atlantic","NJ","08401","609-518-7697","609-263-9243","cheryl@haroldson.org","http://www.newyorklifejohnthune.com"
"Laticia","Merced","Alinabal Inc","72 Mannix Dr","Cincinnati","Hamilton","OH",45203,"513-508-7371","513-418-1566","lmerced@gmail.com","http://www.alinabalinc.com"
"Carissa","Batman","Poletto, Kim David Esq","12270 Caton Cent
 er Dr","Eugene","Lane","OR",97401,"541-326-4074","541-801-5717","carissa.batman@yahoo.com","http://www.polettokimdavidesq.com"
"Lezlie","Craghead","Chang, Carolyn Esq","749 W 18th St #45","Smithfield","Johnston","NC",27577,"919-533-3762","919-885-2453","lezlie.craghead@craghead.org","http://www.changcarolynesq.com"
"Ozell","Shealy","Silver Bros Inc","8 Industry Ln","New York","New York","NY",10002,"212-332-8435","212-880-8865","oshealy@hotmail.com","http://www.silverbrosinc.com"
"Arminda","Parvis","Newtec Inc","1 Huntwood Ave","Phoenix","Maricopa","AZ",85017,"602-906-9419","602-277-3025","arminda@parvis.com","http://www.newtecinc.com"
"Reita","Leto","Creative Business Systems","55262 N French Rd","Indianapolis","Marion","IN",46240,"317-234-1135","317-787-5514","reita.leto@gmail.com","http://www.creativebusinesssystems.com"
"Yolando","Luczki","Dal Tile Corporation","422 E 21st St","Syracuse","Onondaga","NY",13214,"315-304-4759","315-640-6357","yolando@cox.net","http://www.daltilecorp
 oration.com"
"Lizette","Stem","Edward S Katz","501 N 19th Ave","Cherry Hill","Camden","NJ","08002","856-487-5412","856-702-3676","lizette.stem@aol.com","http://www.edwardskatz.com"
"Gregoria","Pawlowicz","Oh My Goodknits Inc","455 N Main Ave","Garden City","Nassau","NY",11530,"516-212-1915","516-376-4230","gpawlowicz@yahoo.com","http://www.ohmygoodknitsinc.com"
"Carin","Deleo","Redeker, Debbie","1844 Southern Blvd","Little Rock","Pulaski","AR",72202,"501-308-1040","501-409-6072","cdeleo@deleo.com","http://www.redekerdebbie.com"
"Chantell","Maynerich","Desert Sands Motel","2023 Greg St","Saint Paul","Ramsey","MN",55101,"651-591-2583","651-776-9688","chantell@yahoo.com","http://www.desertsandsmotel.com"
"Dierdre","Yum","Cummins Southern Plains Inc","63381 Jenks Ave","Philadelphia","Philadelphia","PA",19134,"215-325-3042","215-346-4666","dyum@yahoo.com","http://www.cumminssouthernplainsinc.com"
"Larae","Gudroe","Lehigh Furn Divsn Lehigh","6651 Municipal Rd","Houma","Terrebonne","LA",70
 360,"985-890-7262","985-261-5783","larae_gudroe@gmail.com","http://www.lehighfurndivsnlehigh.com"
"Latrice","Tolfree","United Van Lines Agent","81 Norris Ave #525","Ronkonkoma","Suffolk","NY",11779,"631-957-7624","631-998-2102","latrice.tolfree@hotmail.com","http://www.unitedvanlinesagent.com"
"Kerry","Theodorov","Capitol Reporters","6916 W Main St","Sacramento","Sacramento","CA",95827,"916-591-3277","916-770-7448","kerry.theodorov@gmail.com","http://www.capitolreporters.com"
"Dorthy","Hidvegi","Kwik Kopy Printing","9635 S Main St","Boise","Ada","ID",83704,"208-649-2373","208-690-3315","dhidvegi@yahoo.com","http://www.kwikkopyprinting.com"
"Fannie","Lungren","Centro Inc","17 Us Highway 111","Round Rock","Williamson","TX",78664,"512-587-5746","512-528-9933","fannie.lungren@yahoo.com","http://www.centroinc.com"
"Evangelina","Radde","Campbell, Jan Esq","992 Civic Center Dr","Philadelphia","Philadelphia","PA",19123,"215-964-3284","215-417-5612","evangelina@aol.com","http://www.campbellj
 anesq.com"
"Novella","Degroot","Evans, C Kelly Esq","303 N Radcliffe St","Hilo","Hawaii","HI",96720,"808-477-4775","808-746-1865","novella_degroot@degroot.org","http://www.evansckellyesq.com"
"Clay","Hoa","Scat Enterprises","73 Saint Ann St #86","Reno","Washoe","NV",89502,"775-501-8109","775-848-9135","choa@hoa.org","http://www.scatenterprises.com"
"Jennifer","Fallick","Nagle, Daniel J Esq","44 58th St","Wheeling","Cook","IL",60090,"847-979-9545","847-800-3054","jfallick@yahoo.com","http://www.nagledanieljesq.com"
"Irma","Wolfgramm","Serendiquity Bed & Breakfast","9745 W Main St","Randolph","Morris","NJ","07869","973-545-7355","973-868-8660","irma.wolfgramm@hotmail.com","http://www.serendiquitybedbreakfast.com"
"Eun","Coody","Ray Carolyne Realty","84 Bloomfield Ave","Spartanburg","Spartanburg","SC",29301,"864-256-3620","864-594-4578","eun@yahoo.com","http://www.raycarolynerealty.com"
"Sylvia","Cousey","Berg, Charles E","287 Youngstown Warren Rd","Hampstead","Carroll","MD",21074,"410
 -209-9545","410-863-8263","sylvia_cousey@cousey.org","http://www.bergcharlese.com"
"Nana","Wrinkles","Ray, Milbern D","6 Van Buren St","Mount Vernon","Westchester","NY",10553,"914-855-2115","914-796-3775","nana@aol.com","http://www.raymilbernd.com"
"Layla","Springe","Chadds Ford Winery","229 N Forty Driv","New York","New York","NY",10011,"212-260-3151","212-253-7448","layla.springe@cox.net","http://www.chaddsfordwinery.com"
"Joesph","Degonia","A R Packaging","2887 Knowlton St #5435","Berkeley","Alameda","CA",94710,"510-677-9785","510-942-5916","joesph_degonia@degonia.org","http://www.arpackaging.com"
"Annabelle","Boord","Corn Popper","523 Marquette Ave","Concord","Middlesex","MA","01742","978-697-6263","978-289-7717","annabelle.boord@cox.net","http://www.cornpopper.com"
"Stephaine","Vinning","Birite Foodservice Distr","3717 Hamann Industrial Pky","San Francisco","San Francisco","CA",94104,"415-767-6596","415-712-9530","stephaine@cox.net","http://www.biritefoodservicedistr.com"
"Neli
 da","Sawchuk","Anchorage Museum Of Hist & Art","3 State Route 35 S","Paramus","Bergen","NJ","07652","201-971-1638","201-247-8925","nelida@gmail.com","http://www.anchoragemuseumofhistart.com"
"Marguerita","Hiatt","Haber, George D Md","82 N Highway 67","Oakley","Contra Costa","CA",94561,"925-634-7158","925-541-8521","marguerita.hiatt@gmail.com","http://www.habergeorgedmd.com"
"Carmela","Cookey","Royal Pontiac Olds Inc","9 Murfreesboro Rd","Chicago","Cook","IL",60623,"773-494-4195","773-297-9391","ccookey@cookey.org","http://www.royalpontiacoldsinc.com"
"Junita","Brideau","Leonards Antiques Inc","6 S Broadway St","Cedar Grove","Essex","NJ","07009","973-943-3423","973-582-5469","jbrideau@aol.com","http://www.leonardsantiquesinc.com"
"Claribel","Varriano","Meca","6 Harry L Dr #6327","Perrysburg","Wood","OH",43551,"419-544-4900","419-573-2033","claribel_varriano@cox.net","http://www.meca.com"
"Benton","Skursky","Nercon Engineering & Mfg Inc","47939 Porter Ave","Gardena","Los Angeles","CA"
 ,90248,"310-579-2907","310-694-8466","benton.skursky@aol.com","http://www.nerconengineeringmfginc.com"
"Hillary","Skulski","Replica I","9 Wales Rd Ne #914","Homosassa","Citrus","FL",34448,"352-242-2570","352-990-5946","hillary.skulski@aol.com","http://www.replicai.com"
"Merilyn","Bayless","20 20 Printing Inc","195 13n N","Santa Clara","Santa Clara","CA",95054,"408-758-5015","408-346-2180","merilyn_bayless@cox.net","http://www.printinginc.com"
"Teri","Ennaco","Publishers Group West","99 Tank Farm Rd","Hazleton","Luzerne","PA",18201,"570-889-5187","570-355-1665","tennaco@gmail.com","http://www.publishersgroupwest.com"
"Merlyn","Lawler","Nischwitz, Jeffrey L Esq","4671 Alemany Blvd","Jersey City","Hudson","NJ","07304","201-588-7810","201-858-9960","merlyn_lawler@hotmail.com","http://www.nischwitzjeffreylesq.com"
"Georgene","Montezuma","Payne Blades & Wellborn Pa","98 University Dr","San Ramon","Contra Costa","CA",94583,"925-615-5185","925-943-3449","gmontezuma@cox.net","http://www.payn
 ebladeswellbornpa.com"
"Jettie","Mconnell","Coldwell Bnkr Wright Real Est","50 E Wacker Dr","Bridgewater","Somerset","NJ","08807","908-802-3564","908-602-5258","jmconnell@hotmail.com","http://www.coldwellbnkrwrightrealest.com"
"Lemuel","Latzke","Computer Repair Service","70 Euclid Ave #722","Bohemia","Suffolk","NY",11716,"631-748-6479","631-291-4976","lemuel.latzke@gmail.com","http://www.computerrepairservice.com"
"Melodie","Knipp","Fleetwood Building Block Inc","326 E Main St #6496","Thousand Oaks","Ventura","CA",91362,"805-690-1682","805-810-8964","mknipp@gmail.com","http://www.fleetwoodbuildingblockinc.com"
"Candida","Corbley","Colts Neck Medical Assocs Inc","406 Main St","Somerville","Somerset","NJ","08876","908-275-8357","908-943-6103","candida_corbley@hotmail.com","http://www.coltsneckmedicalassocsinc.com"
"Karan","Karpin","New England Taxidermy","3 Elmwood Dr","Beaverton","Washington","OR",97005,"503-940-8327","503-707-5812","karan_karpin@gmail.com","http://www.newenglandtaxi
 dermy.com"
"Andra","Scheyer","Ludcke, George O Esq","9 Church St","Salem","Marion","OR",97302,"503-516-2189","503-950-3068","andra@gmail.com","http://www.ludckegeorgeoesq.com"
"Felicidad","Poullion","Mccorkle, Tom S Esq","9939 N 14th St","Riverton","Burlington","NJ","08077","856-305-9731","856-828-6021","fpoullion@poullion.com","http://www.mccorkletomsesq.com"
"Belen","Strassner","Eagle Software Inc","5384 Southwyck Blvd","Douglasville","Douglas","GA",30135,"770-507-8791","770-802-4003","belen_strassner@aol.com","http://www.eaglesoftwareinc.com"
"Gracia","Melnyk","Juvenile & Adult Super","97 Airport Loop Dr","Jacksonville","Duval","FL",32216,"904-235-3633","904-627-4341","gracia@melnyk.com","http://www.juvenileadultsuper.com"
"Jolanda","Hanafan","Perez, Joseph J Esq","37855 Nolan Rd","Bangor","Penobscot","ME","04401","207-458-9196","207-233-6185","jhanafan@gmail.com","http://www.perezjosephjesq.com"
"Barrett","Toyama","Case Foundation Co","4252 N Washington Ave #9","Kennedale","Tarr
 ant","TX",76060,"817-765-5781","817-577-6151","barrett.toyama@toyama.org","http://www.casefoundationco.com"
"Helga","Fredicks","Eis Environmental Engrs Inc","42754 S Ash Ave","Buffalo","Erie","NY",14228,"716-752-4114","716-854-9845","helga_fredicks@yahoo.com","http://www.eisenvironmentalengrsinc.com"
"Ashlyn","Pinilla","Art Crafters","703 Beville Rd","Opa Locka","Miami-Dade","FL",33054,"305-670-9628","305-857-5489","apinilla@cox.net","http://www.artcrafters.com"
"Fausto","Agramonte","Marriott Hotels Resorts Suites","5 Harrison Rd","New York","New York","NY",10038,"212-313-1783","212-778-3063","fausto_agramonte@yahoo.com","http://www.marriotthotelsresortssuites.com"
"Ronny","Caiafa","Remaco Inc","73 Southern Blvd","Philadelphia","Philadelphia","PA",19103,"215-605-7570","215-511-3531","ronny.caiafa@caiafa.org","http://www.remacoinc.com"
"Marge","Limmel","Bjork, Robert D Jr","189 Village Park Rd","Crestview","Okaloosa","FL",32536,"850-430-1663","850-330-8079","marge@gmail.com","http://
 www.bjorkrobertdjr.com"
"Norah","Waymire","Carmichael, Jeffery L Esq","6 Middlegate Rd #106","San Francisco","San Francisco","CA",94107,"415-306-7897","415-874-2984","norah.waymire@gmail.com","http://www.carmichaeljefferylesq.com"
"Aliza","Baltimore","Andrews, J Robert Esq","1128 Delaware St","San Jose","Santa Clara","CA",95132,"408-504-3552","408-425-1994","aliza@aol.com","http://www.andrewsjrobertesq.com"
"Mozell","Pelkowski","Winship & Byrne","577 Parade St","South San Francisco","San Mateo","CA",94080,"650-947-1215","650-960-1069","mpelkowski@pelkowski.org","http://www.winshipbyrne.com"
"Viola","Bitsuie","Burton & Davis","70 Mechanic St","Northridge","Los Angeles","CA",91325,"818-864-4875","818-481-5787","viola@gmail.com","http://www.burtondavis.com"
"Franklyn","Emard","Olympic Graphic Arts","4379 Highway 116","Philadelphia","Philadelphia","PA",19103,"215-558-8189","215-483-3003","femard@emard.com","http://www.olympicgraphicarts.com"
"Willodean","Konopacki","Magnuson","55 Hawtho
 rne Blvd","Lafayette","Lafayette","LA",70506,"337-253-8384","337-774-7564","willodean_konopacki@konopacki.org","http://www.magnuson.com"
"Beckie","Silvestrini","A All American Travel Inc","7116 Western Ave","Dearborn","Wayne","MI",48126,"313-533-4884","313-390-7855","beckie.silvestrini@silvestrini.com","http://www.aallamericantravelinc.com"
"Rebecka","Gesick","Polykote Inc","2026 N Plankinton Ave #3","Austin","Travis","TX",78754,"512-213-8574","512-693-8345","rgesick@gesick.org","http://www.polykoteinc.com"
"Frederica","Blunk","Jets Cybernetics","99586 Main St","Dallas","Dallas","TX",75207,"214-428-2285","214-529-1949","frederica_blunk@gmail.com","http://www.jetscybernetics.com"
"Glen","Bartolet","Metlab Testing Services","8739 Hudson St","Vashon","King","WA",98070,"206-697-5796","206-389-1482","glen_bartolet@hotmail.com","http://www.metlabtestingservices.com"
"Freeman","Gochal","Kellermann, William T Esq","383 Gunderman Rd #197","Coatesville","Chester","PA",19320,"610-476-3501","61
 0-752-2683","freeman_gochal@aol.com","http://www.kellermannwilliamtesq.com"
"Vincent","Meinerding","Arturi, Peter D Esq","4441 Point Term Mkt","Philadelphia","Philadelphia","PA",19143,"215-372-1718","215-829-4221","vincent.meinerding@hotmail.com","http://www.arturipeterdesq.com"
"Rima","Bevelacqua","Mcauley Mfg Co","2972 Lafayette Ave","Gardena","Los Angeles","CA",90248,"310-858-5079","310-499-4200","rima@cox.net","http://www.mcauleymfgco.com"
"Glendora","Sarbacher","Defur Voran Hanley Radcliff","2140 Diamond Blvd","Rohnert Park","Sonoma","CA",94928,"707-653-8214","707-881-3154","gsarbacher@gmail.com","http://www.defurvoranhanleyradcliff.com"
"Avery","Steier","Dill Dill Carr & Stonbraker Pc","93 Redmond Rd #492","Orlando","Orange","FL",32803,"407-808-9439","407-945-8566","avery@cox.net","http://www.dilldillcarrstonbrakerpc.com"
"Cristy","Lother","Kleensteel","3989 Portage Tr","Escondido","San Diego","CA",92025,"760-971-4322","760-465-4762","cristy@lother.com","http://www.kleensteel.
 com"
"Nicolette","Brossart","Goulds Pumps Inc Slurry Pump","1 Midway Rd","Westborough","Worcester","MA","01581","508-837-9230","508-504-6388","nicolette_brossart@brossart.com","http://www.gouldspumpsincslurrypump.com"
"Tracey","Modzelewski","Kansas City Insurance Report","77132 Coon Rapids Blvd Nw","Conroe","Montgomery","TX",77301,"936-264-9294","936-988-8171","tracey@hotmail.com","http://www.kansascityinsurancereport.com"
"Virgina","Tegarden","Berhanu International Foods","755 Harbor Way","Milwaukee","Milwaukee","WI",53226,"414-214-8697","414-411-5744","virgina_tegarden@tegarden.com","http://www.berhanuinternationalfoods.com"
"Tiera","Frankel","Roland Ashcroft","87 Sierra Rd","El Monte","Los Angeles","CA",91731,"626-636-4117","626-638-4241","tfrankel@aol.com","http://www.rolandashcroft.com"
"Alaine","Bergesen","Hispanic Magazine","7667 S Hulen St #42","Yonkers","Westchester","NY",10701,"914-300-9193","914-654-1426","alaine_bergesen@cox.net","http://www.hispanicmagazine.com"
"Earlee
 n","Mai","Little Sheet Metal Co","75684 S Withlapopka Dr #32","Dallas","Dallas","TX",75227,"214-289-1973","214-785-6750","earleen_mai@cox.net","http://www.littlesheetmetalco.com"
"Leonida","Gobern","Holmes, Armstead J Esq","5 Elmwood Park Blvd","Biloxi","Harrison","MS",39530,"228-235-5615","228-432-4635","leonida@gobern.org","http://www.holmesarmsteadjesq.com"
"Ressie","Auffrey","Faw, James C Cpa","23 Palo Alto Sq","Miami","Miami-Dade","FL",33134,"305-604-8981","305-287-4743","ressie.auffrey@yahoo.com","http://www.fawjamesccpa.com"
"Justine","Mugnolo","Evans Rule Company","38062 E Main St","New York","New York","NY",10048,"212-304-9225","212-311-6377","jmugnolo@yahoo.com","http://www.evansrulecompany.com"
"Eladia","Saulter","Tyee Productions Inc","3958 S Dupont Hwy #7","Ramsey","Bergen","NJ","07446","201-474-4924","201-365-8698","eladia@saulter.com","http://www.tyeeproductionsinc.com"
"Chaya","Malvin","Dunnells & Duvall","560 Civic Center Dr","Ann Arbor","Washtenaw","MI",48103,"734-
 928-5182","734-408-8174","chaya@malvin.com","http://www.dunnellsduvall.com"
"Gwenn","Suffield","Deltam Systems Inc","3270 Dequindre Rd","Deer Park","Suffolk","NY",11729,"631-258-6558","631-295-9879","gwenn_suffield@suffield.org","http://www.deltamsystemsinc.com"
"Salena","Karpel","Hammill Mfg Co","1 Garfield Ave #7","Canton","Stark","OH",44707,"330-791-8557","330-618-2579","skarpel@cox.net","http://www.hammillmfgco.com"
"Yoko","Fishburne","Sams Corner Store","9122 Carpenter Ave","New Haven","New Haven","CT","06511","203-506-4706","203-840-8634","yoko@fishburne.com","http://www.samscornerstore.com"
"Taryn","Moyd","Siskin, Mark J Esq","48 Lenox St","Fairfax","Fairfax City","VA",22030,"703-322-4041","703-938-7939","taryn.moyd@hotmail.com","http://www.siskinmarkjesq.com"
"Katina","Polidori","Cape & Associates Real Estate","5 Little River Tpke","Wilmington","Middlesex","MA","01887","978-626-2978","978-679-7429","katina_polidori@aol.com","http://www.capeassociatesrealestate.com"
"Rickie",
 "Plumer","Merrill Lynch","3 N Groesbeck Hwy","Toledo","Lucas","OH",43613,"419-693-1334","419-313-5571","rickie.plumer@aol.com","http://www.merrilllynch.com"
"Alex","Loader","Sublett, Scott Esq","37 N Elm St #916","Tacoma","Pierce","WA",98409,"253-660-7821","253-875-9222","alex@loader.com","http://www.sublettscottesq.com"
"Lashon","Vizarro","Sentry Signs","433 Westminster Blvd #590","Roseville","Placer","CA",95661,"916-741-7884","916-289-4526","lashon@aol.com","http://www.sentrysigns.com"
"Lauran","Burnard","Professionals Unlimited","66697 Park Pl #3224","Riverton","Fremont","WY",82501,"307-342-7795","307-453-7589","lburnard@burnard.com","http://www.professionalsunlimited.com"
"Ceola","Setter","Southern Steel Shelving Co","96263 Greenwood Pl","Warren","Knox","ME","04864","207-627-7565","207-297-5029","ceola.setter@setter.org","http://www.southernsteelshelvingco.com"
"My","Rantanen","Bosco, Paul J","8 Mcarthur Ln","Richboro","Bucks","PA",18954,"215-491-5633","215-647-2158","my@hotmail
 .com","http://www.boscopaulj.com"
"Lorrine","Worlds","Longo, Nicholas J Esq","8 Fair Lawn Ave","Tampa","Hillsborough","FL",33614,"813-769-2939","813-863-6467","lorrine.worlds@worlds.com","http://www.longonicholasjesq.com"
"Peggie","Sturiale","Henry County Middle School","9 N 14th St","El Cajon","San Diego","CA",92020,"619-608-1763","619-695-8086","peggie@cox.net","http://www.henrycountymiddleschool.com"
"Marvel","Raymo","Edison Supply & Equipment Co","9 Vanowen St","College Station","Brazos","TX",77840,"979-718-8968","979-809-5770","mraymo@yahoo.com","http://www.edisonsupplyequipmentco.com"
"Daron","Dinos","Wolf, Warren R Esq","18 Waterloo Geneva Rd","Highland Park","Lake","IL",60035,"847-233-3075","847-265-6609","daron_dinos@cox.net","http://www.wolfwarrenresq.com"
"An","Fritz","Linguistic Systems Inc","506 S Hacienda Dr","Atlantic City","Atlantic","NJ","08401","609-228-5265","609-854-7156","an_fritz@hotmail.com","http://www.linguisticsystemsinc.com"
"Portia","Stimmel","Peace Chris
 tian Center","3732 Sherman Ave","Bridgewater","Somerset","NJ","08807","908-722-7128","908-670-4712","portia.stimmel@aol.com","http://www.peacechristiancenter.com"
"Rhea","Aredondo","Double B Foods Inc","25657 Live Oak St","Brooklyn","Kings","NY",11226,"718-560-9537","718-280-4183","rhea_aredondo@cox.net","http://www.doublebfoodsinc.com"
"Benedict","Sama","Alexander & Alexander Inc","4923 Carey Ave","Saint Louis","Saint Louis City","MO",63104,"314-787-1588","314-858-4832","bsama@cox.net","http://www.alexanderalexanderinc.com"
"Alyce","Arias","Fairbanks Scales","3196 S Rider Trl","Stockton","San Joaquin","CA",95207,"209-317-1801","209-242-7022","alyce@arias.org","http://www.fairbanksscales.com"
"Heike","Berganza","Cali Sportswear Cutting Dept","3 Railway Ave #75","Little Falls","Passaic","NJ","07424","973-936-5095","973-822-8827","heike@gmail.com","http://www.calisportswearcuttingdept.com"
"Carey","Dopico","Garofani, John Esq","87393 E Highland Rd","Indianapolis","Marion","IN",46220,"
 317-578-2453","317-441-5848","carey_dopico@dopico.org","http://www.garofanijohnesq.com"
"Dottie","Hellickson","Thompson Fabricating Co","67 E Chestnut Hill Rd","Seattle","King","WA",98133,"206-540-6076","206-295-5631","dottie@hellickson.org","http://www.thompsonfabricatingco.com"
"Deandrea","Hughey","Century 21 Krall Real Estate","33 Lewis Rd #46","Burlington","Alamance","NC",27215,"336-822-7652","336-467-3095","deandrea@yahoo.com","http://www.centurykrallrealestate.com"
"Kimberlie","Duenas","Mid Contntl Rlty & Prop Mgmt","8100 Jacksonville Rd #7","Hays","Ellis","KS",67601,"785-629-8542","785-616-1685","kimberlie_duenas@yahoo.com","http://www.midcontntlrltypropmgmt.com"
"Martina","Staback","Ace Signs Inc","7 W Wabansia Ave #227","Orlando","Orange","FL",32822,"407-471-6908","407-429-2145","martina_staback@staback.com","http://www.acesignsinc.com"
"Skye","Fillingim","Rodeway Inn","25 Minters Chapel Rd #9","Minneapolis","Hennepin","MN",55401,"612-508-2655","612-664-6304","skye_fillingi
 m@yahoo.com","http://www.rodewayinn.com"
"Jade","Farrar","Bonnet & Daughter","6882 Torresdale Ave","Columbia","Richland","SC",29201,"803-352-5387","803-975-3405","jade.farrar@yahoo.com","http://www.bonnetdaughter.com"
"Charlene","Hamilton","Oshins & Gibbons","985 E 6th Ave","Santa Rosa","Sonoma","CA",95407,"707-300-1771","707-821-8037","charlene.hamilton@hotmail.com","http://www.oshinsgibbons.com"
"Geoffrey","Acey","Price Business Services","7 West Ave #1","Palatine","Cook","IL",60067,"847-222-1734","847-556-2909","geoffrey@gmail.com","http://www.pricebusinessservices.com"
"Stevie","Westerbeck","Wise, Dennis W Md","26659 N 13th St","Costa Mesa","Orange","CA",92626,"949-867-4077","949-903-3898","stevie.westerbeck@yahoo.com","http://www.wisedenniswmd.com"
"Pamella","Fortino","Super 8 Motel","669 Packerland Dr #1438","Denver","Denver","CO",80212,"303-404-2210","303-794-1341","pamella@fortino.com","http://www.supermotel.com"
"Harrison","Haufler","John Wagner Associates","759 Eldora St",
 "New Haven","New Haven","CT","06515","203-801-6193","203-801-8497","hhaufler@hotmail.com","http://www.johnwagnerassociates.com"
"Johnna","Engelberg","Thrifty Oil Co","5 S Colorado Blvd #449","Bothell","Snohomish","WA",98021,"425-986-7573","425-700-3751","jengelberg@engelberg.org","http://www.thriftyoilco.com"
"Buddy","Cloney","Larkfield Photo","944 Gaither Dr","Strongsville","Cuyahoga","OH",44136,"440-989-5826","440-327-2093","buddy.cloney@yahoo.com","http://www.larkfieldphoto.com"
"Dalene","Riden","Silverman Planetarium","66552 Malone Rd","Plaistow","Rockingham","NH","03865","603-315-6839","603-745-7497","dalene.riden@aol.com","http://www.silvermanplanetarium.com"
"Jerry","Zurcher","J & F Lumber","77 Massillon Rd #822","Satellite Beach","Brevard","FL",32937,"321-518-5938","321-597-2159","jzurcher@zurcher.org","http://www.jflumber.com"
"Haydee","Denooyer","Cleaning Station Inc","25346 New Rd","New York","New York","NY",10016,"212-792-8658","212-782-3493","hdenooyer@denooyer.org","ht
 tp://www.cleaningstationinc.com"
"Joseph","Cryer","Ames Stationers","60 Fillmore Ave","Huntington Beach","Orange","CA",92647,"714-584-2237","714-698-2170","joseph_cryer@cox.net","http://www.amesstationers.com"
"Deonna","Kippley","Midas Muffler Shops","57 Haven Ave #90","Southfield","Oakland","MI",48075,"248-913-4677","248-793-4966","deonna_kippley@hotmail.com","http://www.midasmufflershops.com"
"Raymon","Calvaresi","Seaboard Securities Inc","6538 E Pomona St #60","Indianapolis","Marion","IN",46222,"317-825-4724","317-342-1532","raymon.calvaresi@gmail.com","http://www.seaboardsecuritiesinc.com"
"Alecia","Bubash","Petersen, James E Esq","6535 Joyce St","Wichita Falls","Wichita","TX",76301,"940-276-7922","940-302-3036","alecia@aol.com","http://www.petersenjameseesq.com"
"Ma","Layous","Development Authority","78112 Morris Ave","North Haven","New Haven","CT","06473","203-721-3388","203-564-1543","mlayous@hotmail.com","http://www.developmentauthority.com"
"Detra","Coyier","Schott Fiber Op
 tics Inc","96950 Hidden Ln","Aberdeen","Harford","MD",21001,"410-739-9277","410-259-2118","detra@aol.com","http://www.schottfiberopticsinc.com"
"Terrilyn","Rodeigues","Stuart J Agins","3718 S Main St","New Orleans","Orleans","LA",70130,"504-463-4384","504-635-8518","terrilyn.rodeigues@cox.net","http://www.stuartjagins.com"
"Salome","Lacovara","Mitsumi Electronics Corp","9677 Commerce Dr","Richmond","Richmond City","VA",23219,"804-550-5097","804-858-1011","slacovara@gmail.com","http://www.mitsumielectronicscorp.com"
"Garry","Keetch","Italian Express Franchise Corp","5 Green Pond Rd #4","Southampton","Bucks","PA",18966,"215-979-8776","215-846-9046","garry_keetch@hotmail.com","http://www.italianexpressfranchisecorp.com"
"Matthew","Neither","American Council On Sci & Hlth","636 Commerce Dr #42","Shakopee","Scott","MN",55379,"952-651-7597","952-906-4597","mneither@yahoo.com","http://www.americancouncilonscihlth.com"
"Theodora","Restrepo","Kleri, Patricia S Esq","42744 Hamann Industrial P
 ky #82","Miami","Miami-Dade","FL",33136,"305-936-8226","305-573-1085","theodora.restrepo@restrepo.com","http://www.kleripatriciasesq.com"
"Noah","Kalafatis","Twiggs Abrams Blanchard","1950 5th Ave","Milwaukee","Milwaukee","WI",53209,"414-263-5287","414-660-9766","noah.kalafatis@aol.com","http://www.twiggsabramsblanchard.com"
"Carmen","Sweigard","Maui Research & Technology Pk","61304 N French Rd","Somerset","Somerset","NJ","08873","732-941-2621","732-445-6940","csweigard@sweigard.com","http://www.mauiresearchtechnologypk.com"
"Lavonda","Hengel","Bradley Nameplate Corp","87 Imperial Ct #79","Fargo","Cass","ND",58102,"701-898-2154","701-421-7080","lavonda@cox.net","http://www.bradleynameplatecorp.com"
"Junita","Stoltzman","Geonex Martel Inc","94 W Dodge Rd","Carson City","Carson City","NV",89701,"775-638-9963","775-578-1214","junita@aol.com","http://www.geonexmartelinc.com"
"Herminia","Nicolozakes","Sea Island Div Of Fstr Ind Inc","4 58th St #3519","Scottsdale","Maricopa","AZ",85254,"6
 02-954-5141","602-304-6433","herminia@nicolozakes.org","http://www.seaislanddivoffstrindinc.com"
"Casie","Good","Papay, Debbie J Esq","5221 Bear Valley Rd","Nashville","Davidson","TN",37211,"615-390-2251","615-825-4297","casie.good@aol.com","http://www.papaydebbiejesq.com"
"Reena","Maisto","Lane Promotions","9648 S Main","Salisbury","Wicomico","MD",21801,"410-351-1863","410-951-2667","reena@hotmail.com","http://www.lanepromotions.com"
"Mirta","Mallett","Stephen Kennerly Archts Inc Pc","7 S San Marcos Rd","New York","New York","NY",10004,"212-870-1286","212-745-6948","mirta_mallett@gmail.com","http://www.stephenkennerlyarchtsincpc.com"
"Cathrine","Pontoriero","Business Systems Of Wis Inc","812 S Haven St","Amarillo","Randall","TX",79109,"806-703-1435","806-558-5848","cathrine.pontoriero@pontoriero.com","http://www.businesssystemsofwisinc.com"
"Filiberto","Tawil","Flash, Elena Salerno Esq","3882 W Congress St #799","Los Angeles","Los Angeles","CA",90016,"323-765-2528","323-842-8226","
 ftawil@hotmail.com","http://www.flashelenasalernoesq.com"
"Raul","Upthegrove","Neeley, Gregory W Esq","4 E Colonial Dr","La Mesa","San Diego","CA",91942,"619-509-5282","619-666-4765","rupthegrove@yahoo.com","http://www.neeleygregorywesq.com"
"Sarah","Candlish","Alabama Educational Tv Comm","45 2nd Ave #9759","Atlanta","Fulton","GA",30328,"770-732-1194","770-531-2842","sarah.candlish@gmail.com","http://www.alabamaeducationaltvcomm.com"
"Lucy","Treston","Franz Inc","57254 Brickell Ave #372","Worcester","Worcester","MA","01602","508-769-5250","508-502-5634","lucy@cox.net","http://www.franzinc.com"
"Judy","Aquas","Plantation Restaurant","8977 Connecticut Ave Nw #3","Niles","Berrien","MI",49120,"269-756-7222","269-431-9464","jaquas@aquas.com","http://www.plantationrestaurant.com"
"Yvonne","Tjepkema","Radio Communications Co","9 Waydell St","Fairfield","Essex","NJ","07004","973-714-1721","973-976-8627","yvonne.tjepkema@hotmail.com","http://www.radiocommunicationsco.com"
"Kayleigh","Lace",
 "Dentalaw Divsn Hlth Care","43 Huey P Long Ave","Lafayette","Lafayette","LA",70508,"337-740-9323","337-751-2326","kayleigh.lace@yahoo.com","http://www.dentalawdivsnhlthcare.com"
"Felix","Hirpara","American Speedy Printing Ctrs","7563 Cornwall Rd #4462","Denver","Lancaster","PA",17517,"717-491-5643","717-583-1497","felix_hirpara@cox.net","http://www.americanspeedyprintingctrs.com"
"Tresa","Sweely","Grayson, Grant S Esq","22 Bridle Ln","Valley Park","Saint Louis","MO",63088,"314-359-9566","314-231-3514","tresa_sweely@hotmail.com","http://www.graysongrantsesq.com"
"Kristeen","Turinetti","Jeanerette Middle School","70099 E North Ave","Arlington","Tarrant","TX",76013,"817-213-8851","817-947-9480","kristeen@gmail.com","http://www.jeanerettemiddleschool.com"
"Jenelle","Regusters","Haavisto, Brian F Esq","3211 E Northeast Loop","Tampa","Hillsborough","FL",33619,"813-932-8715","813-357-7296","jregusters@regusters.com","http://www.haavistobrianfesq.com"
"Renea","Monterrubio","Wmmt Radio Stati
 on","26 Montgomery St","Atlanta","Fulton","GA",30328,"770-679-4752","770-930-9967","renea@hotmail.com","http://www.wmmtradiostation.com"
"Olive","Matuszak","Colony Paints Sales Ofc & Plnt","13252 Lighthouse Ave","Cathedral City","Riverside","CA",92234,"760-938-6069","760-745-2649","olive@aol.com","http://www.colonypaintssalesofcplnt.com"
"Ligia","Reiber","Floral Expressions","206 Main St #2804","Lansing","Ingham","MI",48933,"517-906-1108","517-747-7664","lreiber@cox.net","http://www.floralexpressions.com"
"Christiane","Eschberger","Casco Services Inc","96541 W Central Blvd","Phoenix","Maricopa","AZ",85034,"602-390-4944","602-330-6894","christiane.eschberger@yahoo.com","http://www.cascoservicesinc.com"
"Goldie","Schirpke","Reuter, Arthur C Jr","34 Saint George Ave #2","Bangor","Penobscot","ME","04401","207-295-7569","207-748-3722","goldie.schirpke@yahoo.com","http://www.reuterarthurcjr.com"
"Loreta","Timenez","Kaminski, Katherine Andritsaki","47857 Coney Island Ave","Clinton","Prince
  Georges","MD",20735,"301-696-6420","301-392-6698","loreta.timenez@hotmail.com","http://www.kaminskikatherineandritsaki.com"
"Fabiola","Hauenstein","Sidewinder Products Corp","8573 Lincoln Blvd","York","York","PA",17404,"717-809-3119","717-344-2804","fabiola.hauenstein@hauenstein.org","http://www.sidewinderproductscorp.com"
"Amie","Perigo","General Foam Corporation","596 Santa Maria Ave #7913","Mesquite","Dallas","TX",75150,"972-419-7946","972-898-1033","amie.perigo@yahoo.com","http://www.generalfoamcorporation.com"
"Raina","Brachle","Ikg Borden Divsn Harsco Corp","3829 Ventura Blvd","Butte","Silver Bow","MT",59701,"406-318-1515","406-374-7752","raina.brachle@brachle.org","http://www.ikgbordendivsnharscocorp.com"
"Erinn","Canlas","Anchor Computer Inc","13 S Hacienda Dr","Livingston","Essex","NJ","07039","973-767-3008","973-563-9502","erinn.canlas@canlas.com","http://www.anchorcomputerinc.com"
"Cherry","Lietz","Sebring & Co","40 9th Ave Sw #91","Waterford","Oakland","MI",48329,"248-9
 80-6904","248-697-7722","cherry@lietz.com","http://www.sebringco.com"
"Kattie","Vonasek","H A C Farm Lines Co Optv Assoc","2845 Boulder Crescent St","Cleveland","Cuyahoga","OH",44103,"216-923-3715","216-270-9653","kattie@vonasek.org","http://www.hacfarmlinescooptvassoc.com"
"Lilli","Scriven","Hunter, John J Esq","33 State St","Abilene","Taylor","TX",79601,"325-631-1560","325-667-7868","lilli@aol.com","http://www.hunterjohnjesq.com"
"Whitley","Tomasulo","Freehold Fence Co","2 S 15th St","Fort Worth","Tarrant","TX",76107,"817-526-4408","817-819-7799","whitley.tomasulo@aol.com","http://www.freeholdfenceco.com"
"Barbra","Adkin","Binswanger","4 Kohler Memorial Dr","Brooklyn","Kings","NY",11230,"718-201-3751","718-732-9475","badkin@hotmail.com","http://www.binswanger.com"
"Hermila","Thyberg","Chilton Malting Co","1 Rancho Del Mar Shopping C","Providence","Providence","RI","02903","401-893-4882","401-885-7681","hermila_thyberg@hotmail.com","http://www.chiltonmaltingco.com"
"Jesusita","Flis
 ter","Schoen, Edward J Jr","3943 N Highland Ave","Lancaster","Lancaster","PA",17601,"717-885-9118","717-686-7564","jesusita.flister@hotmail.com","http://www.schoenedwardjjr.com"
"Caitlin","Julia","Helderman, Seymour Cpa","5 Williams St","Johnston","Providence","RI","02919","401-948-4982","401-552-9059","caitlin.julia@julia.org","http://www.heldermanseymourcpa.com"
"Roosevelt","Hoffis","Denbrook, Myron","60 Old Dover Rd","Hialeah","Miami-Dade","FL",33014,"305-622-4739","305-302-1135","roosevelt.hoffis@aol.com","http://www.denbrookmyron.com"
"Helaine","Halter","Lippitt, Mike","8 Sheridan Rd","Jersey City","Hudson","NJ","07304","201-832-4168","201-412-3040","hhalter@yahoo.com","http://www.lippittmike.com"
"Lorean","Martabano","Hiram, Hogg P Esq","85092 Southern Blvd","San Antonio","Bexar","TX",78204,"210-856-4979","210-634-2447","lorean.martabano@hotmail.com","http://www.hiramhoggpesq.com"
"France","Buzick","In Travel Agency","64 Newman Springs Rd E","Brooklyn","Kings","NY",11219,"718-
 478-8504","718-853-3740","france.buzick@yahoo.com","http://www.intravelagency.com"
"Justine","Ferrario","Newhart Foods Inc","48 Stratford Ave","Pomona","Los Angeles","CA",91768,"909-993-3242","909-631-5703","jferrario@hotmail.com","http://www.newhartfoodsinc.com"
"Adelina","Nabours","Courtyard By Marriott","80 Pittsford Victor Rd #9","Cleveland","Cuyahoga","OH",44103,"216-230-4892","216-937-5320","adelina_nabours@gmail.com","http://www.courtyardbymarriott.com"
"Derick","Dhamer","Studer, Eugene A Esq","87163 N Main Ave","New York","New York","NY",10013,"212-304-4515","212-225-9676","ddhamer@cox.net","http://www.studereugeneaesq.com"
"Jerry","Dallen","Seashore Supply Co Waretown","393 Lafayette Ave","Richmond","Richmond City","VA",23219,"804-762-9576","804-808-9574","jerry.dallen@yahoo.com","http://www.seashoresupplycowaretown.com"
"Leota","Ragel","Mayar Silk Inc","99 5th Ave #33","Trion","Chattooga","GA",30753,"706-221-4243","706-616-5131","leota.ragel@gmail.com","http://www.mayarsil
 kinc.com"
"Jutta","Amyot","National Medical Excess Corp","49 N Mays St","Broussard","Lafayette","LA",70518,"337-515-1438","337-991-8070","jamyot@hotmail.com","http://www.nationalmedicalexcesscorp.com"
"Aja","Gehrett","Stero Company","993 Washington Ave","Nutley","Essex","NJ","07110","973-544-2677","973-986-4456","aja_gehrett@hotmail.com","http://www.sterocompany.com"
"Kirk","Herritt","Hasting, H Duane Esq","88 15th Ave Ne","Vestal","Broome","NY",13850,"607-407-3716","607-350-7690","kirk.herritt@aol.com","http://www.hastinghduaneesq.com"
"Leonora","Mauson","Insty Prints","3381 E 40th Ave","Passaic","Passaic","NJ","07055","973-412-2995","973-355-2120","leonora@yahoo.com","http://www.instyprints.com"
"Winfred","Brucato","Glenridge Manor Mobile Home Pk","201 Ridgewood Rd","Moscow","Latah","ID",83843,"208-252-4552","208-793-4108","winfred_brucato@hotmail.com","http://www.glenridgemanormobilehomepk.com"
"Tarra","Nachor","Circuit Solution Inc","39 Moccasin Dr","San Francisco","San Francisc
 o","CA",94104,"415-411-1775","415-284-2730","tarra.nachor@cox.net","http://www.circuitsolutioninc.com"
"Corinne","Loder","Local Office","4 Carroll St","North Attleboro","Bristol","MA","02760","508-942-4186","508-618-7826","corinne@loder.org","http://www.localoffice.com"
"Dulce","Labreche","Lee Kilkelly Paulson & Kabaker","9581 E Arapahoe Rd","Rochester","Oakland","MI",48307,"248-357-8718","248-811-5696","dulce_labreche@yahoo.com","http://www.leekilkellypaulsonkabaker.com"
"Kate","Keneipp","Davis, Maxon R Esq","33 N Michigan Ave","Green Bay","Brown","WI",54301,"920-353-6377","920-355-1610","kate_keneipp@yahoo.com","http://www.davismaxonresq.com"
"Kaitlyn","Ogg","Garrison, Paul E Esq","2 S Biscayne Blvd","Baltimore","Baltimore City","MD",21230,"410-665-4903","410-773-3862","kaitlyn.ogg@gmail.com","http://www.garrisonpauleesq.com"
"Sherita","Saras","Black History Resource Center","8 Us Highway 22","Colorado Springs","El Paso","CO",80937,"719-669-1664","719-547-9543","sherita.saras@cox.
 net","http://www.blackhistoryresourcecenter.com"
"Lashawnda","Stuer","Rodriguez, J Christopher Esq","7422 Martin Ave #8","Toledo","Lucas","OH",43607,"419-588-8719","419-399-1744","lstuer@cox.net","http://www.rodriguezjchristopheresq.com"
"Ernest","Syrop","Grant Family Health Center","94 Chase Rd","Hyattsville","Prince Georges","MD",20785,"301-998-9644","301-257-4883","ernest@cox.net","http://www.grantfamilyhealthcenter.com"
"Nobuko","Halsey","Goeman Wood Products Inc","8139 I Hwy 10 #92","New Bedford","Bristol","MA","02745","508-855-9887","508-897-7916","nobuko.halsey@yahoo.com","http://www.goemanwoodproductsinc.com"
"Lavonna","Wolny","Linhares, Kenneth A Esq","5 Cabot Rd","Mc Lean","Fairfax","VA",22102,"703-483-1970","703-892-2914","lavonna.wolny@hotmail.com","http://www.linhareskennethaesq.com"
"Lashaunda","Lizama","Earnhardt Printing","3387 Ryan Dr","Hanover","Anne Arundel","MD",21076,"410-678-2473","410-912-6032","llizama@cox.net","http://www.earnhardtprinting.com"
"Mariann","Bi
 lden","H P G Industrys Inc","3125 Packer Ave #9851","Austin","Travis","TX",78753,"512-223-4791","512-742-1149","mariann.bilden@aol.com","http://www.hpgindustrysinc.com"
"Helene","Rodenberger","Bailey Transportation Prod Inc","347 Chestnut St","Peoria","Maricopa","AZ",85381,"623-461-8551","623-426-4907","helene@aol.com","http://www.baileytransportationprodinc.com"
"Roselle","Estell","Mcglynn Bliss Pc","8116 Mount Vernon Ave","Bucyrus","Crawford","OH",44820,"419-571-5920","419-488-6648","roselle.estell@hotmail.com","http://www.mcglynnblisspc.com"
"Samira","Heintzman","Mutual Fish Co","8772 Old County Rd #5410","Kent","King","WA",98032,"206-311-4137","206-923-6042","sheintzman@hotmail.com","http://www.mutualfishco.com"
"Margart","Meisel","Yeates, Arthur L Aia","868 State St #38","Cincinnati","Hamilton","OH",45251,"513-617-2362","513-747-9603","margart_meisel@yahoo.com","http://www.yeatesarthurlaia.com"
"Kristofer","Bennick","Logan, Ronald J Esq","772 W River Dr","Bloomington","Monroe",
 "IN",47404,"812-368-1511","812-442-8544","kristofer.bennick@yahoo.com","http://www.loganronaldjesq.com"
"Weldon","Acuff","Advantage Martgage Company","73 W Barstow Ave","Arlington Heights","Cook","IL",60004,"847-353-2156","847-613-5866","wacuff@gmail.com","http://www.advantagemartgagecompany.com"
"Shalon","Shadrick","Germer And Gertz Llp","61047 Mayfield Ave","Brooklyn","Kings","NY",11223,"718-232-2337","718-394-4974","shalon@cox.net","http://www.germerandgertzllp.com"
"Denise","Patak","Spence Law Offices","2139 Santa Rosa Ave","Orlando","Orange","FL",32801,"407-446-4358","407-808-3254","denise@patak.org","http://www.spencelawoffices.com"
"Louvenia","Beech","John Ortiz Nts Therapy Center","598 43rd St","Beverly Hills","Los Angeles","CA",90210,"310-820-2117","310-652-2379","louvenia.beech@beech.com","http://www.johnortizntstherapycenter.com"
"Audry","Yaw","Mike Uchrin Htg & Air Cond Inc","70295 Pioneer Ct","Brandon","Hillsborough","FL",33511,"813-797-4816","813-744-7100","audry.yaw@y
 aw.org","http://www.mikeuchrinhtgaircondinc.com"
"Kristel","Ehmann","Mccoy, Joy Reynolds Esq","92899 Kalakaua Ave","El Paso","El Paso","TX",79925,"915-452-1290","915-300-6100","kristel.ehmann@aol.com","http://www.mccoyjoyreynoldsesq.com"
"Vincenza","Zepp","Kbor 1600 Am","395 S 6th St #2","El Cajon","San Diego","CA",92020,"619-603-5125","619-935-6661","vzepp@gmail.com","http://www.kboram.com"
"Elouise","Gwalthney","Quality Inn Northwest","9506 Edgemore Ave","Bladensburg","Prince Georges","MD",20710,"301-841-5012","301-591-3034","egwalthney@yahoo.com","http://www.qualityinnnorthwest.com"
"Venita","Maillard","Wallace Church Assoc Inc","72119 S Walker Ave #63","Anaheim","Orange","CA",92801,"714-523-6653","714-663-9740","venita_maillard@gmail.com","http://www.wallacechurchassocinc.com"
"Kasandra","Semidey","Can Tron","369 Latham St #500","Saint Louis","Saint Louis City","MO",63102,"314-732-9131","314-697-3652","kasandra_semidey@semidey.com","http://www.cantron.com"
"Xochitl","Discipio","
 Ravaal Enterprises Inc","3158 Runamuck Pl","Round Rock","Williamson","TX",78664,"512-233-1831","512-942-3411","xdiscipio@gmail.com","http://www.ravaalenterprisesinc.com"
"Maile","Linahan","Thompson Steel Company Inc","9 Plainsboro Rd #598","Greensboro","Guilford","NC",27409,"336-670-2640","336-364-6037","mlinahan@yahoo.com","http://www.thompsonsteelcompanyinc.com"
"Krissy","Rauser","Anderson, Mark A Esq","8728 S Broad St","Coram","Suffolk","NY",11727,"631-443-4710","631-288-2866","krauser@cox.net","http://www.andersonmarkaesq.com"
"Pete","Dubaldi","Womack & Galich","2215 Prosperity Dr","Lyndhurst","Bergen","NJ","07071","201-825-2514","201-749-8866","pdubaldi@hotmail.com","http://www.womackgalich.com"
"Linn","Paa","Valerie & Company","1 S Pine St","Memphis","Shelby","TN",38112,"901-412-4381","901-573-9024","linn_paa@paa.com","http://www.valeriecompany.com"
"Paris","Wide","Gehring Pumps Inc","187 Market St","Atlanta","Fulton","GA",30342,"404-505-4445","404-607-8435","paris@hotmail.com
 ","http://www.gehringpumpsinc.com"
"Wynell","Dorshorst","Haehnel, Craig W Esq","94290 S Buchanan St","Pacifica","San Mateo","CA",94044,"650-473-1262","650-749-9879","wynell_dorshorst@dorshorst.org","http://www.haehnelcraigwesq.com"
"Quentin","Birkner","Spoor Behrins Campbell & Young","7061 N 2nd St","Burnsville","Dakota","MN",55337,"952-702-7993","952-314-5871","qbirkner@aol.com","http://www.spoorbehrinscampbellyoung.com"
"Regenia","Kannady","Ken Jeter Store Equipment Inc","10759 Main St","Scottsdale","Maricopa","AZ",85260,"480-726-1280","480-205-5121","regenia.kannady@cox.net","http://www.kenjeterstoreequipmentinc.com"
"Sheron","Louissant","Potter, Brenda J Cpa","97 E 3rd St #9","Long Island City","Queens","NY",11101,"718-976-8610","718-613-9994","sheron@aol.com","http://www.potterbrendajcpa.com"
"Izetta","Funnell","Baird Kurtz & Dobson","82 Winsor St #54","Atlanta","Dekalb","GA",30340,"770-844-3447","770-584-4119","izetta.funnell@hotmail.com","http://www.bairdkurtzdobson.com"
"Rod
 olfo","Butzen","Minor, Cynthia A Esq","41 Steel Ct","Northfield","Rice","MN",55057,"507-210-3510","507-590-5237","rodolfo@hotmail.com","http://www.minorcynthiaaesq.com"
"Zona","Colla","Solove, Robert A Esq","49440 Dearborn St","Norwalk","Fairfield","CT","06854","203-461-1949","203-938-2557","zona@hotmail.com","http://www.soloverobertaesq.com"
"Serina","Zagen","Mark Ii Imports Inc","7 S Beverly Dr","Fort Wayne","Allen","IN",46802,"260-273-3725","260-382-4869","szagen@aol.com","http://www.markiiimportsinc.com"
"Paz","Sahagun","White Sign Div Ctrl Equip Co","919 Wall Blvd","Meridian","Lauderdale","MS",39307,"601-927-8287","601-249-4511","paz_sahagun@cox.net","http://www.whitesigndivctrlequipco.com"
"Markus","Lukasik","M & M Store Fixtures Co Inc","89 20th St E #779","Sterling Heights","Macomb","MI",48310,"586-970-7380","586-247-1614","markus@yahoo.com","http://www.mmstorefixturescoinc.com"
"Jaclyn","Bachman","Judah Caster & Wheel Co","721 Interstate 45 S","Colorado Springs","El Paso","
 CO",80919,"719-853-3600","719-223-2074","jaclyn@aol.com","http://www.judahcasterwheelco.com"
"Cyril","Daufeldt","Galaxy International Inc","3 Lawton St","New York","New York","NY",10013,"212-745-8484","212-422-5427","cyril_daufeldt@daufeldt.com","http://www.galaxyinternationalinc.com"
"Gayla","Schnitzler","Sigma Corp Of America","38 Pleasant Hill Rd","Hayward","Alameda","CA",94545,"510-686-3407","510-441-4055","gschnitzler@gmail.com","http://www.sigmacorpofamerica.com"
"Erick","Nievas","Soward, Anne Esq","45 E Acacia Ct","Chicago","Cook","IL",60624,"773-704-9903","773-359-6109","erick_nievas@aol.com","http://www.sowardanneesq.com"
"Jennie","Drymon","Osborne, Michelle M Esq","63728 Poway Rd #1","Scranton","Lackawanna","PA",18509,"570-218-4831","570-868-8688","jennie@cox.net","http://www.osbornemichellemesq.com"
"Mitsue","Scipione","Students In Free Entrprs Natl","77 222 Dr","Oroville","Butte","CA",95965,"530-986-9272","530-399-3254","mscipione@scipione.com","http://www.studentsinfree
 entrprsnatl.com"
"Ciara","Ventura","Johnson, Robert M Esq","53 W Carey St","Port Jervis","Orange","NY",12771,"845-823-8877","845-694-7919","cventura@yahoo.com","http://www.johnsonrobertmesq.com"
"Galen","Cantres","Del Charro Apartments","617 Nw 36th Ave","Brook Park","Cuyahoga","OH",44142,"216-600-6111","216-871-6876","galen@yahoo.com","http://www.delcharroapartments.com"
"Truman","Feichtner","Legal Search Inc","539 Coldwater Canyon Ave","Bloomfield","Essex","NJ","07003","973-852-2736","973-473-5108","tfeichtner@yahoo.com","http://www.legalsearchinc.com"
"Gail","Kitty","Service Supply Co Inc","735 Crawford Dr","Anchorage","Anchorage","AK",99501,"907-435-9166","907-770-3542","gail@kitty.com","http://www.servicesupplycoinc.com"
"Dalene","Schoeneck","Sameshima, Douglas J Esq","910 Rahway Ave","Philadelphia","Philadelphia","PA",19102,"215-268-1275","215-380-8820","dalene@schoeneck.org","http://www.sameshimadouglasjesq.com"
"Gertude","Witten","Thompson, John Randolph Jr","7 Tarrytown Rd"
 ,"Cincinnati","Hamilton","OH",45217,"513-977-7043","513-863-9471","gertude.witten@gmail.com","http://www.thompsonjohnrandolphjr.com"
"Lizbeth","Kohl","E T Balancing Co Inc","35433 Blake St #588","Gardena","Los Angeles","CA",90248,"310-699-1222","310-955-5788","lizbeth@yahoo.com","http://www.etbalancingcoinc.com"
"Glenn","Berray","Griswold, John E Esq","29 Cherry St #7073","Des Moines","Polk","IA",50315,"515-370-7348","515-372-1738","gberray@gmail.com","http://www.griswoldjohneesq.com"
"Lashandra","Klang","Acqua Group","810 N La Brea Ave","King of Prussia","Montgomery","PA",19406,"610-809-1818","610-378-7332","lashandra@yahoo.com","http://www.acquagroup.com"
"Lenna","Newville","Brooks, Morris J Jr","987 Main St","Raleigh","Wake","NC",27601,"919-623-2524","919-254-5987","lnewville@newville.com","http://www.brooksmorrisjjr.com"
"Laurel","Pagliuca","Printing Images Corp","36 Enterprise St Se","Richland","Benton","WA",99352,"509-695-5199","509-595-6485","laurel@yahoo.com","http://www.pri
 ntingimagescorp.com"
"Mireya","Frerking","Roberts Supply Co Inc","8429 Miller Rd","Pelham","Westchester","NY",10803,"914-868-5965","914-883-3061","mireya.frerking@hotmail.com","http://www.robertssupplycoinc.com"
"Annelle","Tagala","Vico Products Mfg Co","5 W 7th St","Parkville","Baltimore","MD",21234,"410-757-1035","410-234-2267","annelle@yahoo.com","http://www.vicoproductsmfgco.com"
"Dean","Ketelsen","J M Custom Design Millwork","2 Flynn Rd","Hicksville","Nassau","NY",11801,"516-847-4418","516-732-6649","dean_ketelsen@gmail.com","http://www.jmcustomdesignmillwork.com"
"Levi","Munis","Farrell & Johnson Office Equip","2094 Ne 36th Ave","Worcester","Worcester","MA","01603","508-456-4907","508-658-7802","levi.munis@gmail.com","http://www.farrelljohnsonofficeequip.com"
"Sylvie","Ryser","Millers Market & Deli","649 Tulane Ave","Tulsa","Tulsa","OK",74105,"918-644-9555","918-565-1706","sylvie@aol.com","http://www.millersmarketdeli.com"
"Sharee","Maile","Holiday Inn Naperville","2094 Montou
 r Blvd","Muskegon","Muskegon","MI",49442,"231-467-9978","231-265-6940","sharee_maile@aol.com","http://www.holidayinnnaperville.com"
"Cordelia","Storment","Burrows, Jon H Esq","393 Hammond Dr","Lafayette","Lafayette","LA",70506,"337-566-6001","337-255-3427","cordelia_storment@aol.com","http://www.burrowsjonhesq.com"
"Mollie","Mcdoniel","Dock Seal Specialty","8590 Lake Lizzie Dr","Bowling Green","Wood","OH",43402,"419-975-3182","419-417-4674","mollie_mcdoniel@yahoo.com","http://www.docksealspecialty.com"
"Brett","Mccullan","Five Star Limousines Of Tx Inc","87895 Concord Rd","La Mesa","San Diego","CA",91942,"619-461-9984","619-727-3892","brett.mccullan@mccullan.com","http://www.fivestarlimousinesoftxinc.com"
"Teddy","Pedrozo","Barkan, Neal J Esq","46314 Route 130","Bridgeport","Fairfield","CT","06610","203-892-3863","203-918-3939","teddy_pedrozo@aol.com","http://www.barkannealjesq.com"
"Tasia","Andreason","Campbell, Robert A","4 Cowesett Ave","Kearny","Hudson","NJ","07032","201-920-900
 2","201-969-7063","tasia_andreason@yahoo.com","http://www.campbellroberta.com"
"Hubert","Walthall","Dee, Deanna","95 Main Ave #2","Barberton","Summit","OH",44203,"330-903-1345","330-566-8898","hubert@walthall.org","http://www.deedeanna.com"
"Arthur","Farrow","Young, Timothy L Esq","28 S 7th St #2824","Englewood","Bergen","NJ","07631","201-238-5688","201-772-4377","arthur.farrow@yahoo.com","http://www.youngtimothylesq.com"
"Vilma","Berlanga","Wells, D Fred Esq","79 S Howell Ave","Grand Rapids","Kent","MI",49546,"616-737-3085","616-568-4113","vberlanga@berlanga.com","http://www.wellsdfredesq.com"
"Billye","Miro","Gray, Francine H Esq","36 Lancaster Dr Se","Pearl","Rankin","MS",39208,"601-567-5386","601-637-5479","billye_miro@cox.net","http://www.grayfrancinehesq.com"
"Glenna","Slayton","Toledo Iv Care","2759 Livingston Ave","Memphis","Shelby","TN",38118,"901-640-9178","901-869-4314","glenna_slayton@cox.net","http://www.toledoivcare.com"
"Mitzie","Hudnall","Cangro Transmission Co","17 
 Jersey Ave","Englewood","Arapahoe","CO",80110,"303-402-1940","303-997-7760","mitzie_hudnall@yahoo.com","http://www.cangrotransmissionco.com"
"Bernardine","Rodefer","Sat Poly Inc","2 W Grand Ave","Memphis","Shelby","TN",38112,"901-901-4726","901-739-5892","bernardine_rodefer@yahoo.com","http://www.satpolyinc.com"
"Staci","Schmaltz","Midwest Contracting & Mfg Inc","18 Coronado Ave #563","Pasadena","Los Angeles","CA",91106,"626-866-2339","626-293-7678","staci_schmaltz@aol.com","http://www.midwestcontractingmfginc.com"
"Nichelle","Meteer","Print Doctor","72 Beechwood Ter","Chicago","Cook","IL",60657,"773-225-9985","773-857-2231","nichelle_meteer@meteer.com","http://www.printdoctor.com"
"Janine","Rhoden","Nordic Group Inc","92 Broadway","Astoria","Queens","NY",11103,"718-228-5894","718-728-5051","jrhoden@yahoo.com","http://www.nordicgroupinc.com"
"Ettie","Hoopengardner","Jackson Millwork Co","39 Franklin Ave","Richland","Benton","WA",99352,"509-755-5393","509-847-3352","ettie.hoopengardn
 er@hotmail.com","http://www.jacksonmillworkco.com"
"Eden","Jayson","Harris Corporation","4 Iwaena St","Baltimore","Baltimore City","MD",21202,"410-890-7866","410-429-4888","eden_jayson@yahoo.com","http://www.harriscorporation.com"
"Lynelle","Auber","United Cerebral Palsy Of Ne Pa","32820 Corkwood Rd","Newark","Essex","NJ","07104","973-860-8610","973-605-6492","lynelle_auber@gmail.com","http://www.unitedcerebralpalsyofnepa.com"
"Merissa","Tomblin","One Day Surgery Center Inc","34 Raritan Center Pky","Bellflower","Los Angeles","CA",90706,"562-579-6900","562-719-7922","merissa.tomblin@gmail.com","http://www.onedaysurgerycenterinc.com"
"Golda","Kaniecki","Calaveras Prospect","6201 S Nevada Ave","Toms River","Ocean","NJ","08755","732-628-9909","732-617-5310","golda_kaniecki@yahoo.com","http://www.calaverasprospect.com"
"Catarina","Gleich","Terk, Robert E Esq","78 Maryland Dr #146","Denville","Morris","NJ","07834","973-210-3994","973-491-8723","catarina_gleich@hotmail.com","http://www.ter
 kroberteesq.com"
"Virgie","Kiel","Cullen, Terrence P Esq","76598 Rd  I 95 #1","Denver","Denver","CO",80216,"303-776-7548","303-845-5408","vkiel@hotmail.com","http://www.cullenterrencepesq.com"
"Jolene","Ostolaza","Central Die Casting Mfg Co Inc","1610 14th St Nw","Newport News","Newport News City","VA",23608,"757-682-7116","757-940-1741","jolene@yahoo.com","http://www.centraldiecastingmfgcoinc.com"
"Keneth","Borgman","Centerline Engineering","86350 Roszel Rd","Phoenix","Maricopa","AZ",85012,"602-919-4211","602-442-3092","keneth@yahoo.com","http://www.centerlineengineering.com"
"Rikki","Nayar","Targan & Kievit Pa","1644 Clove Rd","Miami","Miami-Dade","FL",33155,"305-968-9487","305-978-2069","rikki@nayar.com","http://www.targankievitpa.com"
"Elke","Sengbusch","Riley Riper Hollin & Colagreco","9 W Central Ave","Phoenix","Maricopa","AZ",85013,"602-896-2993","602-575-3457","elke_sengbusch@yahoo.com","http://www.rileyriperhollincolagreco.com"
"Hoa","Sarao","Kaplan, Joel S Esq","27846 Lafa
 yette Ave","Oak Hill","Volusia","FL",32759,"386-526-7800","386-599-7296","hoa@sarao.org","http://www.kaplanjoelsesq.com"
"Trinidad","Mcrae","Water Office","10276 Brooks St","San Francisco","San Francisco","CA",94105,"415-331-9634","415-419-1597","trinidad_mcrae@yahoo.com","http://www.wateroffice.com"
"Mari","Lueckenbach","Westbrooks, Nelson E Jr","1 Century Park E","San Diego","San Diego","CA",92110,"858-793-9684","858-228-5683","mari_lueckenbach@yahoo.com","http://www.westbrooksnelsonejr.com"
"Selma","Husser","Armon Communications","9 State Highway 57 #22","Jersey City","Hudson","NJ","07306","201-991-8369","201-772-7699","selma.husser@cox.net","http://www.armoncommunications.com"
"Antione","Onofrio","Jacobs & Gerber Inc","4 S Washington Ave","San Bernardino","San Bernardino","CA",92410,"909-430-7765","909-665-3223","aonofrio@onofrio.com","http://www.jacobsgerberinc.com"
"Luisa","Jurney","Forest Fire Laboratory","25 Se 176th Pl","Cambridge","Middlesex","MA","02138","617-365-2134","6
 17-544-2541","ljurney@hotmail.com","http://www.forestfirelaboratory.com"
"Clorinda","Heimann","Haughey, Charles Jr","105 Richmond Valley Rd","Escondido","San Diego","CA",92025,"760-291-5497","760-261-4786","clorinda.heimann@hotmail.com","http://www.haugheycharlesjr.com"
"Dick","Wenzinger","Wheaton Plastic Products","22 Spruce St #595","Gardena","Los Angeles","CA",90248,"310-510-9713","310-936-2258","dick@yahoo.com","http://www.wheatonplasticproducts.com"
"Ahmed","Angalich","Reese Plastics","2 W Beverly Blvd","Harrisburg","Dauphin","PA",17110,"717-528-8996","717-632-5831","ahmed.angalich@angalich.com","http://www.reeseplastics.com"
"Iluminada","Ohms","Nazette Marner Good Wendt","72 Southern Blvd","Mesa","Maricopa","AZ",85204,"480-293-2882","480-866-6544","iluminada.ohms@yahoo.com","http://www.nazettemarnergoodwendt.com"
"Joanna","Leinenbach","Levinson Axelrod Wheaton","1 Washington St","Lake Worth","Palm Beach","FL",33461,"561-470-4574","561-951-9734","joanna_leinenbach@hotmail.com",
 "http://www.levinsonaxelrodwheaton.com"
"Caprice","Suell","Egnor, W Dan Esq","90177 N 55th Ave","Nashville","Davidson","TN",37211,"615-246-1824","615-726-4537","caprice@aol.com","http://www.egnorwdanesq.com"
"Stephane","Myricks","Portland Central Thriftlodge","9 Tower Ave","Burlington","Boone","KY",41005,"859-717-7638","859-308-4286","stephane_myricks@cox.net","http://www.portlandcentralthriftlodge.com"
"Quentin","Swayze","Ulbrich Trucking","278 Bayview Ave","Milan","Monroe","MI",48160,"734-561-6170","734-851-8571","quentin_swayze@yahoo.com","http://www.ulbrichtrucking.com"
"Annmarie","Castros","Tipiak Inc","80312 W 32nd St","Conroe","Montgomery","TX",77301,"936-751-7961","936-937-2334","annmarie_castros@gmail.com","http://www.tipiakinc.com"
"Shonda","Greenbush","Saint George Well Drilling","82 Us Highway 46","Clifton","Passaic","NJ","07011","973-482-2430","973-644-2974","shonda_greenbush@cox.net","http://www.saintgeorgewelldrilling.com"
"Cecil","Lapage","Hawkes, Douglas D","4 Stova
 ll St #72","Union City","Hudson","NJ","07087","201-693-3967","201-856-2720","clapage@lapage.com","http://www.hawkesdouglasd.com"
"Jeanice","Claucherty","Accurel Systems Intrntl Corp","19 Amboy Ave","Miami","Miami-Dade","FL",33142,"305-988-4162","305-306-7834","jeanice.claucherty@yahoo.com","http://www.accurelsystemsintrntlcorp.com"
"Josphine","Villanueva","Santa Cruz Community Internet","63 Smith Ln #8343","Moss","Clay","TN",38575,"931-553-9774","931-486-6946","josphine_villanueva@villanueva.com","http://www.santacruzcommunityinternet.com"
"Daniel","Perruzza","Gersh & Danielson","11360 S Halsted St","Santa Ana","Orange","CA",92705,"714-771-3880","714-531-1391","dperruzza@perruzza.com","http://www.gershdanielson.com"
"Cassi","Wildfong","Cobb, James O Esq","26849 Jefferson Hwy","Rolling Meadows","Cook","IL",60008,"847-633-3216","847-755-9041","cassi.wildfong@aol.com","http://www.cobbjamesoesq.com"
"Britt","Galam","Wheatley Trucking Company","2500 Pringle Rd Se #508","Hatfield","Montgo
 mery","PA",19440,"215-888-3304","215-351-8523","britt@galam.org","http://www.wheatleytruckingcompany.com"
"Adell","Lipkin","Systems Graph Inc Ab Dick Dlr","65 Mountain View Dr","Whippany","Morris","NJ","07981","973-654-1561","973-662-8988","adell.lipkin@lipkin.com","http://www.systemsgraphincabdickdlr.com"
"Jacqueline","Rowling","John Hancock Mutl Life Ins Co","1 N San Saba","Erie","Erie","PA",16501,"814-865-8113","814-481-1700","jacqueline.rowling@yahoo.com","http://www.johnhancockmutllifeinsco.com"
"Lonny","Weglarz","History Division Of State","51120 State Route 18","Salt Lake City","Salt Lake","UT",84115,"801-293-9853","801-892-8781","lonny_weglarz@gmail.com","http://www.historydivisionofstate.com"
"Lonna","Diestel","Dimmock, Thomas J Esq","1482 College Ave","Fayetteville","Cumberland","NC",28301,"910-922-3672","910-200-7912","lonna_diestel@gmail.com","http://www.dimmockthomasjesq.com"
"Cristal","Samara","Intermed Inc","4119 Metropolitan Dr","Los Angeles","Los Angeles","CA",90021
 ,"213-975-8026","213-696-8004","cristal@cox.net","http://www.intermedinc.com"
"Kenneth","Grenet","Bank Of New York","2167 Sierra Rd","East Lansing","Ingham","MI",48823,"517-499-2322","517-867-8077","kenneth.grenet@grenet.org","http://www.bankofnewyork.com"
"Elli","Mclaird","Sportmaster Intrnatl","6 Sunrise Ave","Utica","Oneida","NY",13501,"315-818-2638","315-474-5570","emclaird@mclaird.com","http://www.sportmasterintrnatl.com"
"Alline","Jeanty","W W John Holden Inc","55713 Lake City Hwy","South Bend","St Joseph","IN",46601,"574-656-2800","574-405-1983","ajeanty@gmail.com","http://www.wwjohnholdeninc.com"
"Sharika","Eanes","Maccani & Delp","75698 N Fiesta Blvd","Orlando","Orange","FL",32806,"407-312-1691","407-472-1332","sharika.eanes@aol.com","http://www.maccanidelp.com"
"Nu","Mcnease","Amazonia Film Project","88 Sw 28th Ter","Harrison","Hudson","NJ","07029","973-751-9003","973-903-4175","nu@gmail.com","http://www.amazoniafilmproject.com"
"Daniela","Comnick","Water & Sewer Departmen
 t","7 Flowers Rd #403","Trenton","Mercer","NJ","08611","609-200-8577","609-398-2805","dcomnick@cox.net","http://www.watersewerdepartment.com"
"Cecilia","Colaizzo","Switchcraft Inc","4 Nw 12th St #3849","Madison","Dane","WI",53717,"608-382-4541","608-302-3387","cecilia_colaizzo@colaizzo.com","http://www.switchcraftinc.com"
"Leslie","Threets","C W D C Metal Fabricators","2 A Kelley Dr","Katonah","Westchester","NY",10536,"914-861-9748","914-396-2615","leslie@cox.net","http://www.cwdcmetalfabricators.com"
"Nan","Koppinger","Shimotani, Grace T","88827 Frankford Ave","Greensboro","Guilford","NC",27401,"336-370-5333","336-564-1492","nan@koppinger.com","http://www.shimotanigracet.com"
"Izetta","Dewar","Lisatoni, Jean Esq","2 W Scyene Rd #3","Baltimore","Baltimore City","MD",21217,"410-473-1708","410-522-7621","idewar@dewar.com","http://www.lisatonijeanesq.com"
"Tegan","Arceo","Ceramic Tile Sales Inc","62260 Park Stre","Monroe Township","Middlesex","NJ","08831","732-730-2692","732-705-6719",
 "tegan.arceo@arceo.org","http://www.ceramictilesalesinc.com"
"Ruthann","Keener","Maiden Craft Inc","3424 29th St Se","Kerrville","Kerr","TX",78028,"830-258-2769","830-919-5991","ruthann@hotmail.com","http://www.maidencraftinc.com"
"Joni","Breland","Carriage House Cllsn Rpr Inc","35 E Main St #43","Elk Grove Village","Cook","IL",60007,"847-519-5906","847-740-5304","joni_breland@cox.net","http://www.carriagehousecllsnrprinc.com"
"Vi","Rentfro","Video Workshop","7163 W Clark Rd","Freehold","Monmouth","NJ","07728","732-605-4781","732-724-7251","vrentfro@cox.net","http://www.videoworkshop.com"
"Colette","Kardas","Fresno Tile Center Inc","21575 S Apple Creek Rd","Omaha","Douglas","NE",68124,"402-896-5943","402-707-1602","colette.kardas@yahoo.com","http://www.fresnotilecenterinc.com"
"Malcolm","Tromblay","Versatile Sash & Woodwork","747 Leonis Blvd","Annandale","Fairfax","VA",22003,"703-221-5602","703-874-4248","malcolm_tromblay@cox.net","http://www.versatilesashwoodwork.com"
"Ryan","Harno
 s","Warner Electric Brk & Cltch Co","13 Gunnison St","Plano","Collin","TX",75075,"972-558-1665","972-961-4968","ryan@cox.net","http://www.warnerelectricbrkcltchco.com"
"Jess","Chaffins","New York Public Library","18 3rd Ave","New York","New York","NY",10016,"212-510-4633","212-428-9538","jess.chaffins@chaffins.org","http://www.newyorkpubliclibrary.com"
"Sharen","Bourbon","Mccaleb, John A Esq","62 W Austin St","Syosset","Nassau","NY",11791,"516-816-1541","516-749-3188","sbourbon@yahoo.com","http://www.mccalebjohnaesq.com"
"Nickolas","Juvera","United Oil Co Inc","177 S Rider Trl #52","Crystal River","Citrus","FL",34429,"352-598-8301","352-947-6152","nickolas_juvera@cox.net","http://www.unitedoilcoinc.com"
"Gary","Nunlee","Irving Foot Center","2 W Mount Royal Ave","Fortville","Hancock","IN",46040,"317-542-6023","317-887-8486","gary_nunlee@nunlee.org","http://www.irvingfootcenter.com"
"Diane","Devreese","Acme Supply Co","1953 Telegraph Rd","Saint Joseph","Buchanan","MO",64504,"816-557-9
 673","816-329-5565","diane@cox.net","http://www.acmesupplyco.com"
"Roslyn","Chavous","Mcrae, James L","63517 Dupont St","Jackson","Hinds","MS",39211,"601-234-9632","601-973-5754","roslyn.chavous@chavous.org","http://www.mcraejamesl.com"
"Glory","Schieler","Mcgraths Seafood","5 E Truman Rd","Abilene","Taylor","TX",79602,"325-869-2649","325-740-3778","glory@yahoo.com","http://www.mcgrathsseafood.com"
"Rasheeda","Sayaphon","Kummerer, J Michael Esq","251 Park Ave #979","Saratoga","Santa Clara","CA",95070,"408-805-4309","408-997-7490","rasheeda@aol.com","http://www.kummererjmichaelesq.com"
"Alpha","Palaia","Stoffer, James M Jr","43496 Commercial Dr #29","Cherry Hill","Camden","NJ","08003","856-312-2629","856-513-7024","alpha@yahoo.com","http://www.stofferjamesmjr.com"
"Refugia","Jacobos","North Central Fl Sfty Cncl","2184 Worth St","Hayward","Alameda","CA",94545,"510-974-8671","510-509-3496","refugia.jacobos@jacobos.com","http://www.northcentralflsftycncl.com"
"Shawnda","Yori","Fiorucci 
 Foods Usa Inc","50126 N Plankinton Ave","Longwood","Seminole","FL",32750,"407-53

<TRUNCATED>

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

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 79c602d..e0eb813 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
@@ -23,6 +23,8 @@ 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.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.WriteResult;
 import org.apache.nifi.serialization.record.DataType;
@@ -31,14 +33,14 @@ import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
 import org.apache.nifi.stream.io.NonCloseableOutputStream;
 
-import au.com.bytecode.opencsv.CSVWriter;
-
 public class WriteCSVResult implements RecordSetWriter {
+    private final CSVFormat csvFormat;
     private final String dateFormat;
     private final String timeFormat;
     private final String timestampFormat;
 
-    public WriteCSVResult(final String dateFormat, final String timeFormat, final String timestampFormat) {
+    public WriteCSVResult(final CSVFormat csvFormat, final String dateFormat, final String timeFormat, final String timestampFormat) {
+        this.csvFormat = csvFormat;
         this.dateFormat = dateFormat;
         this.timeFormat = timeFormat;
         this.timestampFormat = timestampFormat;
@@ -66,24 +68,25 @@ public class WriteCSVResult implements RecordSetWriter {
     @Override
     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);
+
         try (final OutputStream nonCloseable = new NonCloseableOutputStream(rawOut);
             final OutputStreamWriter streamWriter = new OutputStreamWriter(nonCloseable);
-            final CSVWriter writer = new CSVWriter(streamWriter)) {
+            final CSVPrinter printer = new CSVPrinter(streamWriter, formatWithHeader)) {
 
             try {
-                final RecordSchema schema = rs.getSchema();
-                final String[] columnNames = schema.getFieldNames().toArray(new String[0]);
-                writer.writeNext(columnNames);
-
                 Record record;
                 while ((record = rs.next()) != null) {
-                    final String[] colVals = new String[schema.getFieldCount()];
+                    final Object[] colVals = new Object[schema.getFieldCount()];
                     int i = 0;
                     for (final String fieldName : schema.getFieldNames()) {
                         colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName));
                     }
 
-                    writer.writeNext(colVals);
+                    printer.printRecord(colVals);
                     count++;
                 }
             } catch (final Exception e) {
@@ -96,22 +99,20 @@ public class WriteCSVResult implements RecordSetWriter {
 
     @Override
     public WriteResult write(final Record record, final OutputStream rawOut) throws IOException {
+
         try (final OutputStream nonCloseable = new NonCloseableOutputStream(rawOut);
             final OutputStreamWriter streamWriter = new OutputStreamWriter(nonCloseable);
-            final CSVWriter writer = new CSVWriter(streamWriter)) {
+            final CSVPrinter printer = new CSVPrinter(streamWriter, csvFormat)) {
 
             try {
                 final RecordSchema schema = record.getSchema();
-                final String[] columnNames = schema.getFieldNames().toArray(new String[0]);
-                writer.writeNext(columnNames);
-
-                final String[] colVals = new String[schema.getFieldCount()];
+                final Object[] colVals = new Object[schema.getFieldCount()];
                 int i = 0;
                 for (final String fieldName : schema.getFieldNames()) {
                     colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName));
                 }
 
-                writer.writeNext(colVals);
+                printer.printRecord(colVals);
             } catch (final Exception e) {
                 throw new IOException("Failed to serialize results", e);
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 f72d5d5..f444b8a 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
@@ -29,11 +29,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.processor.util.StandardValidators;
 import org.apache.nifi.serialization.RecordReader;
 import org.apache.nifi.serialization.RowRecordReaderFactory;
-import org.apache.nifi.serialization.UserTypeOverrideRowReader;
+import org.apache.nifi.serialization.SchemaRegistryRecordReader;
+import org.apache.nifi.serialization.record.RecordSchema;
 
 import io.thekraken.grok.api.Grok;
 import io.thekraken.grok.api.exception.GrokException;
@@ -45,9 +47,11 @@ import io.thekraken.grok.api.exception.GrokException;
     + "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.")
-public class GrokReader extends UserTypeOverrideRowReader implements RowRecordReaderFactory {
+    + "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 {
     private volatile Grok grok;
+    private volatile boolean useSchemaRegistry;
 
     private static final String DEFAULT_PATTERN_NAME = "/default-grok-patterns.txt";
 
@@ -60,6 +64,7 @@ public class GrokReader extends UserTypeOverrideRowReader implements RowRecordRe
         .expressionLanguageSupported(true)
         .required(false)
         .build();
+
     static final PropertyDescriptor GROK_EXPRESSION = new PropertyDescriptor.Builder()
         .name("Grok Expression")
         .description("Specifies the format of a log line in Grok format. This allows the Record Reader to understand how to parse each log line. "
@@ -70,7 +75,7 @@ public class GrokReader extends UserTypeOverrideRowReader implements RowRecordRe
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        final List<PropertyDescriptor> properties = new ArrayList<>();
+        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
         properties.add(PATTERN_FILE);
         properties.add(GROK_EXPRESSION);
         return properties;
@@ -86,14 +91,21 @@ public class GrokReader extends UserTypeOverrideRowReader implements RowRecordRe
         }
 
         if (context.getProperty(PATTERN_FILE).isSet()) {
-            grok.addPatternFromFile(context.getProperty(PATTERN_FILE).getValue());
+            grok.addPatternFromFile(context.getProperty(PATTERN_FILE).evaluateAttributeExpressions().getValue());
         }
 
         grok.compile(context.getProperty(GROK_EXPRESSION).getValue());
+        useSchemaRegistry = context.getProperty(OPTIONAL_SCHEMA_NAME).isSet() && context.getProperty(OPTIONAL_SCHEMA_REGISTRY).isSet();
+    }
+
+    @Override
+    protected boolean isSchemaRequired() {
+        return false;
     }
 
     @Override
-    public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException {
-        return new GrokRecordReader(in, grok, getFieldTypeOverrides());
+    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);
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 bdf12f9..458dbd8 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
@@ -50,10 +50,9 @@ import io.thekraken.grok.api.Match;
 public class GrokRecordReader implements RecordReader {
     private final BufferedReader reader;
     private final Grok grok;
-    private final Map<String, DataType> fieldTypeOverrides;
+    private RecordSchema schema;
 
     private String nextLine;
-    private RecordSchema schema;
 
     static final String STACK_TRACE_COLUMN_NAME = "STACK_TRACE";
     private static final Pattern STACK_TRACE_PATTERN = Pattern.compile(
@@ -74,10 +73,10 @@ public class GrokRecordReader implements RecordReader {
         TIME_FORMAT_TIMESTAMP = FastDateFormat.getInstance("yyyy-MM-dd HH:mm:ss", gmt);
     }
 
-    public GrokRecordReader(final InputStream in, final Grok grok, final Map<String, DataType> fieldTypeOverrides) {
+    public GrokRecordReader(final InputStream in, final Grok grok, final RecordSchema schema) {
         this.reader = new BufferedReader(new InputStreamReader(in));
         this.grok = grok;
-        this.fieldTypeOverrides = fieldTypeOverrides;
+        this.schema = schema;
     }
 
     @Override
@@ -210,46 +209,33 @@ public class GrokRecordReader implements RecordReader {
         if (fieldType == null) {
             return string;
         }
+
+        if (string == null) {
+            return null;
+        }
+
+        // If string is empty then return an empty string if field type is STRING. If field type is
+        // anything else, we can't really convert it so return null
+        if (string.isEmpty() && fieldType.getFieldType() != RecordFieldType.STRING) {
+            return null;
+        }
+
         switch (fieldType.getFieldType()) {
             case BOOLEAN:
-                if (string.length() == 0) {
-                    return null;
-                }
                 return Boolean.parseBoolean(string);
             case BYTE:
-                if (string.length() == 0) {
-                    return null;
-                }
                 return Byte.parseByte(string);
             case SHORT:
-                if (string.length() == 0) {
-                    return null;
-                }
                 return Short.parseShort(string);
             case INT:
-                if (string.length() == 0) {
-                    return null;
-                }
                 return Integer.parseInt(string);
             case LONG:
-                if (string.length() == 0) {
-                    return null;
-                }
                 return Long.parseLong(string);
             case FLOAT:
-                if (string.length() == 0) {
-                    return null;
-                }
                 return Float.parseFloat(string);
             case DOUBLE:
-                if (string.length() == 0) {
-                    return null;
-                }
                 return Double.parseDouble(string);
             case DATE:
-                if (string.length() == 0) {
-                    return null;
-                }
                 try {
                     Date date = TIME_FORMAT_DATE.parse(string);
                     return new java.sql.Date(date.getTime());
@@ -257,9 +243,6 @@ public class GrokRecordReader implements RecordReader {
                     return null;
                 }
             case TIME:
-                if (string.length() == 0) {
-                    return null;
-                }
                 try {
                     Date date = TIME_FORMAT_TIME.parse(string);
                     return new java.sql.Time(date.getTime());
@@ -267,9 +250,6 @@ public class GrokRecordReader implements RecordReader {
                     return null;
                 }
             case TIMESTAMP:
-                if (string.length() == 0) {
-                    return null;
-                }
                 try {
                     Date date = TIME_FORMAT_TIMESTAMP.parse(string);
                     return new java.sql.Timestamp(date.getTime());
@@ -298,11 +278,7 @@ public class GrokRecordReader implements RecordReader {
                 final Map<String, String> namedGroups = GrokUtils.namedGroups(matcher, grokExpression);
                 final String fieldName = namedGroups.get("subname");
 
-                DataType dataType = fieldTypeOverrides.get(fieldName);
-                if (dataType == null) {
-                    dataType = RecordFieldType.STRING.getDataType();
-                }
-
+                DataType dataType = RecordFieldType.STRING.getDataType();
                 final RecordField recordField = new RecordField(fieldName, dataType);
                 fields.add(recordField);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 286326a..ad04912 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,16 +19,11 @@ package org.apache.nifi.json;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
 import java.util.ArrayList;
-import java.util.Date;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.TimeZone;
 
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.serialization.MalformedRecordException;
@@ -45,7 +40,6 @@ import org.codehaus.jackson.JsonParseException;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.JsonToken;
 import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.node.ArrayNode;
 
 
 public abstract class AbstractJsonRowRecordReader implements RecordReader {
@@ -57,8 +51,6 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
 
     private boolean firstObjectConsumed = false;
 
-    private static final TimeZone gmt = TimeZone.getTimeZone("GMT");
-
 
     public AbstractJsonRowRecordReader(final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
         this.logger = logger;
@@ -136,7 +128,7 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
         }
 
         final RecordSchema childSchema = determineSchema(node);
-        return RecordFieldType.RECORD.getDataType(childSchema);
+        return RecordFieldType.RECORD.getRecordDataType(childSchema);
     }
 
     protected RecordSchema determineSchema(final JsonNode jsonNode) {
@@ -155,111 +147,31 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
         return new SimpleRecordSchema(recordFields);
     }
 
-    protected Object convertField(final JsonNode fieldNode, final String fieldName, final DataType desiredType) throws IOException, MalformedRecordException {
-        if (fieldNode == null || fieldNode.isNull()) {
+    protected Object getRawNodeValue(final JsonNode fieldNode) throws IOException {
+        if (fieldNode == null || !fieldNode.isValueNode()) {
             return null;
         }
 
-        switch (desiredType.getFieldType()) {
-            case BOOLEAN:
-                return fieldNode.asBoolean();
-            case BYTE:
-                return (byte) fieldNode.asInt();
-            case CHAR:
-                final String text = fieldNode.asText();
-                if (text.isEmpty()) {
-                    return null;
-                }
-                return text.charAt(0);
-            case DOUBLE:
-                return fieldNode.asDouble();
-            case FLOAT:
-                return (float) fieldNode.asDouble();
-            case INT:
-                return fieldNode.asInt();
-            case LONG:
-                return fieldNode.asLong();
-            case SHORT:
-                return (short) fieldNode.asInt();
-            case STRING:
-                return fieldNode.asText();
-            case DATE: {
-                final String string = fieldNode.asText();
-                if (string.isEmpty()) {
-                    return null;
-                }
-
-                try {
-                    final DateFormat dateFormat = new SimpleDateFormat(desiredType.getFormat());
-                    dateFormat.setTimeZone(gmt);
-                    final Date date = dateFormat.parse(string);
-                    return new java.sql.Date(date.getTime());
-                } catch (ParseException e) {
-                    logger.warn("Failed to convert JSON field to Date for field {} (value {})", new Object[] {fieldName, string, e});
-                    return null;
-                }
-            }
-            case TIME: {
-                final String string = fieldNode.asText();
-                if (string.isEmpty()) {
-                    return null;
-                }
-
-                try {
-                    final DateFormat dateFormat = new SimpleDateFormat(desiredType.getFormat());
-                    dateFormat.setTimeZone(gmt);
-                    final Date date = dateFormat.parse(string);
-                    return new java.sql.Date(date.getTime());
-                } catch (ParseException e) {
-                    logger.warn("Failed to convert JSON field to Time for field {} (value {})", new Object[] {fieldName, string, e});
-                    return null;
-                }
-            }
-            case TIMESTAMP: {
-                final String string = fieldNode.asText();
-                if (string.isEmpty()) {
-                    return null;
-                }
-
-                try {
-                    final DateFormat dateFormat = new SimpleDateFormat(desiredType.getFormat());
-                    dateFormat.setTimeZone(gmt);
-                    final Date date = dateFormat.parse(string);
-                    return new java.sql.Date(date.getTime());
-                } catch (ParseException e) {
-                    logger.warn("Failed to convert JSON field to Timestamp for field {} (value {})", new Object[] {fieldName, string, e});
-                    return null;
-                }
-            }
-            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 Object converted = convertField(node, fieldName, determineFieldType(node));
-                    arrayElements[count++] = converted;
-                }
-
-                return arrayElements;
-            }
-            case RECORD: {
-                if (fieldNode.isObject()) {
-                    final Optional<RecordSchema> childSchema = desiredType.getChildRecordSchema();
-                    if (!childSchema.isPresent()) {
-                        return null;
-                    }
-
-                    return convertJsonNodeToRecord(fieldNode, childSchema.get());
-                } else {
-                    return fieldNode.toString();
-                }
-            }
+        if (fieldNode.isNumber()) {
+            return fieldNode.getNumberValue();
+        }
+
+        if (fieldNode.isBinary()) {
+            return fieldNode.getBinaryValue();
+        }
+
+        if (fieldNode.isBoolean()) {
+            return fieldNode.getBooleanValue();
+        }
+
+        if (fieldNode.isTextual()) {
+            return fieldNode.getTextValue();
         }
 
-        return fieldNode.toString();
+        return null;
     }
 
+
     private JsonNode getNextJsonNode() throws JsonParseException, IOException, MalformedRecordException {
         if (!firstObjectConsumed) {
             firstObjectConsumed = true;
@@ -286,6 +198,7 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
         }
     }
 
+
     @Override
     public void close() throws IOException {
         jsonParser.close();

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 b43b1c1..467ecf8 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
@@ -19,12 +19,11 @@ package org.apache.nifi.json;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Optional;
+import java.util.List;
 
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -34,13 +33,15 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled;
 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.logging.ComponentLog;
+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.record.DataType;
+import org.apache.nifi.serialization.SchemaRegistryRecordReader;
+import org.apache.nifi.serialization.record.RecordSchema;
 
 import com.jayway.jsonpath.JsonPath;
 
@@ -50,16 +51,27 @@ import com.jayway.jsonpath.JsonPath;
     + "User-defined properties define the fields that should be extracted from the JSON in order to form the fields of a Record. Any JSON field "
     + "that is not extracted via a JSONPath will not be returned in the JSON Records.")
 @SeeAlso(JsonTreeReader.class)
-@DynamicProperty(name = "The field name for the record. If it is desirable to enforce that the value be coerced into a given type, its type can be included "
-    + "in the name by using a syntax of <field name>:<field type>. For example, \"balance:double\".",
+@DynamicProperty(name = "The field name for the record.",
     value="A JSONPath Expression that will be evaluated against each JSON record. The result of the JSONPath will be the value of the "
         + "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 AbstractControllerService implements RowRecordReaderFactory {
+public class JsonPathReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory {
 
+    private volatile String dateFormat;
+    private volatile String timeFormat;
+    private volatile String timestampFormat;
     private volatile LinkedHashMap<String, JsonPath> jsonPaths;
-    private volatile Map<String, DataType> fieldTypeOverrides;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        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;
+    }
+
 
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
@@ -74,7 +86,9 @@ public class JsonPathReader extends AbstractControllerService implements RowReco
 
     @OnEnabled
     public void compileJsonPaths(final ConfigurationContext context) {
-        final Map<String, DataType> fieldTypes = new HashMap<>(context.getProperties().size());
+        this.dateFormat = context.getProperty(DateTimeUtils.DATE_FORMAT).getValue();
+        this.timeFormat = context.getProperty(DateTimeUtils.TIME_FORMAT).getValue();
+        this.timestampFormat = context.getProperty(DateTimeUtils.TIMESTAMP_FORMAT).getValue();
 
         final LinkedHashMap<String, JsonPath> compiled = new LinkedHashMap<>();
         for (final PropertyDescriptor descriptor : context.getProperties().keySet()) {
@@ -82,19 +96,13 @@ public class JsonPathReader extends AbstractControllerService implements RowReco
                 continue;
             }
 
-            final String fieldName = PropertyNameUtil.getFieldName(descriptor.getName());
-            final Optional<DataType> dataTypeOption = PropertyNameUtil.getDataType(descriptor.getName());
-            if (dataTypeOption.isPresent()) {
-                fieldTypes.put(fieldName, dataTypeOption.get());
-            }
-
+            final String fieldName = descriptor.getName();
             final String expression = context.getProperty(descriptor).getValue();
             final JsonPath jsonPath = JsonPath.compile(expression);
             compiled.put(fieldName, jsonPath);
         }
 
         jsonPaths = compiled;
-        fieldTypeOverrides = fieldTypes;
     }
 
     @Override
@@ -119,8 +127,9 @@ public class JsonPathReader extends AbstractControllerService implements RowReco
     }
 
     @Override
-    public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
-        return new JsonPathRowRecordReader(jsonPaths, fieldTypeOverrides, in, logger);
+    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
+        final RecordSchema schema = getSchema(flowFile);
+        return new JsonPathRowRecordReader(jsonPaths, schema, in, logger, dateFormat, timeFormat, timestampFormat);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 9654b97..a0f3c32 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
@@ -19,30 +19,23 @@ package org.apache.nifi.json;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Set;
-import java.util.TimeZone;
-import java.util.stream.Collectors;
 
 import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.serialization.DataTypeUtils;
 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.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
 import org.codehaus.jackson.JsonNode;
 
 import com.jayway.jsonpath.Configuration;
@@ -54,23 +47,27 @@ import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
 public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
     private static final Configuration STRICT_PROVIDER_CONFIGURATION = Configuration.builder().jsonProvider(new JacksonJsonProvider()).build();
 
-    private static final String TIME_FORMAT_DATE = "yyyy-MM-dd";
-    private static final String TIME_FORMAT_TIME = "HH:mm:ss";
-    private static final String TIME_FORMAT_TIMESTAMP = "yyyy-MM-dd HH:mm:ss";
-    private static final TimeZone gmt = TimeZone.getTimeZone("GMT");
-
+    private final ComponentLog logger;
     private final LinkedHashMap<String, JsonPath> jsonPaths;
-    private final Map<String, DataType> fieldTypeOverrides;
     private final InputStream in;
     private RecordSchema schema;
+    private final String dateFormat;
+    private final String timeFormat;
+    private final String timestampFormat;
 
-    public JsonPathRowRecordReader(final LinkedHashMap<String, JsonPath> jsonPaths, final Map<String, DataType> fieldTypeOverrides, final InputStream in, final ComponentLog logger)
+    public JsonPathRowRecordReader(final LinkedHashMap<String, JsonPath> jsonPaths, final RecordSchema schema, final InputStream in, final ComponentLog logger,
+        final String dateFormat, final String timeFormat, final String timestampFormat)
         throws MalformedRecordException, IOException {
         super(in, logger);
 
+        this.dateFormat = dateFormat;
+        this.timeFormat = timeFormat;
+        this.timestampFormat = timestampFormat;
+
+        this.schema = schema;
         this.jsonPaths = jsonPaths;
-        this.fieldTypeOverrides = fieldTypeOverrides;
         this.in = in;
+        this.logger = logger;
     }
 
     @Override
@@ -80,60 +77,10 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
 
     @Override
     public RecordSchema getSchema() {
-        if (schema != null) {
-            return schema;
-        }
-
-        final Optional<JsonNode> firstNodeOption = getFirstJsonNode();
-
-        final List<RecordField> recordFields = new ArrayList<>();
-        if (firstNodeOption.isPresent()) {
-            final DocumentContext ctx = JsonPath.using(STRICT_PROVIDER_CONFIGURATION).parse(firstNodeOption.get().toString());
-            for (final Map.Entry<String, JsonPath> entry : jsonPaths.entrySet()) {
-                final String fieldName = PropertyNameUtil.getFieldName(entry.getKey());
-                final JsonPath jsonPath = entry.getValue();
-
-                final DataType dataType;
-                final DataType dataTypeOverride = fieldTypeOverrides.get(fieldName);
-                if (dataTypeOverride == null) {
-                    Object value;
-                    try {
-                        value = ctx.read(jsonPath);
-                    } catch (final PathNotFoundException pnfe) {
-                        value = null;
-                    }
-
-                    if (value == null) {
-                        dataType = RecordFieldType.STRING.getDataType();
-                    } else {
-                        dataType = DataTypeUtils.inferDataType(value);
-                    }
-                } else {
-                    dataType = dataTypeOverride;
-                }
-
-                recordFields.add(new RecordField(fieldName, dataType));
-            }
-        }
-
-        // If there are any overridden field types that we didn't find, add as the last fields.
-        final Set<String> knownFieldNames = recordFields.stream()
-            .map(f -> f.getFieldName())
-            .collect(Collectors.toSet());
-
-        for (final Map.Entry<String, DataType> entry : fieldTypeOverrides.entrySet()) {
-            if (!knownFieldNames.contains(entry.getKey())) {
-                recordFields.add(new RecordField(entry.getKey(), entry.getValue()));
-            }
-        }
-
-        schema = new SimpleRecordSchema(recordFields);
         return schema;
     }
 
-
     @Override
-    @SuppressWarnings("unchecked")
     protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema) throws IOException {
         if (jsonNode == null) {
             return null;
@@ -143,138 +90,72 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
         final Map<String, Object> values = new HashMap<>(schema.getFieldCount());
 
         for (final Map.Entry<String, JsonPath> entry : jsonPaths.entrySet()) {
+            final String fieldName = entry.getKey();
+            final DataType desiredType = schema.getDataType(fieldName).orElse(null);
+            if (desiredType == null) {
+                continue;
+            }
+
             final JsonPath jsonPath = entry.getValue();
 
             Object value;
             try {
                 value = ctx.read(jsonPath);
             } catch (final PathNotFoundException pnfe) {
+                logger.debug("Evaluated JSONPath Expression {} but the path was not found; will use a null value", new Object[] {entry.getValue()});
                 value = null;
             }
 
-            final String fieldName = entry.getKey();
-            if (value != null) {
-                final DataType determinedType = DataTypeUtils.inferDataType(value);
-                final DataType desiredType = schema.getDataType(fieldName).orElse(null);
-
-                if (value instanceof List) {
-                    value = ((List<Object>) value).toArray();
-                } else if (value instanceof Map && desiredType.getFieldType() == RecordFieldType.RECORD) {
-                    value = convert(desiredType, value);
-                } else if (desiredType != null && !determinedType.equals(desiredType) && shouldConvert(value, determinedType.getFieldType())) {
-                    value = convert(desiredType, value);
-                }
-            }
-
+            value = convert(value, desiredType);
             values.put(fieldName, value);
         }
 
         return new MapRecord(schema, values);
     }
 
-    private boolean shouldConvert(final Object value, final RecordFieldType determinedType) {
-        return determinedType != null
-            && determinedType != RecordFieldType.ARRAY;
-    }
-
 
-    protected Object convert(final DataType dataType, final Object value) {
-        if (dataType.getFieldType() == RecordFieldType.RECORD && dataType.getChildRecordSchema().isPresent() && value instanceof Map) {
-            @SuppressWarnings("unchecked")
-            final Map<String, Object> map = (Map<String, Object>) value;
-            return new MapRecord(dataType.getChildRecordSchema().get(), map);
-        } else {
-            return convertString(dataType, value.toString());
+    @SuppressWarnings("unchecked")
+    protected Object convert(final Object value, final DataType dataType) {
+        if (value == null) {
+            return null;
         }
-    }
 
-    /**
-     * Coerces the given string into the provided data type, if possible
-     *
-     * @param dataType the desired type
-     * @param string the string representation of the value
-     * @return an Object representing the same value as the given string but in the requested data type
-     */
-    protected Object convertString(final DataType dataType, final String string) {
-        if (dataType == null) {
-            return string;
+        if (value instanceof List) {
+            if (dataType.getFieldType() != RecordFieldType.ARRAY) {
+                throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type Array to " + dataType);
+            }
+
+            final ArrayDataType arrayType = (ArrayDataType) dataType;
+
+            final List<?> list = (List<?>) value;
+            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);
+            }
+            return coercedValues;
         }
 
-        switch (dataType.getFieldType()) {
-            case BOOLEAN:
-                if (string.length() == 0) {
-                    return null;
-                }
-                return Boolean.parseBoolean(string);
-            case BYTE:
-                if (string.length() == 0) {
-                    return null;
-                }
-                return Byte.parseByte(string);
-            case SHORT:
-                if (string.length() == 0) {
-                    return null;
-                }
-                return Short.parseShort(string);
-            case INT:
-                if (string.length() == 0) {
-                    return null;
-                }
-                return Integer.parseInt(string);
-            case LONG:
-                if (string.length() == 0) {
-                    return null;
-                }
-                return Long.parseLong(string);
-            case FLOAT:
-                if (string.length() == 0) {
-                    return null;
-                }
-                return Float.parseFloat(string);
-            case DOUBLE:
-                if (string.length() == 0) {
-                    return null;
-                }
-                return Double.parseDouble(string);
-            case DATE:
-                if (string.length() == 0) {
-                    return null;
-                }
-                try {
-                    final DateFormat format = new SimpleDateFormat(TIME_FORMAT_DATE);
-                    format.setTimeZone(gmt);
-                    Date date = format.parse(string);
-                    return new java.sql.Date(date.getTime());
-                } catch (ParseException e) {
-                    return null;
-                }
-            case TIME:
-                if (string.length() == 0) {
-                    return null;
-                }
-                try {
-                    final DateFormat format = new SimpleDateFormat(TIME_FORMAT_TIME);
-                    format.setTimeZone(gmt);
-                    Date date = format.parse(string);
-                    return new java.sql.Time(date.getTime());
-                } catch (ParseException e) {
-                    return null;
-                }
-            case TIMESTAMP:
-                if (string.length() == 0) {
-                    return null;
-                }
-                try {
-                    final DateFormat format = new SimpleDateFormat(TIME_FORMAT_TIMESTAMP);
-                    format.setTimeZone(gmt);
-                    Date date = format.parse(string);
-                    return new java.sql.Timestamp(date.getTime());
-                } catch (ParseException e) {
-                    return null;
+        if (dataType.getFieldType() == RecordFieldType.RECORD && value instanceof Map) {
+            final RecordDataType recordDataType = (RecordDataType) dataType;
+            final RecordSchema childSchema = recordDataType.getChildSchema();
+
+            final Map<String, Object> rawValues = (Map<String, Object>) value;
+            final Map<String, Object> coercedValues = new HashMap<>();
+
+            for (final Map.Entry<String, Object> entry : rawValues.entrySet()) {
+                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);
+                    coercedValues.put(key, coercedValue);
                 }
-            case STRING:
-            default:
-                return string;
+            }
+
+            return new MapRecord(childSchema, coercedValues);
+        } else {
+            return DataTypeUtils.convertType(value, dataType, dateFormat, timeFormat, timestampFormat);
         }
     }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathValidator.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/JsonPathValidator.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathValidator.java
index 626f56c..f28c43d 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathValidator.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathValidator.java
@@ -27,18 +27,6 @@ public class JsonPathValidator implements Validator {
 
     @Override
     public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-        if (PropertyNameUtil.hasFieldType(subject) && !PropertyNameUtil.isFieldTypeValid(subject)) {
-            final String fieldType = PropertyNameUtil.getFieldTypeName(subject).get();
-
-            return new ValidationResult.Builder()
-                .subject(subject)
-                .input(input)
-                .valid(false)
-                .explanation("Invalid field type. If property name contains a colon (:) it must use syntax of "
-                    + "<field name>:<field type> but the specified field type ('" + fieldType + "') is not a valid field type")
-                .build();
-        }
-
         try {
             JsonPath.compile(input);
         } catch (final Exception e) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 dc75a51..d09f135 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
@@ -26,14 +26,14 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.serialization.AbstractRecordSetWriter;
+import org.apache.nifi.serialization.DateTimeTextRecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
 
-@Tags({"json", "resultset", "writer", "serialize", "record", "row"})
-@CapabilityDescription("Writes the results of a Database ResultSet as a JSON Array. Even if the ResultSet "
+@Tags({"json", "resultset", "writer", "serialize", "record", "recordset", "row"})
+@CapabilityDescription("Writes the results of a RecordSet as a JSON Array. Even if the RecordSet "
     + "consists of a single row, it will be written as an array with a single element.")
-public class JsonRecordSetWriter extends AbstractRecordSetWriter implements RecordSetWriterFactory {
+public class JsonRecordSetWriter extends DateTimeTextRecordSetWriter implements RecordSetWriterFactory {
 
     static final PropertyDescriptor PRETTY_PRINT_JSON = new PropertyDescriptor.Builder()
         .name("Pretty Print JSON")

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 2d7072a..1abb1f4 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
@@ -19,38 +19,56 @@ package org.apache.nifi.json;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
 
-import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+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.UserTypeOverrideRowReader;
+import org.apache.nifi.serialization.SchemaRegistryRecordReader;
 
 @Tags({"json", "tree", "record", "reader", "parser"})
 @CapabilityDescription("Parses JSON into individual Record objects. The Record that is produced will contain all top-level "
-    + "elements of the corresponding JSON Object. If the JSON has nested arrays, those values will be represented as an Object array for that field. "
-    + "Nested JSON objects will be represented as a Map. "
+    + "elements of the corresponding JSON Object. "
     + "The root JSON element can be either a single element or an array of JSON elements, and each "
-    + "element in that array will be treated as a separate record. If any of the elements has a nested array or a nested "
-    + "element, they will be returned as OBJECT or ARRAY types (respectively), not flattened out into individual fields. "
-    + "The schema for the record is determined by the first JSON element in the array, if the incoming FlowFile is a JSON array. "
-    + "This means that if a field does not exist in the first JSON object, then it will be skipped in all subsequent JSON objects. "
-    + "The data type of a field can be overridden by adding a property to "
-    + "the controller service where the name of the property matches the JSON field name and the value of the property is "
-    + "the data type to use. If that field does not exist in a JSON element, the field will be assumed to be null. "
-    + "See the Usage of the Controller Service for more information.")
+    + "element in that array will be treated as a separate record. "
+    + "If the schema that is configured contains a field that is not present in the JSON, a null value will be used. If the JSON contains "
+    + "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)
-@DynamicProperty(name = "<name of JSON field>", value = "<data type of JSON field>",
-    description = "User-defined properties are used to indicate that the values of a specific field should be interpreted as a "
-    + "user-defined data type (e.g., int, double, float, date, etc.)", supportsExpressionLanguage = false)
-public class JsonTreeReader extends UserTypeOverrideRowReader implements RowRecordReaderFactory {
+public class JsonTreeReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory {
+
+    private volatile String dateFormat;
+    private volatile String timeFormat;
+    private volatile String timestampFormat;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        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
+    public void storeFormats(final ConfigurationContext context) {
+        this.dateFormat = context.getProperty(DateTimeUtils.DATE_FORMAT).getValue();
+        this.timeFormat = context.getProperty(DateTimeUtils.TIME_FORMAT).getValue();
+        this.timestampFormat = context.getProperty(DateTimeUtils.TIMESTAMP_FORMAT).getValue();
+    }
 
     @Override
-    public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
-        return new JsonTreeRowRecordReader(in, logger, getFieldTypeOverrides());
+    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);
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 4a2d212..c8d07f4 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,35 +19,40 @@ 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 java.util.Optional;
-import java.util.Set;
-import java.util.stream.Collectors;
 
 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.RecordSchema;
+import org.apache.nifi.serialization.record.type.ArrayDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
 import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.node.ArrayNode;
 
 
 public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
-    private final Map<String, DataType> fieldTypeOverrides;
-    private RecordSchema schema;
+    private final RecordSchema schema;
+    private final String dateFormat;
+    private final String timeFormat;
+    private final String timestampFormat;
 
-    public JsonTreeRowRecordReader(final InputStream in, final ComponentLog logger, final Map<String, DataType> fieldTypeOverrides) throws IOException, MalformedRecordException {
+    public JsonTreeRowRecordReader(final InputStream in, final ComponentLog logger, final RecordSchema schema,
+        final String dateFormat, final String timeFormat, final String timestampFormat) throws IOException, MalformedRecordException {
         super(in, logger);
-        this.fieldTypeOverrides = fieldTypeOverrides;
+        this.schema = schema;
+
+        this.dateFormat = dateFormat;
+        this.timeFormat = timeFormat;
+        this.timestampFormat = timestampFormat;
     }
 
+
     @Override
     protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema) throws IOException, MalformedRecordException {
         if (jsonNode == null) {
@@ -68,48 +73,76 @@ public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
         return new MapRecord(schema, values);
     }
 
-
-    @Override
-    public RecordSchema getSchema() {
-        if (schema != null) {
-            return schema;
+    protected Object convertField(final JsonNode fieldNode, final String fieldName, final DataType desiredType) throws IOException, MalformedRecordException {
+        if (fieldNode == null || fieldNode.isNull()) {
+            return null;
         }
 
-        final List<RecordField> recordFields = new ArrayList<>();
-        final Optional<JsonNode> firstNodeOption = getFirstJsonNode();
-
-        if (firstNodeOption.isPresent()) {
-            final Iterator<Map.Entry<String, JsonNode>> itr = firstNodeOption.get().getFields();
-            while (itr.hasNext()) {
-                final Map.Entry<String, JsonNode> entry = itr.next();
-                final String elementName = entry.getKey();
-                final JsonNode node = entry.getValue();
+        switch (desiredType.getFieldType()) {
+            case BOOLEAN:
+                return DataTypeUtils.toBoolean(getRawNodeValue(fieldNode));
+            case BYTE:
+                return DataTypeUtils.toByte(getRawNodeValue(fieldNode));
+            case CHAR:
+                return DataTypeUtils.toCharacter(getRawNodeValue(fieldNode));
+            case DOUBLE:
+                return DataTypeUtils.toDouble(getRawNodeValue(fieldNode));
+            case FLOAT:
+                return DataTypeUtils.toFloat(getRawNodeValue(fieldNode));
+            case INT:
+                return DataTypeUtils.toInteger(getRawNodeValue(fieldNode));
+            case LONG:
+                return DataTypeUtils.toLong(getRawNodeValue(fieldNode));
+            case SHORT:
+                return DataTypeUtils.toShort(getRawNodeValue(fieldNode));
+            case STRING:
+                return DataTypeUtils.toString(getRawNodeValue(fieldNode), dateFormat, timeFormat, timestampFormat);
+            case DATE:
+                return DataTypeUtils.toDate(getRawNodeValue(fieldNode), dateFormat);
+            case TIME:
+                return DataTypeUtils.toTime(getRawNodeValue(fieldNode), timeFormat);
+            case TIMESTAMP:
+                return DataTypeUtils.toTimestamp(getRawNodeValue(fieldNode), timestampFormat);
+            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 Object converted = convertField(node, fieldName, elementType);
+                    arrayElements[count++] = converted;
+                }
 
-                DataType dataType;
-                final DataType overriddenDataType = fieldTypeOverrides.get(elementName);
-                if (overriddenDataType == null) {
-                    dataType = determineFieldType(node);
+                return arrayElements;
+            }
+            case RECORD: {
+                if (fieldNode.isObject()) {
+                    final RecordSchema childSchema;
+                    if (desiredType instanceof RecordDataType) {
+                        childSchema = ((RecordDataType) desiredType).getChildSchema();
+                    } else {
+                        return null;
+                    }
+
+                    return convertJsonNodeToRecord(fieldNode, childSchema);
                 } else {
-                    dataType = overriddenDataType;
+                    return null;
                 }
-
-                recordFields.add(new RecordField(elementName, dataType));
             }
         }
 
-        // If there are any overridden field types that we didn't find, add as the last fields.
-        final Set<String> knownFieldNames = recordFields.stream()
-            .map(f -> f.getFieldName())
-            .collect(Collectors.toSet());
-
-        for (final Map.Entry<String, DataType> entry : fieldTypeOverrides.entrySet()) {
-            if (!knownFieldNames.contains(entry.getKey())) {
-                recordFields.add(new RecordField(entry.getKey(), entry.getValue()));
-            }
-        }
+        return null;
+    }
 
-        schema = new SimpleRecordSchema(recordFields);
+    @Override
+    public RecordSchema getSchema() {
         return schema;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/PropertyNameUtil.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/PropertyNameUtil.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/PropertyNameUtil.java
deleted file mode 100644
index 3b7dcf9..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/PropertyNameUtil.java
+++ /dev/null
@@ -1,88 +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.json;
-
-import java.util.Optional;
-
-import org.apache.nifi.serialization.record.DataType;
-import org.apache.nifi.serialization.record.RecordFieldType;
-
-public class PropertyNameUtil {
-
-    public static String getFieldName(final String propertyName) {
-        final int colonIndex = propertyName.indexOf(":");
-        if (colonIndex > -1 && colonIndex < propertyName.length() - 1) {
-            return propertyName.substring(0, colonIndex);
-        }
-
-        return propertyName;
-    }
-
-    public static boolean hasFieldType(final String propertyName) {
-        final int colonIndex = propertyName.indexOf(":");
-        return (colonIndex > -1 && colonIndex < propertyName.length() - 1);
-    }
-
-    public static Optional<String> getFieldTypeName(final String propertyName) {
-        if (hasFieldType(propertyName)) {
-            final String[] splits = propertyName.split("\\:");
-            if (splits.length > 1) {
-                return Optional.of(splits[1]);
-            }
-            return Optional.empty();
-        }
-
-        return Optional.empty();
-    }
-
-    public static Optional<String> getFieldFormat(final String propertyName) {
-        final String[] splits = propertyName.split("\\:");
-        if (splits.length != 3) {
-            return Optional.empty();
-        }
-
-        return Optional.of(splits[2]);
-    }
-
-    public static boolean isFieldTypeValid(final String propertyName) {
-        final Optional<String> fieldType = getFieldTypeName(propertyName);
-        if (!fieldType.isPresent()) {
-            return false;
-        }
-
-        final String typeName = fieldType.get();
-        final RecordFieldType recordFieldType = RecordFieldType.of(typeName);
-        return recordFieldType != null;
-    }
-
-    public static Optional<DataType> getDataType(final String propertyName) {
-        if (isFieldTypeValid(propertyName)) {
-            final String typeName = getFieldTypeName(propertyName).get();
-            final RecordFieldType fieldType = RecordFieldType.of(typeName);
-
-            final Optional<String> format = getFieldFormat(propertyName);
-            if (format.isPresent()) {
-                return Optional.of(fieldType.getDataType(format.get()));
-            } else {
-                return Optional.of(fieldType.getDataType());
-            }
-        }
-
-        return Optional.empty();
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 cf72b19..05895d8 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
@@ -20,17 +20,10 @@ package org.apache.nifi.json;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.math.BigInteger;
-import java.sql.Array;
 import java.sql.SQLException;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
 import java.util.Collections;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
 
 import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.serialization.DataTypeUtils;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.WriteResult;
 import org.apache.nifi.serialization.record.DataType;
@@ -38,25 +31,29 @@ import org.apache.nifi.serialization.record.Record;
 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.util.DataTypeUtils;
 import org.apache.nifi.stream.io.NonCloseableOutputStream;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.JsonGenerator;
 
 public class WriteJsonResult implements RecordSetWriter {
-    private final boolean prettyPrint;
-
     private final ComponentLog logger;
+    private final boolean prettyPrint;
     private final JsonFactory factory = new JsonFactory();
-    private final DateFormat dateFormat;
-    private final DateFormat timeFormat;
-    private final DateFormat timestampFormat;
+    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) {
         this.prettyPrint = prettyPrint;
-        this.dateFormat = new SimpleDateFormat(dateFormat);
-        this.timeFormat = new SimpleDateFormat(timeFormat);
-        this.timestampFormat = new SimpleDateFormat(timestampFormat);
+
+        this.dateFormat = dateFormat;
+        this.timeFormat = timeFormat;
+        this.timestampFormat = timestampFormat;
+
         this.logger = logger;
     }
 
@@ -127,26 +124,6 @@ public class WriteJsonResult implements RecordSetWriter {
         }
     }
 
-    private String createDate(final Object value, final DateFormat format) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Date) {
-            return format.format((Date) value);
-        }
-        if (value instanceof java.sql.Date) {
-            return format.format(new Date(((java.sql.Date) value).getTime()));
-        }
-        if (value instanceof java.sql.Time) {
-            return format.format(new Date(((java.sql.Time) value).getTime()));
-        }
-        if (value instanceof java.sql.Timestamp) {
-            return format.format(new Date(((java.sql.Timestamp) value).getTime()));
-        }
-
-        return null;
-    }
 
     private void writeValue(final JsonGenerator generator, final Object value, final DataType dataType, final boolean moreCols)
         throws JsonGenerationException, IOException, SQLException {
@@ -155,50 +132,46 @@ public class WriteJsonResult implements RecordSetWriter {
             return;
         }
 
-        final DataType resolvedDataType;
-        if (dataType.getFieldType() == RecordFieldType.CHOICE) {
-            resolvedDataType = DataTypeUtils.inferDataType(value);
-        } else {
-            resolvedDataType = dataType;
+        final DataType chosenDataType = dataType.getFieldType() == RecordFieldType.CHOICE ? DataTypeUtils.chooseDataType(value, (ChoiceDataType) dataType) : dataType;
+        final Object coercedValue = DataTypeUtils.convertType(value, chosenDataType);
+        if (coercedValue == null) {
+            generator.writeNull();
+            return;
         }
 
-        switch (resolvedDataType.getFieldType()) {
+        switch (chosenDataType.getFieldType()) {
             case DATE:
-                generator.writeString(createDate(value, dateFormat));
-                break;
             case TIME:
-                generator.writeString(createDate(value, timeFormat));
-                break;
             case TIMESTAMP:
-                generator.writeString(createDate(value, timestampFormat));
+                generator.writeString(DataTypeUtils.toString(coercedValue, dateFormat, timeFormat, timestampFormat));
                 break;
             case DOUBLE:
-                generator.writeNumber(DataTypeUtils.toDouble(value, 0D));
+                generator.writeNumber(DataTypeUtils.toDouble(coercedValue));
                 break;
             case FLOAT:
-                generator.writeNumber(DataTypeUtils.toFloat(value, 0F));
+                generator.writeNumber(DataTypeUtils.toFloat(coercedValue));
                 break;
             case LONG:
-                generator.writeNumber(DataTypeUtils.toLong(value, 0L));
+                generator.writeNumber(DataTypeUtils.toLong(coercedValue));
                 break;
             case INT:
             case BYTE:
             case SHORT:
-                generator.writeNumber(DataTypeUtils.toInteger(value, 0));
+                generator.writeNumber(DataTypeUtils.toInteger(coercedValue));
                 break;
             case CHAR:
             case STRING:
-                generator.writeString(value.toString());
+                generator.writeString(coercedValue.toString());
                 break;
             case BIGINT:
-                if (value instanceof Long) {
-                    generator.writeNumber(((Long) value).longValue());
+                if (coercedValue instanceof Long) {
+                    generator.writeNumber(((Long) coercedValue).longValue());
                 } else {
-                    generator.writeNumber((BigInteger) value);
+                    generator.writeNumber((BigInteger) coercedValue);
                 }
                 break;
             case BOOLEAN:
-                final String stringValue = value.toString();
+                final String stringValue = coercedValue.toString();
                 if ("true".equalsIgnoreCase(stringValue)) {
                     generator.writeBoolean(true);
                 } else if ("false".equalsIgnoreCase(stringValue)) {
@@ -208,95 +181,34 @@ public class WriteJsonResult implements RecordSetWriter {
                 }
                 break;
             case RECORD: {
-                final Record record = (Record) value;
+                final Record record = (Record) coercedValue;
                 writeRecord(record, generator, gen -> gen.writeStartObject(), gen -> gen.writeEndObject());
                 break;
             }
             case ARRAY:
             default:
-                if ("null".equals(value.toString())) {
-                    generator.writeNull();
-                } else if (value instanceof Map) {
-                    final Map<?, ?> map = (Map<?, ?>) value;
-                    generator.writeStartObject();
-
-                    int i = 0;
-                    for (final Map.Entry<?, ?> entry : map.entrySet()) {
-                        generator.writeFieldName(entry.getKey().toString());
-                        final boolean moreEntries = ++i < map.size();
-                        writeValue(generator, entry.getValue(), getColType(entry.getValue()), moreEntries);
-                    }
-                    generator.writeEndObject();
-                } else if (value instanceof List) {
-                    final List<?> list = (List<?>) value;
-                    writeArray(list.toArray(), generator);
-                } else if (value instanceof Array) {
-                    final Array array = (Array) value;
-                    final Object[] values = (Object[]) array.getArray();
-                    writeArray(values, generator);
-                } else if (value instanceof Object[]) {
-                    final Object[] values = (Object[]) value;
-                    writeArray(values, generator);
+                if (coercedValue instanceof Object[]) {
+                    final Object[] values = (Object[]) coercedValue;
+                    final ArrayDataType arrayDataType = (ArrayDataType) dataType;
+                    final DataType elementType = arrayDataType.getElementType();
+                    writeArray(values, generator, elementType);
                 } else {
-                    generator.writeString(value.toString());
+                    generator.writeString(coercedValue.toString());
                 }
                 break;
         }
     }
 
-    private void writeArray(final Object[] values, final JsonGenerator generator) throws JsonGenerationException, IOException, SQLException {
+    private void writeArray(final Object[] values, 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, getColType(element), moreEntries);
+            writeValue(generator, element, elementType, moreEntries);
         }
         generator.writeEndArray();
     }
 
-    private DataType getColType(final Object value) {
-        if (value instanceof String) {
-            return RecordFieldType.STRING.getDataType();
-        }
-        if (value instanceof Double) {
-            return RecordFieldType.DOUBLE.getDataType();
-        }
-        if (value instanceof Float) {
-            return RecordFieldType.FLOAT.getDataType();
-        }
-        if (value instanceof Integer) {
-            return RecordFieldType.INT.getDataType();
-        }
-        if (value instanceof Long) {
-            return RecordFieldType.LONG.getDataType();
-        }
-        if (value instanceof BigInteger) {
-            return RecordFieldType.BIGINT.getDataType();
-        }
-        if (value instanceof Boolean) {
-            return RecordFieldType.BOOLEAN.getDataType();
-        }
-        if (value instanceof Byte || value instanceof Short) {
-            return RecordFieldType.INT.getDataType();
-        }
-        if (value instanceof Character) {
-            return RecordFieldType.STRING.getDataType();
-        }
-        if (value instanceof java.util.Date || value instanceof java.sql.Date) {
-            return RecordFieldType.DATE.getDataType();
-        }
-        if (value instanceof java.sql.Time) {
-            return RecordFieldType.TIME.getDataType();
-        }
-        if (value instanceof java.sql.Timestamp) {
-            return RecordFieldType.TIMESTAMP.getDataType();
-        }
-        if (value instanceof Object[] || value instanceof List || value instanceof Array) {
-            return RecordFieldType.ARRAY.getDataType();
-        }
-
-        return RecordFieldType.RECORD.getDataType();
-    }
 
     @Override
     public String getMimeType() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/AbstractRecordSetWriter.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/AbstractRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/AbstractRecordSetWriter.java
deleted file mode 100644
index b58a22e..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/AbstractRecordSetWriter.java
+++ /dev/null
@@ -1,84 +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.Arrays;
-import java.util.List;
-
-import org.apache.nifi.annotation.lifecycle.OnEnabled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.AbstractControllerService;
-import org.apache.nifi.controller.ConfigurationContext;
-import org.apache.nifi.serialization.record.RecordFieldType;
-
-public abstract class AbstractRecordSetWriter extends AbstractControllerService {
-    static final PropertyDescriptor DATE_FORMAT = new PropertyDescriptor.Builder()
-        .name("Date Format")
-        .description("Specifies the format to use when writing out Date fields")
-        .expressionLanguageSupported(false)
-        .defaultValue(RecordFieldType.DATE.getDefaultFormat())
-        .addValidator(new SimpleDateFormatValidator())
-        .required(true)
-        .build();
-
-    static final PropertyDescriptor TIME_FORMAT = new PropertyDescriptor.Builder()
-        .name("Time Format")
-        .description("Specifies the format to use when writing out Time fields")
-        .expressionLanguageSupported(false)
-        .defaultValue(RecordFieldType.TIME.getDefaultFormat())
-        .addValidator(new SimpleDateFormatValidator())
-        .required(true)
-        .build();
-
-    static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
-        .name("Timestamp Format")
-        .description("Specifies the format to use when writing out Timestamp (date/time) fields")
-        .expressionLanguageSupported(false)
-        .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
-        .addValidator(new SimpleDateFormatValidator())
-        .required(true)
-        .build();
-
-    private volatile String dateFormat;
-    private volatile String timeFormat;
-    private volatile String timestampFormat;
-
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return Arrays.asList(DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
-    }
-
-    @OnEnabled
-    public void captureValues(final ConfigurationContext context) {
-        this.dateFormat = context.getProperty(DATE_FORMAT).getValue();
-        this.timeFormat = context.getProperty(TIME_FORMAT).getValue();
-        this.timestampFormat = context.getProperty(TIMESTAMP_FORMAT).getValue();
-    }
-
-    protected String getDateFormat() {
-        return dateFormat;
-    }
-
-    protected String getTimeFormat() {
-        return timeFormat;
-    }
-
-    protected String getTimestampFormat() {
-        return timestampFormat;
-    }
-}


[03/19] nifi git commit: NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Servic

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..286326a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
@@ -0,0 +1,307 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.json;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TimeZone;
+
+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;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.node.ArrayNode;
+
+
+public abstract class AbstractJsonRowRecordReader implements RecordReader {
+    private final ComponentLog logger;
+    private final JsonParser jsonParser;
+    private final JsonFactory jsonFactory;
+    private final boolean array;
+    private final JsonNode firstJsonNode;
+
+    private boolean firstObjectConsumed = false;
+
+    private static final TimeZone gmt = TimeZone.getTimeZone("GMT");
+
+
+    public AbstractJsonRowRecordReader(final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
+        this.logger = logger;
+
+        jsonFactory = new JsonFactory();
+        try {
+            jsonParser = jsonFactory.createJsonParser(in);
+            jsonParser.setCodec(new ObjectMapper());
+
+            JsonToken token = jsonParser.nextToken();
+            if (token == JsonToken.START_ARRAY) {
+                array = true;
+                token = jsonParser.nextToken(); // advance to START_OBJECT token
+            } else {
+                array = false;
+            }
+
+            if (token == JsonToken.START_OBJECT) { // could be END_ARRAY also
+                firstJsonNode = jsonParser.readValueAsTree();
+            } else {
+                firstJsonNode = null;
+            }
+        } catch (final JsonParseException e) {
+            throw new MalformedRecordException("Could not parse data as JSON", e);
+        }
+    }
+
+    @Override
+    public Record nextRecord() throws IOException, MalformedRecordException {
+        if (firstObjectConsumed && !array) {
+            return null;
+        }
+
+        final JsonNode nextNode = getNextJsonNode();
+        final RecordSchema schema = getSchema();
+        try {
+            return convertJsonNodeToRecord(nextNode, schema);
+        } catch (final MalformedRecordException mre) {
+            throw mre;
+        } catch (final IOException ioe) {
+            throw ioe;
+        } catch (final Exception e) {
+            logger.debug("Failed to convert JSON Element {} into a Record object using schema {} due to {}", new Object[] {nextNode, schema, e.toString(), e});
+            throw new MalformedRecordException("Successfully parsed a JSON object from input but failed to convert into a Record object with the given schema", e);
+        }
+    }
+
+    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.getDataType(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 convertField(final JsonNode fieldNode, final String fieldName, final DataType desiredType) throws IOException, MalformedRecordException {
+        if (fieldNode == null || fieldNode.isNull()) {
+            return null;
+        }
+
+        switch (desiredType.getFieldType()) {
+            case BOOLEAN:
+                return fieldNode.asBoolean();
+            case BYTE:
+                return (byte) fieldNode.asInt();
+            case CHAR:
+                final String text = fieldNode.asText();
+                if (text.isEmpty()) {
+                    return null;
+                }
+                return text.charAt(0);
+            case DOUBLE:
+                return fieldNode.asDouble();
+            case FLOAT:
+                return (float) fieldNode.asDouble();
+            case INT:
+                return fieldNode.asInt();
+            case LONG:
+                return fieldNode.asLong();
+            case SHORT:
+                return (short) fieldNode.asInt();
+            case STRING:
+                return fieldNode.asText();
+            case DATE: {
+                final String string = fieldNode.asText();
+                if (string.isEmpty()) {
+                    return null;
+                }
+
+                try {
+                    final DateFormat dateFormat = new SimpleDateFormat(desiredType.getFormat());
+                    dateFormat.setTimeZone(gmt);
+                    final Date date = dateFormat.parse(string);
+                    return new java.sql.Date(date.getTime());
+                } catch (ParseException e) {
+                    logger.warn("Failed to convert JSON field to Date for field {} (value {})", new Object[] {fieldName, string, e});
+                    return null;
+                }
+            }
+            case TIME: {
+                final String string = fieldNode.asText();
+                if (string.isEmpty()) {
+                    return null;
+                }
+
+                try {
+                    final DateFormat dateFormat = new SimpleDateFormat(desiredType.getFormat());
+                    dateFormat.setTimeZone(gmt);
+                    final Date date = dateFormat.parse(string);
+                    return new java.sql.Date(date.getTime());
+                } catch (ParseException e) {
+                    logger.warn("Failed to convert JSON field to Time for field {} (value {})", new Object[] {fieldName, string, e});
+                    return null;
+                }
+            }
+            case TIMESTAMP: {
+                final String string = fieldNode.asText();
+                if (string.isEmpty()) {
+                    return null;
+                }
+
+                try {
+                    final DateFormat dateFormat = new SimpleDateFormat(desiredType.getFormat());
+                    dateFormat.setTimeZone(gmt);
+                    final Date date = dateFormat.parse(string);
+                    return new java.sql.Date(date.getTime());
+                } catch (ParseException e) {
+                    logger.warn("Failed to convert JSON field to Timestamp for field {} (value {})", new Object[] {fieldName, string, e});
+                    return null;
+                }
+            }
+            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 Object converted = convertField(node, fieldName, determineFieldType(node));
+                    arrayElements[count++] = converted;
+                }
+
+                return arrayElements;
+            }
+            case RECORD: {
+                if (fieldNode.isObject()) {
+                    final Optional<RecordSchema> childSchema = desiredType.getChildRecordSchema();
+                    if (!childSchema.isPresent()) {
+                        return null;
+                    }
+
+                    return convertJsonNodeToRecord(fieldNode, childSchema.get());
+                } else {
+                    return fieldNode.toString();
+                }
+            }
+        }
+
+        return fieldNode.toString();
+    }
+
+    private JsonNode getNextJsonNode() throws JsonParseException, IOException, MalformedRecordException {
+        if (!firstObjectConsumed) {
+            firstObjectConsumed = true;
+            return firstJsonNode;
+        }
+
+        while (true) {
+            final JsonToken token = jsonParser.nextToken();
+            if (token == null) {
+                return null;
+            }
+
+            switch (token) {
+                case END_OBJECT:
+                    continue;
+                case START_OBJECT:
+                    return jsonParser.readValueAsTree();
+                case END_ARRAY:
+                case START_ARRAY:
+                    return null;
+                default:
+                    throw new MalformedRecordException("Expected to get a JSON Object but got a token of type " + token.name());
+            }
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        jsonParser.close();
+    }
+
+    protected JsonParser getJsonParser() {
+        return jsonParser;
+    }
+
+    protected JsonFactory getJsonFactory() {
+        return jsonFactory;
+    }
+
+    protected Optional<JsonNode> getFirstJsonNode() {
+        return Optional.ofNullable(firstJsonNode);
+    }
+
+    protected abstract Record convertJsonNodeToRecord(final JsonNode nextNode, final RecordSchema schema) throws IOException, MalformedRecordException;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..b43b1c1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.json;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.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.logging.ComponentLog;
+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 com.jayway.jsonpath.JsonPath;
+
+@Tags({"json", "jsonpath", "record", "reader", "parser"})
+@CapabilityDescription("Parses JSON records and evaluates user-defined JSON Path's against each JSON object. The root element may be either "
+    + "a single JSON object or a JSON array. If a JSON array is found, each JSON object within that array is treated as a separate record. "
+    + "User-defined properties define the fields that should be extracted from the JSON in order to form the fields of a Record. Any JSON field "
+    + "that is not extracted via a JSONPath will not be returned in the JSON Records.")
+@SeeAlso(JsonTreeReader.class)
+@DynamicProperty(name = "The field name for the record. If it is desirable to enforce that the value be coerced into a given type, its type can be included "
+    + "in the name by using a syntax of <field name>:<field type>. For example, \"balance:double\".",
+    value="A JSONPath Expression that will be evaluated against each JSON record. The result of the JSONPath will be the value of the "
+        + "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 AbstractControllerService implements RowRecordReaderFactory {
+
+    private volatile LinkedHashMap<String, JsonPath> jsonPaths;
+    private volatile Map<String, DataType> fieldTypeOverrides;
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+            .name(propertyDescriptorName)
+            .description("JsonPath Expression that indicates how to retrieve the value from a JSON Object for the '" + propertyDescriptorName + "' column")
+            .dynamic(true)
+            .required(false)
+            .addValidator(new JsonPathValidator())
+            .build();
+    }
+
+    @OnEnabled
+    public void compileJsonPaths(final ConfigurationContext context) {
+        final Map<String, DataType> fieldTypes = new HashMap<>(context.getProperties().size());
+
+        final LinkedHashMap<String, JsonPath> compiled = new LinkedHashMap<>();
+        for (final PropertyDescriptor descriptor : context.getProperties().keySet()) {
+            if (!descriptor.isDynamic()) {
+                continue;
+            }
+
+            final String fieldName = PropertyNameUtil.getFieldName(descriptor.getName());
+            final Optional<DataType> dataTypeOption = PropertyNameUtil.getDataType(descriptor.getName());
+            if (dataTypeOption.isPresent()) {
+                fieldTypes.put(fieldName, dataTypeOption.get());
+            }
+
+            final String expression = context.getProperty(descriptor).getValue();
+            final JsonPath jsonPath = JsonPath.compile(expression);
+            compiled.put(fieldName, jsonPath);
+        }
+
+        jsonPaths = compiled;
+        fieldTypeOverrides = fieldTypes;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        boolean pathSpecified = false;
+        for (final PropertyDescriptor property : validationContext.getProperties().keySet()) {
+            if (property.isDynamic()) {
+                pathSpecified = true;
+                break;
+            }
+        }
+
+        if (pathSpecified) {
+            return Collections.emptyList();
+        }
+
+        return Collections.singleton(new ValidationResult.Builder()
+            .subject("JSON Paths")
+            .valid(false)
+            .explanation("No JSON Paths were specified")
+            .build());
+    }
+
+    @Override
+    public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
+        return new JsonPathRowRecordReader(jsonPaths, fieldTypeOverrides, in, logger);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..9654b97
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java
@@ -0,0 +1,280 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.json;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.DataTypeUtils;
+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.codehaus.jackson.JsonNode;
+
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.DocumentContext;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.PathNotFoundException;
+import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
+
+public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
+    private static final Configuration STRICT_PROVIDER_CONFIGURATION = Configuration.builder().jsonProvider(new JacksonJsonProvider()).build();
+
+    private static final String TIME_FORMAT_DATE = "yyyy-MM-dd";
+    private static final String TIME_FORMAT_TIME = "HH:mm:ss";
+    private static final String TIME_FORMAT_TIMESTAMP = "yyyy-MM-dd HH:mm:ss";
+    private static final TimeZone gmt = TimeZone.getTimeZone("GMT");
+
+    private final LinkedHashMap<String, JsonPath> jsonPaths;
+    private final Map<String, DataType> fieldTypeOverrides;
+    private final InputStream in;
+    private RecordSchema schema;
+
+    public JsonPathRowRecordReader(final LinkedHashMap<String, JsonPath> jsonPaths, final Map<String, DataType> fieldTypeOverrides, final InputStream in, final ComponentLog logger)
+        throws MalformedRecordException, IOException {
+        super(in, logger);
+
+        this.jsonPaths = jsonPaths;
+        this.fieldTypeOverrides = fieldTypeOverrides;
+        this.in = in;
+    }
+
+    @Override
+    public void close() throws IOException {
+        in.close();
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        if (schema != null) {
+            return schema;
+        }
+
+        final Optional<JsonNode> firstNodeOption = getFirstJsonNode();
+
+        final List<RecordField> recordFields = new ArrayList<>();
+        if (firstNodeOption.isPresent()) {
+            final DocumentContext ctx = JsonPath.using(STRICT_PROVIDER_CONFIGURATION).parse(firstNodeOption.get().toString());
+            for (final Map.Entry<String, JsonPath> entry : jsonPaths.entrySet()) {
+                final String fieldName = PropertyNameUtil.getFieldName(entry.getKey());
+                final JsonPath jsonPath = entry.getValue();
+
+                final DataType dataType;
+                final DataType dataTypeOverride = fieldTypeOverrides.get(fieldName);
+                if (dataTypeOverride == null) {
+                    Object value;
+                    try {
+                        value = ctx.read(jsonPath);
+                    } catch (final PathNotFoundException pnfe) {
+                        value = null;
+                    }
+
+                    if (value == null) {
+                        dataType = RecordFieldType.STRING.getDataType();
+                    } else {
+                        dataType = DataTypeUtils.inferDataType(value);
+                    }
+                } else {
+                    dataType = dataTypeOverride;
+                }
+
+                recordFields.add(new RecordField(fieldName, dataType));
+            }
+        }
+
+        // If there are any overridden field types that we didn't find, add as the last fields.
+        final Set<String> knownFieldNames = recordFields.stream()
+            .map(f -> f.getFieldName())
+            .collect(Collectors.toSet());
+
+        for (final Map.Entry<String, DataType> entry : fieldTypeOverrides.entrySet()) {
+            if (!knownFieldNames.contains(entry.getKey())) {
+                recordFields.add(new RecordField(entry.getKey(), entry.getValue()));
+            }
+        }
+
+        schema = new SimpleRecordSchema(recordFields);
+        return schema;
+    }
+
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema) throws IOException {
+        if (jsonNode == null) {
+            return null;
+        }
+
+        final DocumentContext ctx = JsonPath.using(STRICT_PROVIDER_CONFIGURATION).parse(jsonNode.toString());
+        final Map<String, Object> values = new HashMap<>(schema.getFieldCount());
+
+        for (final Map.Entry<String, JsonPath> entry : jsonPaths.entrySet()) {
+            final JsonPath jsonPath = entry.getValue();
+
+            Object value;
+            try {
+                value = ctx.read(jsonPath);
+            } catch (final PathNotFoundException pnfe) {
+                value = null;
+            }
+
+            final String fieldName = entry.getKey();
+            if (value != null) {
+                final DataType determinedType = DataTypeUtils.inferDataType(value);
+                final DataType desiredType = schema.getDataType(fieldName).orElse(null);
+
+                if (value instanceof List) {
+                    value = ((List<Object>) value).toArray();
+                } else if (value instanceof Map && desiredType.getFieldType() == RecordFieldType.RECORD) {
+                    value = convert(desiredType, value);
+                } else if (desiredType != null && !determinedType.equals(desiredType) && shouldConvert(value, determinedType.getFieldType())) {
+                    value = convert(desiredType, value);
+                }
+            }
+
+            values.put(fieldName, value);
+        }
+
+        return new MapRecord(schema, values);
+    }
+
+    private boolean shouldConvert(final Object value, final RecordFieldType determinedType) {
+        return determinedType != null
+            && determinedType != RecordFieldType.ARRAY;
+    }
+
+
+    protected Object convert(final DataType dataType, final Object value) {
+        if (dataType.getFieldType() == RecordFieldType.RECORD && dataType.getChildRecordSchema().isPresent() && value instanceof Map) {
+            @SuppressWarnings("unchecked")
+            final Map<String, Object> map = (Map<String, Object>) value;
+            return new MapRecord(dataType.getChildRecordSchema().get(), map);
+        } else {
+            return convertString(dataType, value.toString());
+        }
+    }
+
+    /**
+     * Coerces the given string into the provided data type, if possible
+     *
+     * @param dataType the desired type
+     * @param string the string representation of the value
+     * @return an Object representing the same value as the given string but in the requested data type
+     */
+    protected Object convertString(final DataType dataType, final String string) {
+        if (dataType == null) {
+            return string;
+        }
+
+        switch (dataType.getFieldType()) {
+            case BOOLEAN:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Boolean.parseBoolean(string);
+            case BYTE:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Byte.parseByte(string);
+            case SHORT:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Short.parseShort(string);
+            case INT:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Integer.parseInt(string);
+            case LONG:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Long.parseLong(string);
+            case FLOAT:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Float.parseFloat(string);
+            case DOUBLE:
+                if (string.length() == 0) {
+                    return null;
+                }
+                return Double.parseDouble(string);
+            case DATE:
+                if (string.length() == 0) {
+                    return null;
+                }
+                try {
+                    final DateFormat format = new SimpleDateFormat(TIME_FORMAT_DATE);
+                    format.setTimeZone(gmt);
+                    Date date = format.parse(string);
+                    return new java.sql.Date(date.getTime());
+                } catch (ParseException e) {
+                    return null;
+                }
+            case TIME:
+                if (string.length() == 0) {
+                    return null;
+                }
+                try {
+                    final DateFormat format = new SimpleDateFormat(TIME_FORMAT_TIME);
+                    format.setTimeZone(gmt);
+                    Date date = format.parse(string);
+                    return new java.sql.Time(date.getTime());
+                } catch (ParseException e) {
+                    return null;
+                }
+            case TIMESTAMP:
+                if (string.length() == 0) {
+                    return null;
+                }
+                try {
+                    final DateFormat format = new SimpleDateFormat(TIME_FORMAT_TIMESTAMP);
+                    format.setTimeZone(gmt);
+                    Date date = format.parse(string);
+                    return new java.sql.Timestamp(date.getTime());
+                } catch (ParseException e) {
+                    return null;
+                }
+            case STRING:
+            default:
+                return string;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathValidator.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/JsonPathValidator.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathValidator.java
new file mode 100644
index 0000000..626f56c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathValidator.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.json;
+
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+
+import com.jayway.jsonpath.JsonPath;
+
+public class JsonPathValidator implements Validator {
+
+    @Override
+    public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+        if (PropertyNameUtil.hasFieldType(subject) && !PropertyNameUtil.isFieldTypeValid(subject)) {
+            final String fieldType = PropertyNameUtil.getFieldTypeName(subject).get();
+
+            return new ValidationResult.Builder()
+                .subject(subject)
+                .input(input)
+                .valid(false)
+                .explanation("Invalid field type. If property name contains a colon (:) it must use syntax of "
+                    + "<field name>:<field type> but the specified field type ('" + fieldType + "') is not a valid field type")
+                .build();
+        }
+
+        try {
+            JsonPath.compile(input);
+        } catch (final Exception e) {
+            return new ValidationResult.Builder()
+                .subject(subject)
+                .input(input)
+                .valid(false)
+                .explanation("Invalid JSON Path Expression: " + e.getMessage())
+                .build();
+        }
+
+        return new ValidationResult.Builder()
+            .subject(subject)
+            .input(input)
+            .valid(true)
+            .build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..dc75a51
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.json;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.AbstractRecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+
+@Tags({"json", "resultset", "writer", "serialize", "record", "row"})
+@CapabilityDescription("Writes the results of a Database ResultSet as a JSON Array. Even if the ResultSet "
+    + "consists of a single row, it will be written as an array with a single element.")
+public class JsonRecordSetWriter extends AbstractRecordSetWriter implements RecordSetWriterFactory {
+
+    static final PropertyDescriptor PRETTY_PRINT_JSON = new PropertyDescriptor.Builder()
+        .name("Pretty Print JSON")
+        .description("Specifies whether or not the JSON should be pretty printed")
+        .expressionLanguageSupported(false)
+        .allowableValues("true", "false")
+        .defaultValue("false")
+        .required(true)
+        .build();
+
+    private boolean prettyPrint;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(PRETTY_PRINT_JSON);
+        return properties;
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        prettyPrint = context.getProperty(PRETTY_PRINT_JSON).asBoolean();
+    }
+
+    @Override
+    public RecordSetWriter createWriter(final ComponentLog logger) {
+        return new WriteJsonResult(logger, prettyPrint, getDateFormat(), getTimeFormat(), getTimestampFormat());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..2d7072a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.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.json;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RowRecordReaderFactory;
+import org.apache.nifi.serialization.UserTypeOverrideRowReader;
+
+@Tags({"json", "tree", "record", "reader", "parser"})
+@CapabilityDescription("Parses JSON into individual Record objects. The Record that is produced will contain all top-level "
+    + "elements of the corresponding JSON Object. If the JSON has nested arrays, those values will be represented as an Object array for that field. "
+    + "Nested JSON objects will be represented as a Map. "
+    + "The root JSON element can be either a single element or an array of JSON elements, and each "
+    + "element in that array will be treated as a separate record. If any of the elements has a nested array or a nested "
+    + "element, they will be returned as OBJECT or ARRAY types (respectively), not flattened out into individual fields. "
+    + "The schema for the record is determined by the first JSON element in the array, if the incoming FlowFile is a JSON array. "
+    + "This means that if a field does not exist in the first JSON object, then it will be skipped in all subsequent JSON objects. "
+    + "The data type of a field can be overridden by adding a property to "
+    + "the controller service where the name of the property matches the JSON field name and the value of the property is "
+    + "the data type to use. If that field does not exist in a JSON element, the field will be assumed to be null. "
+    + "See the Usage of the Controller Service for more information.")
+@SeeAlso(JsonPathReader.class)
+@DynamicProperty(name = "<name of JSON field>", value = "<data type of JSON field>",
+    description = "User-defined properties are used to indicate that the values of a specific field should be interpreted as a "
+    + "user-defined data type (e.g., int, double, float, date, etc.)", supportsExpressionLanguage = false)
+public class JsonTreeReader extends UserTypeOverrideRowReader implements RowRecordReaderFactory {
+
+    @Override
+    public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
+        return new JsonTreeRowRecordReader(in, logger, getFieldTypeOverrides());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..4a2d212
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.json;
+
+import 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 java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+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.RecordSchema;
+import org.codehaus.jackson.JsonNode;
+
+
+public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
+    private final Map<String, DataType> fieldTypeOverrides;
+    private RecordSchema schema;
+
+    public JsonTreeRowRecordReader(final InputStream in, final ComponentLog logger, final Map<String, DataType> fieldTypeOverrides) throws IOException, MalformedRecordException {
+        super(in, logger);
+        this.fieldTypeOverrides = fieldTypeOverrides;
+    }
+
+    @Override
+    protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema) throws IOException, MalformedRecordException {
+        if (jsonNode == null) {
+            return null;
+        }
+
+        final Map<String, Object> values = new HashMap<>(schema.getFieldCount());
+        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);
+
+            final DataType desiredType = field.getDataType();
+            final Object value = convertField(fieldNode, fieldName, desiredType);
+            values.put(fieldName, value);
+        }
+
+        return new MapRecord(schema, values);
+    }
+
+
+    @Override
+    public RecordSchema getSchema() {
+        if (schema != null) {
+            return schema;
+        }
+
+        final List<RecordField> recordFields = new ArrayList<>();
+        final Optional<JsonNode> firstNodeOption = getFirstJsonNode();
+
+        if (firstNodeOption.isPresent()) {
+            final Iterator<Map.Entry<String, JsonNode>> itr = firstNodeOption.get().getFields();
+            while (itr.hasNext()) {
+                final Map.Entry<String, JsonNode> entry = itr.next();
+                final String elementName = entry.getKey();
+                final JsonNode node = entry.getValue();
+
+                DataType dataType;
+                final DataType overriddenDataType = fieldTypeOverrides.get(elementName);
+                if (overriddenDataType == null) {
+                    dataType = determineFieldType(node);
+                } else {
+                    dataType = overriddenDataType;
+                }
+
+                recordFields.add(new RecordField(elementName, dataType));
+            }
+        }
+
+        // If there are any overridden field types that we didn't find, add as the last fields.
+        final Set<String> knownFieldNames = recordFields.stream()
+            .map(f -> f.getFieldName())
+            .collect(Collectors.toSet());
+
+        for (final Map.Entry<String, DataType> entry : fieldTypeOverrides.entrySet()) {
+            if (!knownFieldNames.contains(entry.getKey())) {
+                recordFields.add(new RecordField(entry.getKey(), entry.getValue()));
+            }
+        }
+
+        schema = new SimpleRecordSchema(recordFields);
+        return schema;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/PropertyNameUtil.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/PropertyNameUtil.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/PropertyNameUtil.java
new file mode 100644
index 0000000..3b7dcf9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/PropertyNameUtil.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.json;
+
+import java.util.Optional;
+
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+public class PropertyNameUtil {
+
+    public static String getFieldName(final String propertyName) {
+        final int colonIndex = propertyName.indexOf(":");
+        if (colonIndex > -1 && colonIndex < propertyName.length() - 1) {
+            return propertyName.substring(0, colonIndex);
+        }
+
+        return propertyName;
+    }
+
+    public static boolean hasFieldType(final String propertyName) {
+        final int colonIndex = propertyName.indexOf(":");
+        return (colonIndex > -1 && colonIndex < propertyName.length() - 1);
+    }
+
+    public static Optional<String> getFieldTypeName(final String propertyName) {
+        if (hasFieldType(propertyName)) {
+            final String[] splits = propertyName.split("\\:");
+            if (splits.length > 1) {
+                return Optional.of(splits[1]);
+            }
+            return Optional.empty();
+        }
+
+        return Optional.empty();
+    }
+
+    public static Optional<String> getFieldFormat(final String propertyName) {
+        final String[] splits = propertyName.split("\\:");
+        if (splits.length != 3) {
+            return Optional.empty();
+        }
+
+        return Optional.of(splits[2]);
+    }
+
+    public static boolean isFieldTypeValid(final String propertyName) {
+        final Optional<String> fieldType = getFieldTypeName(propertyName);
+        if (!fieldType.isPresent()) {
+            return false;
+        }
+
+        final String typeName = fieldType.get();
+        final RecordFieldType recordFieldType = RecordFieldType.of(typeName);
+        return recordFieldType != null;
+    }
+
+    public static Optional<DataType> getDataType(final String propertyName) {
+        if (isFieldTypeValid(propertyName)) {
+            final String typeName = getFieldTypeName(propertyName).get();
+            final RecordFieldType fieldType = RecordFieldType.of(typeName);
+
+            final Optional<String> format = getFieldFormat(propertyName);
+            if (format.isPresent()) {
+                return Optional.of(fieldType.getDataType(format.get()));
+            } else {
+                return Optional.of(fieldType.getDataType());
+            }
+        }
+
+        return Optional.empty();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..cf72b19
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java
@@ -0,0 +1,309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.json;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.sql.Array;
+import java.sql.SQLException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.DataTypeUtils;
+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.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.stream.io.NonCloseableOutputStream;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonGenerator;
+
+public class WriteJsonResult implements RecordSetWriter {
+    private final boolean prettyPrint;
+
+    private final ComponentLog logger;
+    private final JsonFactory factory = new JsonFactory();
+    private final DateFormat dateFormat;
+    private final DateFormat timeFormat;
+    private final DateFormat timestampFormat;
+
+    public WriteJsonResult(final ComponentLog logger, final boolean prettyPrint, final String dateFormat, final String timeFormat, final String timestampFormat) {
+        this.prettyPrint = prettyPrint;
+        this.dateFormat = new SimpleDateFormat(dateFormat);
+        this.timeFormat = new SimpleDateFormat(timeFormat);
+        this.timestampFormat = new SimpleDateFormat(timestampFormat);
+        this.logger = logger;
+    }
+
+    @Override
+    public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws IOException {
+        int count = 0;
+
+        try (final JsonGenerator generator = factory.createJsonGenerator(new NonCloseableOutputStream(rawOut))) {
+            if (prettyPrint) {
+                generator.useDefaultPrettyPrinter();
+            }
+
+            generator.writeStartArray();
+
+            Record record;
+            while ((record = rs.next()) != null) {
+                count++;
+                writeRecord(record, generator, g -> g.writeStartObject(), g -> g.writeEndObject());
+            }
+
+            generator.writeEndArray();
+        } catch (final SQLException e) {
+            throw new IOException("Failed to serialize Result Set to stream", e);
+        }
+
+        return WriteResult.of(count, Collections.emptyMap());
+    }
+
+    @Override
+    public WriteResult write(final Record record, final OutputStream rawOut) throws IOException {
+        try (final JsonGenerator generator = factory.createJsonGenerator(new NonCloseableOutputStream(rawOut))) {
+            if (prettyPrint) {
+                generator.useDefaultPrettyPrinter();
+            }
+
+            writeRecord(record, generator, g -> g.writeStartObject(), g -> g.writeEndObject());
+        } catch (final SQLException e) {
+            throw new IOException("Failed to write records to stream", e);
+        }
+
+        return WriteResult.of(1, Collections.emptyMap());
+    }
+
+    private void writeRecord(final Record record, 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);
+                if (value == null) {
+                    generator.writeNullField(fieldName);
+                    continue;
+                }
+
+                generator.writeFieldName(fieldName);
+                final DataType dataType = schema.getDataType(fieldName).get();
+
+                writeValue(generator, value, dataType, i < schema.getFieldCount() - 1);
+            }
+
+            endTask.apply(generator);
+        } catch (final Exception e) {
+            logger.error("Failed to write {} with schema {} as a JSON Object due to {}", new Object[] {record, record.getSchema(), e.toString(), e});
+            throw e;
+        }
+    }
+
+    private String createDate(final Object value, final DateFormat format) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Date) {
+            return format.format((Date) value);
+        }
+        if (value instanceof java.sql.Date) {
+            return format.format(new Date(((java.sql.Date) value).getTime()));
+        }
+        if (value instanceof java.sql.Time) {
+            return format.format(new Date(((java.sql.Time) value).getTime()));
+        }
+        if (value instanceof java.sql.Timestamp) {
+            return format.format(new Date(((java.sql.Timestamp) value).getTime()));
+        }
+
+        return null;
+    }
+
+    private void writeValue(final JsonGenerator generator, final Object value, final DataType dataType, final boolean moreCols)
+        throws JsonGenerationException, IOException, SQLException {
+        if (value == null) {
+            generator.writeNull();
+            return;
+        }
+
+        final DataType resolvedDataType;
+        if (dataType.getFieldType() == RecordFieldType.CHOICE) {
+            resolvedDataType = DataTypeUtils.inferDataType(value);
+        } else {
+            resolvedDataType = dataType;
+        }
+
+        switch (resolvedDataType.getFieldType()) {
+            case DATE:
+                generator.writeString(createDate(value, dateFormat));
+                break;
+            case TIME:
+                generator.writeString(createDate(value, timeFormat));
+                break;
+            case TIMESTAMP:
+                generator.writeString(createDate(value, timestampFormat));
+                break;
+            case DOUBLE:
+                generator.writeNumber(DataTypeUtils.toDouble(value, 0D));
+                break;
+            case FLOAT:
+                generator.writeNumber(DataTypeUtils.toFloat(value, 0F));
+                break;
+            case LONG:
+                generator.writeNumber(DataTypeUtils.toLong(value, 0L));
+                break;
+            case INT:
+            case BYTE:
+            case SHORT:
+                generator.writeNumber(DataTypeUtils.toInteger(value, 0));
+                break;
+            case CHAR:
+            case STRING:
+                generator.writeString(value.toString());
+                break;
+            case BIGINT:
+                if (value instanceof Long) {
+                    generator.writeNumber(((Long) value).longValue());
+                } else {
+                    generator.writeNumber((BigInteger) value);
+                }
+                break;
+            case BOOLEAN:
+                final String stringValue = value.toString();
+                if ("true".equalsIgnoreCase(stringValue)) {
+                    generator.writeBoolean(true);
+                } else if ("false".equalsIgnoreCase(stringValue)) {
+                    generator.writeBoolean(false);
+                } else {
+                    generator.writeString(stringValue);
+                }
+                break;
+            case RECORD: {
+                final Record record = (Record) value;
+                writeRecord(record, generator, gen -> gen.writeStartObject(), gen -> gen.writeEndObject());
+                break;
+            }
+            case ARRAY:
+            default:
+                if ("null".equals(value.toString())) {
+                    generator.writeNull();
+                } else if (value instanceof Map) {
+                    final Map<?, ?> map = (Map<?, ?>) value;
+                    generator.writeStartObject();
+
+                    int i = 0;
+                    for (final Map.Entry<?, ?> entry : map.entrySet()) {
+                        generator.writeFieldName(entry.getKey().toString());
+                        final boolean moreEntries = ++i < map.size();
+                        writeValue(generator, entry.getValue(), getColType(entry.getValue()), moreEntries);
+                    }
+                    generator.writeEndObject();
+                } else if (value instanceof List) {
+                    final List<?> list = (List<?>) value;
+                    writeArray(list.toArray(), generator);
+                } else if (value instanceof Array) {
+                    final Array array = (Array) value;
+                    final Object[] values = (Object[]) array.getArray();
+                    writeArray(values, generator);
+                } else if (value instanceof Object[]) {
+                    final Object[] values = (Object[]) value;
+                    writeArray(values, generator);
+                } else {
+                    generator.writeString(value.toString());
+                }
+                break;
+        }
+    }
+
+    private void writeArray(final Object[] values, final JsonGenerator generator) 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, getColType(element), moreEntries);
+        }
+        generator.writeEndArray();
+    }
+
+    private DataType getColType(final Object value) {
+        if (value instanceof String) {
+            return RecordFieldType.STRING.getDataType();
+        }
+        if (value instanceof Double) {
+            return RecordFieldType.DOUBLE.getDataType();
+        }
+        if (value instanceof Float) {
+            return RecordFieldType.FLOAT.getDataType();
+        }
+        if (value instanceof Integer) {
+            return RecordFieldType.INT.getDataType();
+        }
+        if (value instanceof Long) {
+            return RecordFieldType.LONG.getDataType();
+        }
+        if (value instanceof BigInteger) {
+            return RecordFieldType.BIGINT.getDataType();
+        }
+        if (value instanceof Boolean) {
+            return RecordFieldType.BOOLEAN.getDataType();
+        }
+        if (value instanceof Byte || value instanceof Short) {
+            return RecordFieldType.INT.getDataType();
+        }
+        if (value instanceof Character) {
+            return RecordFieldType.STRING.getDataType();
+        }
+        if (value instanceof java.util.Date || value instanceof java.sql.Date) {
+            return RecordFieldType.DATE.getDataType();
+        }
+        if (value instanceof java.sql.Time) {
+            return RecordFieldType.TIME.getDataType();
+        }
+        if (value instanceof java.sql.Timestamp) {
+            return RecordFieldType.TIMESTAMP.getDataType();
+        }
+        if (value instanceof Object[] || value instanceof List || value instanceof Array) {
+            return RecordFieldType.ARRAY.getDataType();
+        }
+
+        return RecordFieldType.RECORD.getDataType();
+    }
+
+    @Override
+    public String getMimeType() {
+        return "application/json";
+    }
+
+    private static interface GeneratorTask {
+        void apply(JsonGenerator generator) throws JsonGenerationException, IOException;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/AbstractRecordSetWriter.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/AbstractRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/AbstractRecordSetWriter.java
new file mode 100644
index 0000000..b58a22e
--- /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/AbstractRecordSetWriter.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+public abstract class AbstractRecordSetWriter extends AbstractControllerService {
+    static final PropertyDescriptor DATE_FORMAT = new PropertyDescriptor.Builder()
+        .name("Date Format")
+        .description("Specifies the format to use when writing out Date fields")
+        .expressionLanguageSupported(false)
+        .defaultValue(RecordFieldType.DATE.getDefaultFormat())
+        .addValidator(new SimpleDateFormatValidator())
+        .required(true)
+        .build();
+
+    static final PropertyDescriptor TIME_FORMAT = new PropertyDescriptor.Builder()
+        .name("Time Format")
+        .description("Specifies the format to use when writing out Time fields")
+        .expressionLanguageSupported(false)
+        .defaultValue(RecordFieldType.TIME.getDefaultFormat())
+        .addValidator(new SimpleDateFormatValidator())
+        .required(true)
+        .build();
+
+    static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+        .name("Timestamp Format")
+        .description("Specifies the format to use when writing out Timestamp (date/time) fields")
+        .expressionLanguageSupported(false)
+        .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+        .addValidator(new SimpleDateFormatValidator())
+        .required(true)
+        .build();
+
+    private volatile String dateFormat;
+    private volatile String timeFormat;
+    private volatile String timestampFormat;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
+    }
+
+    @OnEnabled
+    public void captureValues(final ConfigurationContext context) {
+        this.dateFormat = context.getProperty(DATE_FORMAT).getValue();
+        this.timeFormat = context.getProperty(TIME_FORMAT).getValue();
+        this.timestampFormat = context.getProperty(TIMESTAMP_FORMAT).getValue();
+    }
+
+    protected String getDateFormat() {
+        return dateFormat;
+    }
+
+    protected String getTimeFormat() {
+        return timeFormat;
+    }
+
+    protected String getTimestampFormat() {
+        return timestampFormat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java
new file mode 100644
index 0000000..de207f4
--- /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/DataTypeUtils.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public class DataTypeUtils {
+
+    public static Double toDouble(final Object value, final Double defaultValue) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).doubleValue();
+        }
+
+        if (value instanceof String) {
+            return Double.parseDouble((String) value);
+        }
+
+        return defaultValue;
+    }
+
+    public static Float toFloat(final Object value, final Float defaultValue) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).floatValue();
+        }
+
+        if (value instanceof String) {
+            return Float.parseFloat((String) value);
+        }
+
+        return defaultValue;
+    }
+
+    public static Long toLong(final Object value, final Long defaultValue) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).longValue();
+        }
+
+        if (value instanceof String) {
+            return Long.parseLong((String) value);
+        }
+
+        return defaultValue;
+    }
+
+
+
+    public static Integer toInteger(final Object value, final Integer defaultValue) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).intValue();
+        }
+
+        if (value instanceof String) {
+            return Integer.parseInt((String) value);
+        }
+
+        return defaultValue;
+    }
+
+
+    /**
+     * Deduces the type of RecordFieldType that should be used for a value of the given type,
+     * or returns <code>null</code> if the value is null
+     *
+     * @param value the value whose type should be deduced
+     * @return the type of RecordFieldType that should be used for a value of the given type,
+     *         or <code>null</code> if the value is null
+     */
+    public static DataType inferDataType(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof String) {
+            return RecordFieldType.STRING.getDataType();
+        }
+        if (value instanceof Long) {
+            return RecordFieldType.LONG.getDataType();
+        }
+        if (value instanceof Integer) {
+            return RecordFieldType.INT.getDataType();
+        }
+        if (value instanceof Double) {
+            return RecordFieldType.DOUBLE.getDataType();
+        }
+        if (value instanceof Float) {
+            return RecordFieldType.FLOAT.getDataType();
+        }
+        if (value instanceof Boolean) {
+            return RecordFieldType.BOOLEAN.getDataType();
+        }
+        if (value instanceof Byte) {
+            return RecordFieldType.BYTE.getDataType();
+        }
+        if (value instanceof Character) {
+            return RecordFieldType.CHAR.getDataType();
+        }
+        if (value instanceof Short) {
+            return RecordFieldType.SHORT.getDataType();
+        }
+        if (value instanceof Date) {
+            return RecordFieldType.DATE.getDataType();
+        }
+        if (value instanceof Object[] || value instanceof List) {
+            return RecordFieldType.ARRAY.getDataType();
+        }
+        if (value instanceof Map) {
+            @SuppressWarnings("unchecked")
+            final Map<String, Object> map = (Map<String, Object>) value;
+            final RecordSchema childSchema = determineSchema(map);
+            return RecordFieldType.RECORD.getDataType(childSchema);
+        }
+
+        return RecordFieldType.RECORD.getDataType();
+    }
+
+    public static RecordSchema determineSchema(final Map<String, Object> valueMap) {
+        final List<RecordField> fields = new ArrayList<>(valueMap.size());
+        for (final Map.Entry<String, Object> entry : valueMap.entrySet()) {
+            final DataType valueType = inferDataType(entry.getValue());
+            final String fieldName = entry.getKey();
+            final RecordField field = new RecordField(fieldName, valueType);
+            fields.add(field);
+        }
+        return new SimpleRecordSchema(fields);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SimpleDateFormatValidator.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/SimpleDateFormatValidator.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SimpleDateFormatValidator.java
new file mode 100644
index 0000000..f25749b
--- /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/SimpleDateFormatValidator.java
@@ -0,0 +1,48 @@
+/*
+ * 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.text.SimpleDateFormat;
+
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+
+public class SimpleDateFormatValidator implements Validator {
+
+    @Override
+    public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+        try {
+            new SimpleDateFormat(input);
+        } catch (final Exception e) {
+            return new ValidationResult.Builder()
+                .input(input)
+                .subject(subject)
+                .valid(false)
+                .explanation("Invalid Date format: " + e.getMessage())
+                .build();
+        }
+
+        return new ValidationResult.Builder()
+            .input(input)
+            .subject(subject)
+            .valid(true)
+            .build();
+    }
+
+}

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..07da00e
--- /dev/null
+++ 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
@@ -0,0 +1,80 @@
+/*
+ * 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.text;
+
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+
+@Tags({"text", "freeform", "expression", "language", "el", "resultset", "writer", "serialize"})
+@CapabilityDescription("Writes the contents of a Database ResultSet as free-form text. The configured "
+    + "text is able to make use of the Expression Language to reference each of the columns that are available "
+    + "in the ResultSet. Each record in the ResultSet will be separated by a single newline character.")
+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)}\"")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .required(true)
+        .build();
+    static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder()
+        .name("Character Set")
+        .description("The Character set to use when writing the data to the FlowFile")
+        .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
+        .defaultValue("UTF-8")
+        .expressionLanguageSupported(false)
+        .required(true)
+        .build();
+
+    private volatile PropertyValue textValue;
+    private volatile Charset characterSet;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(TEXT);
+        properties.add(CHARACTER_SET);
+        return properties;
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        textValue = context.getProperty(TEXT);
+        characterSet = Charset.forName(context.getProperty(CHARACTER_SET).getValue());
+    }
+
+    @Override
+    public RecordSetWriter createWriter(final ComponentLog logger) {
+        return new FreeFormTextWriter(textValue, characterSet);
+    }
+
+}


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

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/NOTICE
index 1848020..f5f0bb1 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/NOTICE
@@ -64,11 +64,14 @@ The following binary components are provided under the Apache Software License v
     The following NOTICE information applies:
       Copyright 2011 JSON-SMART authors
 
-   (ASLv2) JsonPath
-     The following NOTICE information applies:
-       Copyright 2011 JsonPath authors
+  (ASLv2) JsonPath
+    The following NOTICE information applies:
+      Copyright 2011 JsonPath authors
 
-  (ASLv2) opencsv (net.sf.opencsv:opencsv:2.3)
+  (ASLv2) Apache Commons CSV
+	The following NOTICE information applies:
+	  Apache Commons CSV
+	  Copyright 2005-2016 The Apache Software Foundation
 
   (ASLv2) Apache Avro
     The following NOTICE information applies:

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/.gitignore
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/.gitignore b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/.gitignore
deleted file mode 100644
index ae3c172..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/bin/

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/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 9b2a56c..d86a8c5 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
@@ -34,6 +34,10 @@
             <artifactId>nifi-record-serialization-service-api</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-schema-registry-service-api</artifactId>
+        </dependency>
+        <dependency>
             <groupId>com.jayway.jsonpath</groupId>
             <artifactId>json-path</artifactId>
         </dependency>
@@ -50,9 +54,13 @@
             <artifactId>commons-lang3</artifactId>
         </dependency>
         <dependency>
-            <groupId>net.sf.opencsv</groupId>
-            <artifactId>opencsv</artifactId>
-            <version>2.3</version>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-csv</artifactId>
+            <version>1.4</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
         </dependency>
         <dependency>
             <groupId>io.thekraken</groupId>
@@ -62,6 +70,7 @@
         <dependency>
             <groupId>org.apache.avro</groupId>
             <artifactId>avro</artifactId>
+            <version>1.8.1</version>
         </dependency>
     </dependencies>
 
@@ -72,6 +81,8 @@
                 <artifactId>apache-rat-plugin</artifactId>
                 <configuration>
                     <excludes combine.children="append">
+                        <exclude>src/test/resources/avro/datatypes.avsc</exclude>
+                        <exclude>src/test/resources/avro/logical-types.avsc</exclude>
                         <exclude>src/test/resources/csv/extra-white-space.csv</exclude>
                         <exclude>src/test/resources/csv/multi-bank-account.csv</exclude>
                         <exclude>src/test/resources/csv/single-bank-account.csv</exclude>
@@ -80,12 +91,14 @@
                         <exclude>src/test/resources/grok/nifi-log-sample.log</exclude>
                         <exclude>src/test/resources/grok/single-line-log-messages.txt</exclude>
                         <exclude>src/test/resources/json/bank-account-array-different-schemas.json</exclude>
+                        <exclude>src/test/resources/json/bank-account-array-optional-balance.json</exclude>
                         <exclude>src/test/resources/json/bank-account-array.json</exclude>
                         <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-element-nested-array.json</exclude>
                         <exclude>src/test/resources/json/single-element-nested.json</exclude>
+                        <exclude>src/test/resources/json/output/dataTypes.json</exclude>
                     </excludes>
                 </configuration>
             </plugin>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 fc0c598..f92816f 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
@@ -23,18 +23,27 @@ import java.io.InputStream;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 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;
 
 @Tags({"avro", "parse", "record", "row", "reader", "delimited", "comma", "separated", "values"})
-@CapabilityDescription("Parses Avro data and returns each Avro record as an separate record.")
+@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 {
 
     @Override
-    public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws MalformedRecordException, IOException {
+    public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws MalformedRecordException, IOException {
         return new AvroRecordReader(in);
     }
 
+    @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;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 e98a5ad..d725cbf 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
@@ -24,11 +24,12 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
+import java.util.concurrent.TimeUnit;
 
+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.Schema.Type;
 import org.apache.avro.file.DataFileStream;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericData.Array;
@@ -44,8 +45,8 @@ 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;
 
 public class AvroRecordReader implements RecordReader {
     private final InputStream in;
@@ -53,6 +54,7 @@ public class AvroRecordReader implements RecordReader {
     private final DataFileStream<GenericRecord> dataFileStream;
     private RecordSchema recordSchema;
 
+
     public AvroRecordReader(final InputStream in) throws IOException, MalformedRecordException {
         this.in = in;
 
@@ -79,65 +81,78 @@ public class AvroRecordReader implements RecordReader {
         }
 
         final RecordSchema schema = getSchema();
-        final Map<String, Object> values = convertRecordToObjectArray(record, schema);
+        final Map<String, Object> values = convertAvroRecordToMap(record, schema);
         return new MapRecord(schema, values);
     }
 
 
-    private Map<String, Object> convertRecordToObjectArray(final GenericRecord record, final RecordSchema schema) {
-        final Map<String, Object> values = new HashMap<>(schema.getFieldCount());
+    private Map<String, Object> convertAvroRecordToMap(final GenericRecord avroRecord, final RecordSchema recordSchema) {
+        final Map<String, Object> values = new HashMap<>(recordSchema.getFieldCount());
 
-        for (final String fieldName : schema.getFieldNames()) {
-            final Object value = record.get(fieldName);
+        for (final String fieldName : recordSchema.getFieldNames()) {
+            final Object value = avroRecord.get(fieldName);
 
-            final Field avroField = record.getSchema().getField(fieldName);
+            final Field avroField = avroRecord.getSchema().getField(fieldName);
             if (avroField == null) {
                 values.put(fieldName, null);
                 continue;
             }
 
             final Schema fieldSchema = avroField.schema();
-            final DataType dataType = schema.getDataType(fieldName).orElse(null);
-            final Object converted = convertValue(value, fieldSchema, avroField.name(), dataType);
-            values.put(fieldName, converted);
-        }
+            final Object rawValue = normalizeValue(value, fieldSchema);
 
-        return values;
-    }
-
-
-    @Override
-    public RecordSchema getSchema() throws MalformedRecordException {
-        if (recordSchema != null) {
-            return recordSchema;
-        }
-
-        recordSchema = createSchema(avroSchema);
-        return recordSchema;
-    }
+            final DataType desiredType = recordSchema.getDataType(fieldName).get();
+            final Object coercedValue = DataTypeUtils.convertType(rawValue, desiredType);
 
-    private RecordSchema createSchema(final Schema avroSchema) {
-        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));
+            values.put(fieldName, coercedValue);
         }
 
-        final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
-        return recordSchema;
+        return values;
     }
 
-    private Object convertValue(final Object value, final Schema avroSchema, final String fieldName, final DataType desiredType) {
+    private Object normalizeValue(final Object value, final Schema avroSchema) {
         if (value == null) {
             return null;
         }
 
         switch (avroSchema.getType()) {
+            case INT: {
+                final LogicalType logicalType = avroSchema.getLogicalType();
+                if (logicalType == null) {
+                    return value;
+                }
+
+                final String logicalName = logicalType.getName();
+                if (LogicalTypes.date().getName().equals(logicalName)) {
+                    // date logical name means that the value is number of days since Jan 1, 1970
+                    return new java.sql.Date(TimeUnit.DAYS.toMillis((int) value));
+                } else if (LogicalTypes.timeMillis().equals(logicalName)) {
+                    // time-millis logical name means that the value is number of milliseconds since midnight.
+                    return new java.sql.Time((int) value);
+                }
+
+                break;
+            }
+            case LONG: {
+                final LogicalType logicalType = avroSchema.getLogicalType();
+                if (logicalType == null) {
+                    return value;
+                }
+
+                final String logicalName = logicalType.getName();
+                if (LogicalTypes.timeMicros().getName().equals(logicalName)) {
+                    return new java.sql.Time(TimeUnit.MICROSECONDS.toMillis((long) value));
+                } else if (LogicalTypes.timestampMillis().getName().equals(logicalName)) {
+                    return new java.sql.Timestamp((long) value);
+                } else if (LogicalTypes.timestampMicros().getName().equals(logicalName)) {
+                    return new java.sql.Timestamp(TimeUnit.MICROSECONDS.toMillis((long) value));
+                }
+                break;
+            }
             case UNION:
                 if (value instanceof GenericData.Record) {
-                    final GenericData.Record record = (GenericData.Record) value;
-                    return convertValue(value, record.getSchema(), fieldName, desiredType);
+                    final GenericData.Record avroRecord = (GenericData.Record) value;
+                    return normalizeValue(value, avroRecord.getSchema());
                 }
                 break;
             case RECORD:
@@ -146,19 +161,18 @@ public class AvroRecordReader implements RecordReader {
                 final List<Field> recordFields = recordSchema.getFields();
                 final Map<String, Object> values = new HashMap<>(recordFields.size());
                 for (final Field field : recordFields) {
-                    final DataType desiredFieldType = determineDataType(field.schema());
                     final Object avroFieldValue = record.get(field.name());
-                    final Object fieldValue = convertValue(avroFieldValue, field.schema(), field.name(), desiredFieldType);
+                    final Object fieldValue = normalizeValue(avroFieldValue, field.schema());
                     values.put(field.name(), fieldValue);
                 }
-                final RecordSchema childSchema = createSchema(recordSchema);
+                final RecordSchema childSchema = AvroTypeUtil.createSchema(recordSchema);
                 return new MapRecord(childSchema, values);
             case BYTES:
                 final ByteBuffer bb = (ByteBuffer) value;
-                return bb.array();
+                return AvroTypeUtil.convertByteArray(bb.array());
             case FIXED:
                 final GenericFixed fixed = (GenericFixed) value;
-                return fixed.bytes();
+                return AvroTypeUtil.convertByteArray(fixed.bytes());
             case ENUM:
                 return value.toString();
             case NULL:
@@ -170,7 +184,7 @@ public class AvroRecordReader implements RecordReader {
                 final Object[] valueArray = new Object[array.size()];
                 for (int i = 0; i < array.size(); i++) {
                     final Schema elementSchema = avroSchema.getElementType();
-                    valueArray[i] = convertValue(array.get(i), elementSchema, fieldName, determineDataType(elementSchema));
+                    valueArray[i] = normalizeValue(array.get(i), elementSchema);
                 }
                 return valueArray;
             case MAP:
@@ -182,73 +196,32 @@ public class AvroRecordReader implements RecordReader {
                         obj = obj.toString();
                     }
 
-                    map.put(entry.getKey().toString(), obj);
+                    final String key = entry.getKey().toString();
+                    obj = normalizeValue(obj, avroSchema.getValueType());
+
+                    map.put(key, obj);
+                }
+
+                final DataType elementType = AvroTypeUtil.determineDataType(avroSchema.getValueType());
+                final List<RecordField> mapFields = new ArrayList<>();
+                for (final String key : map.keySet()) {
+                    mapFields.add(new RecordField(key, elementType));
                 }
-                return map;
+                final RecordSchema mapSchema = new SimpleRecordSchema(mapFields);
+                return new MapRecord(mapSchema, map);
         }
 
         return value;
     }
 
 
-    private DataType determineDataType(final Schema avroSchema) {
-        final Type avroType = avroSchema.getType();
-
-        switch (avroType) {
-            case ARRAY:
-            case BYTES:
-            case FIXED:
-                return RecordFieldType.ARRAY.getDataType();
-            case BOOLEAN:
-                return RecordFieldType.BOOLEAN.getDataType();
-            case DOUBLE:
-                return RecordFieldType.DOUBLE.getDataType();
-            case ENUM:
-            case STRING:
-                return RecordFieldType.STRING.getDataType();
-            case FLOAT:
-                return RecordFieldType.FLOAT.getDataType();
-            case INT:
-                return RecordFieldType.INT.getDataType();
-            case LONG:
-                return RecordFieldType.LONG.getDataType();
-            case RECORD: {
-                final List<Field> avroFields = avroSchema.getFields();
-                final List<RecordField> recordFields = new ArrayList<>(avroFields.size());
-
-                for (final Field field : avroFields) {
-                    final String fieldName = field.name();
-                    final Schema fieldSchema = field.schema();
-                    final DataType fieldType = determineDataType(fieldSchema);
-                    recordFields.add(new RecordField(fieldName, fieldType));
-                }
-
-                final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
-                return RecordFieldType.RECORD.getDataType(recordSchema);
-            }
-            case NULL:
-            case MAP:
-                return RecordFieldType.RECORD.getDataType();
-            case UNION: {
-                final List<Schema> nonNullSubSchemas = avroSchema.getTypes().stream()
-                    .filter(s -> s.getType() != Type.NULL)
-                    .collect(Collectors.toList());
-
-                if (nonNullSubSchemas.size() == 1) {
-                    return determineDataType(nonNullSubSchemas.get(0));
-                }
-
-                final List<DataType> possibleChildTypes = new ArrayList<>(nonNullSubSchemas.size());
-                for (final Schema subSchema : nonNullSubSchemas) {
-                    final DataType childDataType = determineDataType(subSchema);
-                    possibleChildTypes.add(childDataType);
-                }
-
-                return RecordFieldType.CHOICE.getDataType(possibleChildTypes);
-            }
+    @Override
+    public RecordSchema getSchema() throws MalformedRecordException {
+        if (recordSchema != null) {
+            return recordSchema;
         }
 
-        return null;
+        recordSchema = AvroTypeUtil.createSchema(avroSchema);
+        return recordSchema;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 d56c716..03d766c 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
@@ -25,18 +25,15 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.serialization.AbstractRecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
 
-@Tags({"avro", "result", "set", "writer", "serializer", "record", "row"})
-@CapabilityDescription("Writes the contents of a Database ResultSet in Binary Avro format. The data types in the Result Set must match those "
-    + "specified by the Avro Schema. No type coercion will occur, with the exception of Date, Time, and Timestamps fields because Avro does not provide "
-    + "support for these types specifically. As a result, they will be converted to String fields using the configured formats. In addition, the label"
-    + "of the column must be a valid Avro field name.")
-public class AvroRecordSetWriter extends AbstractRecordSetWriter implements RecordSetWriterFactory {
+@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")
@@ -49,7 +46,7 @@ public class AvroRecordSetWriter extends AbstractRecordSetWriter implements Reco
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        final List<PropertyDescriptor> properties = new ArrayList<>();
         properties.add(SCHEMA);
         return properties;
     }
@@ -61,7 +58,7 @@ public class AvroRecordSetWriter extends AbstractRecordSetWriter implements Reco
 
     @Override
     public RecordSetWriter createWriter(final ComponentLog logger) {
-        return new WriteAvroResult(schema, getDateFormat(), getTimeFormat(), getTimestampFormat());
+        return new WriteAvroResult(schema);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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
new file mode 100644
index 0000000..1810c83
--- /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/AvroTypeUtil.java
@@ -0,0 +1,159 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+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.Schema.Type;
+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.util.IllegalTypeConversionException;
+
+public class AvroTypeUtil {
+
+    public static DataType determineDataType(final Schema avroSchema) {
+        final Type avroType = avroSchema.getType();
+
+        switch (avroType) {
+            case BYTES:
+            case FIXED:
+                return RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType());
+            case ARRAY:
+                final DataType elementType = determineDataType(avroSchema.getElementType());
+                return RecordFieldType.ARRAY.getArrayDataType(elementType);
+            case BOOLEAN:
+                return RecordFieldType.BOOLEAN.getDataType();
+            case DOUBLE:
+                return RecordFieldType.DOUBLE.getDataType();
+            case ENUM:
+            case STRING:
+                return RecordFieldType.STRING.getDataType();
+            case FLOAT:
+                return RecordFieldType.FLOAT.getDataType();
+            case INT: {
+                final LogicalType logicalType = avroSchema.getLogicalType();
+                if (logicalType == null) {
+                    return RecordFieldType.INT.getDataType();
+                }
+
+                if (LogicalTypes.date().getName().equals(logicalType.getName())) {
+                    return RecordFieldType.DATE.getDataType();
+                } else if (LogicalTypes.timeMillis().getName().equals(logicalType.getName())) {
+                    return RecordFieldType.TIME.getDataType();
+                }
+
+                return RecordFieldType.INT.getDataType();
+            }
+            case LONG: {
+                final LogicalType logicalType = avroSchema.getLogicalType();
+                if (logicalType == null) {
+                    return RecordFieldType.LONG.getDataType();
+                }
+
+                if (LogicalTypes.timestampMillis().getName().equals(logicalType.getName())) {
+                    return RecordFieldType.TIMESTAMP.getDataType();
+                } else if (LogicalTypes.timestampMicros().getName().equals(logicalType.getName())) {
+                    return RecordFieldType.TIMESTAMP.getDataType();
+                } else if (LogicalTypes.timeMicros().getName().equals(logicalType.getName())) {
+                    return RecordFieldType.TIME.getDataType();
+                }
+
+                return RecordFieldType.LONG.getDataType();
+            }
+            case RECORD: {
+                final List<Field> avroFields = avroSchema.getFields();
+                final List<RecordField> recordFields = new ArrayList<>(avroFields.size());
+
+                for (final Field field : avroFields) {
+                    final String fieldName = field.name();
+                    final Schema fieldSchema = field.schema();
+                    final DataType fieldType = determineDataType(fieldSchema);
+                    recordFields.add(new RecordField(fieldName, fieldType));
+                }
+
+                final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
+                return RecordFieldType.RECORD.getRecordDataType(recordSchema);
+            }
+            case NULL:
+            case MAP:
+                return RecordFieldType.RECORD.getDataType();
+            case UNION: {
+                final List<Schema> nonNullSubSchemas = avroSchema.getTypes().stream()
+                    .filter(s -> s.getType() != Type.NULL)
+                    .collect(Collectors.toList());
+
+                if (nonNullSubSchemas.size() == 1) {
+                    return determineDataType(nonNullSubSchemas.get(0));
+                }
+
+                final List<DataType> possibleChildTypes = new ArrayList<>(nonNullSubSchemas.size());
+                for (final Schema subSchema : nonNullSubSchemas) {
+                    final DataType childDataType = determineDataType(subSchema);
+                    possibleChildTypes.add(childDataType);
+                }
+
+                return RecordFieldType.CHOICE.getChoiceDataType(possibleChildTypes);
+            }
+        }
+
+        return null;
+    }
+
+    public static RecordSchema createSchema(final Schema avroSchema) {
+        final List<RecordField> recordFields = new ArrayList<>(avroSchema.getFields().size());
+        for (final Field field : avroSchema.getFields()) {
+            final String fieldName = field.name();
+            final DataType dataType = AvroTypeUtil.determineDataType(field.schema());
+            recordFields.add(new RecordField(fieldName, dataType));
+        }
+
+        final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
+        return recordSchema;
+    }
+
+    public static Object[] convertByteArray(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 ByteBuffer convertByteArray(final Object[] bytes) {
+        final ByteBuffer bb = ByteBuffer.allocate(bytes.length);
+        for (final Object o : bytes) {
+            if (o instanceof Byte) {
+                bb.put(((Byte) o).byteValue());
+            } else {
+                throw new IllegalTypeConversionException("Cannot convert value " + bytes + " of type " + bytes.getClass() + " to ByteBuffer");
+            }
+        }
+        bb.flip();
+        return bb;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 d75d86d..b512b82 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
@@ -18,47 +18,41 @@
 package org.apache.nifi.avro;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.io.OutputStream;
-import java.math.BigDecimal;
-import java.math.BigInteger;
 import java.nio.ByteBuffer;
-import java.sql.Blob;
-import java.sql.Clob;
-import java.sql.SQLException;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
 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 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.DataFileWriter;
 import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericData.EnumSymbol;
 import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
 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.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 {
     private final Schema schema;
-    private final DateFormat dateFormat;
-    private final DateFormat timeFormat;
-    private final DateFormat timestampFormat;
 
-    public WriteAvroResult(final Schema schema, final String dateFormat, final String timeFormat, final String timestampFormat) {
+    public WriteAvroResult(final Schema schema) {
         this.schema = schema;
-        this.dateFormat = new SimpleDateFormat(dateFormat);
-        this.timeFormat = new SimpleDateFormat(timeFormat);
-        this.timestampFormat = new SimpleDateFormat(timestampFormat);
     }
 
     @Override
@@ -68,34 +62,13 @@ public class WriteAvroResult implements RecordSetWriter {
             return WriteResult.of(0, Collections.emptyMap());
         }
 
-        final GenericRecord rec = new GenericData.Record(schema);
-
         int nrOfRows = 0;
         final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
         try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
             dataFileWriter.create(schema, outStream);
 
-            final RecordSchema recordSchema = rs.getSchema();
-
             do {
-                for (final String fieldName : recordSchema.getFieldNames()) {
-                    final Object value = record.getValue(fieldName);
-
-                    final Field field = schema.getField(fieldName);
-                    if (field == null) {
-                        continue;
-                    }
-
-                    final Object converted;
-                    try {
-                        converted = convert(value, field.schema(), fieldName);
-                    } catch (final SQLException e) {
-                        throw new IOException("Failed to write records to stream", e);
-                    }
-
-                    rec.put(fieldName, converted);
-                }
-
+                final GenericRecord rec = createAvroRecord(record, schema);
                 dataFileWriter.append(rec);
                 nrOfRows++;
             } while ((record = rs.next()) != null);
@@ -104,169 +77,149 @@ public class WriteAvroResult implements RecordSetWriter {
         return WriteResult.of(nrOfRows, Collections.emptyMap());
     }
 
-    @Override
-    public WriteResult write(final Record record, final OutputStream out) throws IOException {
-        final GenericRecord rec = new GenericData.Record(schema);
+    private GenericRecord createAvroRecord(final Record record, final Schema avroSchema) throws IOException {
+        final GenericRecord rec = new GenericData.Record(avroSchema);
+        final RecordSchema recordSchema = record.getSchema();
 
-        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
-        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
-            dataFileWriter.create(schema, out);
-            final RecordSchema recordSchema = record.getSchema();
-
-            for (final String fieldName : recordSchema.getFieldNames()) {
-                final Object value = record.getValue(fieldName);
-
-                final Field field = schema.getField(fieldName);
-                if (field == null) {
-                    continue;
-                }
+        for (final String fieldName : recordSchema.getFieldNames()) {
+            final Object rawValue = record.getValue(fieldName);
 
-                final Object converted;
-                try {
-                    converted = convert(value, field.schema(), fieldName);
-                } catch (final SQLException e) {
-                    throw new IOException("Failed to write records to stream", e);
-                }
-
-                rec.put(fieldName, converted);
+            final Field field = avroSchema.getField(fieldName);
+            if (field == null) {
+                continue;
             }
 
-            dataFileWriter.append(rec);
+            final Object converted = convertToAvroObject(rawValue, field.schema());
+            rec.put(fieldName, converted);
         }
 
-        return WriteResult.of(1, Collections.emptyMap());
+        return rec;
     }
 
-
-    private Object convert(final Object value, final Schema schema, final String fieldName) throws SQLException, IOException {
-        if (value == null) {
+    private Object convertToAvroObject(final Object rawValue, final Schema fieldSchema) throws IOException {
+        if (rawValue == null) {
             return null;
         }
 
-        // Need to handle CLOB and BLOB before getObject() is called, due to ResultSet's maximum portability statement
-        if (value instanceof Clob) {
-            final Clob clob = (Clob) value;
-
-            long numChars = clob.length();
-            char[] buffer = new char[(int) numChars];
-            InputStream is = clob.getAsciiStream();
-            int index = 0;
-            int c = is.read();
-            while (c > 0) {
-                buffer[index++] = (char) c;
-                c = is.read();
-            }
-
-            clob.free();
-            return new String(buffer);
-        }
+        switch (fieldSchema.getType()) {
+            case INT: {
+                final LogicalType logicalType = fieldSchema.getLogicalType();
+                if (logicalType == null) {
+                    return DataTypeUtils.toInteger(rawValue);
+                }
 
-        if (value instanceof Blob) {
-            final Blob blob = (Blob) value;
+                if (LogicalTypes.date().getName().equals(logicalType.getName())) {
+                    final long longValue = DataTypeUtils.toLong(rawValue);
+                    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 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;
+                }
 
-            final long numChars = blob.length();
-            final byte[] buffer = new byte[(int) numChars];
-            final InputStream is = blob.getBinaryStream();
-            int index = 0;
-            int c = is.read();
-            while (c > 0) {
-                buffer[index++] = (byte) c;
-                c = is.read();
+                return DataTypeUtils.toInteger(rawValue);
             }
+            case LONG: {
+                final LogicalType logicalType = fieldSchema.getLogicalType();
+                if (logicalType == null) {
+                    return DataTypeUtils.toLong(rawValue);
+                }
 
-            final ByteBuffer bb = ByteBuffer.wrap(buffer);
-            blob.free();
-            return bb;
-        }
+                if (LogicalTypes.timeMicros().getName().equals(logicalType.getName())) {
+                    final long longValue = DataTypeUtils.toLong(rawValue);
+                    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);
+                } else if (LogicalTypes.timestampMicros().getName().equals(logicalType.getName())) {
+                    return DataTypeUtils.toLong(rawValue) * 1000L;
+                }
 
-        if (value instanceof byte[]) {
-            // bytes requires little bit different handling
-            return ByteBuffer.wrap((byte[]) value);
-        } else if (value instanceof Byte) {
-            // tinyint(1) type is returned by JDBC driver as java.sql.Types.TINYINT
-            // But value is returned by JDBC as java.lang.Byte
-            // (at least H2 JDBC works this way)
-            // direct put to avro record results:
-            // org.apache.avro.AvroRuntimeException: Unknown datum type java.lang.Byte
-            return ((Byte) value).intValue();
-        } else if (value instanceof Short) {
-            //MS SQL returns TINYINT as a Java Short, which Avro doesn't understand.
-            return ((Short) value).intValue();
-        } else if (value instanceof BigDecimal) {
-            // Avro can't handle BigDecimal as a number - it will throw an AvroRuntimeException such as: "Unknown datum type: java.math.BigDecimal: 38"
-            return value.toString();
-        } else if (value instanceof BigInteger) {
-            // Check the precision of the BIGINT. Some databases allow arbitrary precision (> 19), but Avro won't handle that.
-            // It the SQL type is BIGINT and the precision is between 0 and 19 (inclusive); if so, the BigInteger is likely a
-            // long (and the schema says it will be), so try to get its value as a long.
-            // Otherwise, Avro can't handle BigInteger as a number - it will throw an AvroRuntimeException
-            // such as: "Unknown datum type: java.math.BigInteger: 38". In this case the schema is expecting a string.
-            final BigInteger bigInt = (BigInteger) value;
-            if (bigInt.compareTo(BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
-                return value.toString();
-            } else {
-                return bigInt.longValue();
+                return DataTypeUtils.toLong(rawValue);
             }
-        } else if (value instanceof Boolean) {
-            return value;
-        } else if (value instanceof Map) {
-            // TODO: Revisit how we handle a lot of these cases....
-            switch (schema.getType()) {
-                case MAP:
-                    return value;
-                case RECORD:
-                    final GenericData.Record avroRecord = new GenericData.Record(schema);
+            case BYTES:
+            case FIXED:
+                if (rawValue instanceof byte[]) {
+                    return ByteBuffer.wrap((byte[]) rawValue);
+                }
+                if (rawValue instanceof Object[]) {
+                    return AvroTypeUtil.convertByteArray((Object[]) rawValue);
+                } else {
+                    throw new IllegalTypeConversionException("Cannot convert value " + rawValue + " of type " + rawValue.getClass() + " to a ByteBuffer");
+                }
+            case MAP:
+                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);
+                        if (v != null) {
+                            map.put(recordFieldName, v);
+                        }
+                    }
 
-                    final Record record = (Record) value;
-                    for (final String recordFieldName : record.getSchema().getFieldNames()) {
-                        final Object recordFieldValue = record.getValue(recordFieldName);
+                    return map;
+                } else {
+                    throw new IllegalTypeConversionException("Cannot convert value " + rawValue + " of type " + rawValue.getClass() + " to a Map");
+                }
+            case RECORD:
+                final GenericData.Record avroRecord = new GenericData.Record(fieldSchema);
 
-                        final Field field = schema.getField(recordFieldName);
-                        if (field == null) {
-                            continue;
-                        }
+                final Record record = (Record) rawValue;
+                for (final String recordFieldName : record.getSchema().getFieldNames()) {
+                    final Object recordFieldValue = record.getValue(recordFieldName);
 
-                        final Object converted = convert(recordFieldValue, field.schema(), recordFieldName);
-                        avroRecord.put(recordFieldName, converted);
+                    final Field field = fieldSchema.getField(recordFieldName);
+                    if (field == null) {
+                        continue;
                     }
-                    return avroRecord;
-            }
 
-            return value.toString();
+                    final Object converted = convertToAvroObject(recordFieldValue, field.schema());
+                    avroRecord.put(recordFieldName, converted);
+                }
+                return avroRecord;
+            case ARRAY:
+                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());
+                    list.add(converted);
+                }
+                return list;
+            case BOOLEAN:
+                return DataTypeUtils.toBoolean(rawValue);
+            case DOUBLE:
+                return DataTypeUtils.toDouble(rawValue);
+            case FLOAT:
+                return DataTypeUtils.toFloat(rawValue);
+            case NULL:
+                return null;
+            case ENUM:
+                return new EnumSymbol(fieldSchema, rawValue);
+            case STRING:
+                return DataTypeUtils.toString(rawValue, RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
+        }
+
+        return rawValue;
+    }
 
-        } else if (value instanceof List) {
-            return value;
-        } else if (value instanceof Object[]) {
-            final List<Object> list = new ArrayList<>();
-            for (final Object o : ((Object[]) value)) {
-                final Object converted = convert(o, schema.getElementType(), fieldName);
-                list.add(converted);
-            }
-            return list;
-        } else if (value instanceof Number) {
-            return value;
-        } else if (value instanceof java.util.Date) {
-            final java.util.Date date = (java.util.Date) value;
-            return dateFormat.format(date);
-        } else if (value instanceof java.sql.Date) {
-            final java.sql.Date sqlDate = (java.sql.Date) value;
-            final java.util.Date date = new java.util.Date(sqlDate.getTime());
-            return dateFormat.format(date);
-        } else if (value instanceof Time) {
-            final Time time = (Time) value;
-            final java.util.Date date = new java.util.Date(time.getTime());
-            return timeFormat.format(date);
-        } else if (value instanceof Timestamp) {
-            final Timestamp time = (Timestamp) value;
-            final java.util.Date date = new java.util.Date(time.getTime());
-            return timestampFormat.format(date);
+    @Override
+    public WriteResult write(final Record record, final OutputStream out) throws IOException {
+        final GenericRecord rec = createAvroRecord(record, schema);
+
+        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
+            dataFileWriter.create(schema, out);
+            dataFileWriter.append(rec);
         }
 
-        // The different types that we support are numbers (int, long, double, float),
-        // as well as boolean values and Strings. Since Avro doesn't provide
-        // timestamp types, we want to convert those to Strings. So we will cast anything other
-        // than numbers or booleans to strings by using the toString() method.
-        return value.toString();
+        return WriteResult.of(1, Collections.emptyMap());
     }
 
 
@@ -275,7 +228,6 @@ public class WriteAvroResult implements RecordSetWriter {
         return "application/avro-binary";
     }
 
-
     public static String normalizeNameForAvro(String inputName) {
         String normalizedName = inputName.replaceAll("[^A-Za-z0-9_]", "_");
         if (Character.isDigit(normalizedName.charAt(0))) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 eccad7d..6b06ebf 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
@@ -19,31 +19,63 @@ package org.apache.nifi.csv;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
 
-import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.DateTimeUtils;
 import org.apache.nifi.serialization.RecordReader;
 import org.apache.nifi.serialization.RowRecordReaderFactory;
-import org.apache.nifi.serialization.UserTypeOverrideRowReader;
+import org.apache.nifi.serialization.SchemaRegistryRecordReader;
+import org.apache.nifi.serialization.record.RecordSchema;
 
 @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. By default, the reader will assume that all columns are of 'String' type, but this can be "
-    + "overridden by adding a user-defined Property where the key is the name of a column and the value is the "
-    + "type of the column. For example, if a Property has the name \"balance\" with a value of float, it the "
-    + "reader will attempt to coerce all values in the \"balance\" column into a floating-point number. See "
-    + "Controller Service's Usage for further documentation.")
-@DynamicProperty(name = "<name of column in CSV>", value = "<type of column values in CSV>",
-    description = "User-defined properties are used to indicate that the values of a specific column should be interpreted as a "
-    + "user-defined data type (e.g., int, double, float, date, etc.)", supportsExpressionLanguage = false)
-public class CSVReader extends UserTypeOverrideRowReader implements RowRecordReaderFactory {
+    + "the values. See Controller Service's Usage for further documentation.")
+public class CSVReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory {
+
+    private volatile CSVFormat csvFormat;
+    private volatile String dateFormat;
+    private volatile String timeFormat;
+    private volatile String timestampFormat;
+
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(DateTimeUtils.DATE_FORMAT);
+        properties.add(DateTimeUtils.TIME_FORMAT);
+        properties.add(DateTimeUtils.TIMESTAMP_FORMAT);
+        properties.add(CSVUtils.CSV_FORMAT);
+        properties.add(CSVUtils.VALUE_SEPARATOR);
+        properties.add(CSVUtils.QUOTE_CHAR);
+        properties.add(CSVUtils.ESCAPE_CHAR);
+        properties.add(CSVUtils.COMMENT_MARKER);
+        properties.add(CSVUtils.NULL_STRING);
+        properties.add(CSVUtils.TRIM_FIELDS);
+        return properties;
+    }
+
+    @OnEnabled
+    public void storeCsvFormat(final ConfigurationContext context) {
+        this.csvFormat = CSVUtils.createCSVFormat(context);
+        this.dateFormat = context.getProperty(DateTimeUtils.DATE_FORMAT).getValue();
+        this.timeFormat = context.getProperty(DateTimeUtils.TIME_FORMAT).getValue();
+        this.timestampFormat = context.getProperty(DateTimeUtils.TIMESTAMP_FORMAT).getValue();
+    }
 
     @Override
-    public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException {
-        return new CSVRecordReader(in, logger, getFieldTypeOverrides());
+    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);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 c2e8963..d02768c 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
@@ -17,200 +17,91 @@
 
 package org.apache.nifi.csv;
 
-import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Date;
+import java.io.Reader;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.csv.CSVRecord;
+import org.apache.commons.io.input.BOMInputStream;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.MapRecord;
 import org.apache.nifi.serialization.record.Record;
-import org.apache.nifi.serialization.record.RecordField;
-import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
 
-import au.com.bytecode.opencsv.CSVReader;
 
 public class CSVRecordReader implements RecordReader {
-    private final ComponentLog logger;
-    private final CSVReader reader;
-    private final String[] firstLine;
-    private final Map<String, DataType> fieldTypeOverrides;
-    private RecordSchema schema;
-
-    public CSVRecordReader(final InputStream in, final ComponentLog logger, final Map<String, DataType> fieldTypeOverrides) throws IOException {
-        this.logger = logger;
-        reader = new CSVReader(new InputStreamReader(new BufferedInputStream(in)));
-        firstLine = reader.readNext();
-        this.fieldTypeOverrides = fieldTypeOverrides;
+    private final CSVParser csvParser;
+    private final RecordSchema schema;
+    private final String dateFormat;
+    private final String timeFormat;
+    private final String timestampFormat;
+
+    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;
     }
 
     @Override
     public Record nextRecord() throws IOException, MalformedRecordException {
         final RecordSchema schema = getSchema();
 
-        while (true) {
-            final String[] line = reader.readNext();
-            if (line == null) {
-                return null;
-            }
-
-            final List<DataType> fieldTypes = schema.getDataTypes();
-            if (fieldTypes.size() != line.length) {
-                logger.warn("Found record with incorrect number of fields. Expected {} but found {}; skipping record", new Object[] {fieldTypes.size(), line.length});
-                continue;
-            }
-
-            try {
-                final Map<String, Object> rowValues = new HashMap<>(schema.getFieldCount());
+        for (final CSVRecord csvRecord : csvParser) {
+            final Map<String, Object> rowValues = new HashMap<>(schema.getFieldCount());
 
-                int i = 0;
-                for (final String fieldName : schema.getFieldNames()) {
-                    if (i >= line.length) {
-                        rowValues.put(fieldName, null);
-                        continue;
-                    }
-
-                    final String rawValue = line[i++].trim();
-                    final Object converted = convert(schema.getDataType(fieldName).orElse(null), rawValue);
-                    rowValues.put(fieldName, converted);
+            for (final String fieldName : schema.getFieldNames()) {
+                final String rawValue = csvRecord.get(fieldName);
+                if (rawValue == null) {
+                    rowValues.put(fieldName, null);
+                    continue;
                 }
 
-                return new MapRecord(schema, rowValues);
-            } catch (final Exception e) {
-                throw new MalformedRecordException("Found invalid CSV record", e);
+                final Object converted = convert(rawValue, schema.getDataType(fieldName).orElse(null));
+                rowValues.put(fieldName, converted);
             }
+
+            return new MapRecord(schema, rowValues);
         }
+
+        return null;
     }
 
     @Override
     public RecordSchema getSchema() {
-        if (schema != null) {
-            return schema;
-        }
-
-        final List<RecordField> recordFields = new ArrayList<>();
-        for (final String element : firstLine) {
-
-            final String name = element.trim();
-            final DataType dataType;
-
-            final DataType overriddenDataType = fieldTypeOverrides.get(name);
-            if (overriddenDataType != null) {
-                dataType = overriddenDataType;
-            } else {
-                dataType = RecordFieldType.STRING.getDataType();
-            }
-
-            final RecordField field = new RecordField(name, dataType);
-            recordFields.add(field);
-        }
-
-        if (recordFields.isEmpty()) {
-            recordFields.add(new RecordField("line", RecordFieldType.STRING.getDataType()));
-        }
-
-        schema = new SimpleRecordSchema(recordFields);
         return schema;
     }
 
-    protected Object convert(final DataType dataType, final String value) {
-        if (dataType == null) {
+    protected Object convert(final String value, final DataType dataType) {
+        if (dataType == null || value == null) {
             return value;
         }
 
-        switch (dataType.getFieldType()) {
-            case BOOLEAN:
-                if (value.length() == 0) {
-                    return null;
-                }
-                return Boolean.parseBoolean(value);
-            case BYTE:
-                if (value.length() == 0) {
-                    return null;
-                }
-                return Byte.parseByte(value);
-            case SHORT:
-                if (value.length() == 0) {
-                    return null;
-                }
-                return Short.parseShort(value);
-            case INT:
-                if (value.length() == 0) {
-                    return null;
-                }
-                return Integer.parseInt(value);
-            case LONG:
-            case BIGINT:
-                if (value.length() == 0) {
-                    return null;
-                }
-                return Long.parseLong(value);
-            case FLOAT:
-                if (value.length() == 0) {
-                    return null;
-                }
-                return Float.parseFloat(value);
-            case DOUBLE:
-                if (value.length() == 0) {
-                    return null;
-                }
-                return Double.parseDouble(value);
-            case DATE:
-                if (value.length() == 0) {
-                    return null;
-                }
-                try {
-                    final Date date = new SimpleDateFormat(dataType.getFormat()).parse(value);
-                    return new java.sql.Date(date.getTime());
-                } catch (final ParseException e) {
-                    logger.warn("Found invalid value for DATE field: " + value + " does not match expected format of "
-                        + dataType.getFormat() + "; will substitute a NULL value for this field");
-                    return null;
-                }
-            case TIME:
-                if (value.length() == 0) {
-                    return null;
-                }
-                try {
-                    final Date date = new SimpleDateFormat(dataType.getFormat()).parse(value);
-                    return new java.sql.Time(date.getTime());
-                } catch (final ParseException e) {
-                    logger.warn("Found invalid value for TIME field: " + value + " does not match expected format of "
-                        + dataType.getFormat() + "; will substitute a NULL value for this field");
-                    return null;
-                }
-            case TIMESTAMP:
-                if (value.length() == 0) {
-                    return null;
-                }
-                try {
-                    final Date date = new SimpleDateFormat(dataType.getFormat()).parse(value);
-                    return new java.sql.Timestamp(date.getTime());
-                } catch (final ParseException e) {
-                    logger.warn("Found invalid value for TIMESTAMP field: " + value + " does not match expected format of "
-                        + dataType.getFormat() + "; will substitute a NULL value for this field");
-                    return null;
-                }
-            case STRING:
-            default:
-                return value;
+        final String trimmed = value.startsWith("\"") && value.endsWith("\"") ? value.substring(1, value.length() - 1) : value;
+
+        if (trimmed.isEmpty()) {
+            return null;
         }
+
+        return DataTypeUtils.convertType(trimmed, dataType, dateFormat, timeFormat, timestampFormat);
     }
 
     @Override
     public void close() throws IOException {
-        reader.close();
+        csvParser.close();
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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 906e9c4..6a7b758 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,21 +17,51 @@
 
 package org.apache.nifi.csv;
 
+import java.util.ArrayList;
+import java.util.List;
+
+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.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.serialization.AbstractRecordSetWriter;
+import org.apache.nifi.serialization.DateTimeTextRecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
 
-@Tags({"csv", "result", "set", "writer", "serializer", "record", "row"})
-@CapabilityDescription("Writes the contents of a Database ResultSet as CSV data. The first line written "
+@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 "
     + "will be the column names. All subsequent lines will be the values corresponding to those columns.")
-public class CSVRecordSetWriter extends AbstractRecordSetWriter implements RecordSetWriterFactory {
+public class CSVRecordSetWriter extends DateTimeTextRecordSetWriter implements RecordSetWriterFactory {
+
+    private volatile CSVFormat csvFormat;
+
+    @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.QUOTE_CHAR);
+        properties.add(CSVUtils.ESCAPE_CHAR);
+        properties.add(CSVUtils.COMMENT_MARKER);
+        properties.add(CSVUtils.NULL_STRING);
+        properties.add(CSVUtils.TRIM_FIELDS);
+        properties.add(CSVUtils.QUOTE_MODE);
+        properties.add(CSVUtils.RECORD_SEPARATOR);
+        properties.add(CSVUtils.TRAILING_DELIMITER);
+        return properties;
+    }
+
+    @OnEnabled
+    public void storeCsvFormat(final ConfigurationContext context) {
+        this.csvFormat = CSVUtils.createCSVFormat(context);
+    }
 
     @Override
     public RecordSetWriter createWriter(final ComponentLog logger) {
-        return new WriteCSVResult(getDateFormat(), getTimeFormat(), getTimestampFormat());
+        return new WriteCSVResult(csvFormat, getDateFormat(), getTimeFormat(), getTimestampFormat());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/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
new file mode 100644
index 0000000..e23b6e1
--- /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/CSVUtils.java
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.csv;
+
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.QuoteMode;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public class CSVUtils {
+
+    static final AllowableValue CUSTOM = new AllowableValue("custom", "Custom Format",
+        "The format of the CSV is configured by using the properties of this Controller Service, such as Value Separator");
+    static final AllowableValue RFC_4180 = new AllowableValue("rfc-4180", "RFC 4180", "CSV data follows the RFC 4180 Specification defined at https://tools.ietf.org/html/rfc4180");
+    static final AllowableValue EXCEL = new AllowableValue("excel", "Microsoft Excel", "CSV data follows the format used by Microsoft Excel");
+    static final AllowableValue TDF = new AllowableValue("tdf", "Tab-Delimited", "CSV data is Tab-Delimited instead of Comma Delimited");
+    static final AllowableValue INFORMIX_UNLOAD = new AllowableValue("informix-unload", "Informix Unload", "The format used by Informix when issuing the UNLOAD TO file_name command");
+    static final AllowableValue INFORMIX_UNLOAD_CSV = new AllowableValue("informix-unload", "Informix Unload Escape Disabled",
+        "The format used by Informix when issuing the UNLOAD TO file_name command with escaping disabled");
+    static final AllowableValue MYSQL = new AllowableValue("mysql", "MySQL Format", "CSV data follows the format used by MySQL");
+
+    static final PropertyDescriptor CSV_FORMAT = new PropertyDescriptor.Builder()
+        .name("CSV Format")
+        .description("Specifies which \"format\" the CSV data is in, or specifies if custom formatting should be used.")
+        .expressionLanguageSupported(false)
+        .allowableValues(CUSTOM, RFC_4180, EXCEL, TDF, MYSQL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV)
+        .defaultValue(CUSTOM.getValue())
+        .required(true)
+        .build();
+    static final PropertyDescriptor VALUE_SEPARATOR = new PropertyDescriptor.Builder()
+        .name("Value Separator")
+        .description("The character that is used to separate values/fields in a CSV Record")
+        .addValidator(new SingleCharacterValidator())
+        .expressionLanguageSupported(false)
+        .defaultValue(",")
+        .required(true)
+        .build();
+    static final PropertyDescriptor QUOTE_CHAR = new PropertyDescriptor.Builder()
+        .name("Quote Character")
+        .description("The character that is used to quote values so that escape characters do not have to be used")
+        .addValidator(new SingleCharacterValidator())
+        .expressionLanguageSupported(false)
+        .defaultValue("\"")
+        .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.")
+        .addValidator(new SingleCharacterValidator())
+        .expressionLanguageSupported(false)
+        .required(false)
+        .build();
+    static final PropertyDescriptor ESCAPE_CHAR = new PropertyDescriptor.Builder()
+        .name("Escape Character")
+        .description("The character that is used to escape characters that would otherwise have a specific meaning to the CSV Parser.")
+        .addValidator(new SingleCharacterValidator())
+        .expressionLanguageSupported(false)
+        .defaultValue("\\")
+        .required(true)
+        .build();
+    static final PropertyDescriptor NULL_STRING = new PropertyDescriptor.Builder()
+        .name("Null String")
+        .description("Specifies a String that, if present as a value in the CSV, should be considered a null field instead of using the literal value.")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(false)
+        .required(false)
+        .build();
+    static final PropertyDescriptor TRIM_FIELDS = new PropertyDescriptor.Builder()
+        .name("Trim Fields")
+        .description("Whether or not white space should be removed from the beginning and end of fields")
+        .expressionLanguageSupported(false)
+        .allowableValues("true", "false")
+        .defaultValue("true")
+        .required(true)
+        .build();
+
+    // CSV Format fields for writers only
+    static final AllowableValue QUOTE_ALL = new AllowableValue("ALL", "Quote All Values", "All values will be quoted using the configured quote character.");
+    static final AllowableValue QUOTE_MINIMAL = new AllowableValue("MINIMAL", "Quote Minimal",
+        "Values will be quoted only if they are contain special characters such as newline characters or field separators.");
+    static final AllowableValue QUOTE_NON_NUMERIC = new AllowableValue("NON_NUMERIC", "Quote Non-Numeric Values", "Values will be quoted unless the value is a number.");
+    static final AllowableValue QUOTE_NONE = new AllowableValue("NONE", "Do Not Quote Values",
+        "Values will not be quoted. Instead, all special characters will be escaped using the configured escape character.");
+
+    static final PropertyDescriptor QUOTE_MODE = new PropertyDescriptor.Builder()
+        .name("Quote Mode")
+        .description("Specifies how fields should be quoted when they are written")
+        .expressionLanguageSupported(false)
+        .allowableValues(QUOTE_ALL, QUOTE_MINIMAL, QUOTE_NON_NUMERIC, QUOTE_NONE)
+        .defaultValue(QUOTE_MINIMAL.getValue())
+        .required(true)
+        .build();
+    static final PropertyDescriptor TRAILING_DELIMITER = new PropertyDescriptor.Builder()
+        .name("Include Trailing Delimiter")
+        .description("If true, a trailing delimiter will be added to each CSV Record that is written. If false, the trailing delimiter will be omitted.")
+        .expressionLanguageSupported(false)
+        .allowableValues("true", "false")
+        .defaultValue("false")
+        .required(true)
+        .build();
+    static final PropertyDescriptor RECORD_SEPARATOR = new PropertyDescriptor.Builder()
+        .name("Record Separator")
+        .description("Specifies the characters to use in order to separate CSV Records")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(false)
+        .defaultValue("\\n")
+        .required(true)
+        .build();
+
+
+    static CSVFormat createCSVFormat(final ConfigurationContext context) {
+        final String formatName = context.getProperty(CSV_FORMAT).getValue();
+        if (formatName.equalsIgnoreCase(CUSTOM.getValue())) {
+            return buildCustomFormat(context);
+        }
+        if (formatName.equalsIgnoreCase(RFC_4180.getValue())) {
+            return CSVFormat.RFC4180;
+        } else if (formatName.equalsIgnoreCase(EXCEL.getValue())) {
+            return CSVFormat.EXCEL;
+        } else if (formatName.equalsIgnoreCase(TDF.getValue())) {
+            return CSVFormat.TDF;
+        } else if (formatName.equalsIgnoreCase(MYSQL.getValue())) {
+            return CSVFormat.MYSQL;
+        } else if (formatName.equalsIgnoreCase(INFORMIX_UNLOAD.getValue())) {
+            return CSVFormat.INFORMIX_UNLOAD;
+        } else if (formatName.equalsIgnoreCase(INFORMIX_UNLOAD_CSV.getValue())) {
+            return CSVFormat.INFORMIX_UNLOAD_CSV;
+        } else {
+            return CSVFormat.DEFAULT;
+        }
+    }
+
+    private static char getChar(final ConfigurationContext context, final PropertyDescriptor property) {
+        return CSVUtils.unescape(context.getProperty(property).getValue()).charAt(0);
+    }
+
+    private static CSVFormat buildCustomFormat(final ConfigurationContext context) {
+        final char valueSeparator = getChar(context, VALUE_SEPARATOR);
+        CSVFormat format = CSVFormat.newFormat(valueSeparator)
+            .withAllowMissingColumnNames()
+            .withIgnoreEmptyLines()
+            .withFirstRecordAsHeader();
+
+        format = format.withQuote(getChar(context, QUOTE_CHAR));
+        format = format.withEscape(getChar(context, ESCAPE_CHAR));
+        format = format.withTrim(context.getProperty(TRIM_FIELDS).asBoolean());
+
+        if (context.getProperty(COMMENT_MARKER).isSet()) {
+            format = format.withCommentMarker(getChar(context, COMMENT_MARKER));
+        }
+        if (context.getProperty(NULL_STRING).isSet()) {
+            format = format.withNullString(CSVUtils.unescape(context.getProperty(NULL_STRING).getValue()));
+        }
+
+        final PropertyValue quoteValue = context.getProperty(QUOTE_MODE);
+        if (quoteValue != null) {
+            final QuoteMode quoteMode = QuoteMode.valueOf(quoteValue.getValue());
+            format = format.withQuoteMode(quoteMode);
+        }
+
+        final PropertyValue trailingDelimiterValue = context.getProperty(TRAILING_DELIMITER);
+        if (trailingDelimiterValue != null) {
+            final boolean trailingDelimiter = trailingDelimiterValue.asBoolean();
+            format = format.withTrailingDelimiter(trailingDelimiter);
+        }
+
+        final PropertyValue recordSeparator = context.getProperty(RECORD_SEPARATOR);
+        if (recordSeparator != null) {
+            final String separator = unescape(recordSeparator.getValue());
+            format = format.withRecordSeparator(separator);
+        }
+
+        return format;
+    }
+
+
+    public static String unescape(final String input) {
+        if (input == null) {
+            return input;
+        }
+
+        return input.replace("\\t", "\t")
+            .replace("\\n", "\n")
+            .replace("\\r", "\r");
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/SingleCharacterValidator.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/SingleCharacterValidator.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/SingleCharacterValidator.java
new file mode 100644
index 0000000..b24dea9
--- /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/SingleCharacterValidator.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.csv;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+
+public class SingleCharacterValidator implements Validator {
+    private static final Set<String> illegalChars = new HashSet<>();
+    static {
+        illegalChars.add("\r");
+        illegalChars.add("\n");
+    }
+
+    @Override
+    public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+        final String unescaped = CSVUtils.unescape(input);
+        if (unescaped.length() != 1) {
+            return new ValidationResult.Builder()
+                .input(input)
+                .subject(subject)
+                .valid(false)
+                .explanation("Value must be exactly 1 character but was " + input.length() + " in length")
+                .build();
+        }
+
+        if (illegalChars.contains(input)) {
+            return new ValidationResult.Builder()
+                .input(input)
+                .subject(subject)
+                .valid(false)
+                .explanation(input + " is not a valid character for this property")
+                .build();
+        }
+
+        return new ValidationResult.Builder()
+            .input(input)
+            .subject(subject)
+            .valid(true)
+            .build();
+    }
+
+}


[02/19] nifi git commit: NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Servic

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..781f41f
--- /dev/null
+++ 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
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.text;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.nifi.components.PropertyValue;
+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.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+public class FreeFormTextWriter implements RecordSetWriter {
+    private static final byte NEW_LINE = (byte) '\n';
+    private final PropertyValue propertyValue;
+    private final Charset charset;
+
+    public FreeFormTextWriter(final PropertyValue textPropertyValue, final Charset characterSet) {
+        propertyValue = textPropertyValue;
+        charset = characterSet;
+    }
+
+    @Override
+    public WriteResult write(final RecordSet recordSet, final OutputStream out) throws IOException {
+        int count = 0;
+
+        try {
+            final RecordSchema schema = recordSet.getSchema();
+            final String[] colNames = getColumnNames(schema);
+
+            Record record;
+            while ((record = recordSet.next()) != null) {
+                count++;
+                write(record, out, colNames);
+            }
+        } catch (final Exception e) {
+            throw new ProcessException(e);
+        }
+
+        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();
+        }
+
+        return columnNames;
+    }
+
+    @Override
+    public WriteResult write(final Record record, final OutputStream out) throws IOException {
+        write(record, out, getColumnNames(record.getSchema()));
+        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;
+        final Map<String, String> values = new HashMap<>(numCols);
+        for (int i = 0; i < numCols; i++) {
+            final String columnName = columnNames[i];
+            final String columnValue = record.getAsString(columnName);
+            values.put(columnName, columnValue);
+        }
+
+        final String evaluated = propertyValue.evaluateAttributeExpressions(values).getValue();
+        out.write(evaluated.getBytes(charset));
+        out.write(NEW_LINE);
+    }
+
+    @Override
+    public String getMimeType() {
+        return "text/plain";
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
new file mode 100644
index 0000000..628dbe5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -0,0 +1,28 @@
+# 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.
+
+org.apache.nifi.avro.AvroReader
+org.apache.nifi.avro.AvroRecordSetWriter
+
+org.apache.nifi.json.JsonTreeReader
+org.apache.nifi.json.JsonPathReader
+org.apache.nifi.json.JsonRecordSetWriter
+
+org.apache.nifi.csv.CSVReader
+org.apache.nifi.csv.CSVRecordSetWriter
+
+org.apache.nifi.grok.GrokReader
+
+org.apache.nifi.text.FreeFormTextRecordSetWriter
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/default-grok-patterns.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/default-grok-patterns.txt b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/default-grok-patterns.txt
new file mode 100644
index 0000000..4b110e8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/default-grok-patterns.txt
@@ -0,0 +1,115 @@
+# 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.
+
+
+
+# Log Levels
+LOGLEVEL ([Aa]lert|ALERT|[Tt]race|TRACE|[Dd]ebug|DEBUG|[Nn]otice|NOTICE|[Ii]nfo|INFO|[Ww]arn?(?:ing)?|WARN?(?:ING)?|[Ee]rr?(?:or)?|ERR?(?:OR)?|[Cc]rit?(?:ical)?|CRIT?(?:ICAL)?|[Ff]atal|FATAL|[Ss]evere|SEVERE|EMERG(?:ENCY)?|[Ee]merg(?:ency)?)|FINE|FINER|FINEST|CONFIG
+
+# Syslog Dates: Month Day HH:MM:SS
+SYSLOGTIMESTAMP %{MONTH} +%{MONTHDAY} %{TIME}
+PROG (?:[\w._/%-]+)
+SYSLOGPROG %{PROG:program}(?:\[%{POSINT:pid}\])?
+SYSLOGHOST %{IPORHOST}
+SYSLOGFACILITY <%{NONNEGINT:facility}.%{NONNEGINT:priority}>
+HTTPDATE %{MONTHDAY}/%{MONTH}/%{YEAR}:%{TIME} %{INT}
+
+# Months: January, Feb, 3, 03, 12, December
+MONTH \b(?:Jan(?:uary)?|Feb(?:ruary)?|Mar(?:ch)?|Apr(?:il)?|May|Jun(?:e)?|Jul(?:y)?|Aug(?:ust)?|Sep(?:tember)?|Oct(?:ober)?|Nov(?:ember)?|Dec(?:ember)?)\b
+MONTHNUM (?:0?[1-9]|1[0-2])
+MONTHNUM2 (?:0[1-9]|1[0-2])
+MONTHDAY (?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9])
+
+# Days: Monday, Tue, Thu, etc...
+DAY (?:Mon(?:day)?|Tue(?:sday)?|Wed(?:nesday)?|Thu(?:rsday)?|Fri(?:day)?|Sat(?:urday)?|Sun(?:day)?)
+
+# Years?
+YEAR (?>\d\d){1,2}
+HOUR (?:2[0123]|[01]?[0-9])
+MINUTE (?:[0-5][0-9])
+# '60' is a leap second in most time standards and thus is valid.
+SECOND (?:(?:[0-5]?[0-9]|60)(?:[:.,][0-9]+)?)
+TIME (?!<[0-9])%{HOUR}:%{MINUTE}(?::%{SECOND})(?![0-9])
+
+# datestamp is YYYY/MM/DD-HH:MM:SS.UUUU (or something like it)
+DATE_US_MONTH_DAY_YEAR %{MONTHNUM}[/-]%{MONTHDAY}[/-]%{YEAR}
+DATE_US_YEAR_MONTH_DAY %{YEAR}[/-]%{MONTHNUM}[/-]%{MONTHDAY}
+DATE_US %{DATE_US_MONTH_DAY_YEAR}|%{DATE_US_YEAR_MONTH_DAY}
+DATE_EU %{MONTHDAY}[./-]%{MONTHNUM}[./-]%{YEAR}
+ISO8601_TIMEZONE (?:Z|[+-]%{HOUR}(?::?%{MINUTE}))
+ISO8601_SECOND (?:%{SECOND}|60)
+TIMESTAMP_ISO8601 %{YEAR}-%{MONTHNUM}-%{MONTHDAY}[T ]%{HOUR}:?%{MINUTE}(?::?%{SECOND})?%{ISO8601_TIMEZONE}?
+DATE %{DATE_US}|%{DATE_EU}
+DATESTAMP %{DATE}[- ]%{TIME}
+TZ (?:[PMCE][SD]T|UTC)
+DATESTAMP_RFC822 %{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{TIME} %{TZ}
+DATESTAMP_RFC2822 %{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{TIME} %{ISO8601_TIMEZONE}
+DATESTAMP_OTHER %{DAY} %{MONTH} %{MONTHDAY} %{TIME} %{TZ} %{YEAR}
+DATESTAMP_EVENTLOG %{YEAR}%{MONTHNUM2}%{MONTHDAY}%{HOUR}%{MINUTE}%{SECOND}
+
+
+POSINT \b(?:[1-9][0-9]*)\b
+NONNEGINT \b(?:[0-9]+)\b
+WORD \b\w+\b
+NOTSPACE \S+
+SPACE \s*
+DATA .*?
+GREEDYDATA .*
+QUOTEDSTRING (?>(?<!\\)(?>"(?>\\.|[^\\"]+)+"|""|(?>'(?>\\.|[^\\']+)+')|''|(?>`(?>\\.|[^\\`]+)+`)|``))
+UUID [A-Fa-f0-9]{8}-(?:[A-Fa-f0-9]{4}-){3}[A-Fa-f0-9]{12}
+
+USERNAME [a-zA-Z0-9._-]+
+USER %{USERNAME}
+INT (?:[+-]?(?:[0-9]+))
+BASE10NUM (?<![0-9.+-])(?>[+-]?(?:(?:[0-9]+(?:\.[0-9]+)?)|(?:\.[0-9]+)))
+NUMBER (?:%{BASE10NUM})
+BASE16NUM (?<![0-9A-Fa-f])(?:[+-]?(?:0x)?(?:[0-9A-Fa-f]+))
+BASE16FLOAT \b(?<![0-9A-Fa-f.])(?:[+-]?(?:0x)?(?:(?:[0-9A-Fa-f]+(?:\.[0-9A-Fa-f]*)?)|(?:\.[0-9A-Fa-f]+)))\b
+
+# Networking
+MAC (?:%{CISCOMAC}|%{WINDOWSMAC}|%{COMMONMAC})
+CISCOMAC (?:(?:[A-Fa-f0-9]{4}\.){2}[A-Fa-f0-9]{4})
+WINDOWSMAC (?:(?:[A-Fa-f0-9]{2}-){5}[A-Fa-f0-9]{2})
+COMMONMAC (?:(?:[A-Fa-f0-9]{2}:){5}[A-Fa-f0-9]{2})
+IPV6 ((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5
 ]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:)))(%.+)?
+IPV4 (?<![0-9])(?:(?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2}))(?![0-9])
+IP (?:%{IPV6}|%{IPV4})
+HOSTNAME \b(?:[0-9A-Za-z][0-9A-Za-z-]{0,62})(?:\.(?:[0-9A-Za-z][0-9A-Za-z-]{0,62}))*(\.?|\b)
+HOST %{HOSTNAME}
+IPORHOST (?:%{HOSTNAME}|%{IP})
+HOSTPORT %{IPORHOST}:%{POSINT}
+
+# paths
+PATH (?:%{UNIXPATH}|%{WINPATH})
+UNIXPATH (?>/(?>[\w_%!$@:.,-]+|\\.)*)+
+TTY (?:/dev/(pts|tty([pq])?)(\w+)?/?(?:[0-9]+))
+WINPATH (?>[A-Za-z]+:|\\)(?:\\[^\\?*]*)+
+URIPROTO [A-Za-z]+(\+[A-Za-z+]+)?
+URIHOST %{IPORHOST}(?::%{POSINT:port})?
+# uripath comes loosely from RFC1738, but mostly from what Firefox
+# doesn't turn into %XX
+URIPATH (?:/[A-Za-z0-9$.+!*'(){},~:;=@#%_\-]*)+
+#URIPARAM \?(?:[A-Za-z0-9]+(?:=(?:[^&]*))?(?:&(?:[A-Za-z0-9]+(?:=(?:[^&]*))?)?)*)?
+URIPARAM \?[A-Za-z0-9$.+!*'|(){},~@#%&/=:;_?\-\[\]]*
+URIPATHPARAM %{URIPATH}(?:%{URIPARAM})?
+URI %{URIPROTO}://(?:%{USER}(?::[^@]*)?@)?(?:%{URIHOST})?(?:%{URIPATHPARAM})?
+
+# Shortcuts
+QS %{QUOTEDSTRING}
+
+# Log formats
+SYSLOGBASE %{SYSLOGTIMESTAMP:timestamp} (?:%{SYSLOGFACILITY} )?%{SYSLOGHOST:logsource} %{SYSLOGPROG}:
+COMMONAPACHELOG %{IPORHOST:clientip} %{USER:ident} %{USER:auth} \[%{HTTPDATE:timestamp}\] "(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest})" %{NUMBER:response} (?:%{NUMBER:bytes}|-)
+COMBINEDAPACHELOG %{COMMONAPACHELOG} %{QS:referrer} %{QS:agent}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.csv.CSVReader/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.csv.CSVReader/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.csv.CSVReader/additionalDetails.html
new file mode 100644
index 0000000..e6dfd0c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.csv.CSVReader/additionalDetails.html
@@ -0,0 +1,185 @@
+<!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>CSVReader</title>
+        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
+    </head>
+
+    <body>
+        <p>
+        	The CSVReader Controller Service, expects input in such a way that the first line of a FlowFile specifies the name of
+        	each column in the data. Following the first line, the rest of the FlowFile is expected to be valid CSV data from which
+        	to form appropriate Records. By default, the schema for a FlowFile is inferred by extracting the name of each column from
+        	the first line of the CSV and assumes that all columns are of type <code>string</code>. Of course, we may want to treat some
+        	columns as a data type other than <code>string</code>. This can be accomplished by adding a user-defined property where the
+        	name of the property is the same as the name of a CSV column and the value of the property is the data type to use.
+        </p>
+        
+        <p>
+        	When specifying a data type for a field, the following values are valid:
+        </p>
+        
+        <ul>
+        	<li><b>string</b></li>
+        	<li><b>boolean</b></li>
+        	<li><b>byte</b></li>
+        	<li><b>char</b></li>
+        	<li><b>short</b></li>
+        	<li><b>int</b></li>
+        	<li><b>bigint</b></li>
+        	<li><b>long</b></li>
+        	<li><b>float</b></li>
+        	<li><b>double</b></li>
+        	<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
+        		by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).
+        	</li>
+        	<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
+        		by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).
+        	</li>
+        	<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
+        		<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
+        		<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).</li>
+        	<li><b>object</b> - <i>This data type does not apply to CSV data.</i></li>
+        	<li><b>array</b> - <i>This data type does not apply to CSV data.</i></li>
+        </ul>
+        
+        <p>
+        	As an example, consider a FlowFile whose contents consists of the following:
+        </p>
+        
+        <code>
+        	id, name, balance, notes<br />
+        	1, John, 48.23, "Our very<br />
+first customer!"<br />
+        	2, Jane, 1245.89,<br />
+        	3, Frank Franklin, "48481.29",<br />
+        </code>
+        
+        <p>
+        	Additionally, let's consider that this Controller Service is configured with the following user-defined properties:
+        </p>
+        
+		<table>
+    		<head>
+    			<th>Property Name</th>
+    			<th>Property Value</th>
+    		</head>
+    		<body>
+    			<tr>
+    				<td>balance</td>
+    				<td><code>float</code></td>
+    			</tr>
+    		</body>
+    	</table>
+    	
+    	<p>
+    		In this case, the result will be that this FlowFile consists of 3 different records. The first record will contain the following values:
+    	</p>
+
+		<table>
+    		<head>
+    			<th>Field Name</th>
+    			<th>Field Value</th>
+    		</head>
+    		<body>
+    			<tr>
+    				<td>id</td>
+    				<td>1</td>
+    			</tr>
+    			<tr>
+    				<td>name</td>
+    				<td>John</td>
+    			</tr>
+    			<tr>
+    				<td>balance</td>
+    				<td>48.23</td>
+    			</tr>
+    			<tr>
+    				<td>notes</td>
+    				<td>Our very<br />first customer!</td>
+    			</tr>
+    		</body>
+    	</table>
+    	
+    	<p>
+    		The second record will contain the following values:
+    	</p>
+    	
+		<table>
+    		<head>
+    			<th>Field Name</th>
+    			<th>Field Value</th>
+    		</head>
+    		<body>
+    			<tr>
+    				<td>id</td>
+    				<td>2</td>
+    			</tr>
+    			<tr>
+    				<td>name</td>
+    				<td>Jane</td>
+    			</tr>
+    			<tr>
+    				<td>balance</td>
+    				<td>1245.89</td>
+    			</tr>
+    			<tr>
+    				<td>notes</td>
+    				<td></td>
+    			</tr>
+    		</body>
+    	</table>
+    	
+		<p>
+			The third record will contain the following values:
+		</p>    	
+    	
+		<table>
+    		<head>
+    			<th>Field Name</th>
+    			<th>Field Value</th>
+    		</head>
+    		<body>
+    			<tr>
+    				<td>id</td>
+    				<td>3</td>
+    			</tr>
+    			<tr>
+    				<td>name</td>
+    				<td>Frank Franklin</td>
+    			</tr>
+    			<tr>
+    				<td>balance</td>
+    				<td>48481.29</td>
+    			</tr>
+    			<tr>
+    				<td>notes</td>
+    				<td></td>
+    			</tr>
+    		</body>
+    	</table>
+    	
+    	
+    </body>
+</html>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.grok.GrokReader/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.grok.GrokReader/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.grok.GrokReader/additionalDetails.html
new file mode 100644
index 0000000..3a41f47
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.grok.GrokReader/additionalDetails.html
@@ -0,0 +1,396 @@
+<!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>GrokReader</title>
+        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
+    </head>
+
+    <body>
+        <p>
+        	The GrokReader Controller Service, provides a means for parsing and structuring input that is
+        	made up of unstructured text, such as log files. Grok allows users to add a naming construct to
+        	Regular Expressions such that they can be composed in order to create expressions that are easier
+        	to manage and work with. This Controller Service consists of one Required Property and one Optional
+        	Property. The Optional Property is named <code>Grok Pattern File</code> and specifies the filename of
+        	a file that contains Grok Patterns that can be used for parsing log data. If not specified, a default
+        	patterns file will be used. Its contains are provided below.
+		</p>
+		
+		<p>
+        	The Required Property is named <code>Grok Expression</code> and specifies how to parse each
+        	incoming record. This is done by providing a Grok Expression such as:
+        	<code>%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \[%{DATA:thread}\] %{DATA:class} %{GREEDYDATA:message}</code>.
+        	This Expression will parse Apache NiFi log messages. This is accomplished by specifying that a line begins
+        	with the <code>TIMESTAMP_ISO8601</code> pattern (which is a Regular Expression defined in the default
+        	Grok Patterns File). The value that matches this pattern is then given the name <code>timestamp</code>. As a result,
+        	the value that matches this pattern will be assigned to a field named <code>timestamp</code> in the Record that
+        	produced by this Controller Service.
+        </p>
+        
+        <p>
+        	If a line is encountered in the FlowFile that does not match the configured Grok Expression, it is assumed that the line
+        	is part of the previous message. If the line is the start of a stack trace, then the entire stack trace is read in and assigned
+        	to a field named <code>STACK_TRACE</code>. Otherwise, the line is appended to the last field defined in the Grok Expression. This
+        	is done because typically the last field is a 'message' type of field, which can consist of new-lines.
+        </p>
+        
+        <p>
+        	By default, all fields that are extracted are considered to be of type <code>string</code>. This can be overridden
+        	by adding a user-defined property where the name of the property matches the name of the field that is present in the
+        	configured Grok Expression. The value of the user-defined property is the data type to use.
+        	When specifying a data type for a field, the following values are valid:
+        </p>
+        
+        <ul>
+        	<li><b>string</b></li>
+        	<li><b>boolean</b></li>
+        	<li><b>byte</b></li>
+        	<li><b>char</b></li>
+        	<li><b>short</b></li>
+        	<li><b>int</b></li>
+        	<li><b>bigint</b></li>
+        	<li><b>long</b></li>
+        	<li><b>float</b></li>
+        	<li><b>double</b></li>
+        	<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
+        		by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).
+        	</li>
+        	<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
+        		by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).
+        	</li>
+        	<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
+        		<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
+        		<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).</li>
+        	<li><b>object</b> - <i>This data type does not apply to CSV data.</i></li>
+        	<li><b>array</b> - <i>This data type does not apply to CSV data.</i></li>
+        </ul>
+        
+        
+        <h2>
+        	Examples
+		</h2>
+        
+        <p>
+        	As an example, consider that this Controller Service is configured with the following properties:
+        </p>
+
+		<table>
+    		<head>
+    			<th>Property Name</th>
+    			<th>Property Value</th>
+    		</head>
+    		<body>
+    			<tr>
+    				<td>Grok Expression</td>
+    				<td><code>%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \[%{DATA:thread}\] %{DATA:class} %{GREEDYDATA:message}</code></td>
+    			</tr>
+    		</body>
+    	</table>
+
+        <p>
+        	Additionally, let's consider a FlowFile whose contents consists of the following:
+        </p>
+
+        <code><pre>
+2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.c.l.e.CuratorLeaderElectionManager org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 has been interrupted; no longer leader for role 'Cluster Coordinator'
+2016-08-04 13:26:32,474 ERROR [Leader Election Notification Thread-2] o.apache.nifi.controller.FlowController One
+Two
+Three
+org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_45]
+	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_45]
+        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_45]
+        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_45]
+        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_45]
+        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_45]
+        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
+Caused by: org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+    ... 12 common frames omitted
+2016-08-04 13:26:35,475 WARN [Curator-Framework-0] org.apache.curator.ConnectionState Connection attempt unsuccessful after 3008 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.
+        </pre></code>
+    	
+    	<p>
+    		In this case, the result will be that this FlowFile consists of 3 different records. The first record will contain the following values:
+    	</p>
+
+		<table>
+    		<head>
+    			<th>Field Name</th>
+    			<th>Field Value</th>
+    		</head>
+    		<body>
+    			<tr>
+    				<td>timestamp</td>
+    				<td>2016-08-04 13:26:32,473</td>
+    			</tr>
+    			<tr>
+    				<td>level</td>
+    				<td>INFO</td>
+    			</tr>
+    			<tr>
+    				<td>thread</td>
+    				<td>Leader Election Notification Thread-1</td>
+    			</tr>
+    			<tr>
+    				<td>class</td>
+    				<td>o.a.n.c.l.e.CuratorLeaderElectionManager</td>
+    			</tr>
+    			<tr>
+    				<td>message</td>
+    				<td>org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 has been interrupted; no longer leader for role 'Cluster Coordinator'</td>
+    			</tr>
+    			<tr>
+    				<td>STACK_TRACE</td>
+    				<td><i>null</i></td>
+    			</tr>
+    		</body>
+    	</table>
+    	
+    	<p>
+    		The second record will contain the following values:
+    	</p>
+    	
+		<table>
+    		<head>
+    			<th>Field Name</th>
+    			<th>Field Value</th>
+    		</head>
+    		<body>
+    			<tr>
+    				<td>timestamp</td>
+    				<td>2016-08-04 13:26:32,474</td>
+    			</tr>
+    			<tr>
+    				<td>level</td>
+    				<td>ERROR</td>
+    			</tr>
+    			<tr>
+    				<td>thread</td>
+    				<td>Leader Election Notification Thread-2</td>
+    			</tr>
+    			<tr>
+    				<td>class</td>
+    				<td>o.apache.nifi.controller.FlowController</td>
+    			</tr>
+    			<tr>
+    				<td>message</td>
+    				<td>One<br />
+Two<br />
+Three</td>
+    			</tr>
+    			<tr>
+    				<td>STACK_TRACE</td>
+    				<td>
+<pre>
+org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_45]
+	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_45]
+        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_45]
+        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_45]
+        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_45]
+        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_45]
+        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
+Caused by: org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+    at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
+    ... 12 common frames omitted
+</pre></td>
+    			</tr>
+    		</body>
+    	</table>
+    	
+		<p>
+			The third record will contain the following values:
+		</p>    	
+    	
+		<table>
+    		<head>
+    			<th>Field Name</th>
+    			<th>Field Value</th>
+    		</head>
+    		<body>
+    			<tr>
+    				<td>timestamp</td>
+    				<td>2016-08-04 13:26:35,475</td>
+    			</tr>
+    			<tr>
+    				<td>level</td>
+    				<td>WARN</td>
+    			</tr>
+    			<tr>
+    				<td>thread</td>
+    				<td>Curator-Framework-0</td>
+    			</tr>
+    			<tr>
+    				<td>class</td>
+    				<td>org.apache.curator.ConnectionState</td>
+    			</tr>
+    			<tr>
+    				<td>message</td>
+    				<td>Connection attempt unsuccessful after 3008 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.</td>
+    			</tr>
+    			<tr>
+    				<td>STACK_TRACE</td>
+    				<td><i>null</i></td>
+    			</tr>
+    		</body>
+    	</table>    	
+
+		
+		<h2>
+		</h2>
+    	
+    	<h2>Default Patterns</h2>
+
+    	<p>
+    		The following patterns are available in the default Grok Pattern File:
+    	</p>
+
+		<code>
+		<pre>
+# Log Levels
+LOGLEVEL ([Aa]lert|ALERT|[Tt]race|TRACE|[Dd]ebug|DEBUG|[Nn]otice|NOTICE|[Ii]nfo|INFO|[Ww]arn?(?:ing)?|WARN?(?:ING)?|[Ee]rr?(?:or)?|ERR?(?:OR)?|[Cc]rit?(?:ical)?|CRIT?(?:ICAL)?|[Ff]atal|FATAL|[Ss]evere|SEVERE|EMERG(?:ENCY)?|[Ee]merg(?:ency)?)|FINE|FINER|FINEST|CONFIG
+
+# Syslog Dates: Month Day HH:MM:SS
+SYSLOGTIMESTAMP %{MONTH} +%{MONTHDAY} %{TIME}
+PROG (?:[\w._/%-]+)
+SYSLOGPROG %{PROG:program}(?:\[%{POSINT:pid}\])?
+SYSLOGHOST %{IPORHOST}
+SYSLOGFACILITY <%{NONNEGINT:facility}.%{NONNEGINT:priority}>
+HTTPDATE %{MONTHDAY}/%{MONTH}/%{YEAR}:%{TIME} %{INT}
+
+# Months: January, Feb, 3, 03, 12, December
+MONTH \b(?:Jan(?:uary)?|Feb(?:ruary)?|Mar(?:ch)?|Apr(?:il)?|May|Jun(?:e)?|Jul(?:y)?|Aug(?:ust)?|Sep(?:tember)?|Oct(?:ober)?|Nov(?:ember)?|Dec(?:ember)?)\b
+MONTHNUM (?:0?[1-9]|1[0-2])
+MONTHNUM2 (?:0[1-9]|1[0-2])
+MONTHDAY (?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9])
+
+# Days: Monday, Tue, Thu, etc...
+DAY (?:Mon(?:day)?|Tue(?:sday)?|Wed(?:nesday)?|Thu(?:rsday)?|Fri(?:day)?|Sat(?:urday)?|Sun(?:day)?)
+
+# Years?
+YEAR (?>\d\d){1,2}
+HOUR (?:2[0123]|[01]?[0-9])
+MINUTE (?:[0-5][0-9])
+# '60' is a leap second in most time standards and thus is valid.
+SECOND (?:(?:[0-5]?[0-9]|60)(?:[:.,][0-9]+)?)
+TIME (?!<[0-9])%{HOUR}:%{MINUTE}(?::%{SECOND})(?![0-9])
+
+# datestamp is YYYY/MM/DD-HH:MM:SS.UUUU (or something like it)
+DATE_US_MONTH_DAY_YEAR %{MONTHNUM}[/-]%{MONTHDAY}[/-]%{YEAR}
+DATE_US_YEAR_MONTH_DAY %{YEAR}[/-]%{MONTHNUM}[/-]%{MONTHDAY}
+DATE_US %{DATE_US_MONTH_DAY_YEAR}|%{DATE_US_YEAR_MONTH_DAY}
+DATE_EU %{MONTHDAY}[./-]%{MONTHNUM}[./-]%{YEAR}
+ISO8601_TIMEZONE (?:Z|[+-]%{HOUR}(?::?%{MINUTE}))
+ISO8601_SECOND (?:%{SECOND}|60)
+TIMESTAMP_ISO8601 %{YEAR}-%{MONTHNUM}-%{MONTHDAY}[T ]%{HOUR}:?%{MINUTE}(?::?%{SECOND})?%{ISO8601_TIMEZONE}?
+DATE %{DATE_US}|%{DATE_EU}
+DATESTAMP %{DATE}[- ]%{TIME}
+TZ (?:[PMCE][SD]T|UTC)
+DATESTAMP_RFC822 %{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{TIME} %{TZ}
+DATESTAMP_RFC2822 %{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{TIME} %{ISO8601_TIMEZONE}
+DATESTAMP_OTHER %{DAY} %{MONTH} %{MONTHDAY} %{TIME} %{TZ} %{YEAR}
+DATESTAMP_EVENTLOG %{YEAR}%{MONTHNUM2}%{MONTHDAY}%{HOUR}%{MINUTE}%{SECOND}
+
+
+POSINT \b(?:[1-9][0-9]*)\b
+NONNEGINT \b(?:[0-9]+)\b
+WORD \b\w+\b
+NOTSPACE \S+
+SPACE \s*
+DATA .*?
+GREEDYDATA .*
+QUOTEDSTRING (?>(?<!\\)(?>"(?>\\.|[^\\"]+)+"|""|(?>'(?>\\.|[^\\']+)+')|''|(?>`(?>\\.|[^\\`]+)+`)|``))
+UUID [A-Fa-f0-9]{8}-(?:[A-Fa-f0-9]{4}-){3}[A-Fa-f0-9]{12}
+
+USERNAME [a-zA-Z0-9._-]+
+USER %{USERNAME}
+INT (?:[+-]?(?:[0-9]+))
+BASE10NUM (?<![0-9.+-])(?>[+-]?(?:(?:[0-9]+(?:\.[0-9]+)?)|(?:\.[0-9]+)))
+NUMBER (?:%{BASE10NUM})
+BASE16NUM (?<![0-9A-Fa-f])(?:[+-]?(?:0x)?(?:[0-9A-Fa-f]+))
+BASE16FLOAT \b(?<![0-9A-Fa-f.])(?:[+-]?(?:0x)?(?:(?:[0-9A-Fa-f]+(?:\.[0-9A-Fa-f]*)?)|(?:\.[0-9A-Fa-f]+)))\b
+
+# Networking
+MAC (?:%{CISCOMAC}|%{WINDOWSMAC}|%{COMMONMAC})
+CISCOMAC (?:(?:[A-Fa-f0-9]{4}\.){2}[A-Fa-f0-9]{4})
+WINDOWSMAC (?:(?:[A-Fa-f0-9]{2}-){5}[A-Fa-f0-9]{2})
+COMMONMAC (?:(?:[A-Fa-f0-9]{2}:){5}[A-Fa-f0-9]{2})
+IPV6 ((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5
 ]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:)))(%.+)?
+IPV4 (?<![0-9])(?:(?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2}))(?![0-9])
+IP (?:%{IPV6}|%{IPV4})
+HOSTNAME \b(?:[0-9A-Za-z][0-9A-Za-z-]{0,62})(?:\.(?:[0-9A-Za-z][0-9A-Za-z-]{0,62}))*(\.?|\b)
+HOST %{HOSTNAME}
+IPORHOST (?:%{HOSTNAME}|%{IP})
+HOSTPORT %{IPORHOST}:%{POSINT}
+
+# paths
+PATH (?:%{UNIXPATH}|%{WINPATH})
+UNIXPATH (?>/(?>[\w_%!$@:.,-]+|\\.)*)+
+TTY (?:/dev/(pts|tty([pq])?)(\w+)?/?(?:[0-9]+))
+WINPATH (?>[A-Za-z]+:|\\)(?:\\[^\\?*]*)+
+URIPROTO [A-Za-z]+(\+[A-Za-z+]+)?
+URIHOST %{IPORHOST}(?::%{POSINT:port})?
+# uripath comes loosely from RFC1738, but mostly from what Firefox
+# doesn't turn into %XX
+URIPATH (?:/[A-Za-z0-9$.+!*'(){},~:;=@#%_\-]*)+
+#URIPARAM \?(?:[A-Za-z0-9]+(?:=(?:[^&]*))?(?:&(?:[A-Za-z0-9]+(?:=(?:[^&]*))?)?)*)?
+URIPARAM \?[A-Za-z0-9$.+!*'|(){},~@#%&/=:;_?\-\[\]]*
+URIPATHPARAM %{URIPATH}(?:%{URIPARAM})?
+URI %{URIPROTO}://(?:%{USER}(?::[^@]*)?@)?(?:%{URIHOST})?(?:%{URIPATHPARAM})?
+
+# Shortcuts
+QS %{QUOTEDSTRING}
+
+# Log formats
+SYSLOGBASE %{SYSLOGTIMESTAMP:timestamp} (?:%{SYSLOGFACILITY} )?%{SYSLOGHOST:logsource} %{SYSLOGPROG}:
+COMMONAPACHELOG %{IPORHOST:clientip} %{USER:ident} %{USER:auth} \[%{HTTPDATE:timestamp}\] "(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest})" %{NUMBER:response} (?:%{NUMBER:bytes}|-)
+COMBINEDAPACHELOG %{COMMONAPACHELOG} %{QS:referrer} %{QS:agent}
+		</pre>
+		</code>
+
+    </body>
+</html>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonPathReader/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonPathReader/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonPathReader/additionalDetails.html
new file mode 100644
index 0000000..2b69f7e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonPathReader/additionalDetails.html
@@ -0,0 +1,227 @@
+<!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>JsonPathReader</title>
+        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
+    </head>
+
+    <body>
+        <p>
+        	The JsonPathReader Controller Service, parses FlowFiles that are in the JSON format. User-defined properties
+        	specify how to extract all relevant fields from the JSON in order to create a row-oriented record. The Controller
+        	Service will not be valid unless at least one JSON Path is provided. Unlike the
+        	<a href="../org.apache.nifi.json.FlatJsonReader/additionalDetails.html">FlatJsonReader</a> Controller Service, this
+        	service will return a record that contains only those fields that have been configured via JSON Path.
+        </p>
+        
+        <p>
+        	If the root of the FlowFile's JSON is a JSON Array, each JSON Object found in that array will be treated as a separate
+        	Record, not as a single record made up of an array. If the root of the FlowFile's JSON is a JSON Object, it will be
+        	evaluated as a single Record.
+        </p>
+        
+        <p>
+        	Supplying a JSON Path is accomplished by adding a user-defined property where the name of the property becomes the name
+        	of the field in the Record that is returned. The value of the property must be a valid JSON Path expression. This JSON Path
+        	will be evaluated against each top-level JSON Object in the FlowFile, and the result will be the value of the field whose
+        	name is specified by the property name. By default, the type of each field is inferred automatically based on the values of
+        	the first JSON Object encountered for the FlowFile. This can be overridden by changing the name of the user-defined property
+        	by adding a colon (:) and specifying the data type. For example: <code>balance:double</code> or <code>dob:date:MM/dd/yyyy</code>.
+        	In this case, the data type and option format are not included in the field name. So for the aforementioned examples, we would
+        	end up with field names <code>balance</code> and <code>dob</code>.
+        </p>
+        
+		<p>
+        	When specifying a data type for a field, the following values are valid:
+        </p>
+        
+        <ul>
+        	<li><b>string</b></li>
+        	<li><b>boolean</b></li>
+        	<li><b>byte</b></li>
+        	<li><b>char</b></li>
+        	<li><b>short</b></li>
+        	<li><b>int</b></li>
+        	<li><b>bigint</b></li>
+        	<li><b>long</b></li>
+        	<li><b>float</b></li>
+        	<li><b>double</b></li>
+        	<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
+        		by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).
+        	</li>
+        	<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
+        		by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).
+        	</li>
+        	<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
+        		<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
+        		<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).</li>
+        	<li><b>object</b> - The value will be returned as a <code>Map&lt;String, Object&gt;</code>. The types of the values in the Map
+        		are always inferred. The type used for the values may not be the same for each record. For example, consider the following
+        		JSON array:
+        			<br /><br />
+        			<code>
+        			[{
+        			    id: 17,
+        			  	name: "John",
+        			    child: {
+        			    	id: "1"
+        			    },
+        			    siblingIds: [4, "8"]
+        			  },
+        			  <br />{
+        			  	id: 98,
+        			    name: "Jane",
+        			    child: {
+        			        id: 2
+        			    },
+        			    siblingIds: []
+        			  }]
+        			</code>
+        			<br /><br />
+        		In this case, the <code>child</code> element would be inferred to be of type <code>object</code>. Since nested types
+        		are inferred on a per-record basis, for the first record, the <code>child</code> field would return a <code>Map</code>
+        		where the value of the <code>id</code> entry is a <code>string</code>. However, for the second record, the <code>child</code>
+        		field would return a <code>Map</code> where the value of the <code>id</code> entry is an <code>int</code>.
+        		<br />
+        		Moreover, the <code>siblingIds</code> of the John will be an <code>array</code> where the first element is an <code>int</code>
+        		and the second element is a <code>string</code>. The <code>siblingIds</code> of Jane will be an empty array.
+        		</li>
+        	<li><b>array</b> - An array of values. The types of the values are always inferred and may not be the same for each element
+        		in the array, or for two arrays from different JSON objects.</li>
+        </ul>
+
+
+        <p>
+        	As an example, consider a FlowFile whose content contains the following JSON:
+        </p>
+        
+        <code>
+			[{
+			    id: 17,
+			  	name: "John",
+			    child: {
+			    	id: "1"
+			    },
+			    siblingIds: [4, "8"]
+			  },
+			  <br />{
+			  	id: 98,
+			    name: "Jane",
+			    child: {
+			        id: 2
+			    },
+			    gender: "F",
+			    siblingIds: []
+			  }]
+        </code>
+        
+        <p>
+        	If we configure this Controller Service with the following user-defined properties:
+        	
+        	<table>
+        		<head>
+        			<th>Property Name</th>
+        			<th>Property Value</th>
+        		</head>
+        		<body>
+        			<tr>
+        				<td>id</td>
+        				<td><code>$.id</code></td>
+        			</tr>
+        			<tr>
+        				<td>name</td>
+        				<td><code>$.name</code></td>
+        			</tr>
+        			<tr>
+        				<td>childId:long</td>
+        				<td><code>$.child.id</code></td>
+        			</tr>
+        			<tr>
+        				<td>gender:string</td>
+        				<td><code>$.gender</code></td>
+        			</tr>
+        		</body>
+        	</table>
+        </p>
+        
+		<p>
+			In this case, the FlowFile will generate two Records. The first record will consist of the following key/value pairs:
+
+        	<table>
+        		<head>
+        			<th>Field Name</th>
+        			<th>Field Value</th>
+        		</head>
+        		<body>
+        			<tr>
+        				<td>id</td>
+        				<td>17</td>
+        			</tr>
+        			<tr>
+        				<td>name</td>
+        				<td>John</td>
+        			</tr>
+        			<tr>
+        				<td>childId</td>
+        				<td>1</td>
+        			</tr>
+        			<tr>
+        				<td>gender</td>
+        				<td><i>null</i></td>
+        			</tr>
+				</body>
+			</table>
+		</p>
+		
+		<p>
+			The second record will consist of the following key/value pairs:
+
+        	<table>
+        		<head>
+        			<th>Field Name</th>
+        			<th>Field Value</th>
+        		</head>
+        		<body>
+        			<tr>
+        				<td>id</td>
+        				<td>98</td>
+        			</tr>
+        			<tr>
+        				<td>name</td>
+        				<td>Jane</td>
+        			</tr>
+        			<tr>
+        				<td>childId</td>
+        				<td>2</td>
+        			</tr>
+        			<tr>
+        				<td>gender</td>
+        				<td>F</td>
+        			</tr>
+				</body>
+			</table>
+		</p>
+		
+    </body>
+</html>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonTreeReader/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonTreeReader/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonTreeReader/additionalDetails.html
new file mode 100644
index 0000000..7d6be7a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.json.JsonTreeReader/additionalDetails.html
@@ -0,0 +1,102 @@
+<!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>JsonTreeReader</title>
+        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
+    </head>
+
+    <body>
+        <p>
+        	The JsonTreeReader Controller Service, by default, derives the schema for a FlowFile
+        	based on the first JSON Object in the FlowFile. For each field found, the data type
+        	is inferred. However, the type of a field can be overridden by adding a user-defined property to
+        	the Controller Service. The name of the property should be the same as the name of the
+        	JSON field. The value of the property denotes the data type of the corresponding field.
+        	If no JSON field is found with a matching name, then a field will be added to the schema,
+        	and a <code>null</code> value will be used for any record for which the JSON field
+        	is not present. If a field is found with a matching name, but the type is different,
+        	the Controller Service will attempt to coerce the value into the user-defined type. If unable
+        	to do so, an Exception will be thrown.
+        </p>
+        
+        <p>
+        	When specifying a data type for a field, the following values are valid:
+        </p>
+        
+        <ul>
+        	<li><b>string</b></li>
+        	<li><b>boolean</b></li>
+        	<li><b>byte</b></li>
+        	<li><b>char</b></li>
+        	<li><b>short</b></li>
+        	<li><b>int</b></li>
+        	<li><b>bigint</b></li>
+        	<li><b>long</b></li>
+        	<li><b>float</b></li>
+        	<li><b>double</b></li>
+        	<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
+        		by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).
+        	</li>
+        	<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
+        		by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).
+        	</li>
+        	<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
+        		<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
+        		<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
+        		that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
+        		SimpleDateFormat Patterns</a> for more information).</li>
+        	<li><b>object</b> - The value will be returned as a <code>Map&lt;String, Object&gt;</code>. The types of the values in the Map
+        		are always inferred. The type used for the values may not be the same for each record. For example, consider the following
+        		JSON array:
+        			<br /><br />
+        			<code>
+        			[{
+        			    id: 17,
+        			  	name: "John",
+        			    child: {
+        			    	id: "1"
+        			    },
+        			    siblingIds: [4, "8"]
+        			  },
+        			  <br />{
+        			  	id: 98,
+        			    name: "Jane",
+        			    child: {
+        			        id: 2
+        			    },
+        			    siblingIds: []
+        			  }]
+        			</code>
+        			<br /><br />
+        		In this case, the <code>child</code> element would be inferred to be of type <code>object</code>. Since nested types
+        		are inferred on a per-record basis, for the first record, the <code>child</code> field would return a <code>Map</code>
+        		where the value of the <code>id</code> entry is a <code>string</code>. However, for the second record, the <code>child</code>
+        		field would return a <code>Map</code> where the value of the <code>id</code> entry is an <code>int</code>.
+        		<br />
+        		Moreover, the <code>siblingIds</code> of the John will be an <code>array</code> where the first element is an <code>int</code>
+        		and the second element is a <code>string</code>. The <code>siblingIds</code> of Jane will be an empty array.
+        		</li>
+        	<li><b>array</b> - An array of values. The types of the values are always inferred and may not be the same for each element
+        		in the array, or for two arrays from different JSON objects.</li>
+        </ul>
+    </body>
+</html>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..2ec3441
--- /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/TestAvroRecordReader.java
@@ -0,0 +1,221 @@
+/*
+ * 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.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+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 testDataTypes() throws IOException, MalformedRecordException {
+        final List<Field> accountFields = new ArrayList<>();
+        accountFields.add(new Field("accountId", Schema.create(Type.LONG), null, null));
+        accountFields.add(new Field("accountName", Schema.create(Type.STRING), null, null));
+        final Schema accountSchema = Schema.createRecord("account", null, null, false);
+        accountSchema.setFields(accountFields);
+
+        final List<Field> catFields = new ArrayList<>();
+        catFields.add(new Field("catTailLength", Schema.create(Type.INT), null, null));
+        catFields.add(new Field("catName", Schema.create(Type.STRING), null, null));
+        final Schema catSchema = Schema.createRecord("cat", null, null, false);
+        catSchema.setFields(catFields);
+
+        final List<Field> dogFields = new ArrayList<>();
+        dogFields.add(new Field("dogTailLength", Schema.create(Type.INT), null, null));
+        dogFields.add(new Field("dogName", Schema.create(Type.STRING), null, null));
+        final Schema dogSchema = Schema.createRecord("dog", null, null, false);
+        dogSchema.setFields(dogFields);
+
+        final List<Field> fields = new ArrayList<>();
+        fields.add(new Field("name", Schema.create(Type.STRING), null, null));
+        fields.add(new Field("age", Schema.create(Type.INT), null, null));
+        fields.add(new Field("balance", Schema.create(Type.DOUBLE), null, null));
+        fields.add(new Field("rate", Schema.create(Type.FLOAT), null, null));
+        fields.add(new Field("debt", Schema.create(Type.BOOLEAN), null, null));
+        fields.add(new Field("nickname", Schema.create(Type.NULL), null, null));
+        fields.add(new Field("binary", Schema.create(Type.BYTES), null, null));
+        fields.add(new Field("fixed", Schema.createFixed("fixed", null, null, 5), null, null));
+        fields.add(new Field("map", Schema.createMap(Schema.create(Type.STRING)), null, null));
+        fields.add(new Field("array", Schema.createArray(Schema.create(Type.LONG)), null, null));
+        fields.add(new Field("account", accountSchema, null, null));
+        fields.add(new Field("desiredbalance", Schema.createUnion( // test union of NULL and other type with no value
+            Arrays.asList(Schema.create(Type.NULL), Schema.create(Type.DOUBLE))),
+            null, null));
+        fields.add(new Field("dreambalance", Schema.createUnion( // test union of NULL and other type with a value
+            Arrays.asList(Schema.create(Type.NULL), Schema.create(Type.DOUBLE))),
+            null, null));
+        fields.add(new Field("favAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, null));
+        fields.add(new Field("otherFavAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, null));
+
+        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 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("binary".getBytes(StandardCharsets.UTF_8), (byte[]) values[6]);
+            assertArrayEquals("fixed".getBytes(StandardCharsets.UTF_8), (byte[]) 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]);
+        }
+    }
+
+    public static enum Status {
+        GOOD, BAD;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.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/TestCSVRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java
new file mode 100644
index 0000000..1e53d89
--- /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/TestCSVRecordReader.java
@@ -0,0 +1,122 @@
+/*
+ * 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.assertNull;
+
+import java.io.File;
+import java.io.FileInputStream;
+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.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestCSVRecordReader {
+    private final DataType stringDataType = RecordFieldType.STRING.getDataType();
+    private final DataType doubleDataType = RecordFieldType.DOUBLE.getDataType();
+    private final DataType timeDataType = RecordFieldType.TIME.getDataType();
+
+    @Test
+    public void testSimpleParse() throws IOException, MalformedRecordException {
+        final Map<String, DataType> overrides = new HashMap<>();
+        overrides.put("balance", doubleDataType);
+        overrides.put("other", timeDataType);
+
+        try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/single-bank-account.csv"))) {
+            final CSVRecordReader reader = new CSVRecordReader(fis, null, overrides);
+
+            final RecordSchema schema = reader.getSchema();
+            verifyFields(schema);
+
+            final Object[] record = reader.nextRecord().getValues();
+            final Object[] expectedValues = new Object[] {"1", "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"};
+            Assert.assertArrayEquals(expectedValues, record);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testMultipleRecords() throws IOException, MalformedRecordException {
+        final Map<String, DataType> overrides = new HashMap<>();
+        overrides.put("balance", doubleDataType);
+
+        try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/multi-bank-account.csv"))) {
+            final CSVRecordReader reader = new CSVRecordReader(fis, null, overrides);
+
+            final RecordSchema schema = reader.getSchema();
+            verifyFields(schema);
+
+            final Object[] firstRecord = reader.nextRecord().getValues();
+            final Object[] firstExpectedValues = new Object[] {"1", "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"};
+            Assert.assertArrayEquals(firstExpectedValues, firstRecord);
+
+            final Object[] secondRecord = reader.nextRecord().getValues();
+            final Object[] secondExpectedValues = new Object[] {"2", "Jane Doe", 4820.09D, "321 Your Street", "Your City", "NY", "33333", "USA"};
+            Assert.assertArrayEquals(secondExpectedValues, secondRecord);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    @Test
+    public void testExtraWhiteSpace() throws IOException, MalformedRecordException {
+        final Map<String, DataType> overrides = new HashMap<>();
+        overrides.put("balance", doubleDataType);
+
+        try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/extra-white-space.csv"))) {
+            final CSVRecordReader reader = new CSVRecordReader(fis, Mockito.mock(ComponentLog.class), overrides);
+
+            final RecordSchema schema = reader.getSchema();
+            verifyFields(schema);
+
+            final Object[] firstRecord = reader.nextRecord().getValues();
+            final Object[] firstExpectedValues = new Object[] {"1", "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"};
+            Assert.assertArrayEquals(firstExpectedValues, firstRecord);
+
+            final Object[] secondRecord = reader.nextRecord().getValues();
+            final Object[] secondExpectedValues = new Object[] {"2", "Jane Doe", 4820.09D, "321 Your Street", "Your City", "NY", "33333", "USA"};
+            Assert.assertArrayEquals(secondExpectedValues, secondRecord);
+
+            assertNull(reader.nextRecord());
+        }
+    }
+
+    private void verifyFields(final RecordSchema schema) {
+        final List<String> fieldNames = schema.getFieldNames();
+        final List<String> expectedFieldNames = Arrays.asList("id", "name", "balance", "address", "city", "state", "zipCode", "country");
+        assertEquals(expectedFieldNames, fieldNames);
+
+        final List<DataType> dataTypes = schema.getDataTypes();
+        final List<DataType> expectedDataTypes = Arrays.asList(stringDataType, stringDataType, doubleDataType,
+            stringDataType, stringDataType, stringDataType, stringDataType, stringDataType);
+        assertEquals(expectedDataTypes, dataTypes);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..04f8479
--- /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/TestWriteCSVResult.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.csv;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.junit.Test;
+
+
+public class TestWriteCSVResult {
+
+    @Test
+    public void testDataTypes() throws IOException {
+        final WriteCSVResult result = new WriteCSVResult(RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
+
+        final StringBuilder headerBuilder = new StringBuilder();
+        final List<RecordField> fields = new ArrayList<>();
+        for (final RecordFieldType fieldType : RecordFieldType.values()) {
+            if (fieldType == RecordFieldType.CHOICE) {
+                final List<DataType> possibleTypes = new ArrayList<>();
+                possibleTypes.add(RecordFieldType.INT.getDataType());
+                possibleTypes.add(RecordFieldType.LONG.getDataType());
+
+                fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType(possibleTypes)));
+            } else {
+                fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType()));
+            }
+
+            headerBuilder.append('"').append(fieldType.name().toLowerCase()).append('"').append(",");
+        }
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+
+        final long now = System.currentTimeMillis();
+        final Map<String, Object> valueMap = new HashMap<>();
+        valueMap.put("string", "string");
+        valueMap.put("boolean", true);
+        valueMap.put("byte", (byte) 1);
+        valueMap.put("char", 'c');
+        valueMap.put("short", (short) 8);
+        valueMap.put("int", 9);
+        valueMap.put("bigint", BigInteger.valueOf(8L));
+        valueMap.put("long", 8L);
+        valueMap.put("float", 8.0F);
+        valueMap.put("double", 8.0D);
+        valueMap.put("date", new Date(now));
+        valueMap.put("time", new Time(now));
+        valueMap.put("timestamp", new Timestamp(now));
+        valueMap.put("object", null);
+        valueMap.put("choice", 48L);
+        valueMap.put("array", null);
+
+        final Record record = new MapRecord(schema, valueMap);
+        final RecordSet rs = RecordSet.of(schema, record);
+
+        final String output;
+        try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+            result.write(rs, baos);
+            output = new String(baos.toByteArray(), StandardCharsets.UTF_8);
+        }
+
+        headerBuilder.deleteCharAt(headerBuilder.length() - 1);
+        final String headerLine = headerBuilder.toString();
+
+        final String[] splits = output.split("\n");
+        assertEquals(2, splits.length);
+        assertEquals(headerLine, splits[0]);
+
+        final String values = splits[1];
+        final StringBuilder expectedBuilder = new StringBuilder();
+        expectedBuilder.append("\"string\",\"true\",\"1\",\"c\",\"8\",\"9\",\"8\",\"8\",\"8.0\",\"8.0\",");
+
+        final String dateValue = new SimpleDateFormat(RecordFieldType.DATE.getDefaultFormat()).format(now);
+        final String timeValue = new SimpleDateFormat(RecordFieldType.TIME.getDefaultFormat()).format(now);
+        final String timestampValue = new SimpleDateFormat(RecordFieldType.TIMESTAMP.getDefaultFormat()).format(now);
+
+        expectedBuilder.append('"').append(dateValue).append('"').append(',');
+        expectedBuilder.append('"').append(timeValue).append('"').append(',');
+        expectedBuilder.append('"').append(timestampValue).append('"').append(',');
+        expectedBuilder.append(",\"48\",");
+        final String expectedValues = expectedBuilder.toString();
+
+        assertEquals(expectedValues, values);
+    }
+
+}


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

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformJsonToAvro.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformJsonToAvro.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformJsonToAvro.java
deleted file mode 100644
index f54a4b5..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformJsonToAvro.java
+++ /dev/null
@@ -1,45 +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.schemaregistry.processors;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collections;
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.nifi.annotation.behavior.InputRequirement;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-
-@Tags({ "registry", "schema", "avro", "json", "transform" })
-@CapabilityDescription("Transforms JSON content of the Flow File to Avro using the schema provided by the Schema Registry Service.")
-@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
-public final class TransformJsonToAvro extends AbstractContentTransformer {
-
-    /**
-     *
-     */
-    @Override
-    protected Map<String, String> transform(InputStream in, OutputStream out, InvocationContextProperties contextProperties, Schema schema) {
-        GenericRecord avroRecord = JsonUtils.read(in, schema);
-        AvroUtils.write(avroRecord, out);
-        return Collections.singletonMap(CoreAttributes.MIME_TYPE.key(), "binary/avro");
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformJsonToCSV.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformJsonToCSV.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformJsonToCSV.java
deleted file mode 100644
index c026570..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformJsonToCSV.java
+++ /dev/null
@@ -1,45 +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.schemaregistry.processors;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collections;
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.nifi.annotation.behavior.InputRequirement;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-
-@Tags({ "registry", "schema", "csv", "json", "transform" })
-@CapabilityDescription("Transforms JSON content of the Flow File to CSV using the schema provided by the Schema Registry Service.")
-@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
-public final class TransformJsonToCSV extends AbstractCSVTransformer {
-
-    /**
-     *
-     */
-    @Override
-    protected Map<String, String> transform(InputStream in, OutputStream out, InvocationContextProperties contextProperties, Schema schema) {
-        GenericRecord avroRecord = JsonUtils.read(in, schema);
-        CSVUtils.write(avroRecord, this.delimiter, out);
-        return Collections.singletonMap(CoreAttributes.MIME_TYPE.key(), "text/csv");
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
deleted file mode 100644
index 0bb067e..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ /dev/null
@@ -1,21 +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.
-org.apache.nifi.schemaregistry.processors.TransformJsonToAvro
-org.apache.nifi.schemaregistry.processors.TransformAvroToJson
-org.apache.nifi.schemaregistry.processors.TransformCSVToAvro
-org.apache.nifi.schemaregistry.processors.TransformCSVToJson
-org.apache.nifi.schemaregistry.processors.TransformAvroToCSV
-org.apache.nifi.schemaregistry.processors.TransformJsonToCSV
-org.apache.nifi.schemaregistry.processors.ExtractAvroFields
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/java/org/apache/nifi/schemaregistry/processors/TransformersTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/java/org/apache/nifi/schemaregistry/processors/TransformersTest.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/java/org/apache/nifi/schemaregistry/processors/TransformersTest.java
deleted file mode 100644
index 058af62..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/java/org/apache/nifi/schemaregistry/processors/TransformersTest.java
+++ /dev/null
@@ -1,220 +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.schemaregistry.processors;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.FileSystems;
-import java.nio.file.Files;
-import java.text.DecimalFormat;
-import java.text.NumberFormat;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
-import static org.junit.Assume.assumeFalse;
-
-@RunWith(JUnitParamsRunner.class)
-public class TransformersTest {
-
-    private final ClassLoader classLoader = getClass().getClassLoader();
-
-    @Test
-    public void validateCSVtoAvroPair() throws Exception {
-        String data = "John Dow|13|blue";
-        String fooSchemaText = "{\"namespace\": \"example.avro\", " + "\"type\": \"record\", " + "\"name\": \"User\", "
-                + "\"fields\": [ " + "{\"name\": \"name\", \"type\": \"string\"}, "
-                + "{\"name\": \"favorite_number\",  \"type\": \"int\"}, "
-                + "{\"name\": \"favorite_color\", \"type\": \"string\"} " + "]" + "}";
-
-        Schema schema = new Schema.Parser().parse(fooSchemaText);
-
-        // CSV -> AVRO -> CSV
-        ByteArrayInputStream in = new ByteArrayInputStream(data.getBytes());
-        GenericRecord record = CSVUtils.read(in, '|', schema, '\"');
-        ByteArrayOutputStream out = new ByteArrayOutputStream();
-        AvroUtils.write(record, out);
-        byte[] avro = out.toByteArray();
-
-        in = new ByteArrayInputStream(avro);
-        record = AvroUtils.read(in, schema);
-        out = new ByteArrayOutputStream();
-        CSVUtils.write(record, '|', out);
-        byte[] csv = out.toByteArray();
-        assertEquals(data, new String(csv, StandardCharsets.UTF_8));
-    }
-
-    @Test
-    public void validateCSVtoJsonPair() throws Exception {
-        String data = "John Dow|13|blue";
-        String fooSchemaText = "{\"namespace\": \"example.avro\", " + "\"type\": \"record\", " + "\"name\": \"User\", "
-                + "\"fields\": [ " + "{\"name\": \"name\", \"type\": \"string\"}, "
-                + "{\"name\": \"favorite_number\",  \"type\": \"int\"}, "
-                + "{\"name\": \"favorite_color\", \"type\": \"string\"} " + "]" + "}";
-
-        Schema schema = new Schema.Parser().parse(fooSchemaText);
-
-        // CSV -> JSON -> CSV
-        ByteArrayInputStream in = new ByteArrayInputStream(data.getBytes());
-        GenericRecord record = CSVUtils.read(in, '|', schema, '\"');
-        ByteArrayOutputStream out = new ByteArrayOutputStream();
-        JsonUtils.write(record, out);
-        byte[] json = out.toByteArray();
-
-        assertEquals("{\"name\":\"John Dow\",\"favorite_number\":13,\"favorite_color\":\"blue\"}", new String(json, StandardCharsets.UTF_8));
-
-        in = new ByteArrayInputStream(json);
-        record = JsonUtils.read(in, schema);
-        out = new ByteArrayOutputStream();
-        CSVUtils.write(record, '|', out);
-        byte[] csv = out.toByteArray();
-        assertEquals(data, new String(csv, StandardCharsets.UTF_8));
-    }
-
-    @Test
-    public void validateJsonToAvroPair() throws Exception {
-        String data = "{\"name\":\"John Dow\",\"favorite_number\":13,\"favorite_color\":\"blue\"}";
-        String fooSchemaText = "{\"namespace\": \"example.avro\", " + "\"type\": \"record\", " + "\"name\": \"User\", "
-                + "\"fields\": [ " + "{\"name\": \"name\", \"type\": \"string\"}, "
-                + "{\"name\": \"favorite_number\",  \"type\": \"int\"}, "
-                + "{\"name\": \"favorite_color\", \"type\": \"string\"} " + "]" + "}";
-
-        Schema schema = new Schema.Parser().parse(fooSchemaText);
-
-        // JSON -> AVRO -> JSON
-        ByteArrayInputStream in = new ByteArrayInputStream(data.getBytes());
-        GenericRecord record = JsonUtils.read(in, schema);
-        ByteArrayOutputStream out = new ByteArrayOutputStream();
-        AvroUtils.write(record, out);
-        byte[] avro = out.toByteArray();
-
-        in = new ByteArrayInputStream(avro);
-        record = AvroUtils.read(in, schema);
-        out = new ByteArrayOutputStream();
-        JsonUtils.write(record, out);
-        byte[] csv = out.toByteArray();
-        assertEquals(data, new String(csv, StandardCharsets.UTF_8));
-    }
-
-    @Test
-    @Parameters({"input_csv/union_null_last_field_with_default.txt,input_avro/union_and_matching_defaults.txt,expected_ouput_csv/union_null_last_field_with_default.txt",
-            "input_csv/union_with_default.txt,input_avro/union_and_matching_defaults.txt,expected_ouput_csv/union_with_default.txt",
-            "input_csv/union_null_middle_field_with_default.txt,input_avro/union_and_matching_defaults.txt,expected_ouput_csv/union_null_middle_field_with_default.txt",
-            "input_csv/primitive_types.txt,input_avro/primitive_types_no_defaults.txt,expected_ouput_csv/primitive_types.txt",
-            "input_csv/primitive_types_with_matching_default.txt,input_avro/primitive_types_with_matching_default.txt,expected_ouput_csv/primitive_types_with_matching_default.txt",
-            "input_csv/decimal_logicalType_missing_value.txt,input_avro/decimal_logicalType_invalid_scale_with_default.txt,expected_ouput_csv/decimal_logicalType_with_default.txt"})
-    public void testCSVRoundtrip(final String inputCSVFileName, final String inputAvroSchema, final String expectedOuput) throws Exception {
-        assumeFalse(isWindowsEnvironment());
-        final String data = getResourceAsString(inputCSVFileName);
-        final String schemaText = getResourceAsString(inputAvroSchema);
-        final String result = getResourceAsString(expectedOuput);
-        csvRoundTrip(data, schemaText, result);
-    }
-
-    private boolean isWindowsEnvironment() {
-        return System.getProperty("os.name").toLowerCase().startsWith("windows");
-    }
-
-    @Test
-    @Parameters({"input_csv/union_with_missing_value.txt,input_avro/union_and_mismatch_defaults.txt",
-            "input_csv/primitive_types_with_matching_default.txt,input_avro/primitive_types_with_mismatch_default.txt"})
-    public void testCSVMismatchDefaults(final String inputCSVFileName, final String inputAvroSchema)  {
-        assumeFalse(isWindowsEnvironment());
-        try {
-            final String data = getResourceAsString(inputCSVFileName);
-            final String schemaText = getResourceAsString(inputAvroSchema);
-            Schema schema = new Schema.Parser().parse(schemaText);
-
-            ByteArrayInputStream in = new ByteArrayInputStream(data.getBytes());
-            CSVUtils.read(in, '|', schema, '\"');
-        }catch (IOException ioe){
-            assertTrue(false);
-        }catch(IllegalArgumentException iae){
-            assertTrue(true);
-        }
-    }
-
-    @Test
-    public void testCSVRoundTrip() throws IOException {
-        assumeFalse(isWindowsEnvironment());
-        NumberFormat numberFormat = DecimalFormat.getInstance();
-        numberFormat.setGroupingUsed(false);
-        ((DecimalFormat) numberFormat).setParseBigDecimal(true);
-
-        //"input_csv/decimal_logicalType.txt,input_avro/decimal_logicalType_invalid_scale_with_default.txt,expected_ouput_csv/decimal_logicalType_invalid_scale.txt",
-        String decimalLogicalType = "\"fake_transactionid\"|" + numberFormat.format(new BigDecimal(11234567.89));
-        String data = getResourceAsString("input_csv/decimal_logicalType.txt");
-        String schemaText = getResourceAsString("input_avro/decimal_logicalType_invalid_scale_with_default.txt");
-        csvRoundTrip(data, schemaText, decimalLogicalType);
-
-        // needs to be set now because scale < precision
-        numberFormat.setMaximumIntegerDigits(10);
-        numberFormat.setMaximumFractionDigits(3);
-        numberFormat.setMinimumFractionDigits(3);
-
-        //"input_csv/decimal_logicalType.txt,input_avro/decimal_logicalType_valid_scale_with_no_default.txt,expected_ouput_csv/decimal_logicalType.txt",
-        decimalLogicalType = "\"fake_transactionid\"|" + numberFormat.format(new BigDecimal(11234567.890));
-        data = getResourceAsString("input_csv/decimal_logicalType.txt");
-        schemaText = getResourceAsString("input_avro/decimal_logicalType_valid_scale_with_no_default.txt");
-
-        //"input_csv/decimal_logicalType_missing_value.txt,input_avro/decimal_logicalType_valid_scale_with_default.txt,expected_ouput_csv/decimal_logicalType_valid_scale_with_default.txt",
-        decimalLogicalType = "\"fake_transactionid\"|" + numberFormat.format(new BigDecimal(0.000));
-        data = getResourceAsString("input_csv/decimal_logicalType_missing_value.txt");
-        schemaText = getResourceAsString("input_avro/decimal_logicalType_valid_scale_with_default.txt");
-        csvRoundTrip(data, schemaText, decimalLogicalType);
-    }
-
-    private void csvRoundTrip(final String data, final String schemaText, final String result) {
-        Schema schema = new Schema.Parser().parse(schemaText);
-
-        // CSV -> AVRO -> CSV
-        ByteArrayInputStream in = new ByteArrayInputStream(data.getBytes());
-        GenericRecord record = CSVUtils.read(in, '|', schema, '\"');
-        ByteArrayOutputStream out = new ByteArrayOutputStream();
-        AvroUtils.write(record, out);
-        byte[] avro = out.toByteArray();
-
-        in = new ByteArrayInputStream(avro);
-        record = AvroUtils.read(in, schema);
-        out = new ByteArrayOutputStream();
-        CSVUtils.write(record, '|', out);
-        byte[] csv = out.toByteArray();
-        assertEquals(result, new String(csv, StandardCharsets.UTF_8));
-    }
-
-    /**
-     * Simple wrapper around getting the test resource file that is used by the above test cases
-     *
-     * @param fileName - the filename of the file to read
-     * @return A string that contains the body of the file.
-     * @throws IOException - if an error occurs reading the file.
-     */
-    private String getResourceAsString(String fileName) throws IOException {
-        return new String(Files.readAllBytes(FileSystems.getDefault().getPath(classLoader.getResource(fileName).getPath())));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType.txt
deleted file mode 100644
index 1a53f85..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType.txt
+++ /dev/null
@@ -1 +0,0 @@
-"fake_transactionid"|11234567.890
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_invalid_scale.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_invalid_scale.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_invalid_scale.txt
deleted file mode 100644
index 9506ad4..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_invalid_scale.txt
+++ /dev/null
@@ -1 +0,0 @@
-"fake_transactionid"|11234567.89
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_valid_scale_with_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_valid_scale_with_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_valid_scale_with_default.txt
deleted file mode 100644
index 2309e71..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_valid_scale_with_default.txt
+++ /dev/null
@@ -1 +0,0 @@
-"fake_transactionid"|0.000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_with_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_with_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_with_default.txt
deleted file mode 100644
index 3a9689c..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/decimal_logicalType_with_default.txt
+++ /dev/null
@@ -1 +0,0 @@
-"fake_transactionid"|0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/primitive_types.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/primitive_types.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/primitive_types.txt
deleted file mode 100644
index 77f353f..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/primitive_types.txt
+++ /dev/null
@@ -1 +0,0 @@
-"this is a simple string."|10|21474836470|1.7976931348623157E308|true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/primitive_types_with_matching_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/primitive_types_with_matching_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/primitive_types_with_matching_default.txt
deleted file mode 100644
index 095f81e..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/primitive_types_with_matching_default.txt
+++ /dev/null
@@ -1 +0,0 @@
-"default_string"|1234|21474836470|1.7976931348623157E308|true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_null_last_field_with_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_null_last_field_with_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_null_last_field_with_default.txt
deleted file mode 100644
index 83cbf75..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_null_last_field_with_default.txt
+++ /dev/null
@@ -1 +0,0 @@
-andrew|13|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_null_middle_field_with_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_null_middle_field_with_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_null_middle_field_with_default.txt
deleted file mode 100644
index 1b03c97..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_null_middle_field_with_default.txt
+++ /dev/null
@@ -1 +0,0 @@
-andrew|21474|blue
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_with_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_with_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_with_default.txt
deleted file mode 100644
index 9c7abb5..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/expected_ouput_csv/union_with_default.txt
+++ /dev/null
@@ -1 +0,0 @@
-andrew|13|blue
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_invalid_scale_with_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_invalid_scale_with_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_invalid_scale_with_default.txt
deleted file mode 100644
index 54ba8b1..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_invalid_scale_with_default.txt
+++ /dev/null
@@ -1,16 +0,0 @@
-{
-	"name": "trx_table",
-	"type": "record",
-	"fields": [
-	 {
-		"name": "transactionid",
-		"type": ["string", "null"]
-	}, {
-		"name": "amount",
-		"type": "bytes",
-		"logicalType": "decimal",
-		"precision": 10,
-		"scale": 13,
-		"default": 0.0
-	}]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_valid_scale_with_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_valid_scale_with_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_valid_scale_with_default.txt
deleted file mode 100644
index 8385fb1..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_valid_scale_with_default.txt
+++ /dev/null
@@ -1,16 +0,0 @@
-{
-	"name": "trx_table",
-	"type": "record",
-	"fields": [
-	 {
-		"name": "transactionid",
-		"type": ["string", "null"]
-	}, {
-		"name": "amount",
-		"type": "bytes",
-		"logicalType": "decimal",
-		"precision": 10,
-		"scale": 3,
-		"default": 0.0
-	}]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_valid_scale_with_no_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_valid_scale_with_no_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_valid_scale_with_no_default.txt
deleted file mode 100644
index 9878590..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/decimal_logicalType_valid_scale_with_no_default.txt
+++ /dev/null
@@ -1,15 +0,0 @@
-{
-	"name": "trx_table",
-	"type": "record",
-	"fields": [
-	 {
-		"name": "transactionid",
-		"type": ["string", "null"]
-	}, {
-		"name": "amount",
-		"type": "bytes",
-		"logicalType": "decimal",
-		"precision": 10,
-		"scale": 3
-	}]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_no_defaults.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_no_defaults.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_no_defaults.txt
deleted file mode 100644
index 934a53c..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_no_defaults.txt
+++ /dev/null
@@ -1,11 +0,0 @@
-{
-    "type":"record",
-    "name":"basic_primitive_type_check",
-    "fields":[
-      {"name":"string","type":"string"},
-      {"name":"integer","type":"int"},
-      {"name":"long","type":"long"},
-      {"name":"double","type":"double"},
-      {"name":"boolean","type":"boolean"}
-    ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_union_with_defaults.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_union_with_defaults.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_union_with_defaults.txt
deleted file mode 100644
index abc80ca..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_union_with_defaults.txt
+++ /dev/null
@@ -1,11 +0,0 @@
-{
-    "type":"record",
-    "name":"basic_primitive_type_check",
-    "fields":[
-      {"name":"string","type":["null","string"],"default":null},
-      {"name":"integer","type":["null","int"],"default":null},
-      {"name":"long","type":["null","long"],"default":null},
-      {"name":"double","type":["null","double"],"default":null},
-      {"name":"boolean","type":["null","boolean"],"default":null}
-    ]
-  }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_with_matching_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_with_matching_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_with_matching_default.txt
deleted file mode 100644
index b3ea951..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_with_matching_default.txt
+++ /dev/null
@@ -1,11 +0,0 @@
-{
-    "type":"record",
-    "name":"basic_primitive_type_check",
-    "fields":[
-      {"name":"string","type":"string","default":"default_string"},
-      {"name":"integer","type":"int","default":1234},
-      {"name":"long","type":"long","default":21474836470},
-      {"name":"double","type":"double","default":1.7976931348623157E308},
-      {"name":"boolean","type":"boolean","default":true}
-    ]
-  }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_with_mismatch_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_with_mismatch_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_with_mismatch_default.txt
deleted file mode 100644
index e8f0e28..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/primitive_types_with_mismatch_default.txt
+++ /dev/null
@@ -1,11 +0,0 @@
-{
-    "type":"record",
-    "name":"basic_primitive_type_check",
-    "fields":[
-      {"name":"string","type":"string","default":1234},
-      {"name":"integer","type":"int","default":"mismatch_int"},
-      {"name":"long","type":"long","default":"mismatch_long"},
-      {"name":"double","type":"double","default":"mismatch_double"},
-      {"name":"boolean","type":"boolean","default":"mismatch_boolean"}
-    ]
-  }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/union_and_matching_defaults.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/union_and_matching_defaults.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/union_and_matching_defaults.txt
deleted file mode 100644
index 442a3a4..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/union_and_matching_defaults.txt
+++ /dev/null
@@ -1,18 +0,0 @@
-{
-	"namespace": "example.avro",
-	"type": "record",
-	"name": "User",
-	"fields": [{
-		"name": "name",
-		"type": "string",
-		"default": "default_name"
-	}, {
-		"name": "favorite_number",
-		"type": "int",
-		"default": 21474
-	}, {
-		"name": "favorite_color",
-		"type": ["null", "string"],
-		"default": null
-	}]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/union_and_mismatch_defaults.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/union_and_mismatch_defaults.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/union_and_mismatch_defaults.txt
deleted file mode 100644
index 5222074..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_avro/union_and_mismatch_defaults.txt
+++ /dev/null
@@ -1,18 +0,0 @@
-{
-	"namespace": "example.avro",
-	"type": "record",
-	"name": "User",
-	"fields": [{
-		"name": "name",
-		"type": "string",
-		"default": "default_name"
-	}, {
-		"name": "favorite_number",
-		"type": "int",
-		"default": "mismatched_int_default"
-	}, {
-		"name": "favorite_color",
-		"type": ["null", "string"],
-		"default": null
-	}]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/decimal_logicalType.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/decimal_logicalType.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/decimal_logicalType.txt
deleted file mode 100644
index 1a53f85..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/decimal_logicalType.txt
+++ /dev/null
@@ -1 +0,0 @@
-"fake_transactionid"|11234567.890
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/decimal_logicalType_missing_value.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/decimal_logicalType_missing_value.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/decimal_logicalType_missing_value.txt
deleted file mode 100644
index 1ee2a9b..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/decimal_logicalType_missing_value.txt
+++ /dev/null
@@ -1 +0,0 @@
-"fake_transactionid"|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/primitive_types.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/primitive_types.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/primitive_types.txt
deleted file mode 100644
index 77f353f..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/primitive_types.txt
+++ /dev/null
@@ -1 +0,0 @@
-"this is a simple string."|10|21474836470|1.7976931348623157E308|true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/primitive_types_with_matching_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/primitive_types_with_matching_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/primitive_types_with_matching_default.txt
deleted file mode 100644
index b60c01b..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/primitive_types_with_matching_default.txt
+++ /dev/null
@@ -1 +0,0 @@
-"default_string"||21474836470||true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_null_last_field_with_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_null_last_field_with_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_null_last_field_with_default.txt
deleted file mode 100644
index 83cbf75..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_null_last_field_with_default.txt
+++ /dev/null
@@ -1 +0,0 @@
-andrew|13|
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_null_middle_field_with_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_null_middle_field_with_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_null_middle_field_with_default.txt
deleted file mode 100644
index 5a706ac..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_null_middle_field_with_default.txt
+++ /dev/null
@@ -1 +0,0 @@
-andrew||blue
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_with_default.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_with_default.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_with_default.txt
deleted file mode 100644
index 9c7abb5..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_with_default.txt
+++ /dev/null
@@ -1 +0,0 @@
-andrew|13|blue
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_with_missing_value.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_with_missing_value.txt b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_with_missing_value.txt
deleted file mode 100644
index 5a706ac..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/test/resources/input_csv/union_with_missing_value.txt
+++ /dev/null
@@ -1 +0,0 @@
-andrew||blue
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/pom.xml b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/pom.xml
index d99dc64..3cfae30 100644
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/pom.xml
+++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/pom.xml
@@ -23,6 +23,18 @@
 	<packaging>jar</packaging>
 
 	<dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-schema-registry-service-api</artifactId>
+        </dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-record-serialization-service-api</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.avro</groupId>
+			<artifactId>avro</artifactId>
+		</dependency>
 		<dependency>
 			<groupId>org.apache.nifi</groupId>
 			<artifactId>nifi-api</artifactId>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
new file mode 100644
index 0000000..13b1d5d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java
@@ -0,0 +1,217 @@
+/*
+ * 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.schemaregistry.services;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.avro.LogicalType;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.Schema.Type;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.reporting.InitializationException;
+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;
+
+@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 final Map<String, String> schemaNameToSchemaMap;
+
+    private static final String LOGICAL_TYPE_DATE = "date";
+    private static final String LOGICAL_TYPE_TIME_MILLIS = "time-millis";
+    private static final String LOGICAL_TYPE_TIME_MICROS = "time-micros";
+    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(String schemaName) {
+        if (!this.schemaNameToSchemaMap.containsKey(schemaName)) {
+            throw new IllegalArgumentException("Failed to find schema; Name: '" + schemaName + ".");
+        } else {
+            return this.schemaNameToSchemaMap.get(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.");
+    }
+
+    @Override
+    @OnDisabled
+    public void close() throws Exception {
+        this.schemaNameToSchemaMap.clear();
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+            .name(propertyDescriptorName)
+            .required(false)
+            .addValidator(new AvroSchemaValidator())
+            .dynamic(true)
+            .expressionLanguageSupported(true)
+            .build();
+    }
+
+
+    @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
+     * @return the Corresponding Record Schema
+     */
+    private RecordSchema createRecordSchema(final Schema avroSchema) {
+        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));
+        }
+
+        final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
+        return recordSchema;
+    }
+
+    /**
+     * Returns a DataType for the given Avro Schema
+     *
+     * @param avroSchema the Avro Schema to convert
+     * @return a Data Type that corresponds to the given Avro Schema
+     */
+    private DataType determineDataType(final Schema avroSchema) {
+        final Type avroType = avroSchema.getType();
+
+        final LogicalType logicalType = avroSchema.getLogicalType();
+        if (logicalType != null) {
+            final String logicalTypeName = logicalType.getName();
+            switch (logicalTypeName) {
+                case LOGICAL_TYPE_DATE:
+                    return RecordFieldType.DATE.getDataType();
+                case LOGICAL_TYPE_TIME_MILLIS:
+                case LOGICAL_TYPE_TIME_MICROS:
+                    return RecordFieldType.TIME.getDataType();
+                case LOGICAL_TYPE_TIMESTAMP_MILLIS:
+                case LOGICAL_TYPE_TIMESTAMP_MICROS:
+                    return RecordFieldType.TIMESTAMP.getDataType();
+            }
+        }
+
+        switch (avroType) {
+            case ARRAY:
+                return RecordFieldType.ARRAY.getArrayDataType(determineDataType(avroSchema.getElementType()));
+            case BYTES:
+            case FIXED:
+                return RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType());
+            case BOOLEAN:
+                return RecordFieldType.BOOLEAN.getDataType();
+            case DOUBLE:
+                return RecordFieldType.DOUBLE.getDataType();
+            case ENUM:
+            case STRING:
+                return RecordFieldType.STRING.getDataType();
+            case FLOAT:
+                return RecordFieldType.FLOAT.getDataType();
+            case INT:
+                return RecordFieldType.INT.getDataType();
+            case LONG:
+                return RecordFieldType.LONG.getDataType();
+            case RECORD: {
+                final List<Field> avroFields = avroSchema.getFields();
+                final List<RecordField> recordFields = new ArrayList<>(avroFields.size());
+
+                for (final Field field : avroFields) {
+                    final String fieldName = field.name();
+                    final Schema fieldSchema = field.schema();
+                    final DataType fieldType = determineDataType(fieldSchema);
+                    recordFields.add(new RecordField(fieldName, fieldType));
+                }
+
+                final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
+                return RecordFieldType.RECORD.getRecordDataType(recordSchema);
+            }
+            case NULL:
+            case MAP:
+                return RecordFieldType.RECORD.getDataType();
+            case UNION: {
+                final List<Schema> nonNullSubSchemas = avroSchema.getTypes().stream()
+                    .filter(s -> s.getType() != Type.NULL)
+                    .collect(Collectors.toList());
+
+                if (nonNullSubSchemas.size() == 1) {
+                    return determineDataType(nonNullSubSchemas.get(0));
+                }
+
+                final List<DataType> possibleChildTypes = new ArrayList<>(nonNullSubSchemas.size());
+                for (final Schema subSchema : nonNullSubSchemas) {
+                    final DataType childDataType = determineDataType(subSchema);
+                    possibleChildTypes.add(childDataType);
+                }
+
+                return RecordFieldType.CHOICE.getChoiceDataType(possibleChildTypes);
+            }
+        }
+
+        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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaValidator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaValidator.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaValidator.java
new file mode 100644
index 0000000..32b700f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaValidator.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.schemaregistry.services;
+
+import org.apache.avro.Schema;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+
+public class AvroSchemaValidator 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 is present")
+                .build();
+        }
+
+        try {
+            new Schema.Parser().parse(input);
+
+            return new ValidationResult.Builder()
+                .input(input)
+                .subject(subject)
+                .valid(true)
+                .explanation("Schema is valid")
+                .build();
+        } catch (final Exception e) {
+            return new ValidationResult.Builder()
+                .input(input)
+                .subject(subject)
+                .valid(false)
+                .explanation("Not a valid Avro Schema: " + e.getMessage())
+                .build();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java
deleted file mode 100644
index fd5d0c5..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java
+++ /dev/null
@@ -1,46 +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.schemaregistry.services;
-
-import java.util.Properties;
-
-import org.apache.nifi.controller.ControllerService;
-
-/**
- * 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";
-
-
-    /**
-     * 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.
-     */
-    String retrieveSchemaText(String schemaName);
-
-    /**
-     * 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.
-     */
-    String retrieveSchemaText(String schemaName, Properties attributes);
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/SimpleKeyValueSchemaRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/SimpleKeyValueSchemaRegistry.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/SimpleKeyValueSchemaRegistry.java
deleted file mode 100644
index aaedea2..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/SimpleKeyValueSchemaRegistry.java
+++ /dev/null
@@ -1,96 +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.schemaregistry.services;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.stream.Collectors;
-
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.annotation.lifecycle.OnDisabled;
-import org.apache.nifi.annotation.lifecycle.OnEnabled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.AbstractControllerService;
-import org.apache.nifi.controller.ConfigurationContext;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.reporting.InitializationException;
-
-@Tags({ "schema", "registry", "avro", "json", "csv" })
-@CapabilityDescription("Provides a service for registering and accessing schemas. You can register schema "
-        + "as a dynamic property where 'name' represents the schema name and 'value' represents the textual "
-        + "representation of the actual schema.")
-public class SimpleKeyValueSchemaRegistry extends AbstractControllerService implements SchemaRegistry {
-
-    private static final List<PropertyDescriptor> propertyDescriptors;
-
-    static {
-        propertyDescriptors = Collections.emptyList();
-    }
-
-    private final Map<String, String> schemaNameToSchemaMap;
-
-    public SimpleKeyValueSchemaRegistry() {
-        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(String schemaName) {
-        if (!this.schemaNameToSchemaMap.containsKey(schemaName)) {
-            throw new IllegalArgumentException("Failed to find schema; Name: '" + schemaName + ".");
-        } else {
-            return this.schemaNameToSchemaMap.get(schemaName);
-        }
-    }
-
-    @Override
-    public String retrieveSchemaText(String schemaName, Properties attributes) {
-        throw new UnsupportedOperationException("This version of schema registry does not "
-                + "support this operation, since schemas are only identofied by name.");
-    }
-
-    @Override
-    @OnDisabled
-    public void close() throws Exception {
-        this.schemaNameToSchemaMap.clear();
-    }
-
-    @Override
-    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder().required(false).name(propertyDescriptorName)
-                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).dynamic(true).expressionLanguageSupported(true)
-                .build();
-    }
-
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return propertyDescriptors;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
index 1775b76..a000cd7 100644
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
+++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -12,4 +12,4 @@
 # 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.
-org.apache.nifi.schemaregistry.services.SimpleKeyValueSchemaRegistry
\ No newline at end of file
+org.apache.nifi.schemaregistry.services.AvroSchemaRegistry
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/SimpleKeyValueSchemaRegistryTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/SimpleKeyValueSchemaRegistryTest.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/SimpleKeyValueSchemaRegistryTest.java
deleted file mode 100644
index 29179ab..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/SimpleKeyValueSchemaRegistryTest.java
+++ /dev/null
@@ -1,70 +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.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;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.ConfigurationContext;
-import org.apache.nifi.schemaregistry.services.SchemaRegistry;
-import org.apache.nifi.schemaregistry.services.SimpleKeyValueSchemaRegistry;
-import org.junit.Test;
-
-public class SimpleKeyValueSchemaRegistryTest {
-
-    @Test
-    public void validateSchemaRegistrationFromrDynamicProperties() throws Exception {
-        String schemaName = "fooSchema";
-        ConfigurationContext configContext = mock(ConfigurationContext.class);
-        Map<PropertyDescriptor, String> properties = new HashMap<>();
-        PropertyDescriptor fooSchema = new PropertyDescriptor.Builder()
-                .name(schemaName)
-                .dynamic(true)
-                .build();
-        String fooSchemaText = "{\"namespace\": \"example.avro\", " + "\"type\": \"record\", " + "\"name\": \"User\", "
-                + "\"fields\": [ " + "{\"name\": \"name\", \"type\": [\"string\", \"null\"]}, "
-                + "{\"name\": \"favorite_number\",  \"type\": [\"int\", \"null\"]}, "
-                + "{\"name\": \"foo\",  \"type\": [\"int\", \"null\"]}, "
-                + "{\"name\": \"favorite_color\", \"type\": [\"string\", \"null\"]} " + "]" + "}";
-        PropertyDescriptor barSchema = new PropertyDescriptor.Builder()
-                .name("barSchema")
-                .dynamic(false)
-                .build();
-        properties.put(fooSchema, fooSchemaText);
-        properties.put(barSchema, "");
-        when(configContext.getProperties()).thenReturn(properties);
-        SchemaRegistry delegate = new SimpleKeyValueSchemaRegistry();
-        ((SimpleKeyValueSchemaRegistry)delegate).enable(configContext);
-
-        String locatedSchemaText = delegate.retrieveSchemaText(schemaName);
-        assertEquals(fooSchemaText, locatedSchemaText);
-        try {
-            locatedSchemaText = delegate.retrieveSchemaText("barSchema");
-            fail();
-        } catch (Exception e) {
-            // ignore
-        }
-        delegate.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
new file mode 100644
index 0000000..929aab9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/TestAvroSchemaRegistry.java
@@ -0,0 +1,111 @@
+/*
+ * 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.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;
+
+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.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.junit.Test;
+
+public class TestAvroSchemaRegistry {
+
+    @Test
+    public void validateSchemaRegistrationFromrDynamicProperties() throws Exception {
+        String schemaName = "fooSchema";
+        ConfigurationContext configContext = mock(ConfigurationContext.class);
+        Map<PropertyDescriptor, String> properties = new HashMap<>();
+        PropertyDescriptor fooSchema = new PropertyDescriptor.Builder()
+            .name(schemaName)
+            .dynamic(true)
+            .build();
+        String fooSchemaText = "{\"namespace\": \"example.avro\", " + "\"type\": \"record\", " + "\"name\": \"User\", "
+            + "\"fields\": [ " + "{\"name\": \"name\", \"type\": [\"string\", \"null\"]}, "
+            + "{\"name\": \"favorite_number\",  \"type\": [\"int\", \"null\"]}, "
+            + "{\"name\": \"foo\",  \"type\": [\"int\", \"null\"]}, "
+            + "{\"name\": \"favorite_color\", \"type\": [\"string\", \"null\"]} " + "]" + "}";
+        PropertyDescriptor barSchema = new PropertyDescriptor.Builder()
+            .name("barSchema")
+            .dynamic(false)
+            .build();
+        properties.put(fooSchema, fooSchemaText);
+        properties.put(barSchema, "");
+        when(configContext.getProperties()).thenReturn(properties);
+        SchemaRegistry delegate = new AvroSchemaRegistry();
+        ((AvroSchemaRegistry) delegate).enable(configContext);
+
+        String locatedSchemaText = delegate.retrieveSchemaText(schemaName);
+        assertEquals(fooSchemaText, locatedSchemaText);
+        try {
+            locatedSchemaText = delegate.retrieveSchemaText("barSchema");
+            fail();
+        } catch (Exception e) {
+            // ignore
+        }
+        delegate.close();
+    }
+
+
+    @Test
+    public void validateRecordSchemaRetrieval() throws Exception {
+        String schemaName = "fooSchema";
+        ConfigurationContext configContext = mock(ConfigurationContext.class);
+        Map<PropertyDescriptor, String> properties = new HashMap<>();
+        PropertyDescriptor fooSchema = new PropertyDescriptor.Builder()
+            .name(schemaName)
+            .dynamic(true)
+            .build();
+        String fooSchemaText = "{\"namespace\": \"example.avro\", " + "\"type\": \"record\", " + "\"name\": \"User\", "
+            + "\"fields\": [ " + "{\"name\": \"name\", \"type\": [\"string\", \"null\"]}, "
+            + "{\"name\": \"favorite_number\",  \"type\": \"int\"}, "
+            + "{\"name\": \"foo\",  \"type\": \"boolean\"}, "
+            + "{\"name\": \"favorite_color\", \"type\": [\"string\", \"null\"]} " + "]" + "}";
+        PropertyDescriptor barSchema = new PropertyDescriptor.Builder()
+            .name("barSchema")
+            .dynamic(false)
+            .build();
+        properties.put(fooSchema, fooSchemaText);
+        properties.put(barSchema, "");
+        when(configContext.getProperties()).thenReturn(properties);
+        SchemaRegistry delegate = new AvroSchemaRegistry();
+        ((AvroSchemaRegistry) delegate).enable(configContext);
+
+        RecordSchema locatedSchema = delegate.retrieveSchema(schemaName);
+        List<RecordField> recordFields = locatedSchema.getFields();
+        assertEquals(4, recordFields.size());
+        assertEquals(RecordFieldType.STRING.getDataType(), recordFields.get(0).getDataType());
+        assertEquals("name", recordFields.get(0).getFieldName());
+        assertEquals(RecordFieldType.INT.getDataType(), recordFields.get(1).getDataType());
+        assertEquals("favorite_number", recordFields.get(1).getFieldName());
+        assertEquals(RecordFieldType.BOOLEAN.getDataType(), recordFields.get(2).getDataType());
+        assertEquals("foo", recordFields.get(2).getFieldName());
+        assertEquals(RecordFieldType.STRING.getDataType(), recordFields.get(3).getDataType());
+        assertEquals("favorite_color", recordFields.get(3).getFieldName());
+        delegate.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/pom.xml b/nifi-nar-bundles/nifi-registry-bundle/pom.xml
index fa6d30f..5287a02 100644
--- a/nifi-nar-bundles/nifi-registry-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-registry-bundle/pom.xml
@@ -21,12 +21,7 @@
 	<packaging>pom</packaging>
 	<description>A bundle of processors that rely on external service to obtain schema.</description>
 
-	<properties>
-		<commons-lang3.version>3.0</commons-lang3.version>
-	</properties>
-
 	<modules>
-		<module>nifi-registry-processors</module>
 		<module>nifi-registry-service</module>
 		<module>nifi-registry-nar</module>
 	</modules>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index e390097..295ae96 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -290,8 +290,8 @@
         </dependency>
         <dependency>
             <groupId>org.apache.calcite</groupId>
-            <artifactId>calcite-example-csv</artifactId>
-            <version>1.11.0</version>
+            <artifactId>calcite-core</artifactId>
+            <version>1.12.0</version>
         </dependency>
     </dependencies>
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
index 833a5d6..83a3d4b 100644
--- 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
@@ -98,19 +98,22 @@ import org.apache.nifi.util.StopWatch;
         + "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")
+        .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")
+        .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")
+        .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)
@@ -119,7 +122,8 @@ public class QueryFlowFile extends AbstractProcessor {
         .required(true)
         .build();
     static final PropertyDescriptor CACHE_SCHEMA = new PropertyDescriptor.Builder()
-        .name("Cache Schema")
+        .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 "
@@ -391,7 +395,7 @@ public class QueryFlowFile extends AbstractProcessor {
 
         final Supplier<CalciteConnection> connectionSupplier = () -> {
             final Properties properties = new Properties();
-            properties.put(CalciteConnectionProperty.LEX.camelName(), Lex.JAVA.name());
+            properties.put(CalciteConnectionProperty.LEX.camelName(), Lex.MYSQL_ANSI.name());
 
             try {
                 final Connection connection = DriverManager.getConnection("jdbc:calcite:", properties);
@@ -491,6 +495,15 @@ public class QueryFlowFile extends AbstractProcessor {
     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 {

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
index 1a62d14..7daa002 100644
--- 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
@@ -99,10 +99,6 @@ public class FlowFileEnumerator<InternalType> implements Enumerator<Object> {
         }
 
         // Create a new Object array that contains only the desired fields.
-        if (row.length <= fields.length) {
-            return row;
-        }
-
         final Object[] filtered = new Object[fields.length];
         for (int i = 0; i < fields.length; i++) {
             final int indexToKeep = fields[i];
@@ -125,7 +121,7 @@ public class FlowFileEnumerator<InternalType> implements Enumerator<Object> {
         rawIn = session.read(flowFile);
 
         try {
-            recordParser = recordParserFactory.createRecordReader(rawIn, logger);
+            recordParser = recordParserFactory.createRecordReader(flowFile, rawIn, logger);
         } catch (final MalformedRecordException | IOException e) {
             throw new ProcessException("Failed to reset stream", e);
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
index a23dcfa..27f0c42 100644
--- 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
@@ -136,7 +136,7 @@ public class FlowFileTable<S, E> extends AbstractTable implements QueryableTable
 
         RecordSchema schema;
         try (final InputStream in = session.read(flowFile)) {
-            final RecordReader recordParser = recordParserFactory.createRecordReader(in, logger);
+            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);
@@ -189,7 +189,7 @@ public class FlowFileTable<S, E> extends AbstractTable implements QueryableTable
                 return typeFactory.createJavaType(String.class);
             case ARRAY:
                 return typeFactory.createJavaType(Object[].class);
-            case OBJECT:
+            case RECORD:
                 return typeFactory.createJavaType(Object.class);
         }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
index 1cc7923..0dffc0d 100644
--- 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
@@ -41,7 +41,8 @@
     	</p>
     	
     	<p>
-    		The SQL syntax that is supported by this Processor is ANSI SQL and is powered by Apache Calcite.
+			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


[12/19] nifi git commit: NIFI-1280 Create FilterCSVColumns Processor.

Posted by ma...@apache.org.
NIFI-1280 Create FilterCSVColumns Processor.

Signed-off-by: Matt Burgess <ma...@apache.org>


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

Branch: refs/heads/master
Commit: 4d5872a38500ed0541d7689107b34daedb5b7e34
Parents: 52cf9a7
Author: Toivo Adams <to...@gmail.com>
Authored: Sat May 7 12:29:15 2016 +0300
Committer: Matt Burgess <ma...@apache.org>
Committed: Tue Apr 11 19:29:04 2017 -0400

----------------------------------------------------------------------
 .../nifi-standard-processors/pom.xml            |   5 +
 .../calcite/adapter/csv/CsvEnumerator2.java     | 303 +++++++++++++++++++
 .../apache/calcite/adapter/csv/CsvSchema2.java  |  98 ++++++
 .../calcite/adapter/csv/CsvSchemaFactory2.java  |  53 ++++
 .../calcite/adapter/csv/CsvTableScan2.java      | 104 +++++++
 .../adapter/csv/CsvTranslatableTable2.java      | 121 ++++++++
 .../processors/standard/FilterCSVColumns.java   | 258 ++++++++++++++++
 .../standard/TestFilterCSVColumns.java          | 117 +++++++
 .../resources/TestFilterCSVColumns/Numeric.csv  |   5 +
 .../resources/TestFilterCSVColumns/US500.csv    |   1 +
 .../TestFilterCSVColumns/US500_typeless.csv     |   1 +
 11 files changed, 1066 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index dc77309..d410f43 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -283,6 +283,11 @@ language governing permissions and limitations under the License. -->
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>
+            <groupId>org.apache.calcite</groupId>
+            <artifactId>calcite-example-csv</artifactId>
+            <version>1.11.0</version>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator2.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator2.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator2.java
new file mode 100644
index 0000000..0f928ce
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator2.java
@@ -0,0 +1,303 @@
+/*
+ * 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.calcite.adapter.csv;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.Pair;
+import org.apache.commons.lang3.time.FastDateFormat;
+
+import au.com.bytecode.opencsv.CSVReader;
+
+
+/** Enumerator that reads from a CSV stream.
+ *
+ * @param <E> Row type
+ */
+class CsvEnumerator2<E> implements Enumerator<E> {
+  private final CSVReader reader;
+  private final String[] filterValues;
+  private final RowConverter<E> rowConverter;
+  private E current;
+
+  private static final FastDateFormat TIME_FORMAT_DATE;
+  private static final FastDateFormat TIME_FORMAT_TIME;
+  private static final FastDateFormat TIME_FORMAT_TIMESTAMP;
+
+  static {
+    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 CsvEnumerator2(CSVReader csvReader, List<CsvFieldType> fieldTypes) {
+    this(verifyNotNullReader(csvReader), fieldTypes, identityList(fieldTypes.size()));
+  }
+
+  public CsvEnumerator2(CSVReader csvReader, List<CsvFieldType> fieldTypes, int[] fields) {
+    //noinspection unchecked
+    this(csvReader, null, (RowConverter<E>) converter(fieldTypes, fields));
+  }
+
+  public CsvEnumerator2(CSVReader csvReader, String[] filterValues, RowConverter<E> rowConverter) {
+    this.rowConverter = rowConverter;
+    this.filterValues = filterValues;
+    this.reader = csvReader;
+  }
+
+  static public CSVReader verifyNotNullReader(CSVReader csvReader) {
+    if (csvReader==null)
+      throw new IllegalArgumentException("csvReader cannot be null");
+    return csvReader;
+  }
+
+  private static RowConverter<?> converter(List<CsvFieldType> fieldTypes,
+      int[] fields) {
+    if (fields.length == 1) {
+      final int field = fields[0];
+      return new SingleColumnRowConverter(fieldTypes.get(field), field);
+    } else {
+      return new ArrayRowConverter(fieldTypes, fields);
+    }
+  }
+
+  /** Deduces the names and types of a table's columns by reading the first line
+   * of a CSV stream. */
+  static public RelDataType deduceRowType(JavaTypeFactory typeFactory, String[] firstLine,
+      List<CsvFieldType> fieldTypes) {
+    final List<RelDataType> types = new ArrayList<>();
+    final List<String> names = new ArrayList<>();
+      for (String string : firstLine) {
+        final String name;
+        final CsvFieldType fieldType;
+        final int colon = string.indexOf(':');
+        if (colon >= 0) {
+          name = string.substring(0, colon);
+          String typeString = string.substring(colon + 1);
+          typeString = typeString.trim();
+          fieldType = CsvFieldType.of(typeString);
+          if (fieldType == null) {
+            System.out.println("WARNING: Found unknown type: "
+              + typeString + " in first line: "
+              + " for column: " + name
+              + ". Will assume the type of column is string");
+          }
+        } else {
+          name = string;
+          fieldType = null;
+        }
+        final RelDataType type;
+        if (fieldType == null) {
+          type = typeFactory.createJavaType(String.class);
+        } else {
+          type = fieldType.toType(typeFactory);
+        }
+        names.add(name);
+        types.add(type);
+        if (fieldTypes != null) {
+          fieldTypes.add(fieldType);
+        }
+      }
+
+    if (names.isEmpty()) {
+      names.add("line");
+      types.add(typeFactory.createJavaType(String.class));
+    }
+    return typeFactory.createStructType(Pair.zip(names, types));
+  }
+
+  public E current() {
+    return current;
+  }
+
+  public boolean moveNext() {
+    try {
+    outer:
+      for (;;) {
+        final String[] strings = reader.readNext();
+        if (strings == null) {
+          current = null;
+          reader.close();
+          return false;
+        }
+        if (filterValues != null) {
+          for (int i = 0; i < strings.length; i++) {
+            String filterValue = filterValues[i];
+            if (filterValue != null) {
+              if (!filterValue.equals(strings[i])) {
+                continue outer;
+              }
+            }
+          }
+        }
+        current = rowConverter.convertRow(strings);
+        return true;
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void reset() {
+    throw new UnsupportedOperationException();
+  }
+
+  public void close() {
+    try {
+      reader.close();
+    } catch (IOException e) {
+      throw new RuntimeException("Error closing CSV reader", e);
+    }
+  }
+
+  /** Returns an array of integers {0, ..., n - 1}. */
+  static int[] identityList(int n) {
+    int[] integers = new int[n];
+    for (int i = 0; i < n; i++) {
+      integers[i] = i;
+    }
+    return integers;
+  }
+
+  /** Row converter. */
+  abstract static class RowConverter<E> {
+    abstract E convertRow(String[] rows);
+
+    protected Object convert(CsvFieldType fieldType, String string) {
+      if (fieldType == null) {
+        return string;
+      }
+      switch (fieldType) {
+      case BOOLEAN:
+        if (string.length() == 0) {
+          return null;
+        }
+        return Boolean.parseBoolean(string);
+      case BYTE:
+        if (string.length() == 0) {
+          return null;
+        }
+        return Byte.parseByte(string);
+      case SHORT:
+        if (string.length() == 0) {
+          return null;
+        }
+        return Short.parseShort(string);
+      case INT:
+        if (string.length() == 0) {
+          return null;
+        }
+        return Integer.parseInt(string);
+      case LONG:
+        if (string.length() == 0) {
+          return null;
+        }
+        return Long.parseLong(string);
+      case FLOAT:
+        if (string.length() == 0) {
+          return null;
+        }
+        return Float.parseFloat(string);
+      case DOUBLE:
+        if (string.length() == 0) {
+          return null;
+        }
+        return Double.parseDouble(string);
+      case DATE:
+        if (string.length() == 0) {
+          return null;
+        }
+        try {
+          Date date = TIME_FORMAT_DATE.parse(string);
+          return new java.sql.Date(date.getTime());
+        } catch (ParseException e) {
+          return null;
+        }
+      case TIME:
+        if (string.length() == 0) {
+          return null;
+        }
+        try {
+          Date date = TIME_FORMAT_TIME.parse(string);
+          return new java.sql.Time(date.getTime());
+        } catch (ParseException e) {
+          return null;
+        }
+      case TIMESTAMP:
+        if (string.length() == 0) {
+          return null;
+        }
+        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;
+      }
+    }
+  }
+
+  /** Array row converter. */
+  static class ArrayRowConverter extends RowConverter<Object[]> {
+    private final CsvFieldType[] fieldTypes;
+    private final int[] fields;
+
+    ArrayRowConverter(List<CsvFieldType> fieldTypes, int[] fields) {
+      this.fieldTypes = fieldTypes.toArray(new CsvFieldType[fieldTypes.size()]);
+      this.fields = fields;
+    }
+
+    public Object[] convertRow(String[] strings) {
+      final Object[] objects = new Object[fields.length];
+      for (int i = 0; i < fields.length; i++) {
+        int field = fields[i];
+        objects[i] = convert(fieldTypes[field], strings[field]);
+      }
+      return objects;
+    }
+  }
+
+  /** Single column row converter. */
+  private static class SingleColumnRowConverter extends RowConverter {
+    private final CsvFieldType fieldType;
+    private final int fieldIndex;
+
+    private SingleColumnRowConverter(CsvFieldType fieldType, int fieldIndex) {
+      this.fieldType = fieldType;
+      this.fieldIndex = fieldIndex;
+    }
+
+    public Object convertRow(String[] strings) {
+      return convert(fieldType, strings[fieldIndex]);
+    }
+  }
+}
+
+// End CsvEnumerator2.java

http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchema2.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchema2.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchema2.java
new file mode 100644
index 0000000..f724f79
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchema2.java
@@ -0,0 +1,98 @@
+/*
+ * 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.calcite.adapter.csv;
+
+import java.io.Reader;
+import java.util.Map;
+
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
+
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * Schema mapped onto a directory of CSV files. Each table in the schema
+ * is a CSV file in that directory.
+ */
+public class CsvSchema2 extends AbstractSchema {
+  final private Map<String, Reader> inputs;
+  private final CsvTable.Flavor flavor;
+  private Map<String, Table> tableMap;
+
+  /**
+   * Creates a CSV schema.
+   *
+   * @param inputs     Inputs map
+   * @param flavor     Whether to instantiate flavor tables that undergo
+   *                   query optimization
+   */
+  public CsvSchema2(Map<String, Reader> inputs, CsvTable.Flavor flavor) {
+    super();
+    this.inputs = inputs;
+    this.flavor = flavor;
+  }
+
+  /** Looks for a suffix on a string and returns
+   * either the string with the suffix removed
+   * or the original string. */
+  private static String trim(String s, String suffix) {
+    String trimmed = trimOrNull(s, suffix);
+    return trimmed != null ? trimmed : s;
+  }
+
+  /** Looks for a suffix on a string and returns
+   * either the string with the suffix removed
+   * or null. */
+  private static String trimOrNull(String s, String suffix) {
+    return s.endsWith(suffix)
+        ? s.substring(0, s.length() - suffix.length())
+        : null;
+  }
+
+  @Override protected Map<String, Table> getTableMap() {
+
+    if (tableMap!=null)
+      return tableMap;
+
+    // Build a map from table name to table; each file becomes a table.
+    final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
+
+    for (Map.Entry<String, Reader> entry : inputs.entrySet()) {
+      final Table table = createTable(entry.getValue());
+      builder.put(entry.getKey(), table);
+    }
+
+    tableMap = builder.build();
+    return tableMap;
+  }
+
+  /** Creates different sub-type of table based on the "flavor" attribute. */
+  private Table createTable(Reader readerx) {
+    switch (flavor) {
+    case TRANSLATABLE:
+      return new CsvTranslatableTable2(readerx, null);
+//    case SCANNABLE:
+//      return new CsvScannableTable(file, null);
+//    case FILTERABLE:
+//      return new CsvFilterableTable(file, null);
+    default:
+      throw new AssertionError("Unknown flavor " + flavor);
+    }
+  }
+}
+
+// End CsvSchema2.java

http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory2.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory2.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory2.java
new file mode 100644
index 0000000..f8ec576
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory2.java
@@ -0,0 +1,53 @@
+/*
+ * 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.calcite.adapter.csv;
+
+import java.io.Reader;
+import java.util.Map;
+
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
+
+/**
+ * Factory that creates a {@link CsvSchema}.
+ *
+ * <p>Allows a custom schema to be included in a <code><i>model</i>.json</code>
+ * file.</p>
+ */
+@SuppressWarnings("UnusedDeclaration")
+public class CsvSchemaFactory2 implements SchemaFactory {
+  final private Map<String, Reader> inputs;
+  // public constructor, per factory contract
+  public CsvSchemaFactory2(Map<String, Reader> inputs) {
+      this.inputs = inputs;
+  }
+
+  public Schema create(SchemaPlus parentSchema, String name, Map<String, Object> operand) {
+    String flavorName = (String) operand.get("flavor");
+    CsvTable.Flavor flavor;
+    if (flavorName == null) {
+      flavor = CsvTable.Flavor.SCANNABLE;
+    } else {
+      flavor = CsvTable.Flavor.valueOf(flavorName.toUpperCase());
+    }
+
+    return new CsvSchema2(inputs, flavor);
+  }
+}
+
+// End CsvSchemaFactory2.java

http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan2.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan2.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan2.java
new file mode 100644
index 0000000..75f013c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan2.java
@@ -0,0 +1,104 @@
+/*
+ * 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.calcite.adapter.csv;
+
+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;
+
+import java.util.List;
+
+/**
+ * Relational expression representing a scan of a CSV stream.
+ *
+ * <p>Like any table scan, it serves as a leaf node of a query tree.</p>
+ */
+public class CsvTableScan2 extends TableScan implements EnumerableRel {
+  final CsvTranslatableTable2 csvTable;
+  final int[] fields;
+
+  protected CsvTableScan2(RelOptCluster cluster, RelOptTable table,
+      CsvTranslatableTable2 csvTable, int[] fields) {
+    super(cluster, cluster.traitSetOf(EnumerableConvention.INSTANCE), table);
+    this.csvTable = csvTable;
+    this.fields = fields;
+
+    assert csvTable != null;
+  }
+
+  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    assert inputs.isEmpty();
+    return new CsvTableScan2(getCluster(), table, csvTable, fields);
+  }
+
+  @Override public RelWriter explainTerms(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(CsvProjectTableScanRule.INSTANCE);
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(),
+            getRowType(),
+            pref.preferArray());
+
+    if (table instanceof JsonTable) {
+      return implementor.result(
+          physType,
+          Blocks.toBlock(
+              Expressions.call(table.getExpression(JsonTable.class),
+                  "enumerable")));
+    }
+    return implementor.result(
+        physType,
+        Blocks.toBlock(
+            Expressions.call(table.getExpression(CsvTranslatableTable2.class),
+                "project", Expressions.constant(fields))));
+  }
+}
+
+// End CsvTableScan.java

http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable2.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable2.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable2.java
new file mode 100644
index 0000000..bc28fdd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable2.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.csv;
+
+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.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.TranslatableTable;
+
+import au.com.bytecode.opencsv.CSVReader;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+
+/**
+ * Table based on a CSV stream.
+ */
+public class CsvTranslatableTable2 extends CsvTable
+    implements QueryableTable, TranslatableTable {
+
+  final private CSVReader csvReader;
+  private CsvEnumerator2<Object> csvEnumerator2;
+  final private String[] firstLine;
+
+  /** Creates a CsvTable.
+   */
+  CsvTranslatableTable2(Reader readerx, RelProtoDataType protoRowType) {
+    super(null, protoRowType);
+    this.csvReader = new CSVReader(readerx);
+    try {
+        this.firstLine = csvReader.readNext();
+    } catch (IOException e) {
+        throw new RuntimeException("csvReader.readNext() failed ", e);
+    }
+  }
+
+  public String toString() {
+    return "CsvTranslatableTable2";
+  }
+
+  /** 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>() {
+      public Enumerator<Object> enumerator() {
+        return csvEnumerator2;
+      }
+    };
+  }
+
+  public Expression getExpression(SchemaPlus schema, String tableName,
+      Class clazz) {
+    return Schemas.tableExpression(schema, getElementType(), tableName, clazz);
+  }
+
+  public Type getElementType() {
+    return Object[].class;
+  }
+
+  public <T> Queryable<T> asQueryable(QueryProvider queryProvider,
+      SchemaPlus schema, String tableName) {
+    throw new UnsupportedOperationException();
+  }
+
+  public RelNode toRel(
+      RelOptTable.ToRelContext context,
+      RelOptTable relOptTable) {
+    // Request all fields.
+    final int fieldCount = relOptTable.getRowType().getFieldCount();
+    final int[] fields = CsvEnumerator.identityList(fieldCount);
+    return new CsvTableScan2(context.getCluster(), relOptTable, this, fields);
+  }
+
+  @Override
+  public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+      RelDataType rowType = null;
+
+      if (fieldTypes == null) {
+          fieldTypes = new ArrayList<CsvFieldType>();
+          rowType =  CsvEnumerator2.deduceRowType((JavaTypeFactory) typeFactory, firstLine, fieldTypes);
+      } else {
+          rowType = CsvEnumerator2.deduceRowType((JavaTypeFactory) typeFactory, firstLine, null);
+      }
+
+      if (csvEnumerator2==null)
+          csvEnumerator2 = new CsvEnumerator2<Object>(csvReader, fieldTypes);
+
+          return rowType;
+      }
+}
+
+// End CsvTranslatableTable2.java

http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterCSVColumns.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterCSVColumns.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterCSVColumns.java
new file mode 100644
index 0000000..718f462
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterCSVColumns.java
@@ -0,0 +1,258 @@
+/*
+ * 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 static java.sql.Types.CHAR;
+import static java.sql.Types.LONGNVARCHAR;
+import static java.sql.Types.LONGVARCHAR;
+import static java.sql.Types.NCHAR;
+import static java.sql.Types.NVARCHAR;
+import static java.sql.Types.VARCHAR;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.calcite.adapter.csv.CsvSchemaFactory2;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.commons.lang3.StringEscapeUtils;
+import org.apache.commons.lang3.StringUtils;
+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.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ProcessorLog;
+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.StreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.stream.io.BufferedInputStream;
+import org.apache.nifi.util.StopWatch;
+
+import com.google.common.collect.ImmutableMap;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"xml", "xslt", "transform"})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@CapabilityDescription("Filter out specific columns from CSV data. Some other transformations are also supported."
+        + "Columns can be renamed, simple calculations performed, aggregations, etc."
+        + "SQL select statement is used to specify how CSV data should be transformed."
+        + "SQL statement follows standard SQL, some restrictions may apply."
+        + "Successfully transformed CSV data is routed to the 'success' relationship."
+        + "If transform fails, the original FlowFile is routed to the 'failure' relationship")
+public class FilterCSVColumns  extends AbstractProcessor {
+
+    public static final PropertyDescriptor SQL_SELECT = new PropertyDescriptor.Builder()
+            .name("SQL select statement")
+            .description("SQL select statement specifies how CSV data should be transformed. "
+                       + "Sql select should select from CSV.A table")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("The FlowFile with transformed content will be 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 CSV), it will be routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(SQL_SELECT);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        final FlowFile original = session.get();
+        if (original == null) {
+            return;
+        }
+
+        final ProcessorLog logger = getLogger();
+        final StopWatch stopWatch = new StopWatch(true);
+
+        try {
+            FlowFile transformed = session.write(original, new StreamCallback() {
+                @Override
+                public void process(final InputStream rawIn, final OutputStream out) throws IOException {
+                    try (final InputStream in = new BufferedInputStream(rawIn)) {
+
+                        String sql = context.getProperty(SQL_SELECT).getValue();
+                        final ResultSet resultSet = transform(rawIn, sql);
+                        convertToCSV(resultSet, out);
+
+                    } catch (final Exception e) {
+                        throw new IOException(e);
+                    }
+                }
+            });
+            session.transfer(transformed, REL_SUCCESS);
+            session.getProvenanceReporter().modifyContent(transformed, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
+            logger.info("Transformed {}", new Object[]{original});
+        } catch (ProcessException e) {
+            logger.error("Unable to transform {} due to {}", new Object[]{original, e});
+            session.transfer(original, REL_FAILURE);
+        }
+    }
+
+    static protected ResultSet transform(InputStream rawIn, String sql) throws SQLException {
+
+        Reader readerx = new InputStreamReader(rawIn);
+        HashMap<String, Reader> inputs = new HashMap<>();
+        inputs.put("A", readerx);
+
+        Statement statement = null;
+        final Properties properties = new Properties();
+//      properties.setProperty("caseSensitive", "true");
+        try (final Connection connection = DriverManager.getConnection("jdbc:calcite:", properties)) {
+            final CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class);
+
+            final SchemaPlus rootSchema = calciteConnection.getRootSchema();
+            final Schema schema =
+              new CsvSchemaFactory2(inputs)
+                  .create(rootSchema, "CSV", ImmutableMap.<String, Object>of("flavor", "TRANSLATABLE"));
+
+            calciteConnection.getRootSchema().add("CSV", schema);
+            rootSchema.add("default", schema);
+
+            statement = connection.createStatement();
+            final ResultSet resultSet = statement.executeQuery(sql);
+            return resultSet;
+        }
+    }
+
+    static protected void convertToCSV(ResultSet resultSet, OutputStream out) throws SQLException, IOException {
+
+        convertToCsvStream(resultSet, out);
+    }
+
+    public static long convertToCsvStream(final ResultSet rs, final OutputStream outStream) throws SQLException, IOException {
+        return convertToCsvStream(rs, outStream, null, null);
+    }
+
+    public static long convertToCsvStream(final ResultSet rs, final OutputStream outStream, String recordName, ResultSetRowCallback callback)
+            throws SQLException, IOException {
+
+        final ResultSetMetaData meta = rs.getMetaData();
+        final int nrOfColumns = meta.getColumnCount();
+        List<String> columnNames = new ArrayList<>(nrOfColumns);
+
+        for (int i = 1; i <= nrOfColumns; i++) {
+            String columnNameFromMeta = meta.getColumnName(i);
+            // Hive returns table.column for column name. Grab the column name as the string after the last period
+            int columnNameDelimiter = columnNameFromMeta.lastIndexOf(".");
+            columnNames.add(columnNameFromMeta.substring(columnNameDelimiter + 1));
+        }
+
+        // Write column names as header row
+        outStream.write(StringUtils.join(columnNames, ",").getBytes(StandardCharsets.UTF_8));
+        outStream.write("\n".getBytes(StandardCharsets.UTF_8));
+
+        // Iterate over the rows
+        long nrOfRows = 0;
+        while (rs.next()) {
+            if (callback != null) {
+                callback.processRow(rs);
+            }
+            List<String> rowValues = new ArrayList<>(nrOfColumns);
+            for (int i = 1; i <= nrOfColumns; i++) {
+                final int javaSqlType = meta.getColumnType(i);
+                final Object value = rs.getObject(i);
+
+                switch (javaSqlType) {
+                    case CHAR:
+                    case LONGNVARCHAR:
+                    case LONGVARCHAR:
+                    case NCHAR:
+                    case NVARCHAR:
+                    case VARCHAR:
+                        rowValues.add("\"" + StringEscapeUtils.escapeCsv(rs.getString(i)) + "\"");
+                        break;
+                    default:
+                        rowValues.add(value.toString());
+                }
+            }
+            // Write row values
+            outStream.write(StringUtils.join(rowValues, ",").getBytes(StandardCharsets.UTF_8));
+            outStream.write("\n".getBytes(StandardCharsets.UTF_8));
+            nrOfRows++;
+        }
+        return nrOfRows;
+    }
+
+    /**
+     * An interface for callback methods which allows processing of a row during the convertToXYZStream() processing.
+     * <b>IMPORTANT:</b> This method should only work on the row pointed at by the current ResultSet reference.
+     * Advancing the cursor (e.g.) can cause rows to be skipped during Avro transformation.
+     */
+    public interface ResultSetRowCallback {
+        void processRow(ResultSet resultSet) throws IOException;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFilterCSVColumns.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFilterCSVColumns.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFilterCSVColumns.java
new file mode 100644
index 0000000..421da98
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFilterCSVColumns.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.standard;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestFilterCSVColumns {
+
+    private static final Logger LOGGER;
+
+    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.nifi.processors.standard.FilterCSVColumns", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestFilterCSVColumns", "debug");
+        LOGGER = LoggerFactory.getLogger(TestFilterCSVColumns.class);
+    }
+
+    @Test
+    public void testTransformSimple() throws InitializationException, IOException, SQLException {
+        String sql = "select first_name, last_name, company_name, address, city from CSV.A where city='New York'";
+
+        Path inpath = Paths.get("src/test/resources/TestFilterCSVColumns/US500.csv");
+        InputStream in = new FileInputStream(inpath.toFile());
+
+        ResultSet resultSet = FilterCSVColumns.transform(in, sql);
+
+        int nrofColumns = resultSet.getMetaData().getColumnCount();
+
+        for (int i = 1; i <= nrofColumns; i++) {
+            System.out.print(resultSet.getMetaData().getColumnLabel(i) + "      ");
+        }
+        System.out.println();
+
+        while (resultSet.next()) {
+            for (int i = 1; i <= nrofColumns; i++) {
+                System.out.print(resultSet.getString(i)+ "  ");
+            }
+            System.out.println();
+        }
+    }
+
+    @Test
+    public void testTransformCalc() throws InitializationException, IOException, SQLException {
+        String sql = "select ID, AMOUNT1+AMOUNT2+AMOUNT3 as TOTAL from CSV.A where ID=100";
+
+        Path inpath = Paths.get("src/test/resources/TestFilterCSVColumns/Numeric.csv");
+        InputStream in = new FileInputStream(inpath.toFile());
+
+        ResultSet resultSet = FilterCSVColumns.transform(in, sql);
+
+        int nrofColumns = resultSet.getMetaData().getColumnCount();
+
+        for (int i = 1; i <= nrofColumns; i++) {
+            System.out.print(resultSet.getMetaData().getColumnLabel(i) + "      ");
+        }
+        System.out.println();
+
+        while (resultSet.next()) {
+            for (int i = 1; i <= nrofColumns; i++) {
+                System.out.print(resultSet.getString(i)+ "  ");
+            }
+            double total = resultSet.getDouble("TOTAL");
+            System.out.println();
+            assertEquals(90.75, total, 0.0001);
+        }
+    }
+
+    @Test
+    public void testSimpleTypeless() throws InitializationException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(FilterCSVColumns.class);
+        String sql = "select first_name, last_name, company_name, address, city from CSV.A where city='New York'";
+        runner.setProperty(FilterCSVColumns.SQL_SELECT, sql);
+
+        runner.enqueue(Paths.get("src/test/resources/TestFilterCSVColumns/US500_typeless.csv"));
+        runner.run();
+
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ExecuteProcess.REL_SUCCESS);
+        for (final MockFlowFile flowFile : flowFiles) {
+            System.out.println(flowFile);
+            System.out.println(new String(flowFile.toByteArray()));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/Numeric.csv
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/Numeric.csv b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/Numeric.csv
new file mode 100644
index 0000000..2d56bb7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/Numeric.csv
@@ -0,0 +1,5 @@
+ID:int,AMOUNT1: float,AMOUNT2:float,AMOUNT3:float
+008, 10.05, 15.45, 89.99
+100, 20.25, 25.25, 45.25
+105, 20.05, 25.05, 45.05
+200, 34.05, 25.05, 75.05
\ No newline at end of file


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

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java
deleted file mode 100644
index de207f4..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DataTypeUtils.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.nifi.serialization;
-
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.nifi.serialization.record.DataType;
-import org.apache.nifi.serialization.record.RecordField;
-import org.apache.nifi.serialization.record.RecordFieldType;
-import org.apache.nifi.serialization.record.RecordSchema;
-
-public class DataTypeUtils {
-
-    public static Double toDouble(final Object value, final Double defaultValue) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).doubleValue();
-        }
-
-        if (value instanceof String) {
-            return Double.parseDouble((String) value);
-        }
-
-        return defaultValue;
-    }
-
-    public static Float toFloat(final Object value, final Float defaultValue) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).floatValue();
-        }
-
-        if (value instanceof String) {
-            return Float.parseFloat((String) value);
-        }
-
-        return defaultValue;
-    }
-
-    public static Long toLong(final Object value, final Long defaultValue) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).longValue();
-        }
-
-        if (value instanceof String) {
-            return Long.parseLong((String) value);
-        }
-
-        return defaultValue;
-    }
-
-
-
-    public static Integer toInteger(final Object value, final Integer defaultValue) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).intValue();
-        }
-
-        if (value instanceof String) {
-            return Integer.parseInt((String) value);
-        }
-
-        return defaultValue;
-    }
-
-
-    /**
-     * Deduces the type of RecordFieldType that should be used for a value of the given type,
-     * or returns <code>null</code> if the value is null
-     *
-     * @param value the value whose type should be deduced
-     * @return the type of RecordFieldType that should be used for a value of the given type,
-     *         or <code>null</code> if the value is null
-     */
-    public static DataType inferDataType(final Object value) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof String) {
-            return RecordFieldType.STRING.getDataType();
-        }
-        if (value instanceof Long) {
-            return RecordFieldType.LONG.getDataType();
-        }
-        if (value instanceof Integer) {
-            return RecordFieldType.INT.getDataType();
-        }
-        if (value instanceof Double) {
-            return RecordFieldType.DOUBLE.getDataType();
-        }
-        if (value instanceof Float) {
-            return RecordFieldType.FLOAT.getDataType();
-        }
-        if (value instanceof Boolean) {
-            return RecordFieldType.BOOLEAN.getDataType();
-        }
-        if (value instanceof Byte) {
-            return RecordFieldType.BYTE.getDataType();
-        }
-        if (value instanceof Character) {
-            return RecordFieldType.CHAR.getDataType();
-        }
-        if (value instanceof Short) {
-            return RecordFieldType.SHORT.getDataType();
-        }
-        if (value instanceof Date) {
-            return RecordFieldType.DATE.getDataType();
-        }
-        if (value instanceof Object[] || value instanceof List) {
-            return RecordFieldType.ARRAY.getDataType();
-        }
-        if (value instanceof Map) {
-            @SuppressWarnings("unchecked")
-            final Map<String, Object> map = (Map<String, Object>) value;
-            final RecordSchema childSchema = determineSchema(map);
-            return RecordFieldType.RECORD.getDataType(childSchema);
-        }
-
-        return RecordFieldType.RECORD.getDataType();
-    }
-
-    public static RecordSchema determineSchema(final Map<String, Object> valueMap) {
-        final List<RecordField> fields = new ArrayList<>(valueMap.size());
-        for (final Map.Entry<String, Object> entry : valueMap.entrySet()) {
-            final DataType valueType = inferDataType(entry.getValue());
-            final String fieldName = entry.getKey();
-            final RecordField field = new RecordField(fieldName, valueType);
-            fields.add(field);
-        }
-        return new SimpleRecordSchema(fields);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java
new file mode 100644
index 0000000..5545090
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+
+public abstract class DateTimeTextRecordSetWriter extends AbstractControllerService {
+
+    private volatile String dateFormat;
+    private volatile String timeFormat;
+    private volatile String timestampFormat;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(DateTimeUtils.DATE_FORMAT, DateTimeUtils.TIME_FORMAT, DateTimeUtils.TIMESTAMP_FORMAT);
+    }
+
+    @OnEnabled
+    public void captureValues(final ConfigurationContext context) {
+        this.dateFormat = context.getProperty(DateTimeUtils.DATE_FORMAT).getValue();
+        this.timeFormat = context.getProperty(DateTimeUtils.TIME_FORMAT).getValue();
+        this.timestampFormat = context.getProperty(DateTimeUtils.TIMESTAMP_FORMAT).getValue();
+    }
+
+    protected String getDateFormat() {
+        return dateFormat;
+    }
+
+    protected String getTimeFormat() {
+        return timeFormat;
+    }
+
+    protected String getTimestampFormat() {
+        return timestampFormat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java
new file mode 100644
index 0000000..d5ab8c5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeUtils.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+public class DateTimeUtils {
+    public static final PropertyDescriptor DATE_FORMAT = new PropertyDescriptor.Builder()
+        .name("Date Format")
+        .description("Specifies the format to use when reading/writing Date fields")
+        .expressionLanguageSupported(false)
+        .defaultValue(RecordFieldType.DATE.getDefaultFormat())
+        .addValidator(new SimpleDateFormatValidator())
+        .required(true)
+        .build();
+
+    public static final PropertyDescriptor TIME_FORMAT = new PropertyDescriptor.Builder()
+        .name("Time Format")
+        .description("Specifies the format to use when reading/writing Time fields")
+        .expressionLanguageSupported(false)
+        .defaultValue(RecordFieldType.TIME.getDefaultFormat())
+        .addValidator(new SimpleDateFormatValidator())
+        .required(true)
+        .build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+        .name("Timestamp Format")
+        .description("Specifies the format to use when reading/writing Timestamp fields")
+        .expressionLanguageSupported(false)
+        .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+        .addValidator(new SimpleDateFormatValidator())
+        .required(true)
+        .build();
+}

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/UserTypeOverrideRowReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/UserTypeOverrideRowReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/UserTypeOverrideRowReader.java
deleted file mode 100644
index be0b8ad..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/UserTypeOverrideRowReader.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.nifi.serialization;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.nifi.annotation.lifecycle.OnEnabled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.AbstractControllerService;
-import org.apache.nifi.controller.ConfigurationContext;
-import org.apache.nifi.serialization.DataTypeValidator;
-import org.apache.nifi.serialization.record.DataType;
-import org.apache.nifi.serialization.record.RecordFieldType;
-
-public abstract class UserTypeOverrideRowReader extends AbstractControllerService {
-    private volatile Map<String, DataType> fieldTypeOverrides;
-
-    @Override
-    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder()
-            .name(propertyDescriptorName)
-            .dynamic(true)
-            .addValidator(new DataTypeValidator())
-            .build();
-    }
-
-    @OnEnabled
-    public void createFieldTypeOverrides(final ConfigurationContext context) {
-        final Map<String, DataType> overrides = new HashMap<>(context.getProperties().size());
-        for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
-            if (!entry.getKey().isDynamic()) {
-                continue;
-            }
-
-            final String fieldName = entry.getKey().getName();
-            final String dataTypeName = entry.getValue();
-            if (dataTypeName == null) {
-                continue;
-            }
-
-            final DataType dataType;
-            final String[] splits = dataTypeName.split("\\:");
-            if (splits.length == 2) {
-                final RecordFieldType fieldType = RecordFieldType.of(splits[0]);
-                final String format = splits[1];
-                dataType = fieldType.getDataType(format);
-            } else {
-                final RecordFieldType fieldType = RecordFieldType.of(dataTypeName);
-                dataType = fieldType.getDataType();
-            }
-
-            overrides.put(fieldName, dataType);
-        }
-
-        this.fieldTypeOverrides = Collections.unmodifiableMap(overrides);
-    }
-
-    protected Map<String, DataType> getFieldTypeOverrides() {
-        return fieldTypeOverrides;
-    }
-}

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

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

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

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

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

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


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

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
index 41469ba..8e1c7ed 100644
--- 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
@@ -17,31 +17,22 @@
 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.HashMap;
-import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 
 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.MalformedRecordException;
-import org.apache.nifi.serialization.RecordReader;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
-import org.apache.nifi.serialization.RowRecordReaderFactory;
-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;
 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.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
@@ -247,58 +238,14 @@ public class TestQueryFlowFile {
 
                     Assert.assertEquals(columnNames, colNames);
 
-                    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;
-                }
-            };
-        }
-
-    }
-
-    private static class MockRecordWriter extends AbstractControllerService implements RecordSetWriterFactory {
-        private final String header;
-
-        public MockRecordWriter(final String header) {
-            this.header = header;
-        }
-
-        @Override
-        public RecordSetWriter createWriter(final ComponentLog logger) {
-            return new RecordSetWriter() {
-                @Override
-                public WriteResult write(final RecordSet rs, final OutputStream out) throws IOException {
-                    out.write(header.getBytes());
-                    out.write("\n".getBytes());
-
-                    int recordCount = 0;
-                    final int numCols = rs.getSchema().getFieldCount();
-                    Record record = null;
+                    // 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) {
-                        recordCount++;
-                        int i = 0;
-                        for (final String fieldName : record.getSchema().getFieldNames()) {
-                            final String val = record.getAsString(fieldName);
-                            out.write("\"".getBytes());
-                            out.write(val.getBytes());
-                            out.write("\"".getBytes());
-
-                            if (i++ < numCols - 1) {
-                                out.write(",".getBytes());
-                            }
-                        }
-                        out.write("\n".getBytes());
+                        System.out.println(record);
                     }
 
-                    return WriteResult.of(recordCount, Collections.emptyMap());
+                    return WriteResult.of(0, Collections.emptyMap());
                 }
 
                 @Override
@@ -312,68 +259,7 @@ public class TestQueryFlowFile {
                 }
             };
         }
-    }
-
-    private static class MockRecordParser extends AbstractControllerService implements RowRecordReaderFactory {
-        private final List<Object[]> records = new ArrayList<>();
-        private final List<RecordField> fields = new ArrayList<>();
-        private final int failAfterN;
 
-        public MockRecordParser() {
-            this(-1);
-        }
-
-        public MockRecordParser(final int failAfterN) {
-            this.failAfterN = failAfterN;
-        }
-
-
-        public void addSchemaField(final String fieldName, final RecordFieldType type) {
-            fields.add(new RecordField(fieldName, type.getDataType()));
-        }
-
-        public void addRecord(Object... values) {
-            records.add(values);
-        }
-
-        @Override
-        public RecordReader createRecordReader(InputStream in, ComponentLog logger) throws IOException {
-            final Iterator<Object[]> itr = records.iterator();
-
-            return new RecordReader() {
-                private int recordCount = 0;
-
-                @Override
-                public void close() throws IOException {
-                }
-
-                @Override
-                public Record nextRecord() throws IOException, MalformedRecordException {
-                    if (failAfterN >= recordCount) {
-                        throw new MalformedRecordException("Intentional Unit Test Exception because " + recordCount + " records have been read");
-                    }
-                    recordCount++;
-
-                    if (!itr.hasNext()) {
-                        return null;
-                    }
-
-                    final Object[] values = itr.next();
-                    final Map<String, Object> valueMap = new HashMap<>();
-                    int i = 0;
-                    for (final RecordField field : fields) {
-                        final String fieldName = field.getFieldName();
-                        valueMap.put(fieldName, values[i++]);
-                    }
-
-                    return new MapRecord(new SimpleRecordSchema(fields), valueMap);
-                }
-
-                @Override
-                public RecordSchema getSchema() {
-                    return new SimpleRecordSchema(fields);
-                }
-            };
-        }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
new file mode 100644
index 0000000..1a39b82
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java
@@ -0,0 +1,107 @@
+/*
+ * 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.util.record;
+
+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.controller.AbstractControllerService;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RowRecordReaderFactory;
+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;
+
+public class MockRecordParser extends AbstractControllerService implements RowRecordReaderFactory {
+    private final List<Object[]> records = new ArrayList<>();
+    private final List<RecordField> fields = new ArrayList<>();
+    private final int failAfterN;
+
+    public MockRecordParser() {
+        this(-1);
+    }
+
+    public MockRecordParser(final int failAfterN) {
+        this.failAfterN = failAfterN;
+    }
+
+
+    public void addSchemaField(final String fieldName, final RecordFieldType type) {
+        fields.add(new RecordField(fieldName, type.getDataType()));
+    }
+
+    public void addRecord(Object... values) {
+        records.add(values);
+    }
+
+    @Override
+    public RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws IOException {
+        final Iterator<Object[]> itr = records.iterator();
+
+        return new RecordReader() {
+            private int recordCount = 0;
+
+            @Override
+            public void close() throws IOException {
+            }
+
+            @Override
+            public Record nextRecord() throws IOException, MalformedRecordException {
+                if (failAfterN >= recordCount) {
+                    throw new MalformedRecordException("Intentional Unit Test Exception because " + recordCount + " records have been read");
+                }
+                recordCount++;
+
+                if (!itr.hasNext()) {
+                    return null;
+                }
+
+                final Object[] values = itr.next();
+                final Map<String, Object> valueMap = new HashMap<>();
+                int i = 0;
+                for (final RecordField field : fields) {
+                    final String fieldName = field.getFieldName();
+                    valueMap.put(fieldName, values[i++]);
+                }
+
+                return new MapRecord(new SimpleRecordSchema(fields), valueMap);
+            }
+
+            @Override
+            public RecordSchema getSchema() {
+                return new SimpleRecordSchema(fields);
+            }
+        };
+    }
+
+    @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/68c592ea/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
new file mode 100644
index 0000000..1cf2a28
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java
@@ -0,0 +1,80 @@
+/*
+ * 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.util.record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.logging.ComponentLog;
+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.RecordSet;
+
+public class MockRecordWriter extends AbstractControllerService implements RecordSetWriterFactory {
+    private final String header;
+
+    public MockRecordWriter(final String header) {
+        this.header = header;
+    }
+
+    @Override
+    public RecordSetWriter createWriter(final ComponentLog logger) {
+        return new RecordSetWriter() {
+            @Override
+            public WriteResult write(final RecordSet rs, final OutputStream out) throws IOException {
+                out.write(header.getBytes());
+                out.write("\n".getBytes());
+
+                int recordCount = 0;
+                final int numCols = rs.getSchema().getFieldCount();
+                Record record = null;
+                while ((record = rs.next()) != null) {
+                    recordCount++;
+                    int i = 0;
+                    for (final String fieldName : record.getSchema().getFieldNames()) {
+                        final String val = record.getAsString(fieldName);
+                        out.write("\"".getBytes());
+                        out.write(val.getBytes());
+                        out.write("\"".getBytes());
+
+                        if (i++ < numCols - 1) {
+                            out.write(",".getBytes());
+                        }
+                    }
+                    out.write("\n".getBytes());
+                }
+
+                return WriteResult.of(recordCount, Collections.emptyMap());
+            }
+
+            @Override
+            public String getMimeType() {
+                return "text/plain";
+            }
+
+            @Override
+            public WriteResult write(Record record, OutputStream out) throws IOException {
+                return null;
+            }
+        };
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/pom.xml
index d7d5605..78c0381 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-standard-services</artifactId>
-        <version>1.1.0-SNAPSHOT</version>
+        <version>1.2.0-SNAPSHOT</version>
     </parent>
     
     <artifactId>nifi-record-serialization-service-api</artifactId>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReader.java
index a0cfc79..b728498 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReader.java
@@ -39,7 +39,6 @@ public interface RecordReader extends Closeable {
     /**
      * Returns the next record in the stream or <code>null</code> if no more records are available.
      *
-     * @param schema the schema to use in order to determine how to interprets the fields in a record
      * @return the next record in the stream or <code>null</code> if no more records are available.
      *
      * @throws IOException if unable to read from the underlying data

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordWriter.java
index eef8d82..aa298d9 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordWriter.java
@@ -26,7 +26,7 @@ public interface RecordWriter {
     /**
      * Writes the given result set to the given output stream
      *
-     * @param recordSet the record set to serialize
+     * @param record the record set to serialize
      * @param out the OutputStream to write to
      * @return the results of writing the data
      * @throws IOException if unable to write to the given OutputStream

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
index 5ef4c7c..fbd8a21 100644
--- 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
@@ -21,7 +21,9 @@ 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>
@@ -29,5 +31,8 @@ import org.apache.nifi.logging.ComponentLog;
  * </p>
  */
 public interface RowRecordReaderFactory extends ControllerService {
-    RecordReader createRecordReader(InputStream in, ComponentLog logger) throws MalformedRecordException, IOException;
+
+    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/68c592ea/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 0c187f1..b72c107 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,36 +17,15 @@
 
 package org.apache.nifi.serialization.record;
 
-import java.util.Collections;
-import java.util.List;
-import java.util.Optional;
-
 public class DataType {
     private final RecordFieldType fieldType;
     private final String format;
 
-    private final RecordSchema childSchema;
-    private final List<DataType> childTypes;
-
-    DataType(final RecordFieldType fieldType, final String format) {
-        this(fieldType, format, (RecordSchema) null);
-    }
-
-    DataType(final RecordFieldType fieldType, final String format, final RecordSchema childSchema) {
+    protected DataType(final RecordFieldType fieldType, final String format) {
         this.fieldType = fieldType;
         this.format = format;
-        this.childSchema = childSchema;
-        this.childTypes = Collections.emptyList();
     }
 
-    DataType(final RecordFieldType fieldType, final String format, final List<DataType> childTypes) {
-        this.fieldType = fieldType;
-        this.format = format;
-        this.childSchema = null;
-        this.childTypes = Collections.unmodifiableList(childTypes);
-    }
-
-
     public String getFormat() {
         return format;
     }
@@ -55,14 +34,6 @@ public class DataType {
         return fieldType;
     }
 
-    public Optional<RecordSchema> getChildRecordSchema() {
-        return Optional.ofNullable(childSchema);
-    }
-
-    public List<DataType> getPossibleTypes() {
-        return childTypes;
-    }
-
     @Override
     public int hashCode() {
         return 31 + 41 * fieldType.hashCode() + 41 * (format == null ? 0 : format.hashCode());

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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 f3f9024..0bbb534 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
@@ -17,16 +17,13 @@
 
 package org.apache.nifi.serialization.record;
 
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
 import java.util.Date;
-import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
 public class MapRecord implements Record {
     private final RecordSchema schema;
     private final Map<String, Object> values;
@@ -80,220 +77,52 @@ public class MapRecord implements Record {
             return null;
         }
 
-        if (value instanceof java.sql.Date) {
-            java.sql.Date date = (java.sql.Date) value;
-            final long time = date.getTime();
-            return new SimpleDateFormat(getFormat(format, RecordFieldType.DATE)).format(new java.util.Date(time));
-        }
-        if (value instanceof java.util.Date) {
-            return new SimpleDateFormat(getFormat(format, RecordFieldType.DATE)).format((java.util.Date) value);
-        }
-        if (value instanceof Timestamp) {
-            java.sql.Timestamp date = (java.sql.Timestamp) value;
-            final long time = date.getTime();
-            return new SimpleDateFormat(getFormat(format, RecordFieldType.TIMESTAMP)).format(new java.util.Date(time));
-        }
-        if (value instanceof Time) {
-            java.sql.Time date = (java.sql.Time) value;
-            final long time = date.getTime();
-            return new SimpleDateFormat(getFormat(format, RecordFieldType.TIME)).format(new java.util.Date(time));
-        }
-
-        return value.toString();
+        final String dateFormat = getFormat(format, RecordFieldType.DATE);
+        final String timestampFormat = getFormat(format, RecordFieldType.TIMESTAMP);
+        final String timeFormat = getFormat(format, RecordFieldType.TIME);
+        return DataTypeUtils.toString(value, dateFormat, timeFormat, timestampFormat);
     }
 
     @Override
     public Long getAsLong(final String fieldName) {
-        return convertToLong(getValue(fieldName), fieldName);
-    }
-
-    private Long convertToLong(final Object value, final Object fieldDesc) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).longValue();
-        }
-        if (value instanceof String) {
-            return Long.parseLong((String) value);
-        }
-        if (value instanceof Date) {
-            return ((Date) value).getTime();
-        }
-
-        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Long for field " + fieldDesc);
+        return DataTypeUtils.toLong(getValue(fieldName));
     }
 
     @Override
     public Integer getAsInt(final String fieldName) {
-        return convertToInt(getValue(fieldName), fieldName);
-    }
-
-    private Integer convertToInt(final Object value, final Object fieldDesc) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).intValue();
-        }
-        if (value instanceof String) {
-            return Integer.parseInt((String) value);
-        }
-
-        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Integer for field " + fieldDesc);
+        return DataTypeUtils.toInteger(getValue(fieldName));
     }
 
-
     @Override
     public Double getAsDouble(final String fieldName) {
-        return convertToDouble(getValue(fieldName), fieldName);
-    }
-
-    private Double convertToDouble(final Object value, final Object fieldDesc) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).doubleValue();
-        }
-        if (value instanceof String) {
-            return Double.parseDouble((String) value);
-        }
-
-        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Double for field " + fieldDesc);
+        return DataTypeUtils.toDouble(getValue(fieldName));
     }
 
     @Override
     public Float getAsFloat(final String fieldName) {
-        return convertToFloat(getValue(fieldName), fieldName);
-    }
-
-    private Float convertToFloat(final Object value, final Object fieldDesc) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Number) {
-            return ((Number) value).floatValue();
-        }
-        if (value instanceof String) {
-            return Float.parseFloat((String) value);
-        }
-
-        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Float for field " + fieldDesc);
+        return DataTypeUtils.toFloat(getValue(fieldName));
     }
 
     @Override
-    public Record getAsRecord(String fieldName) {
-        return convertToRecord(getValue(fieldName), fieldName);
-    }
-
-    private Record convertToRecord(final Object value, final Object fieldDesc) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Record) {
-            return (Record) value;
-        }
-
-        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Record for field " + fieldDesc);
+    public Record getAsRecord(String fieldName, final RecordSchema schema) {
+        return DataTypeUtils.toRecord(getValue(fieldName), schema);
     }
 
-
     @Override
     public Boolean getAsBoolean(final String fieldName) {
-        return convertToBoolean(getValue(fieldName), fieldName);
-    }
-
-    private Boolean convertToBoolean(final Object value, final Object fieldDesc) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Boolean) {
-            return (Boolean) value;
-        }
-        if (value instanceof String) {
-            final String string = (String) value;
-            if (string.equalsIgnoreCase("true") || string.equalsIgnoreCase("t")) {
-                return Boolean.TRUE;
-            }
-
-            if (string.equalsIgnoreCase("false") || string.equals("f")) {
-                return Boolean.FALSE;
-            }
-
-            throw new TypeMismatchException("Cannot convert String value to Boolean for field " + fieldDesc + " because it is not a valid boolean value");
-        }
-
-        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Boolean for field " + fieldDesc);
-    }
-
-    @Override
-    public Date getAsDate(final String fieldName) {
-        final Optional<DataType> dataTypeOption = schema.getDataType(fieldName);
-        if (!dataTypeOption.isPresent()) {
-            return null;
-        }
-
-        return convertToDate(getValue(fieldName), fieldName, dataTypeOption.get().getFormat());
+        return DataTypeUtils.toBoolean(getValue(fieldName));
     }
 
     @Override
     public Date getAsDate(final String fieldName, final String format) {
-        return convertToDate(getValue(fieldName), fieldName, format);
-    }
-
-    private Date convertToDate(final Object value, final Object fieldDesc, final String format) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Date) {
-            return (Date) value;
-        }
-        if (value instanceof Number) {
-            final Long time = ((Number) value).longValue();
-            return new Date(time);
-        }
-        if (value instanceof java.sql.Date) {
-            return new Date(((java.sql.Date) value).getTime());
-        }
-        if (value instanceof String) {
-            try {
-                return new SimpleDateFormat(getFormat(format, RecordFieldType.DATE)).parse((String) value);
-            } catch (final ParseException e) {
-                throw new TypeMismatchException("Cannot convert String value to date for field " + fieldDesc + " because it is not in the correct format of: " + format, e);
-            }
-        }
-
-        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Boolean for field " + fieldDesc);
+        return DataTypeUtils.toDate(getValue(fieldName), format);
     }
 
     @Override
     public Object[] getAsArray(final String fieldName) {
-        return convertToArray(getValue(fieldName));
+        return DataTypeUtils.toArray(getValue(fieldName));
     }
 
-    private Object[] convertToArray(final Object value) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Object[]) {
-            return (Object[]) value;
-        }
-
-        if (value instanceof List) {
-            return ((List<?>) value).toArray();
-        }
-
-        return new Object[] {value};
-    }
 
     @Override
     public int hashCode() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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 ca85741..e1d52e9 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
@@ -50,12 +50,10 @@ public interface Record {
 
     Float getAsFloat(String fieldName);
 
-    Record getAsRecord(String fieldName);
+    Record getAsRecord(String fieldName, RecordSchema schema);
 
     Boolean getAsBoolean(String fieldName);
 
-    Date getAsDate(String fieldName);
-
     Date getAsDate(String fieldName, String format);
 
     Object[] getAsArray(String fieldName);

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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 8ad212b..cc83a41 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
@@ -17,35 +17,171 @@
 
 package org.apache.nifi.serialization.record;
 
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 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.RecordDataType;
+
 public enum RecordFieldType {
+    /**
+     * A String field type. Fields of this type use a {@code java.lang.String} value.
+     */
     STRING("string"),
+
+    /**
+     * A boolean field type. Fields of this type use a {@code boolean} value.
+     */
     BOOLEAN("boolean"),
+
+    /**
+     * A byte field type. Fields of this type use a {@code byte} value.
+     */
     BYTE("byte"),
+
+    /**
+     * A char field type. Fields of this type use a {@code char} value.
+     */
     CHAR("char"),
+
+    /**
+     * A short field type. Fields of this type use a {@code short} value.
+     */
     SHORT("short"),
+
+    /**
+     * An int field type. Fields of this type use an {@code int} value.
+     */
     INT("int"),
+
+    /**
+     * A bigint field type. Fields of this type use a {@code java.math.BigInteger} value.
+     */
     BIGINT("bigint"),
+
+    /**
+     * A long field type. Fields of this type use a {@code long} value.
+     */
     LONG("long"),
+
+    /**
+     * A float field type. Fields of this type use a {@code float} value.
+     */
     FLOAT("float"),
+
+    /**
+     * A double field type. Fields of this type use a {@code double} value.
+     */
     DOUBLE("double"),
+
+    /**
+     * A date field type. Fields of this type use a {@code java.sql.Date} value.
+     */
     DATE("date", "yyyy-MM-dd"),
+
+    /**
+     * A time field type. Fields of this type use a {@code java.sql.Time} value.
+     */
     TIME("time", "HH:mm:ss"),
+
+    /**
+     * A timestamp field type. Fields of this type use a {@code java.sql.Timestamp} value.
+     */
     TIMESTAMP("timestamp", "yyyy-MM-dd HH:mm:ss"),
-    RECORD("record"),
-    CHOICE("choice"),
-    ARRAY("array");
+
+    /**
+     * <p>
+     * A record field type. Fields of this type use a {@code org.apache.nifi.serialization.record.Record} value. A Record DataType should be
+     * created by providing the {@link RecordSchema} for the record:
+     * </p>
+     *
+     * <code>
+     * final DataType recordType = RecordFieldType.RECORD.getRecordDataType(recordSchema);
+     * </code>
+     *
+     * <p>
+     * A field of type RECORD should always have a {@link RecordDataType}, so the following idiom is acceptable for use:
+     * </p>
+     *
+     * <code>
+     * <pre>
+     * final DataType dataType = ...;
+     * if (dataType.getFieldType() == RecordFieldType.RECORD) {
+     *     final RecordDataType recordDataType = (RecordDataType) dataType;
+     *     final RecordSchema childSchema = recordDataType.getChildSchema();
+     *     ...
+     * }
+     * </pre>
+     * </code>
+     */
+    RECORD("record", null, new RecordDataType(null)),
+
+    /**
+     * <p>
+     * A choice field type. A field of type choice can be one of any number of different types, which are defined by the DataType that is used.
+     * For example, if a field should allow either a Long or an Integer, this can be accomplished by using:
+     * </p>
+     *
+     * <code>
+     * final DataType choiceType = RecordFieldType.CHOICE.getChoiceDataType( RecordFieldType.INT.getDataType(), RecordFieldType.LONG.getDataType() );
+     * </code>
+     *
+     * <p>
+     * A field of type CHOICE should always have a {@link ChoiceDataType}, so the following idiom is acceptable for use:
+     * </p>
+     *
+     * <code>
+     * <pre>
+     * final DataType dataType = ...;
+     * if (dataType.getFieldType() == RecordFieldType.CHOICE) {
+     *     final ChoiceDataType choiceDataType = (ChoiceDataType) dataType;
+     *     final List&lt;DataType&gt; allowableTypes = choiceDataType.getPossibleSubTypes();
+     *     ...
+     * }
+     * </pre>
+     * </code>
+     */
+    CHOICE("choice", null, new ChoiceDataType(Collections.emptyList())),
+
+    /**
+     * <p>
+     * An array field type. Records should be updated using an {@code Object[]} value for this field. 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>
+     *
+     * <code>
+     * final DataType arrayType = RecordFieldType.ARRAY.getArrayDataType( RecordFieldType.INT.getDataType() );
+     * </code>
+     *
+     * <p>
+     * A field of type ARRAY should always have an {@link ArrayDataType}, so the following idiom is acceptable for use:
+     * </p>
+     *
+     * <code>
+     * <pre>
+     * final DataType dataType = ...;
+     * if (dataType.getFieldType() == RecordFieldType.ARRAY) {
+     *     final ArrayDataType arrayDataType = (ArrayDataType) dataType;
+     *     final DataType elementType = arrayDataType.getElementType();
+     *     ...
+     * }
+     * </pre>
+     * </code>
+     */
+    ARRAY("array", null, new ArrayDataType(null));
 
 
     private static final Map<String, RecordFieldType> SIMPLE_NAME_MAP = new HashMap<String, RecordFieldType>();
 
     static {
-      for (RecordFieldType value : values()) {
-        SIMPLE_NAME_MAP.put(value.simpleName, value);
-      }
+        for (RecordFieldType value : values()) {
+            SIMPLE_NAME_MAP.put(value.simpleName, value);
+        }
     }
 
     private final String simpleName;
@@ -62,6 +198,12 @@ public enum RecordFieldType {
         this.defaultDataType = new DataType(this, defaultFormat);
     }
 
+    private RecordFieldType(final String simpleName, final String defaultFormat, final DataType defaultDataType) {
+        this.simpleName = simpleName;
+        this.defaultFormat = defaultFormat;
+        this.defaultDataType = defaultDataType;
+    }
+
     public String getDefaultFormat() {
         return defaultFormat;
     }
@@ -78,18 +220,50 @@ public enum RecordFieldType {
     }
 
     /**
-     * Returns a Data Type that represents a "RECORD" type with the given schema.
+     * Returns a Data Type that represents a "RECORD" or "ARRAY" type with the given schema.
      *
-     * @param childSchema the Schema for the Record
-     * @return a DataType that represents a Record with the given schema, or <code>null</code> if this RecordFieldType
-     *         is not the RECORD type.
+     * @param childSchema the Schema for the Record or Array
+     * @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.
      */
-    public DataType getDataType(final RecordSchema childSchema) {
+    public DataType getRecordDataType(final RecordSchema childSchema) {
         if (this != RECORD) {
             return null;
         }
 
-        return new DataType(this, getDefaultFormat(), childSchema);
+        return new RecordDataType(childSchema);
+    }
+
+    /**
+     * Returns a Data Type that represents a "RECORD" or "ARRAY" type with the given schema.
+     *
+     * @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.
+     */
+    public DataType getArrayDataType(final DataType elementType) {
+        if (this != ARRAY) {
+            return null;
+        }
+
+        return new ArrayDataType(elementType);
+    }
+
+
+    /**
+     * Returns a Data Type that represents a "CHOICE" of multiple possible types. This method is
+     * only applicable for a RecordFieldType of {@link #CHOICE}.
+     *
+     * @param possibleChildTypes the possible types that are allowable
+     * @return a DataType that represents a "CHOICE" of multiple possible types, or <code>null</code> if this RecordFieldType
+     *         is not the CHOICE type.
+     */
+    public DataType getChoiceDataType(final List<DataType> possibleChildTypes) {
+        if (this != CHOICE) {
+            return null;
+        }
+
+        return new ChoiceDataType(possibleChildTypes);
     }
 
     /**
@@ -100,14 +274,20 @@ public enum RecordFieldType {
      * @return a DataType that represents a "CHOICE" of multiple possible types, or <code>null</code> if this RecordFieldType
      *         is not the CHOICE type.
      */
-    public DataType getDataType(final List<DataType> possibleChildTypes) {
+    public DataType getChoiceDataType(final DataType... possibleChildTypes) {
         if (this != CHOICE) {
             return null;
         }
 
-        return new DataType(this, getDefaultFormat(), possibleChildTypes);
+        final List<DataType> list = new ArrayList<>(possibleChildTypes.length);
+        for (final DataType type : possibleChildTypes) {
+            list.add(type);
+        }
+
+        return new ChoiceDataType(list);
     }
 
+
     public static RecordFieldType of(final String typeString) {
       return SIMPLE_NAME_MAP.get(typeString);
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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 e166918..be064ab 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
@@ -19,6 +19,8 @@ package org.apache.nifi.serialization.record;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.math.BigInteger;
+import java.sql.Array;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
@@ -39,9 +41,11 @@ public class ResultSetRecordSet implements RecordSet, Closeable {
     private final ResultSet rs;
     private final RecordSchema schema;
     private final Set<String> rsColumnNames;
+    private boolean moreRows;
 
     public ResultSetRecordSet(final ResultSet rs) throws SQLException {
         this.rs = rs;
+        moreRows = rs.next();
         this.schema = createSchema(rs);
 
         rsColumnNames = new HashSet<>();
@@ -59,14 +63,16 @@ public class ResultSetRecordSet implements RecordSet, Closeable {
     @Override
     public Record next() throws IOException {
         try {
-            if (rs.next()) {
-                return createRecord(rs);
+            if (moreRows) {
+                final Record record = createRecord(rs);
+                moreRows = rs.next();
+                return record;
+            } else {
+                return null;
             }
         } catch (final SQLException e) {
             throw new IOException("Could not obtain next record from ResultSet", e);
         }
-
-        return null;
     }
 
     @Override
@@ -86,7 +92,7 @@ public class ResultSetRecordSet implements RecordSet, Closeable {
 
             final Object value;
             if (rsColumnNames.contains(fieldName)) {
-                value = rs.getObject(field.getFieldName());
+                value = normalizeValue(rs.getObject(fieldName));
             } else {
                 value = null;
             }
@@ -97,6 +103,19 @@ public class ResultSetRecordSet implements RecordSet, Closeable {
         return new MapRecord(schema, values);
     }
 
+    @SuppressWarnings("rawtypes")
+    private Object normalizeValue(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof List) {
+            return ((List) value).toArray();
+        }
+
+        return value;
+    }
+
     private static RecordSchema createSchema(final ResultSet rs) throws SQLException {
         final ResultSetMetaData metadata = rs.getMetaData();
         final int numCols = metadata.getColumnCount();
@@ -106,26 +125,149 @@ public class ResultSetRecordSet implements RecordSet, Closeable {
             final int column = i + 1;
             final int sqlType = metadata.getColumnType(column);
 
-            final RecordFieldType fieldType = getFieldType(sqlType);
+            final DataType dataType = getDataType(sqlType, rs, column);
             final String fieldName = metadata.getColumnLabel(column);
-            final RecordField field = new RecordField(fieldName, fieldType.getDataType());
+            final RecordField field = new RecordField(fieldName, dataType);
             fields.add(field);
         }
 
         return new SimpleRecordSchema(fields);
     }
 
-    private static RecordFieldType getFieldType(final int sqlType) {
+    private static DataType getDataType(final int sqlType, final ResultSet rs, final int columnIndex) throws SQLException {
         switch (sqlType) {
             case Types.ARRAY:
-                return RecordFieldType.ARRAY;
-            case Types.BIGINT:
-            case Types.ROWID:
-                return RecordFieldType.LONG;
+                // The JDBC API does not allow us to know what the base type of an array is through the metadata.
+                // As a result, we have to obtain the actual Array for this record. Once we have this, we can determine
+                // the base type. However, if the base type is, itself, an array, we will simply return a base type of
+                // String because otherwise, we need the ResultSet for the array itself, and many JDBC Drivers do not
+                // support calling Array.getResultSet() and will throw an Exception if that is not supported.
+                if (rs.isAfterLast()) {
+                    return RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.STRING.getDataType());
+                }
+
+                final Array array = rs.getArray(columnIndex);
+                if (array == null) {
+                    return RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.STRING.getDataType());
+                }
+
+                final DataType baseType = getArrayBaseType(array);
+                return RecordFieldType.ARRAY.getArrayDataType(baseType);
             case Types.BINARY:
             case Types.LONGVARBINARY:
             case Types.VARBINARY:
-                return RecordFieldType.ARRAY;
+                return RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType());
+            default:
+                return getFieldType(sqlType).getDataType();
+        }
+    }
+
+    private static DataType getArrayBaseType(final Array array) throws SQLException {
+        final Object arrayValue = array.getArray();
+        if (arrayValue == null) {
+            return RecordFieldType.STRING.getDataType();
+        }
+
+        if (arrayValue instanceof byte[]) {
+            return RecordFieldType.BYTE.getDataType();
+        }
+        if (arrayValue instanceof int[]) {
+            return RecordFieldType.INT.getDataType();
+        }
+        if (arrayValue instanceof long[]) {
+            return RecordFieldType.LONG.getDataType();
+        }
+        if (arrayValue instanceof boolean[]) {
+            return RecordFieldType.BOOLEAN.getDataType();
+        }
+        if (arrayValue instanceof short[]) {
+            return RecordFieldType.SHORT.getDataType();
+        }
+        if (arrayValue instanceof byte[]) {
+            return RecordFieldType.BYTE.getDataType();
+        }
+        if (arrayValue instanceof float[]) {
+            return RecordFieldType.FLOAT.getDataType();
+        }
+        if (arrayValue instanceof double[]) {
+            return RecordFieldType.DOUBLE.getDataType();
+        }
+        if (arrayValue instanceof char[]) {
+            return RecordFieldType.CHAR.getDataType();
+        }
+        if (arrayValue instanceof Object[]) {
+            final Object[] values = (Object[]) arrayValue;
+            if (values.length == 0) {
+                return RecordFieldType.STRING.getDataType();
+            }
+
+            Object valueToLookAt = null;
+            for (int i = 0; i < values.length; i++) {
+                valueToLookAt = values[i];
+                if (valueToLookAt != null) {
+                    break;
+                }
+            }
+            if (valueToLookAt == null) {
+                return RecordFieldType.STRING.getDataType();
+            }
+
+            if (valueToLookAt instanceof String) {
+                return RecordFieldType.STRING.getDataType();
+            }
+            if (valueToLookAt instanceof Long) {
+                return RecordFieldType.LONG.getDataType();
+            }
+            if (valueToLookAt instanceof Integer) {
+                return RecordFieldType.INT.getDataType();
+            }
+            if (valueToLookAt instanceof Short) {
+                return RecordFieldType.SHORT.getDataType();
+            }
+            if (valueToLookAt instanceof Byte) {
+                return RecordFieldType.BYTE.getDataType();
+            }
+            if (valueToLookAt instanceof Float) {
+                return RecordFieldType.FLOAT.getDataType();
+            }
+            if (valueToLookAt instanceof Double) {
+                return RecordFieldType.DOUBLE.getDataType();
+            }
+            if (valueToLookAt instanceof Boolean) {
+                return RecordFieldType.BOOLEAN.getDataType();
+            }
+            if (valueToLookAt instanceof Character) {
+                return RecordFieldType.CHAR.getDataType();
+            }
+            if (valueToLookAt instanceof BigInteger) {
+                return RecordFieldType.BIGINT.getDataType();
+            }
+            if (valueToLookAt instanceof Integer) {
+                return RecordFieldType.INT.getDataType();
+            }
+            if (valueToLookAt instanceof java.sql.Time) {
+                return RecordFieldType.TIME.getDataType();
+            }
+            if (valueToLookAt instanceof java.sql.Date) {
+                return RecordFieldType.DATE.getDataType();
+            }
+            if (valueToLookAt instanceof java.sql.Timestamp) {
+                return RecordFieldType.TIMESTAMP.getDataType();
+            }
+            if (valueToLookAt instanceof Record) {
+                return RecordFieldType.RECORD.getDataType();
+            }
+        }
+
+        return RecordFieldType.STRING.getDataType();
+    }
+
+
+    private static RecordFieldType getFieldType(final int sqlType) {
+        switch (sqlType) {
+            case Types.BIGINT:
+            case Types.ROWID:
+                return RecordFieldType.LONG;
             case Types.BIT:
             case Types.BOOLEAN:
                 return RecordFieldType.BOOLEAN;

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
new file mode 100644
index 0000000..b74cdcc
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization.record.type;
+
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+public class ChoiceDataType extends DataType {
+    private final List<DataType> possibleSubTypes;
+
+    public ChoiceDataType(final List<DataType> possibleSubTypes) {
+        super(RecordFieldType.CHOICE, null);
+        this.possibleSubTypes = Objects.requireNonNull(possibleSubTypes);
+    }
+
+    public List<DataType> getPossibleSubTypes() {
+        return possibleSubTypes;
+    }
+
+    @Override
+    public RecordFieldType getFieldType() {
+        return RecordFieldType.CHOICE;
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 + 41 * getFieldType().hashCode() + 41 * (possibleSubTypes == null ? 0 : possibleSubTypes.hashCode());
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (obj == this) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (!(obj instanceof RecordDataType)) {
+            return false;
+        }
+
+        final ChoiceDataType other = (ChoiceDataType) obj;
+        return getFieldType().equals(other.getFieldType()) && Objects.equals(possibleSubTypes, other.possibleSubTypes);
+    }
+
+    @Override
+    public String toString() {
+        return "CHOICE" + possibleSubTypes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
new file mode 100644
index 0000000..f24d036
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.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.serialization.record.type;
+
+import java.util.Objects;
+
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public class RecordDataType extends DataType {
+    private final RecordSchema childSchema;
+
+    public RecordDataType(final RecordSchema childSchema) {
+        super(RecordFieldType.RECORD, null);
+        this.childSchema = childSchema;
+    }
+
+    @Override
+    public RecordFieldType getFieldType() {
+        return RecordFieldType.RECORD;
+    }
+
+    public RecordSchema getChildSchema() {
+        return childSchema;
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 + 41 * getFieldType().hashCode() + 41 * (childSchema == null ? 0 : childSchema.hashCode());
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (obj == this) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (!(obj instanceof RecordDataType)) {
+            return false;
+        }
+
+        final RecordDataType other = (RecordDataType) obj;
+        return getFieldType().equals(other.getFieldType()) && Objects.equals(childSchema, other.childSchema);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
new file mode 100644
index 0000000..1cdefb8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
@@ -0,0 +1,608 @@
+/*
+ * 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.util;
+
+import java.math.BigInteger;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TimeZone;
+import java.util.function.Consumer;
+
+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.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.ChoiceDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+
+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 dateFormat, final String timeFormat, final String timestampFormat) {
+        switch (dataType.getFieldType()) {
+            case BIGINT:
+                return toBigInt(value);
+            case BOOLEAN:
+                return toBoolean(value);
+            case BYTE:
+                return toByte(value);
+            case CHAR:
+                return toCharacter(value);
+            case DATE:
+                return toDate(value, dateFormat);
+            case DOUBLE:
+                return toDouble(value);
+            case FLOAT:
+                return toFloat(value);
+            case INT:
+                return toInteger(value);
+            case LONG:
+                return toLong(value);
+            case SHORT:
+                return toShort(value);
+            case STRING:
+                return toString(value, dateFormat, timeFormat, timestampFormat);
+            case TIME:
+                return toTime(value, timeFormat);
+            case TIMESTAMP:
+                return toTimestamp(value, timestampFormat);
+            case ARRAY:
+                return toArray(value);
+            case RECORD:
+                final RecordDataType recordType = (RecordDataType) dataType;
+                final RecordSchema childSchema = recordType.getChildSchema();
+                return toRecord(value, childSchema);
+            case CHOICE: {
+                if (value == null) {
+                    return null;
+                }
+
+                final ChoiceDataType choiceDataType = (ChoiceDataType) dataType;
+                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());
+                }
+
+                return convertType(value, chosenDataType);
+            }
+        }
+
+        return null;
+    }
+
+
+    public static boolean isCompatibleDataType(final Object value, final DataType dataType) {
+        switch (dataType.getFieldType()) {
+            case ARRAY:
+                return isArrayTypeCompatible(value);
+            case BIGINT:
+                return isBigIntTypeCompatible(value);
+            case BOOLEAN:
+                return isBooleanTypeCompatible(value);
+            case BYTE:
+                return isByteTypeCompatible(value);
+            case CHAR:
+                return isCharacterTypeCompatible(value);
+            case DATE:
+                return isDateTypeCompatible(value, dataType.getFormat());
+            case DOUBLE:
+                return isDoubleTypeCompatible(value);
+            case FLOAT:
+                return isFloatTypeCompatible(value);
+            case INT:
+                return isIntegerTypeCompatible(value);
+            case LONG:
+                return isLongTypeCompatible(value);
+            case RECORD:
+                return isRecordTypeCompatible(value);
+            case SHORT:
+                return isShortTypeCompatible(value);
+            case TIME:
+                return isTimeTypeCompatible(value, dataType.getFormat());
+            case TIMESTAMP:
+                return isTimestampTypeCompatible(value, dataType.getFormat());
+            case STRING:
+                return isStringTypeCompatible(value);
+            case CHOICE: {
+                final DataType chosenDataType = chooseDataType(value, (ChoiceDataType) dataType);
+                return chosenDataType != null;
+            }
+        }
+
+        return false;
+    }
+
+    public static DataType chooseDataType(final Object value, final ChoiceDataType choiceType) {
+        for (final DataType subType : choiceType.getPossibleSubTypes()) {
+            if (isCompatibleDataType(value, subType)) {
+                return subType;
+            }
+        }
+
+        return null;
+    }
+
+    public static Record toRecord(final Object value, final RecordSchema recordSchema) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Record) {
+            return ((Record) value);
+        }
+
+        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");
+            }
+
+            final Map<?, ?> map = (Map<?, ?>) value;
+            final Map<String, Object> coercedValues = new HashMap<>();
+
+            for (final Map.Entry<?, ?> entry : map.entrySet()) {
+                final Object keyValue = entry.getKey();
+                if (keyValue == null) {
+                    continue;
+                }
+
+                final String key = keyValue.toString();
+                final Optional<DataType> desiredTypeOption = recordSchema.getDataType(key);
+                if (!desiredTypeOption.isPresent()) {
+                    continue;
+                }
+
+                final Object rawValue = entry.getValue();
+                final Object coercedValue = convertType(rawValue, desiredTypeOption.get());
+                coercedValues.put(key, coercedValue);
+            }
+
+            return new MapRecord(recordSchema, coercedValues);
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Record");
+    }
+
+    public static boolean isRecordTypeCompatible(final Object value) {
+        return value != null && value instanceof Record;
+    }
+
+    public static Object[] toArray(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Object[]) {
+            return (Object[]) value;
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Object Array");
+    }
+
+    public static boolean isArrayTypeCompatible(final Object value) {
+        return value != null && value instanceof Object[];
+    }
+
+    public static String toString(final Object value, final String dateFormat, final String timeFormat, final String timestampFormat) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof String) {
+            return (String) value;
+        }
+
+        if (value instanceof java.sql.Date) {
+            return getDateFormat(dateFormat).format((java.util.Date) value);
+        }
+        if (value instanceof java.sql.Time) {
+            return getDateFormat(timeFormat).format((java.util.Date) value);
+        }
+        if (value instanceof java.sql.Timestamp) {
+            return getDateFormat(timestampFormat).format((java.util.Date) value);
+        }
+        if (value instanceof java.util.Date) {
+            return getDateFormat(timestampFormat).format((java.util.Date) value);
+        }
+
+        return value.toString();
+    }
+
+    public static boolean isStringTypeCompatible(final Object value) {
+        return value != null && (value instanceof String || value instanceof java.util.Date);
+    }
+
+    public static java.sql.Date toDate(final Object value, final String format) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Date) {
+            return (Date) value;
+        }
+
+        if (value instanceof Number) {
+            final long longValue = ((Number) value).longValue();
+            return new Date(longValue);
+        }
+
+        if (value instanceof String) {
+            try {
+                final java.util.Date utilDate = getDateFormat(format).parse((String) value);
+                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);
+            }
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Date");
+    }
+
+    public static boolean isDateTypeCompatible(final Object value, final String format) {
+        if (value == null) {
+            return false;
+        }
+
+        if (value instanceof java.util.Date || value instanceof Number) {
+            return true;
+        }
+
+        if (value instanceof String) {
+            try {
+                getDateFormat(format).parse((String) value);
+                return true;
+            } catch (final ParseException e) {
+                return false;
+            }
+        }
+
+        return false;
+    }
+
+    public static Time toTime(final Object value, final String format) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Time) {
+            return (Time) value;
+        }
+
+        if (value instanceof Number) {
+            final long longValue = ((Number) value).longValue();
+            return new Time(longValue);
+        }
+
+        if (value instanceof String) {
+            try {
+                final java.util.Date utilDate = getDateFormat(format).parse((String) value);
+                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);
+            }
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Time");
+    }
+
+    private static DateFormat getDateFormat(final String format) {
+        final DateFormat df = new SimpleDateFormat(format);
+        df.setTimeZone(gmt);
+        return df;
+    }
+
+    public static boolean isTimeTypeCompatible(final Object value, final String format) {
+        return isDateTypeCompatible(value, format);
+    }
+
+    public static Timestamp toTimestamp(final Object value, final String format) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Timestamp) {
+            return (Timestamp) value;
+        }
+
+        if (value instanceof Number) {
+            final long longValue = ((Number) value).longValue();
+            return new Timestamp(longValue);
+        }
+
+        if (value instanceof String) {
+            try {
+                final java.util.Date utilDate = getDateFormat(format).parse((String) value);
+                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);
+            }
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Timestamp");
+    }
+
+    public static boolean isTimestampTypeCompatible(final Object value, final String format) {
+        return isDateTypeCompatible(value, format);
+    }
+
+
+    public static BigInteger toBigInt(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof BigInteger) {
+            return (BigInteger) value;
+        }
+        if (value instanceof Long) {
+            return BigInteger.valueOf((Long) value);
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to BigInteger");
+    }
+
+    public static boolean isBigIntTypeCompatible(final Object value) {
+        return value == null && (value instanceof BigInteger || value instanceof Long);
+    }
+
+    public static Boolean toBoolean(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Boolean) {
+            return (Boolean) value;
+        }
+        if (value instanceof String) {
+            final String string = (String) value;
+            if (string.equalsIgnoreCase("true")) {
+                return Boolean.TRUE;
+            } else if (string.equalsIgnoreCase("false")) {
+                return Boolean.FALSE;
+            }
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Boolean");
+    }
+
+    public static boolean isBooleanTypeCompatible(final Object value) {
+        if (value == null) {
+            return false;
+        }
+        if (value instanceof Boolean) {
+            return true;
+        }
+        if (value instanceof String) {
+            final String string = (String) value;
+            return string.equalsIgnoreCase("true") || string.equalsIgnoreCase("false");
+        }
+        return false;
+    }
+
+    public static Double toDouble(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).doubleValue();
+        }
+
+        if (value instanceof String) {
+            return Double.parseDouble((String) value);
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Double");
+    }
+
+    public static boolean isDoubleTypeCompatible(final Object value) {
+        return isNumberTypeCompatible(value, s -> Double.parseDouble(s));
+    }
+
+    private static boolean isNumberTypeCompatible(final Object value, final Consumer<String> stringValueVerifier) {
+        if (value == null) {
+            return false;
+        }
+
+        if (value instanceof Number) {
+            return true;
+        }
+
+        if (value instanceof String) {
+            try {
+                stringValueVerifier.accept((String) value);
+                return true;
+            } catch (final NumberFormatException nfe) {
+                return false;
+            }
+        }
+
+        return false;
+    }
+
+    public static Float toFloat(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).floatValue();
+        }
+
+        if (value instanceof String) {
+            return Float.parseFloat((String) value);
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Float");
+    }
+
+    public static boolean isFloatTypeCompatible(final Object value) {
+        return isNumberTypeCompatible(value, s -> Float.parseFloat(s));
+    }
+
+    public static Long toLong(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).longValue();
+        }
+
+        if (value instanceof String) {
+            return Long.parseLong((String) value);
+        }
+
+        if (value instanceof java.util.Date) {
+            return ((java.util.Date) value).getTime();
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Long");
+    }
+
+    public static boolean isLongTypeCompatible(final Object value) {
+        if (value == null) {
+            return false;
+        }
+
+        if (value instanceof Number) {
+            return true;
+        }
+
+        if (value instanceof java.util.Date) {
+            return true;
+        }
+
+        if (value instanceof String) {
+            try {
+                Long.parseLong((String) value);
+                return true;
+            } catch (final NumberFormatException nfe) {
+                return false;
+            }
+        }
+
+        return false;
+    }
+
+
+    public static Integer toInteger(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).intValue();
+        }
+
+        if (value instanceof String) {
+            return Integer.parseInt((String) value);
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Integer");
+    }
+
+    public static boolean isIntegerTypeCompatible(final Object value) {
+        return isNumberTypeCompatible(value, s -> Integer.parseInt(s));
+    }
+
+
+    public static Short toShort(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).shortValue();
+        }
+
+        if (value instanceof String) {
+            return Short.parseShort((String) value);
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Short");
+    }
+
+    public static boolean isShortTypeCompatible(final Object value) {
+        return isNumberTypeCompatible(value, s -> Short.parseShort(s));
+    }
+
+    public static Byte toByte(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).byteValue();
+        }
+
+        if (value instanceof String) {
+            return Byte.parseByte((String) value);
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Byte");
+    }
+
+    public static boolean isByteTypeCompatible(final Object value) {
+        return isNumberTypeCompatible(value, s -> Byte.parseByte(s));
+    }
+
+
+    public static Character toCharacter(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Character) {
+            return ((Character) value);
+        }
+
+        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");
+            }
+
+            return charSeq.charAt(0);
+        }
+
+        throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Character");
+    }
+
+    public static boolean isCharacterTypeCompatible(final Object value) {
+        return value != null && (value instanceof Character || (value instanceof CharSequence && ((CharSequence) value).length() > 0));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/IllegalTypeConversionException.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/IllegalTypeConversionException.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/IllegalTypeConversionException.java
new file mode 100644
index 0000000..38b5d20
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/IllegalTypeConversionException.java
@@ -0,0 +1,29 @@
+/*
+ * 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.util;
+
+public class IllegalTypeConversionException extends RuntimeException {
+
+    public IllegalTypeConversionException(final String message) {
+        super(message);
+    }
+
+    public IllegalTypeConversionException(final String message, final Throwable cause) {
+        super(message, cause);
+    }
+}


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

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/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
new file mode 100644
index 0000000..2102813
--- /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/TestWriteAvroResult.java
@@ -0,0 +1,202 @@
+/*
+ * 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.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+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.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+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.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.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.junit.Test;
+
+public class TestWriteAvroResult {
+
+    @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 List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("timeMillis", RecordFieldType.TIME.getDataType()));
+        fields.add(new RecordField("timeMicros", RecordFieldType.TIME.getDataType()));
+        fields.add(new RecordField("timestampMillis", RecordFieldType.TIMESTAMP.getDataType()));
+        fields.add(new RecordField("timestampMicros", RecordFieldType.TIMESTAMP.getDataType()));
+        fields.add(new RecordField("date", RecordFieldType.DATE.getDataType()));
+        final RecordSchema recordSchema = new SimpleRecordSchema(fields);
+
+        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 Map<String, Object> values = new HashMap<>();
+        values.put("timeMillis", new Time(timeLong));
+        values.put("timeMicros", new Time(timeLong));
+        values.put("timestampMillis", new Timestamp(timeLong));
+        values.put("timestampMicros", new Timestamp(timeLong));
+        values.put("date", new Date(timeLong));
+        final Record record = new MapRecord(recordSchema, values);
+
+        final byte[] data;
+        try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+            writer.write(RecordSet.of(record.getSchema(), record), baos);
+            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 long secondsSinceMidnight = 33 + (20 * 60) + (14 * 60 * 60);
+            final long millisSinceMidnight = secondsSinceMidnight * 1000L;
+
+            assertEquals((int) millisSinceMidnight, avroRecord.get("timeMillis"));
+            assertEquals(millisSinceMidnight * 1000L, avroRecord.get("timeMicros"));
+            assertEquals(timeLong, avroRecord.get("timestampMillis"));
+            assertEquals(timeLong * 1000L, avroRecord.get("timestampMicros"));
+            assertEquals(17260, avroRecord.get("date"));
+        }
+    }
+
+
+    @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 List<RecordField> subRecordFields = Collections.singletonList(new RecordField("field1", RecordFieldType.STRING.getDataType()));
+        final RecordSchema subRecordSchema = new SimpleRecordSchema(subRecordFields);
+
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("string", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("int", RecordFieldType.INT.getDataType()));
+        fields.add(new RecordField("long", RecordFieldType.LONG.getDataType()));
+        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("nullOrLong", RecordFieldType.LONG.getDataType()));
+        fields.add(new RecordField("array", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.INT.getDataType())));
+        fields.add(new RecordField("record", RecordFieldType.RECORD.getRecordDataType(subRecordSchema)));
+        final RecordSchema recordSchema = new SimpleRecordSchema(fields);
+
+        final Record innerRecord = new MapRecord(subRecordSchema, Collections.singletonMap("field1", "hello"));
+
+        final Map<String, Object> values = new HashMap<>();
+        values.put("string", "hello");
+        values.put("int", 8);
+        values.put("long", 42L);
+        values.put("double", 3.14159D);
+        values.put("float", 1.23456F);
+        values.put("boolean", true);
+        values.put("bytes", AvroTypeUtil.convertByteArray("hello".getBytes()));
+        values.put("nullOrLong", null);
+        values.put("array", new Integer[] {1, 2, 3});
+        values.put("record", innerRecord);
+
+        final Record record = new MapRecord(recordSchema, values);
+
+        final byte[] data;
+        try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+            writer.write(RecordSet.of(record.getSchema(), record), baos);
+            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();
+            assertMatch(record, avroRecord);
+        }
+    }
+
+    private 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);
+
+            if (recordValue instanceof String) {
+                assertNotNull(fieldName + " should not have been null", avroValue);
+                avroValue = avroValue.toString();
+            }
+
+            if (recordValue instanceof Object[] && avroValue instanceof ByteBuffer) {
+                final ByteBuffer bb = (ByteBuffer) avroValue;
+                final Object[] objectArray = (Object[]) recordValue;
+                assertEquals("For field " + fieldName + ", byte buffer remaining should have been " + objectArray.length + " but was " + bb.remaining(),
+                    objectArray.length, bb.remaining());
+
+                for (int i = 0; i < objectArray.length; i++) {
+                    assertEquals(objectArray[i], bb.get());
+                }
+            } else if (recordValue instanceof Object[]) {
+                assertTrue(fieldName + " should have been instanceof Array", avroValue instanceof Array);
+                final Array<?> avroArray = (Array<?>) avroValue;
+                final Object[] recordArray = (Object[]) recordValue;
+                assertEquals(fieldName + " not equal", recordArray.length, avroArray.size());
+                for (int i = 0; i < recordArray.length; i++) {
+                    assertEquals(fieldName + "[" + i + "] not equal", recordArray[i], avroArray.get(i));
+                }
+            } else if (recordValue instanceof byte[]) {
+                final ByteBuffer bb = ByteBuffer.wrap((byte[]) recordValue);
+                assertEquals(fieldName + " not equal", bb, avroValue);
+            } else if (recordValue instanceof Record) {
+                assertMatch((Record) recordValue, (GenericRecord) avroValue);
+            } else {
+                assertEquals(fieldName + " not equal", recordValue, avroValue);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.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/TestCSVRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java
index 1e53d89..cb790f1 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java
@@ -20,18 +20,24 @@ package org.apache.nifi.csv;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
+import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.Arrays;
-import java.util.HashMap;
+import java.sql.Date;
+import java.util.ArrayList;
+import java.util.Calendar;
 import java.util.List;
-import java.util.Map;
+import java.util.TimeZone;
 
+import org.apache.commons.csv.CSVFormat;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.junit.Assert;
@@ -39,21 +45,50 @@ import org.junit.Test;
 import org.mockito.Mockito;
 
 public class TestCSVRecordReader {
-    private final DataType stringDataType = RecordFieldType.STRING.getDataType();
     private final DataType doubleDataType = RecordFieldType.DOUBLE.getDataType();
-    private final DataType timeDataType = RecordFieldType.TIME.getDataType();
+    private final CSVFormat format = CSVFormat.DEFAULT.withFirstRecordAsHeader().withTrim().withQuote('"');
+
+    private List<RecordField> getDefaultFields() {
+        final List<RecordField> fields = new ArrayList<>();
+        for (final String fieldName : new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"}) {
+            fields.add(new RecordField(fieldName, RecordFieldType.STRING.getDataType()));
+        }
+        return fields;
+    }
+
+    @Test
+    public void testDate() throws IOException, MalformedRecordException {
+        final String text = "date\n11/30/1983";
+
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("date", RecordFieldType.DATE.getDataType()));
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+
+        try (final InputStream bais = new ByteArrayInputStream(text.getBytes());
+            final CSVRecordReader reader = new CSVRecordReader(bais, Mockito.mock(ComponentLog.class), schema, format,
+                "MM/dd/yyyy", RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat())) {
+
+            final Record record = reader.nextRecord();
+            final java.sql.Date date = (Date) record.getValue("date");
+            final Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("gmt"));
+            calendar.setTimeInMillis(date.getTime());
+
+            assertEquals(1983, calendar.get(Calendar.YEAR));
+            assertEquals(10, calendar.get(Calendar.MONTH));
+            assertEquals(30, calendar.get(Calendar.DAY_OF_MONTH));
+        }
+    }
 
     @Test
     public void testSimpleParse() throws IOException, MalformedRecordException {
-        final Map<String, DataType> overrides = new HashMap<>();
-        overrides.put("balance", doubleDataType);
-        overrides.put("other", timeDataType);
+        final List<RecordField> fields = getDefaultFields();
+        fields.replaceAll(f -> f.getFieldName().equals("balance") ? new RecordField("balance", doubleDataType) : f);
 
-        try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/single-bank-account.csv"))) {
-            final CSVRecordReader reader = new CSVRecordReader(fis, null, overrides);
+        final RecordSchema schema = new SimpleRecordSchema(fields);
 
-            final RecordSchema schema = reader.getSchema();
-            verifyFields(schema);
+        try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/single-bank-account.csv"))) {
+            final CSVRecordReader reader = new CSVRecordReader(fis, Mockito.mock(ComponentLog.class), schema, format,
+                RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
 
             final Object[] record = reader.nextRecord().getValues();
             final Object[] expectedValues = new Object[] {"1", "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"};
@@ -65,14 +100,14 @@ public class TestCSVRecordReader {
 
     @Test
     public void testMultipleRecords() throws IOException, MalformedRecordException {
-        final Map<String, DataType> overrides = new HashMap<>();
-        overrides.put("balance", doubleDataType);
+        final List<RecordField> fields = getDefaultFields();
+        fields.replaceAll(f -> f.getFieldName().equals("balance") ? new RecordField("balance", doubleDataType) : f);
 
-        try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/multi-bank-account.csv"))) {
-            final CSVRecordReader reader = new CSVRecordReader(fis, null, overrides);
+        final RecordSchema schema = new SimpleRecordSchema(fields);
 
-            final RecordSchema schema = reader.getSchema();
-            verifyFields(schema);
+        try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/multi-bank-account.csv"))) {
+            final CSVRecordReader reader = new CSVRecordReader(fis, Mockito.mock(ComponentLog.class), schema, format,
+                RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
 
             final Object[] firstRecord = reader.nextRecord().getValues();
             final Object[] firstExpectedValues = new Object[] {"1", "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"};
@@ -88,14 +123,14 @@ public class TestCSVRecordReader {
 
     @Test
     public void testExtraWhiteSpace() throws IOException, MalformedRecordException {
-        final Map<String, DataType> overrides = new HashMap<>();
-        overrides.put("balance", doubleDataType);
+        final List<RecordField> fields = getDefaultFields();
+        fields.replaceAll(f -> f.getFieldName().equals("balance") ? new RecordField("balance", doubleDataType) : f);
 
-        try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/extra-white-space.csv"))) {
-            final CSVRecordReader reader = new CSVRecordReader(fis, Mockito.mock(ComponentLog.class), overrides);
+        final RecordSchema schema = new SimpleRecordSchema(fields);
 
-            final RecordSchema schema = reader.getSchema();
-            verifyFields(schema);
+        try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/extra-white-space.csv"))) {
+            final CSVRecordReader reader = new CSVRecordReader(fis, Mockito.mock(ComponentLog.class), schema, format,
+                RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
 
             final Object[] firstRecord = reader.nextRecord().getValues();
             final Object[] firstExpectedValues = new Object[] {"1", "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"};
@@ -108,15 +143,4 @@ public class TestCSVRecordReader {
             assertNull(reader.nextRecord());
         }
     }
-
-    private void verifyFields(final RecordSchema schema) {
-        final List<String> fieldNames = schema.getFieldNames();
-        final List<String> expectedFieldNames = Arrays.asList("id", "name", "balance", "address", "city", "state", "zipCode", "country");
-        assertEquals(expectedFieldNames, fieldNames);
-
-        final List<DataType> dataTypes = schema.getDataTypes();
-        final List<DataType> expectedDataTypes = Arrays.asList(stringDataType, stringDataType, doubleDataType,
-            stringDataType, stringDataType, stringDataType, stringDataType, stringDataType);
-        assertEquals(expectedDataTypes, dataTypes);
-    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/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 04f8479..1e8997b 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
@@ -26,12 +26,16 @@ import java.nio.charset.StandardCharsets;
 import java.sql.Date;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.TimeZone;
 
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.QuoteMode;
 import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.MapRecord;
@@ -47,7 +51,8 @@ public class TestWriteCSVResult {
 
     @Test
     public void testDataTypes() throws IOException {
-        final WriteCSVResult result = new WriteCSVResult(RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
+        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<>();
@@ -57,7 +62,7 @@ public class TestWriteCSVResult {
                 possibleTypes.add(RecordFieldType.INT.getDataType());
                 possibleTypes.add(RecordFieldType.LONG.getDataType());
 
-                fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType(possibleTypes)));
+                fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getChoiceDataType(possibleTypes)));
             } else {
                 fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType()));
             }
@@ -81,7 +86,7 @@ public class TestWriteCSVResult {
         valueMap.put("date", new Date(now));
         valueMap.put("time", new Time(now));
         valueMap.put("timestamp", new Timestamp(now));
-        valueMap.put("object", null);
+        valueMap.put("record", null);
         valueMap.put("choice", 48L);
         valueMap.put("array", null);
 
@@ -105,9 +110,9 @@ public class TestWriteCSVResult {
         final StringBuilder expectedBuilder = new StringBuilder();
         expectedBuilder.append("\"string\",\"true\",\"1\",\"c\",\"8\",\"9\",\"8\",\"8\",\"8.0\",\"8.0\",");
 
-        final String dateValue = new SimpleDateFormat(RecordFieldType.DATE.getDefaultFormat()).format(now);
-        final String timeValue = new SimpleDateFormat(RecordFieldType.TIME.getDefaultFormat()).format(now);
-        final String timestampValue = new SimpleDateFormat(RecordFieldType.TIMESTAMP.getDefaultFormat()).format(now);
+        final String dateValue = getDateFormat(RecordFieldType.DATE.getDefaultFormat()).format(now);
+        final String timeValue = getDateFormat(RecordFieldType.TIME.getDefaultFormat()).format(now);
+        final String timestampValue = getDateFormat(RecordFieldType.TIMESTAMP.getDefaultFormat()).format(now);
 
         expectedBuilder.append('"').append(dateValue).append('"').append(',');
         expectedBuilder.append('"').append(timeValue).append('"').append(',');
@@ -118,4 +123,10 @@ public class TestWriteCSVResult {
         assertEquals(expectedValues, values);
     }
 
+    private DateFormat getDateFormat(final String format) {
+        final DateFormat df = new SimpleDateFormat(format);
+        df.setTimeZone(TimeZone.getTimeZone("gmt"));
+        return df;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/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 3757ab1..a741ad1 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
@@ -28,7 +28,6 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.charset.StandardCharsets;
-import java.util.Collections;
 
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.record.Record;
@@ -46,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, Collections.emptyMap());
+            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null);
 
             final String[] logLevels = new String[] {"INFO", "WARN", "ERROR", "FATAL", "FINE"};
             final String[] messages = new String[] {"Test Message 1", "Red", "Green", "Blue", "Yellow"};
@@ -76,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, Collections.emptyMap());
+        final GrokRecordReader deserializer = new GrokRecordReader(bais, grok, null);
 
         final Object[] values = deserializer.nextRecord().getValues();
 
@@ -99,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, Collections.emptyMap());
+            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null);
 
             final String[] logLevels = new String[] {"INFO", "INFO", "INFO", "WARN", "WARN"};
 
@@ -123,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, Collections.emptyMap());
+            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null);
 
             final String[] logLevels = new String[] {"INFO", "INFO", "ERROR", "WARN", "WARN"};
 
@@ -155,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, Collections.emptyMap());
+            final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null);
 
             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/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.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/TestJsonPathRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.java
index fa41396..11e2828 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.java
@@ -26,17 +26,18 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.stream.Collectors;
 
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.junit.Assert;
@@ -47,6 +48,10 @@ import org.mockito.Mockito;
 import com.jayway.jsonpath.JsonPath;
 
 public class TestJsonPathRowRecordReader {
+    private final String dateFormat = RecordFieldType.DATE.getDefaultFormat();
+    private final String timeFormat = RecordFieldType.TIME.getDefaultFormat();
+    private final String timestampFormat = RecordFieldType.TIMESTAMP.getDefaultFormat();
+
     private final LinkedHashMap<String, JsonPath> allJsonPaths = new LinkedHashMap<>();
 
     @Before
@@ -63,12 +68,36 @@ public class TestJsonPathRowRecordReader {
         allJsonPaths.put("country", JsonPath.compile("$.country"));
     }
 
+
+    private List<RecordField> getDefaultFields() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("id", RecordFieldType.INT.getDataType()));
+        fields.add(new RecordField("name", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("balance", RecordFieldType.DOUBLE.getDataType()));
+        fields.add(new RecordField("address", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("city", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("state", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("zipCode", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("country", RecordFieldType.STRING.getDataType()));
+        return fields;
+    }
+
+    private RecordSchema getAccountSchema() {
+        final List<RecordField> accountFields = new ArrayList<>();
+        accountFields.add(new RecordField("id", RecordFieldType.INT.getDataType()));
+        accountFields.add(new RecordField("balance", RecordFieldType.DOUBLE.getDataType()));
+
+        final RecordSchema accountSchema = new SimpleRecordSchema(accountFields);
+        return accountSchema;
+    }
+
+
     @Test
     public void testReadArray() throws IOException, MalformedRecordException {
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array.json"));
-            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+        final RecordSchema schema = new SimpleRecordSchema(getDefaultFields());
 
-            final RecordSchema schema = reader.getSchema();
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, schema, in, Mockito.mock(ComponentLog.class), dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
@@ -91,10 +120,10 @@ public class TestJsonPathRowRecordReader {
 
     @Test
     public void testSingleJsonElement() throws IOException, MalformedRecordException {
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-bank-account.json"));
-            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+        final RecordSchema schema = new SimpleRecordSchema(getDefaultFields());
 
-            final RecordSchema schema = reader.getSchema();
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-bank-account.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, schema, in, Mockito.mock(ComponentLog.class), dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
@@ -119,17 +148,20 @@ public class TestJsonPathRowRecordReader {
         final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
         jsonPaths.put("account", JsonPath.compile("$.account"));
 
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested.json"));
-            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+        final DataType accountType = RecordFieldType.RECORD.getRecordDataType(getAccountSchema());
+        final List<RecordField> fields = getDefaultFields();
+        fields.add(new RecordField("account", accountType));
+        final RecordSchema schema = new SimpleRecordSchema(fields);
 
-            final RecordSchema schema = reader.getSchema();
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, schema, in, Mockito.mock(ComponentLog.class), dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "account"});
             assertEquals(expectedFieldNames, fieldNames);
 
             final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
-            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.STRING,
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.DOUBLE,
                 RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.RECORD});
             assertEquals(expectedTypes, dataTypes);
 
@@ -152,10 +184,15 @@ public class TestJsonPathRowRecordReader {
         final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
         jsonPaths.put("accounts", JsonPath.compile("$.accounts"));
 
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested-array.json"));
-            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+        final DataType accountRecordType = RecordFieldType.RECORD.getRecordDataType(getAccountSchema());
+        final DataType accountsType = RecordFieldType.ARRAY.getArrayDataType(accountRecordType);
+        final List<RecordField> fields = getDefaultFields();
+        fields.add(new RecordField("accounts", accountsType));
+        final RecordSchema schema = new SimpleRecordSchema(fields);
 
-            final RecordSchema schema = reader.getSchema();
+
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested-array.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, schema, in, Mockito.mock(ComponentLog.class), dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {
@@ -163,7 +200,7 @@ public class TestJsonPathRowRecordReader {
             assertEquals(expectedFieldNames, fieldNames);
 
             final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
-            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.STRING,
+            final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.DOUBLE,
                 RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.ARRAY});
             assertEquals(expectedTypes, dataTypes);
 
@@ -177,17 +214,17 @@ public class TestJsonPathRowRecordReader {
             final Object[] array = (Object[]) lastRecord;
             assertEquals(2, array.length);
             final Object firstElement = array[0];
-            assertTrue(firstElement instanceof Map);
+            assertTrue(firstElement instanceof Record);
 
-            final Map<?, ?> firstMap = (Map<?, ?>) firstElement;
-            assertEquals(42, firstMap.get("id"));
-            assertEquals(4750.89D, firstMap.get("balance"));
+            final Record firstRecord = (Record) firstElement;
+            assertEquals(42, firstRecord.getValue("id"));
+            assertEquals(4750.89D, firstRecord.getValue("balance"));
 
             final Object secondElement = array[1];
-            assertTrue(secondElement instanceof Map);
-            final Map<?, ?> secondMap = (Map<?, ?>) secondElement;
-            assertEquals(43, secondMap.get("id"));
-            assertEquals(48212.38D, secondMap.get("balance"));
+            assertTrue(secondElement instanceof Record);
+            final Record secondRecord = (Record) secondElement;
+            assertEquals(43, secondRecord.getValue("id"));
+            assertEquals(48212.38D, secondRecord.getValue("balance"));
 
             assertNull(reader.nextRecord());
         }
@@ -195,10 +232,10 @@ public class TestJsonPathRowRecordReader {
 
     @Test
     public void testReadArrayDifferentSchemas() throws IOException, MalformedRecordException {
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
-            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+        final RecordSchema schema = new SimpleRecordSchema(getDefaultFields());
 
-            final RecordSchema schema = reader.getSchema();
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, schema, in, Mockito.mock(ComponentLog.class), dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
@@ -226,11 +263,14 @@ public class TestJsonPathRowRecordReader {
     public void testReadArrayDifferentSchemasWithOverride() throws IOException, MalformedRecordException {
         final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
         jsonPaths.put("address2", JsonPath.compile("$.address2"));
-        final Map<String, DataType> typeOverrides = Collections.singletonMap("address2", RecordFieldType.STRING.getDataType());
+
+        final List<RecordField> fields = getDefaultFields();
+        fields.add(new RecordField("address2", RecordFieldType.STRING.getDataType()));
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+
 
         try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
-            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, typeOverrides, in, Mockito.mock(ComponentLog.class))) {
-            final RecordSchema schema = reader.getSchema();
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, schema, in, Mockito.mock(ComponentLog.class), dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "address2"});
@@ -259,10 +299,13 @@ public class TestJsonPathRowRecordReader {
         final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
         jsonPaths.put("accountIds", JsonPath.compile("$.accountIds"));
 
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/primitive-type-array.json"));
-            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
+        final List<RecordField> fields = getDefaultFields();
+        final DataType idsType = RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.INT.getDataType());
+        fields.add(new RecordField("accountIds", idsType));
+        final RecordSchema schema = new SimpleRecordSchema(fields);
 
-            final RecordSchema schema = reader.getSchema();
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/primitive-type-array.json"));
+            final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, schema, in, Mockito.mock(ComponentLog.class), dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "accountIds"});

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/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 c5ee0e3..2422206 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
@@ -25,8 +25,8 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -34,8 +34,10 @@ import java.util.stream.Collectors;
 
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.junit.Assert;
@@ -43,13 +45,38 @@ import org.junit.Test;
 import org.mockito.Mockito;
 
 public class TestJsonTreeRowRecordReader {
+    private final String dateFormat = RecordFieldType.DATE.getDefaultFormat();
+    private final String timeFormat = RecordFieldType.TIME.getDefaultFormat();
+    private final String timestampFormat = RecordFieldType.TIMESTAMP.getDefaultFormat();
+
+    private List<RecordField> getDefaultFields() {
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("id", RecordFieldType.INT.getDataType()));
+        fields.add(new RecordField("name", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("balance", RecordFieldType.DOUBLE.getDataType()));
+        fields.add(new RecordField("address", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("city", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("state", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("zipCode", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("country", RecordFieldType.STRING.getDataType()));
+        return fields;
+    }
+
+    private RecordSchema getAccountSchema() {
+        final List<RecordField> accountFields = new ArrayList<>();
+        accountFields.add(new RecordField("id", RecordFieldType.INT.getDataType()));
+        accountFields.add(new RecordField("balance", RecordFieldType.DOUBLE.getDataType()));
+
+        final RecordSchema accountSchema = new SimpleRecordSchema(accountFields);
+        return accountSchema;
+    }
 
     @Test
     public void testReadArray() throws IOException, MalformedRecordException {
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array.json"));
-            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+        final RecordSchema schema = new SimpleRecordSchema(getDefaultFields());
 
-            final RecordSchema schema = reader.getSchema();
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
@@ -72,10 +99,10 @@ public class TestJsonTreeRowRecordReader {
 
     @Test
     public void testSingleJsonElement() throws IOException, MalformedRecordException {
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-bank-account.json"));
-            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+        final RecordSchema schema = new SimpleRecordSchema(getDefaultFields());
 
-            final RecordSchema schema = reader.getSchema();
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-bank-account.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
@@ -95,14 +122,14 @@ public class TestJsonTreeRowRecordReader {
 
     @Test
     public void testElementWithNestedData() throws IOException, MalformedRecordException {
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested.json"));
-            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
-
-            final RecordSchema schema = reader.getSchema();
+        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);
 
-            final List<String> fieldNames = schema.getFieldNames();
-            final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "address", "city", "state", "zipCode", "country", "account"});
-            assertEquals(expectedFieldNames, fieldNames);
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
 
             final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
             final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
@@ -125,10 +152,16 @@ public class TestJsonTreeRowRecordReader {
 
     @Test
     public void testElementWithNestedArray() throws IOException, MalformedRecordException {
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested-array.json"));
-            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+        final DataType accountRecordType = RecordFieldType.RECORD.getRecordDataType(getAccountSchema());
+        final DataType accountsType = RecordFieldType.ARRAY.getArrayDataType(accountRecordType);
+
+        final List<RecordField> fields = getDefaultFields();
+        fields.add(new RecordField("accounts", accountsType));
+        fields.remove(new RecordField("balance", RecordFieldType.DOUBLE.getDataType()));
+        final RecordSchema schema = new SimpleRecordSchema(fields);
 
-            final RecordSchema schema = reader.getSchema();
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested-array.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {
@@ -153,10 +186,10 @@ public class TestJsonTreeRowRecordReader {
 
     @Test
     public void testReadArrayDifferentSchemas() throws IOException, MalformedRecordException {
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
-            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+        final RecordSchema schema = new SimpleRecordSchema(getDefaultFields());
 
-            final RecordSchema schema = reader.getSchema();
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
@@ -185,10 +218,12 @@ public class TestJsonTreeRowRecordReader {
         final Map<String, DataType> overrides = new HashMap<>();
         overrides.put("address2", RecordFieldType.STRING.getDataType());
 
-        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
-            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), overrides)) {
+        final List<RecordField> fields = getDefaultFields();
+        fields.add(new RecordField("address2", RecordFieldType.STRING.getDataType()));
+        final RecordSchema schema = new SimpleRecordSchema(fields);
 
-            final RecordSchema schema = reader.getSchema();
+        try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "address2"});
@@ -214,13 +249,10 @@ public class TestJsonTreeRowRecordReader {
 
     @Test
     public void testReadArrayDifferentSchemasWithOptionalElementOverridden() throws IOException, MalformedRecordException {
-        final Map<String, DataType> overrides = new HashMap<>();
-        overrides.put("balance", RecordFieldType.DOUBLE.getDataType());
+        final RecordSchema schema = new SimpleRecordSchema(getDefaultFields());
 
         try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-optional-balance.json"));
-            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), overrides)) {
-
-            final RecordSchema schema = reader.getSchema();
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
 
             final List<String> fieldNames = schema.getFieldNames();
             final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
@@ -247,8 +279,22 @@ public class TestJsonTreeRowRecordReader {
 
     @Test
     public void testReadUnicodeCharacters() throws IOException, MalformedRecordException {
+
+        final List<RecordField> fromFields = new ArrayList<>();
+        fromFields.add(new RecordField("id", RecordFieldType.LONG.getDataType()));
+        fromFields.add(new RecordField("name", RecordFieldType.STRING.getDataType()));
+        final RecordSchema fromSchema = new SimpleRecordSchema(fromFields);
+        final DataType fromType = RecordFieldType.RECORD.getRecordDataType(fromSchema);
+
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("created_at", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("id", RecordFieldType.LONG.getDataType()));
+        fields.add(new RecordField("unicode", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("from", fromType));
+        final RecordSchema schema = new SimpleRecordSchema(fields);
+
         try (final InputStream in = new FileInputStream(new File("src/test/resources/json/json-with-unicode.json"));
-            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
+            final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) {
 
             final Object[] firstRecordValues = reader.nextRecord().getValues();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/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 f9849ba..6119d36 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
@@ -27,12 +27,14 @@ import java.nio.file.Paths;
 import java.sql.Date;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.text.DateFormat;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.TimeZone;
 
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.serialization.SimpleRecordSchema;
@@ -60,14 +62,17 @@ public class TestWriteJsonResult {
                 possibleTypes.add(RecordFieldType.INT.getDataType());
                 possibleTypes.add(RecordFieldType.LONG.getDataType());
 
-                fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType(possibleTypes)));
+                fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getChoiceDataType(possibleTypes)));
             } else {
                 fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType()));
             }
         }
         final RecordSchema schema = new SimpleRecordSchema(fields);
 
-        final long time = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS").parse("2017/01/01 17:00:00.000").getTime();
+        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> valueMap = new LinkedHashMap<>();
         valueMap.put("string", "string");
         valueMap.put("boolean", true);

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/avro/datatypes.avsc
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/avro/datatypes.avsc b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/avro/datatypes.avsc
new file mode 100644
index 0000000..cc7f60e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/avro/datatypes.avsc
@@ -0,0 +1,47 @@
+{
+  "namespace": "nifi",
+  "name": "data_types",
+  "type": "record",
+  "fields": [
+    {
+      "name": "string",
+      "type": "string"
+    }, {
+      "name": "int",
+      "type": "int"
+    }, {
+      "name": "long",
+      "type": "long"
+    }, {
+      "name": "double",
+      "type": "double"
+    }, {
+      "name": "float",
+      "type": "float"
+    }, {
+      "name": "boolean",
+      "type": "boolean"
+    }, {
+      "name": "bytes",
+      "type": "bytes"
+    }, {
+      "name": "nullOrLong",
+      "type": [ "null", "long" ]
+    }, {
+      "name": "array",
+      "type" : {
+      	"type": "array",
+      	"items": "int"
+      }
+    }, {
+      "name": "record",
+      "type": {
+	      "type": "record",
+	      "name": "subRecord",
+	      "fields": [
+	      	 { "name": "field1", "type": "string" }
+	      ]
+      }
+    }
+ ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/avro/logical-types.avsc
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/avro/logical-types.avsc b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/avro/logical-types.avsc
new file mode 100644
index 0000000..d8315b2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/avro/logical-types.avsc
@@ -0,0 +1,34 @@
+{
+  "namespace": "nifi",
+  "name": "data_types",
+  "type": "record",
+  "fields": [
+    {
+    	"name" : "timeMillis",
+    	"type": {
+    		"type": "int",
+    		"logicalType": "time-millis"
+    	}
+    }, {
+    	"name" : "timeMicros", "type": {
+    		"type" : "long",
+    		"logicalType" : "time-micros"
+		}
+    }, {
+    	"name" : "timestampMillis", "type": {
+    		"type" : "long",
+    		"logicalType" : "timestamp-millis"
+    	}
+	}, {
+    	"name" : "timestampMicros", "type": {
+			"type" : "long",
+			"logicalType" : "timestamp-micros"
+    	}
+    }, {
+		"name" : "date", "type": {
+			"type" : "int",
+			"logicalType" : "date"
+		}
+	}
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/pom.xml
new file mode 100644
index 0000000..265eb71
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/pom.xml
@@ -0,0 +1,32 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!-- Licensed to the Apache Software Foundation (ASF) under one or more 
+        contributor license agreements. See the NOTICE file distributed with this 
+        work for additional information regarding copyright ownership. The ASF licenses 
+        this file to You under the Apache License, Version 2.0 (the "License"); you 
+        may not use this file except in compliance with the License. You may obtain 
+        a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless 
+        required by applicable law or agreed to in writing, software distributed 
+        under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES 
+        OR CONDITIONS OF ANY KIND, either express or implied. See the License for 
+        the specific language governing permissions and limitations under the License. -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-standard-services</artifactId>
+        <version>1.2.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-schema-registry-service-api</artifactId>
+    <packaging>jar</packaging>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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
new file mode 100644
index 0000000..68c2461
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java
@@ -0,0 +1,53 @@
+/*
+ * 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.schemaregistry.services;
+
+import java.util.Map;
+
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+/**
+ * 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";
+
+
+    /**
+     * 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.
+     */
+    String retrieveSchemaText(String schemaName);
+
+    /**
+     * 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.
+     */
+    String retrieveSchemaText(String schemaName, Map<String, String> attributes);
+
+
+    RecordSchema retrieveSchema(String schemaName);
+
+
+    RecordSchema retrieveSchema(String schemaName, Map<String, String> attributes);
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml
index eae3515..5cee52e 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml
@@ -58,6 +58,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-schema-registry-service-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-record-serialization-service-api</artifactId>
             <scope>compile</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-standard-services/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/pom.xml
index 3948a1b..4fac7d2 100644
--- a/nifi-nar-bundles/nifi-standard-services/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-services/pom.xml
@@ -35,6 +35,7 @@
         <module>nifi-dbcp-service-bundle</module>
         <module>nifi-hbase-client-service-api</module>
         <module>nifi-hbase_1_1_2-client-service-bundle</module>
+        <module>nifi-schema-registry-service-api</module>
         <module>nifi-record-serialization-service-api</module>
         <module>nifi-record-serialization-services-bundle</module>
     </modules>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 057832b..0173b04 100644
--- a/pom.xml
+++ b/pom.xml
@@ -681,7 +681,7 @@ language governing permissions and limitations under the License. -->
             <dependency>
                 <groupId>org.apache.avro</groupId>
                 <artifactId>avro</artifactId>
-                <version>1.7.7</version>
+                <version>1.8.1</version>
             </dependency>
             <dependency>
                 <groupId>com.sun.jersey</groupId>
@@ -921,6 +921,11 @@ language governing permissions and limitations under the License. -->
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-registry-service</artifactId>
+                <version>1.2.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-resources</artifactId>
                 <version>1.2.0-SNAPSHOT</version>
                 <classifier>resources</classifier>
@@ -971,6 +976,11 @@ language governing permissions and limitations under the License. -->
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-schema-registry-service-api</artifactId>
+                <version>1.2.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-distributed-cache-services-nar</artifactId>
                 <version>1.2.0-SNAPSHOT</version>
                 <type>nar</type>


[07/19] nifi git commit: NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Servic

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500.csv
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500.csv b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500.csv
deleted file mode 100644
index 61ce4bd..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500.csv
+++ /dev/null
@@ -1 +0,0 @@
-FIRST_NAME:string,LAST_NAME,COMPANY_NAME,ADDRESS,CITY,COUNTY,STATE,zip,phone1,phone2,email,web
"James","Butt","Benton, John B Jr","6649 N Blue Gum St","New Orleans","Orleans","LA",70116,"504-621-8927","504-845-1427","jbutt@gmail.com","http://www.bentonjohnbjr.com"
"Josephine","Darakjy","Chanay, Jeffrey A Esq","4 B Blue Ridge Blvd","Brighton","Livingston","MI",48116,"810-292-9388","810-374-9840","josephine_darakjy@darakjy.org","http://www.chanayjeffreyaesq.com"
"Art","Venere","Chemel, James L Cpa","8 W Cerritos Ave #54","Bridgeport","Gloucester","NJ","08014","856-636-8749","856-264-4130","art@venere.org","http://www.chemeljameslcpa.com"
"Lenna","Paprocki","Feltz Printing Service","639 Main St","Anchorage","Anchorage","AK",99501,"907-385-4412","907-921-2010","lpaprocki@hotmail.com","http://www.feltzprintingservice.com"
"Donette","Foller","Printing Dimensions","34 Center St","Hamilton","Butler","OH",45011,"513-570-1893","513-549-4561","donette.foller@cox.net","http://www.printingdimens
 ions.com"
"Simona","Morasca","Chapman, Ross E Esq","3 Mcauley Dr","Ashland","Ashland","OH",44805,"419-503-2484","419-800-6759","simona@morasca.com","http://www.chapmanrosseesq.com"
"Mitsue","Tollner","Morlong Associates","7 Eads St","Chicago","Cook","IL",60632,"773-573-6914","773-924-8565","mitsue_tollner@yahoo.com","http://www.morlongassociates.com"
"Leota","Dilliard","Commercial Press","7 W Jackson Blvd","San Jose","Santa Clara","CA",95111,"408-752-3500","408-813-1105","leota@hotmail.com","http://www.commercialpress.com"
"Sage","Wieser","Truhlar And Truhlar Attys","5 Boston Ave #88","Sioux Falls","Minnehaha","SD",57105,"605-414-2147","605-794-4895","sage_wieser@cox.net","http://www.truhlarandtruhlarattys.com"
"Kris","Marrier","King, Christopher A Esq","228 Runamuck Pl #2808","Baltimore","Baltimore City","MD",21224,"410-655-8723","410-804-4694","kris@gmail.com","http://www.kingchristopheraesq.com"
"Minna","Amigon","Dorl, James J Esq","2371 Jerrold Ave","Kulpsville","Montgomery","PA
 ",19443,"215-874-1229","215-422-8694","minna_amigon@yahoo.com","http://www.dorljamesjesq.com"
"Abel","Maclead","Rangoni Of Florence","37275 St  Rt 17m M","Middle Island","Suffolk","NY",11953,"631-335-3414","631-677-3675","amaclead@gmail.com","http://www.rangoniofflorence.com"
"Kiley","Caldarera","Feiner Bros","25 E 75th St #69","Los Angeles","Los Angeles","CA",90034,"310-498-5651","310-254-3084","kiley.caldarera@aol.com","http://www.feinerbros.com"
"Graciela","Ruta","Buckley Miller & Wright","98 Connecticut Ave Nw","Chagrin Falls","Geauga","OH",44023,"440-780-8425","440-579-7763","gruta@cox.net","http://www.buckleymillerwright.com"
"Cammy","Albares","Rousseaux, Michael Esq","56 E Morehead St","Laredo","Webb","TX",78045,"956-537-6195","956-841-7216","calbares@gmail.com","http://www.rousseauxmichaelesq.com"
"Mattie","Poquette","Century Communications","73 State Road 434 E","Phoenix","Maricopa","AZ",85013,"602-277-4385","602-953-6360","mattie@aol.com","http://www.centurycommunications.
 com"
"Meaghan","Garufi","Bolton, Wilbur Esq","69734 E Carrillo St","Mc Minnville","Warren","TN",37110,"931-313-9635","931-235-7959","meaghan@hotmail.com","http://www.boltonwilburesq.com"
"Gladys","Rim","T M Byxbee Company Pc","322 New Horizon Blvd","Milwaukee","Milwaukee","WI",53207,"414-661-9598","414-377-2880","gladys.rim@rim.org","http://www.tmbyxbeecompanypc.com"
"Yuki","Whobrey","Farmers Insurance Group","1 State Route 27","Taylor","Wayne","MI",48180,"313-288-7937","313-341-4470","yuki_whobrey@aol.com","http://www.farmersinsurancegroup.com"
"Fletcher","Flosi","Post Box Services Plus","394 Manchester Blvd","Rockford","Winnebago","IL",61109,"815-828-2147","815-426-5657","fletcher.flosi@yahoo.com","http://www.postboxservicesplus.com"
"Bette","Nicka","Sport En Art","6 S 33rd St","Aston","Delaware","PA",19014,"610-545-3615","610-492-4643","bette_nicka@cox.net","http://www.sportenart.com"
"Veronika","Inouye","C 4 Network Inc","6 Greenleaf Ave","San Jose","Santa Clara","CA",95111,"408
 -540-1785","408-813-4592","vinouye@aol.com","http://www.cnetworkinc.com"
"Willard","Kolmetz","Ingalls, Donald R Esq","618 W Yakima Ave","Irving","Dallas","TX",75062,"972-303-9197","972-896-4882","willard@hotmail.com","http://www.ingallsdonaldresq.com"
"Maryann","Royster","Franklin, Peter L Esq","74 S Westgate St","Albany","Albany","NY",12204,"518-966-7987","518-448-8982","mroyster@royster.com","http://www.franklinpeterlesq.com"
"Alisha","Slusarski","Wtlz Power 107 Fm","3273 State St","Middlesex","Middlesex","NJ","08846","732-658-3154","732-635-3453","alisha@slusarski.com","http://www.wtlzpowerfm.com"
"Allene","Iturbide","Ledecky, David Esq","1 Central Ave","Stevens Point","Portage","WI",54481,"715-662-6764","715-530-9863","allene_iturbide@cox.net","http://www.ledeckydavidesq.com"
"Chanel","Caudy","Professional Image Inc","86 Nw 66th St #8673","Shawnee","Johnson","KS",66218,"913-388-2079","913-899-1103","chanel.caudy@caudy.org","http://www.professionalimageinc.com"
"Ezekiel","Chui","
 Sider, Donald C Esq","2 Cedar Ave #84","Easton","Talbot","MD",21601,"410-669-1642","410-235-8738","ezekiel@chui.com","http://www.siderdonaldcesq.com"
"Willow","Kusko","U Pull It","90991 Thorburn Ave","New York","New York","NY",10011,"212-582-4976","212-934-5167","wkusko@yahoo.com","http://www.upullit.com"
"Bernardo","Figeroa","Clark, Richard Cpa","386 9th Ave N","Conroe","Montgomery","TX",77301,"936-336-3951","936-597-3614","bfigeroa@aol.com","http://www.clarkrichardcpa.com"
"Ammie","Corrio","Moskowitz, Barry S","74874 Atlantic Ave","Columbus","Franklin","OH",43215,"614-801-9788","614-648-3265","ammie@corrio.com","http://www.moskowitzbarrys.com"
"Francine","Vocelka","Cascade Realty Advisors Inc","366 South Dr","Las Cruces","Dona Ana","NM",88011,"505-977-3911","505-335-5293","francine_vocelka@vocelka.com","http://www.cascaderealtyadvisorsinc.com"
"Ernie","Stenseth","Knwz Newsradio","45 E Liberty St","Ridgefield Park","Bergen","NJ","07660","201-709-6245","201-387-9093","ernie_stenseth
 @aol.com","http://www.knwznewsradio.com"
"Albina","Glick","Giampetro, Anthony D","4 Ralph Ct","Dunellen","Middlesex","NJ","08812","732-924-7882","732-782-6701","albina@glick.com","http://www.giampetroanthonyd.com"
"Alishia","Sergi","Milford Enterprises Inc","2742 Distribution Way","New York","New York","NY",10025,"212-860-1579","212-753-2740","asergi@gmail.com","http://www.milfordenterprisesinc.com"
"Solange","Shinko","Mosocco, Ronald A","426 Wolf St","Metairie","Jefferson","LA",70002,"504-979-9175","504-265-8174","solange@shinko.com","http://www.mosoccoronalda.com"
"Jose","Stockham","Tri State Refueler Co","128 Bransten Rd","New York","New York","NY",10011,"212-675-8570","212-569-4233","jose@yahoo.com","http://www.tristaterefuelerco.com"
"Rozella","Ostrosky","Parkway Company","17 Morena Blvd","Camarillo","Ventura","CA",93012,"805-832-6163","805-609-1531","rozella.ostrosky@ostrosky.com","http://www.parkwaycompany.com"
"Valentine","Gillian","Fbs Business Finance","775 W 17th St","San
  Antonio","Bexar","TX",78204,"210-812-9597","210-300-6244","valentine_gillian@gmail.com","http://www.fbsbusinessfinance.com"
"Kati","Rulapaugh","Eder Assocs Consltng Engrs Pc","6980 Dorsett Rd","Abilene","Dickinson","KS",67410,"785-463-7829","785-219-7724","kati.rulapaugh@hotmail.com","http://www.ederassocsconsltngengrspc.com"
"Youlanda","Schemmer","Tri M Tool Inc","2881 Lewis Rd","Prineville","Crook","OR",97754,"541-548-8197","541-993-2611","youlanda@aol.com","http://www.trimtoolinc.com"
"Dyan","Oldroyd","International Eyelets Inc","7219 Woodfield Rd","Overland Park","Johnson","KS",66204,"913-413-4604","913-645-8918","doldroyd@aol.com","http://www.internationaleyeletsinc.com"
"Roxane","Campain","Rapid Trading Intl","1048 Main St","Fairbanks","Fairbanks North Star","AK",99708,"907-231-4722","907-335-6568","roxane@hotmail.com","http://www.rapidtradingintl.com"
"Lavera","Perin","Abc Enterprises Inc","678 3rd Ave","Miami","Miami-Dade","FL",33196,"305-606-7291","305-995-2078","lperin@pe
 rin.org","http://www.abcenterprisesinc.com"
"Erick","Ferencz","Cindy Turner Associates","20 S Babcock St","Fairbanks","Fairbanks North Star","AK",99712,"907-741-1044","907-227-6777","erick.ferencz@aol.com","http://www.cindyturnerassociates.com"
"Fatima","Saylors","Stanton, James D Esq","2 Lighthouse Ave","Hopkins","Hennepin","MN",55343,"952-768-2416","952-479-2375","fsaylors@saylors.org","http://www.stantonjamesdesq.com"
"Jina","Briddick","Grace Pastries Inc","38938 Park Blvd","Boston","Suffolk","MA","02128","617-399-5124","617-997-5771","jina_briddick@briddick.com","http://www.gracepastriesinc.com"
"Kanisha","Waycott","Schroer, Gene E Esq","5 Tomahawk Dr","Los Angeles","Los Angeles","CA",90006,"323-453-2780","323-315-7314","kanisha_waycott@yahoo.com","http://www.schroergeneeesq.com"
"Emerson","Bowley","Knights Inn","762 S Main St","Madison","Dane","WI",53711,"608-336-7444","608-658-7940","emerson.bowley@bowley.org","http://www.knightsinn.com"
"Blair","Malet","Bollinger Mach Shp & S
 hipyard","209 Decker Dr","Philadelphia","Philadelphia","PA",19132,"215-907-9111","215-794-4519","bmalet@yahoo.com","http://www.bollingermachshpshipyard.com"
"Brock","Bolognia","Orinda News","4486 W O St #1","New York","New York","NY",10003,"212-402-9216","212-617-5063","bbolognia@yahoo.com","http://www.orindanews.com"
"Lorrie","Nestle","Ballard Spahr Andrews","39 S 7th St","Tullahoma","Coffee","TN",37388,"931-875-6644","931-303-6041","lnestle@hotmail.com","http://www.ballardspahrandrews.com"
"Sabra","Uyetake","Lowy Limousine Service","98839 Hawthorne Blvd #6101","Columbia","Richland","SC",29201,"803-925-5213","803-681-3678","sabra@uyetake.org","http://www.lowylimousineservice.com"
"Marjory","Mastella","Vicon Corporation","71 San Mateo Ave","Wayne","Delaware","PA",19087,"610-814-5533","610-379-7125","mmastella@mastella.com","http://www.viconcorporation.com"
"Karl","Klonowski","Rossi, Michael M","76 Brooks St #9","Flemington","Hunterdon","NJ","08822","908-877-6135","908-470-4661","kar
 l_klonowski@yahoo.com","http://www.rossimichaelm.com"
"Tonette","Wenner","Northwest Publishing","4545 Courthouse Rd","Westbury","Nassau","NY",11590,"516-968-6051","516-333-4861","twenner@aol.com","http://www.northwestpublishing.com"
"Amber","Monarrez","Branford Wire & Mfg Co","14288 Foster Ave #4121","Jenkintown","Montgomery","PA",19046,"215-934-8655","215-329-6386","amber_monarrez@monarrez.org","http://www.branfordwiremfgco.com"
"Shenika","Seewald","East Coast Marketing","4 Otis St","Van Nuys","Los Angeles","CA",91405,"818-423-4007","818-749-8650","shenika@gmail.com","http://www.eastcoastmarketing.com"
"Delmy","Ahle","Wye Technologies Inc","65895 S 16th St","Providence","Providence","RI","02909","401-458-2547","401-559-8961","delmy.ahle@hotmail.com","http://www.wyetechnologiesinc.com"
"Deeanna","Juhas","Healy, George W Iv","14302 Pennsylvania Ave","Huntingdon Valley","Montgomery","PA",19006,"215-211-9589","215-417-9563","deeanna_juhas@gmail.com","http://www.healygeorgewiv.com"
"Blo
 ndell","Pugh","Alpenlite Inc","201 Hawk Ct","Providence","Providence","RI","02904","401-960-8259","401-300-8122","bpugh@aol.com","http://www.alpenliteinc.com"
"Jamal","Vanausdal","Hubbard, Bruce Esq","53075 Sw 152nd Ter #615","Monroe Township","Middlesex","NJ","08831","732-234-1546","732-904-2931","jamal@vanausdal.org","http://www.hubbardbruceesq.com"
"Cecily","Hollack","Arthur A Oliver & Son Inc","59 N Groesbeck Hwy","Austin","Travis","TX",78731,"512-486-3817","512-861-3814","cecily@hollack.org","http://www.arthuraoliversoninc.com"
"Carmelina","Lindall","George Jessop Carter Jewelers","2664 Lewis Rd","Littleton","Douglas","CO",80126,"303-724-7371","303-874-5160","carmelina_lindall@lindall.com","http://www.georgejessopcarterjewelers.com"
"Maurine","Yglesias","Schultz, Thomas C Md","59 Shady Ln #53","Milwaukee","Milwaukee","WI",53214,"414-748-1374","414-573-7719","maurine_yglesias@yglesias.com","http://www.schultzthomascmd.com"
"Tawna","Buvens","H H H Enterprises Inc","3305 Nabell Av
 e #679","New York","New York","NY",10009,"212-674-9610","212-462-9157","tawna@gmail.com","http://www.hhhenterprisesinc.com"
"Penney","Weight","Hawaiian King Hotel","18 Fountain St","Anchorage","Anchorage","AK",99515,"907-797-9628","907-873-2882","penney_weight@aol.com","http://www.hawaiiankinghotel.com"
"Elly","Morocco","Killion Industries","7 W 32nd St","Erie","Erie","PA",16502,"814-393-5571","814-420-3553","elly_morocco@gmail.com","http://www.killionindustries.com"
"Ilene","Eroman","Robinson, William J Esq","2853 S Central Expy","Glen Burnie","Anne Arundel","MD",21061,"410-914-9018","410-937-4543","ilene.eroman@hotmail.com","http://www.robinsonwilliamjesq.com"
"Vallie","Mondella","Private Properties","74 W College St","Boise","Ada","ID",83707,"208-862-5339","208-737-8439","vmondella@mondella.com","http://www.privateproperties.com"
"Kallie","Blackwood","Rowley Schlimgen Inc","701 S Harrison Rd","San Francisco","San Francisco","CA",94104,"415-315-2761","415-604-7609","kallie.blackwo
 od@gmail.com","http://www.rowleyschlimgeninc.com"
"Johnetta","Abdallah","Forging Specialties","1088 Pinehurst St","Chapel Hill","Orange","NC",27514,"919-225-9345","919-715-3791","johnetta_abdallah@aol.com","http://www.forgingspecialties.com"
"Bobbye","Rhym","Smits, Patricia Garity","30 W 80th St #1995","San Carlos","San Mateo","CA",94070,"650-528-5783","650-811-9032","brhym@rhym.com","http://www.smitspatriciagarity.com"
"Micaela","Rhymes","H Lee Leonard Attorney At Law","20932 Hedley St","Concord","Contra Costa","CA",94520,"925-647-3298","925-522-7798","micaela_rhymes@gmail.com","http://www.hleeleonardattorneyatlaw.com"
"Tamar","Hoogland","A K Construction Co","2737 Pistorio Rd #9230","London","Madison","OH",43140,"740-343-8575","740-526-5410","tamar@hotmail.com","http://www.akconstructionco.com"
"Moon","Parlato","Ambelang, Jessica M Md","74989 Brandon St","Wellsville","Allegany","NY",14895,"585-866-8313","585-498-4278","moon@yahoo.com","http://www.ambelangjessicammd.com"
"Laurel","
 Reitler","Q A Service","6 Kains Ave","Baltimore","Baltimore City","MD",21215,"410-520-4832","410-957-6903","laurel_reitler@reitler.com","http://www.qaservice.com"
"Delisa","Crupi","Wood & Whitacre Contractors","47565 W Grand Ave","Newark","Essex","NJ","07105","973-354-2040","973-847-9611","delisa.crupi@crupi.com","http://www.woodwhitacrecontractors.com"
"Viva","Toelkes","Mark Iv Press Ltd","4284 Dorigo Ln","Chicago","Cook","IL",60647,"773-446-5569","773-352-3437","viva.toelkes@gmail.com","http://www.markivpressltd.com"
"Elza","Lipke","Museum Of Science & Industry","6794 Lake Dr E","Newark","Essex","NJ","07104","973-927-3447","973-796-3667","elza@yahoo.com","http://www.museumofscienceindustry.com"
"Devorah","Chickering","Garrison Ind","31 Douglas Blvd #950","Clovis","Curry","NM",88101,"505-975-8559","505-950-1763","devorah@hotmail.com","http://www.garrisonind.com"
"Timothy","Mulqueen","Saronix Nymph Products","44 W 4th St","Staten Island","Richmond","NY",10309,"718-332-6527","718-654
 -7063","timothy_mulqueen@mulqueen.org","http://www.saronixnymphproducts.com"
"Arlette","Honeywell","Smc Inc","11279 Loytan St","Jacksonville","Duval","FL",32254,"904-775-4480","904-514-9918","ahoneywell@honeywell.com","http://www.smcinc.com"
"Dominque","Dickerson","E A I Electronic Assocs Inc","69 Marquette Ave","Hayward","Alameda","CA",94545,"510-993-3758","510-901-7640","dominque.dickerson@dickerson.org","http://www.eaielectronicassocsinc.com"
"Lettie","Isenhower","Conte, Christopher A Esq","70 W Main St","Beachwood","Cuyahoga","OH",44122,"216-657-7668","216-733-8494","lettie_isenhower@yahoo.com","http://www.contechristopheraesq.com"
"Myra","Munns","Anker Law Office","461 Prospect Pl #316","Euless","Tarrant","TX",76040,"817-914-7518","817-451-3518","mmunns@cox.net","http://www.ankerlawoffice.com"
"Stephaine","Barfield","Beutelschies & Company","47154 Whipple Ave Nw","Gardena","Los Angeles","CA",90247,"310-774-7643","310-968-1219","stephaine@barfield.com","http://www.beutelschiesco
 mpany.com"
"Lai","Gato","Fligg, Kenneth I Jr","37 Alabama Ave","Evanston","Cook","IL",60201,"847-728-7286","847-957-4614","lai.gato@gato.org","http://www.fliggkennethijr.com"
"Stephen","Emigh","Sharp, J Daniel Esq","3777 E Richmond St #900","Akron","Summit","OH",44302,"330-537-5358","330-700-2312","stephen_emigh@hotmail.com","http://www.sharpjdanielesq.com"
"Tyra","Shields","Assink, Anne H Esq","3 Fort Worth Ave","Philadelphia","Philadelphia","PA",19106,"215-255-1641","215-228-8264","tshields@gmail.com","http://www.assinkannehesq.com"
"Tammara","Wardrip","Jewel My Shop Inc","4800 Black Horse Pike","Burlingame","San Mateo","CA",94010,"650-803-1936","650-216-5075","twardrip@cox.net","http://www.jewelmyshopinc.com"
"Cory","Gibes","Chinese Translation Resources","83649 W Belmont Ave","San Gabriel","Los Angeles","CA",91776,"626-572-1096","626-696-2777","cory.gibes@gmail.com","http://www.chinesetranslationresources.com"
"Danica","Bruschke","Stevens, Charles T","840 15th Ave","Waco","McLen
 nan","TX",76708,"254-782-8569","254-205-1422","danica_bruschke@gmail.com","http://www.stevenscharlest.com"
"Wilda","Giguere","Mclaughlin, Luther W Cpa","1747 Calle Amanecer #2","Anchorage","Anchorage","AK",99501,"907-870-5536","907-914-9482","wilda@cox.net","http://www.mclaughlinlutherwcpa.com"
"Elvera","Benimadho","Tree Musketeers","99385 Charity St #840","San Jose","Santa Clara","CA",95110,"408-703-8505","408-440-8447","elvera.benimadho@cox.net","http://www.treemusketeers.com"
"Carma","Vanheusen","Springfield Div Oh Edison Co","68556 Central Hwy","San Leandro","Alameda","CA",94577,"510-503-7169","510-452-4835","carma@cox.net","http://www.springfielddivohedisonco.com"
"Malinda","Hochard","Logan Memorial Hospital","55 Riverside Ave","Indianapolis","Marion","IN",46202,"317-722-5066","317-472-2412","malinda.hochard@yahoo.com","http://www.loganmemorialhospital.com"
"Natalie","Fern","Kelly, Charles G Esq","7140 University Ave","Rock Springs","Sweetwater","WY",82901,"307-704-8713","307-2
 79-3793","natalie.fern@hotmail.com","http://www.kellycharlesgesq.com"
"Lisha","Centini","Industrial Paper Shredders Inc","64 5th Ave #1153","Mc Lean","Fairfax","VA",22102,"703-235-3937","703-475-7568","lisha@centini.org","http://www.industrialpapershreddersinc.com"
"Arlene","Klusman","Beck Horizon Builders","3 Secor Rd","New Orleans","Orleans","LA",70112,"504-710-5840","504-946-1807","arlene_klusman@gmail.com","http://www.beckhorizonbuilders.com"
"Alease","Buemi","Porto Cayo At Hawks Cay","4 Webbs Chapel Rd","Boulder","Boulder","CO",80303,"303-301-4946","303-521-9860","alease@buemi.com","http://www.portocayoathawkscay.com"
"Louisa","Cronauer","Pacific Grove Museum Ntrl Hist","524 Louisiana Ave Nw","San Leandro","Alameda","CA",94577,"510-828-7047","510-472-7758","louisa@cronauer.com","http://www.pacificgrovemuseumntrlhist.com"
"Angella","Cetta","Bender & Hatley Pc","185 Blackstone Bldge","Honolulu","Honolulu","HI",96817,"808-892-7943","808-475-2310","angella.cetta@hotmail.com","http:
 //www.benderhatleypc.com"
"Cyndy","Goldammer","Di Cristina J & Son","170 Wyoming Ave","Burnsville","Dakota","MN",55337,"952-334-9408","952-938-9457","cgoldammer@cox.net","http://www.dicristinajson.com"
"Rosio","Cork","Green Goddess","4 10th St W","High Point","Guilford","NC",27263,"336-243-5659","336-497-4407","rosio.cork@gmail.com","http://www.greengoddess.com"
"Celeste","Korando","American Arts & Graphics","7 W Pinhook Rd","Lynbrook","Nassau","NY",11563,"516-509-2347","516-365-7266","ckorando@hotmail.com","http://www.americanartsgraphics.com"
"Twana","Felger","Opryland Hotel","1 Commerce Way","Portland","Washington","OR",97224,"503-939-3153","503-909-7167","twana.felger@felger.org","http://www.oprylandhotel.com"
"Estrella","Samu","Marking Devices Pubg Co","64 Lakeview Ave","Beloit","Rock","WI",53511,"608-976-7199","608-942-8836","estrella@aol.com","http://www.markingdevicespubgco.com"
"Donte","Kines","W Tc Industries Inc","3 Aspen St","Worcester","Worcester","MA","01602","508-429-
 8576","508-843-1426","dkines@hotmail.com","http://www.wtcindustriesinc.com"
"Tiffiny","Steffensmeier","Whitehall Robbins Labs Divsn","32860 Sierra Rd","Miami","Miami-Dade","FL",33133,"305-385-9695","305-304-6573","tiffiny_steffensmeier@cox.net","http://www.whitehallrobbinslabsdivsn.com"
"Edna","Miceli","Sampler","555 Main St","Erie","Erie","PA",16502,"814-460-2655","814-299-2877","emiceli@miceli.org","http://www.sampler.com"
"Sue","Kownacki","Juno Chefs Incorporated","2 Se 3rd Ave","Mesquite","Dallas","TX",75149,"972-666-3413","972-742-4000","sue@aol.com","http://www.junochefsincorporated.com"
"Jesusa","Shin","Carroccio, A Thomas Esq","2239 Shawnee Mission Pky","Tullahoma","Coffee","TN",37388,"931-273-8709","931-739-1551","jshin@shin.com","http://www.carroccioathomasesq.com"
"Rolland","Francescon","Stanley, Richard L Esq","2726 Charcot Ave","Paterson","Passaic","NJ","07501","973-649-2922","973-284-4048","rolland@cox.net","http://www.stanleyrichardlesq.com"
"Pamella","Schmierer","K C
 s Cstm Mouldings Windows","5161 Dorsett Rd","Homestead","Miami-Dade","FL",33030,"305-420-8970","305-575-8481","pamella.schmierer@schmierer.org","http://www.kcscstmmouldingswindows.com"
"Glory","Kulzer","Comfort Inn","55892 Jacksonville Rd","Owings Mills","Baltimore","MD",21117,"410-224-9462","410-916-8015","gkulzer@kulzer.org","http://www.comfortinn.com"
"Shawna","Palaspas","Windsor, James L Esq","5 N Cleveland Massillon Rd","Thousand Oaks","Ventura","CA",91362,"805-275-3566","805-638-6617","shawna_palaspas@palaspas.org","http://www.windsorjameslesq.com"
"Brandon","Callaro","Jackson Shields Yeiser","7 Benton Dr","Honolulu","Honolulu","HI",96819,"808-215-6832","808-240-5168","brandon_callaro@hotmail.com","http://www.jacksonshieldsyeiser.com"
"Scarlet","Cartan","Box, J Calvin Esq","9390 S Howell Ave","Albany","Dougherty","GA",31701,"229-735-3378","229-365-9658","scarlet.cartan@yahoo.com","http://www.boxjcalvinesq.com"
"Oretha","Menter","Custom Engineering Inc","8 County Center Dr #647
 ","Boston","Suffolk","MA","02210","617-418-5043","617-697-6024","oretha_menter@yahoo.com","http://www.customengineeringinc.com"
"Ty","Smith","Bresler Eitel Framg Gllry Ltd","4646 Kaahumanu St","Hackensack","Bergen","NJ","07601","201-672-1553","201-995-3149","tsmith@aol.com","http://www.breslereitelframggllryltd.com"
"Xuan","Rochin","Carol, Drake Sparks Esq","2 Monroe St","San Mateo","San Mateo","CA",94403,"650-933-5072","650-247-2625","xuan@gmail.com","http://www.caroldrakesparksesq.com"
"Lindsey","Dilello","Biltmore Investors Bank","52777 Leaders Heights Rd","Ontario","San Bernardino","CA",91761,"909-639-9887","909-589-1693","lindsey.dilello@hotmail.com","http://www.biltmoreinvestorsbank.com"
"Devora","Perez","Desco Equipment Corp","72868 Blackington Ave","Oakland","Alameda","CA",94606,"510-955-3016","510-755-9274","devora_perez@perez.org","http://www.descoequipmentcorp.com"
"Herman","Demesa","Merlin Electric Co","9 Norristown Rd","Troy","Rensselaer","NY",12180,"518-497-2940","518-
 931-7852","hdemesa@cox.net","http://www.merlinelectricco.com"
"Rory","Papasergi","Bailey Cntl Co Div Babcock","83 County Road 437 #8581","Clarks Summit","Lackawanna","PA",18411,"570-867-7489","570-469-8401","rpapasergi@cox.net","http://www.baileycntlcodivbabcock.com"
"Talia","Riopelle","Ford Brothers Wholesale Inc","1 N Harlem Ave #9","Orange","Essex","NJ","07050","973-245-2133","973-818-9788","talia_riopelle@aol.com","http://www.fordbrotherswholesaleinc.com"
"Van","Shire","Cambridge Inn","90131 J St","Pittstown","Hunterdon","NJ","08867","908-409-2890","908-448-1209","van.shire@shire.com","http://www.cambridgeinn.com"
"Lucina","Lary","Matricciani, Albert J Jr","8597 W National Ave","Cocoa","Brevard","FL",32922,"321-749-4981","321-632-4668","lucina_lary@cox.net","http://www.matriccianialbertjjr.com"
"Bok","Isaacs","Nelson Hawaiian Ltd","6 Gilson St","Bronx","Bronx","NY",10468,"718-809-3762","718-478-8568","bok.isaacs@aol.com","http://www.nelsonhawaiianltd.com"
"Rolande","Spickerman",
 "Neland Travel Agency","65 W Maple Ave","Pearl City","Honolulu","HI",96782,"808-315-3077","808-526-5863","rolande.spickerman@spickerman.com","http://www.nelandtravelagency.com"
"Howard","Paulas","Asendorf, J Alan Esq","866 34th Ave","Denver","Denver","CO",80231,"303-623-4241","303-692-3118","hpaulas@gmail.com","http://www.asendorfjalanesq.com"
"Kimbery","Madarang","Silberman, Arthur L Esq","798 Lund Farm Way","Rockaway","Morris","NJ","07866","973-310-1634","973-225-6259","kimbery_madarang@cox.net","http://www.silbermanarthurlesq.com"
"Thurman","Manno","Honey Bee Breeding Genetics &","9387 Charcot Ave","Absecon","Atlantic","NJ","08201","609-524-3586","609-234-8376","thurman.manno@yahoo.com","http://www.honeybeebreedinggenetics.com"
"Becky","Mirafuentes","Wells Kravitz Schnitzer","30553 Washington Rd","Plainfield","Union","NJ","07062","908-877-8409","908-426-8272","becky.mirafuentes@mirafuentes.com","http://www.wellskravitzschnitzer.com"
"Beatriz","Corrington","Prohab Rehabilitation S
 ervs","481 W Lemon St","Middleboro","Plymouth","MA","02346","508-584-4279","508-315-3867","beatriz@yahoo.com","http://www.prohabrehabilitationservs.com"
"Marti","Maybury","Eldridge, Kristin K Esq","4 Warehouse Point Rd #7","Chicago","Cook","IL",60638,"773-775-4522","773-539-1058","marti.maybury@yahoo.com","http://www.eldridgekristinkesq.com"
"Nieves","Gotter","Vlahos, John J Esq","4940 Pulaski Park Dr","Portland","Multnomah","OR",97202,"503-527-5274","503-455-3094","nieves_gotter@gmail.com","http://www.vlahosjohnjesq.com"
"Leatha","Hagele","Ninas Indian Grs & Videos","627 Walford Ave","Dallas","Dallas","TX",75227,"214-339-1809","214-225-5850","lhagele@cox.net","http://www.ninasindiangrsvideos.com"
"Valentin","Klimek","Schmid, Gayanne K Esq","137 Pioneer Way","Chicago","Cook","IL",60604,"312-303-5453","312-512-2338","vklimek@klimek.org","http://www.schmidgayannekesq.com"
"Melissa","Wiklund","Moapa Valley Federal Credit Un","61 13 Stoneridge #835","Findlay","Hancock","OH",45840,"419-9
 39-3613","419-254-4591","melissa@cox.net","http://www.moapavalleyfederalcreditun.com"
"Sheridan","Zane","Kentucky Tennessee Clay Co","2409 Alabama Rd","Riverside","Riverside","CA",92501,"951-645-3605","951-248-6822","sheridan.zane@zane.com","http://www.kentuckytennesseeclayco.com"
"Bulah","Padilla","Admiral Party Rentals & Sales","8927 Vandever Ave","Waco","McLennan","TX",76707,"254-463-4368","254-816-8417","bulah_padilla@hotmail.com","http://www.admiralpartyrentalssales.com"
"Audra","Kohnert","Nelson, Karolyn King Esq","134 Lewis Rd","Nashville","Davidson","TN",37211,"615-406-7854","615-448-9249","audra@kohnert.com","http://www.nelsonkarolynkingesq.com"
"Daren","Weirather","Panasystems","9 N College Ave #3","Milwaukee","Milwaukee","WI",53216,"414-959-2540","414-838-3151","dweirather@aol.com","http://www.panasystems.com"
"Fernanda","Jillson","Shank, Edward L Esq","60480 Old Us Highway 51","Preston","Caroline","MD",21655,"410-387-5260","410-724-6472","fjillson@aol.com","http://www.sh
 ankedwardlesq.com"
"Gearldine","Gellinger","Megibow & Edwards","4 Bloomfield Ave","Irving","Dallas","TX",75061,"972-934-6914","972-821-7118","gearldine_gellinger@gellinger.com","http://www.megibowedwards.com"
"Chau","Kitzman","Benoff, Edward Esq","429 Tiger Ln","Beverly Hills","Los Angeles","CA",90212,"310-560-8022","310-969-7230","chau@gmail.com","http://www.benoffedwardesq.com"
"Theola","Frey","Woodbridge Free Public Library","54169 N Main St","Massapequa","Nassau","NY",11758,"516-948-5768","516-357-3362","theola_frey@frey.com","http://www.woodbridgefreepubliclibrary.com"
"Cheryl","Haroldson","New York Life John Thune","92 Main St","Atlantic City","Atlantic","NJ","08401","609-518-7697","609-263-9243","cheryl@haroldson.org","http://www.newyorklifejohnthune.com"
"Laticia","Merced","Alinabal Inc","72 Mannix Dr","Cincinnati","Hamilton","OH",45203,"513-508-7371","513-418-1566","lmerced@gmail.com","http://www.alinabalinc.com"
"Carissa","Batman","Poletto, Kim David Esq","12270 Caton Cent
 er Dr","Eugene","Lane","OR",97401,"541-326-4074","541-801-5717","carissa.batman@yahoo.com","http://www.polettokimdavidesq.com"
"Lezlie","Craghead","Chang, Carolyn Esq","749 W 18th St #45","Smithfield","Johnston","NC",27577,"919-533-3762","919-885-2453","lezlie.craghead@craghead.org","http://www.changcarolynesq.com"
"Ozell","Shealy","Silver Bros Inc","8 Industry Ln","New York","New York","NY",10002,"212-332-8435","212-880-8865","oshealy@hotmail.com","http://www.silverbrosinc.com"
"Arminda","Parvis","Newtec Inc","1 Huntwood Ave","Phoenix","Maricopa","AZ",85017,"602-906-9419","602-277-3025","arminda@parvis.com","http://www.newtecinc.com"
"Reita","Leto","Creative Business Systems","55262 N French Rd","Indianapolis","Marion","IN",46240,"317-234-1135","317-787-5514","reita.leto@gmail.com","http://www.creativebusinesssystems.com"
"Yolando","Luczki","Dal Tile Corporation","422 E 21st St","Syracuse","Onondaga","NY",13214,"315-304-4759","315-640-6357","yolando@cox.net","http://www.daltilecorp
 oration.com"
"Lizette","Stem","Edward S Katz","501 N 19th Ave","Cherry Hill","Camden","NJ","08002","856-487-5412","856-702-3676","lizette.stem@aol.com","http://www.edwardskatz.com"
"Gregoria","Pawlowicz","Oh My Goodknits Inc","455 N Main Ave","Garden City","Nassau","NY",11530,"516-212-1915","516-376-4230","gpawlowicz@yahoo.com","http://www.ohmygoodknitsinc.com"
"Carin","Deleo","Redeker, Debbie","1844 Southern Blvd","Little Rock","Pulaski","AR",72202,"501-308-1040","501-409-6072","cdeleo@deleo.com","http://www.redekerdebbie.com"
"Chantell","Maynerich","Desert Sands Motel","2023 Greg St","Saint Paul","Ramsey","MN",55101,"651-591-2583","651-776-9688","chantell@yahoo.com","http://www.desertsandsmotel.com"
"Dierdre","Yum","Cummins Southern Plains Inc","63381 Jenks Ave","Philadelphia","Philadelphia","PA",19134,"215-325-3042","215-346-4666","dyum@yahoo.com","http://www.cumminssouthernplainsinc.com"
"Larae","Gudroe","Lehigh Furn Divsn Lehigh","6651 Municipal Rd","Houma","Terrebonne","LA",70
 360,"985-890-7262","985-261-5783","larae_gudroe@gmail.com","http://www.lehighfurndivsnlehigh.com"
"Latrice","Tolfree","United Van Lines Agent","81 Norris Ave #525","Ronkonkoma","Suffolk","NY",11779,"631-957-7624","631-998-2102","latrice.tolfree@hotmail.com","http://www.unitedvanlinesagent.com"
"Kerry","Theodorov","Capitol Reporters","6916 W Main St","Sacramento","Sacramento","CA",95827,"916-591-3277","916-770-7448","kerry.theodorov@gmail.com","http://www.capitolreporters.com"
"Dorthy","Hidvegi","Kwik Kopy Printing","9635 S Main St","Boise","Ada","ID",83704,"208-649-2373","208-690-3315","dhidvegi@yahoo.com","http://www.kwikkopyprinting.com"
"Fannie","Lungren","Centro Inc","17 Us Highway 111","Round Rock","Williamson","TX",78664,"512-587-5746","512-528-9933","fannie.lungren@yahoo.com","http://www.centroinc.com"
"Evangelina","Radde","Campbell, Jan Esq","992 Civic Center Dr","Philadelphia","Philadelphia","PA",19123,"215-964-3284","215-417-5612","evangelina@aol.com","http://www.campbellj
 anesq.com"
"Novella","Degroot","Evans, C Kelly Esq","303 N Radcliffe St","Hilo","Hawaii","HI",96720,"808-477-4775","808-746-1865","novella_degroot@degroot.org","http://www.evansckellyesq.com"
"Clay","Hoa","Scat Enterprises","73 Saint Ann St #86","Reno","Washoe","NV",89502,"775-501-8109","775-848-9135","choa@hoa.org","http://www.scatenterprises.com"
"Jennifer","Fallick","Nagle, Daniel J Esq","44 58th St","Wheeling","Cook","IL",60090,"847-979-9545","847-800-3054","jfallick@yahoo.com","http://www.nagledanieljesq.com"
"Irma","Wolfgramm","Serendiquity Bed & Breakfast","9745 W Main St","Randolph","Morris","NJ","07869","973-545-7355","973-868-8660","irma.wolfgramm@hotmail.com","http://www.serendiquitybedbreakfast.com"
"Eun","Coody","Ray Carolyne Realty","84 Bloomfield Ave","Spartanburg","Spartanburg","SC",29301,"864-256-3620","864-594-4578","eun@yahoo.com","http://www.raycarolynerealty.com"
"Sylvia","Cousey","Berg, Charles E","287 Youngstown Warren Rd","Hampstead","Carroll","MD",21074,"410
 -209-9545","410-863-8263","sylvia_cousey@cousey.org","http://www.bergcharlese.com"
"Nana","Wrinkles","Ray, Milbern D","6 Van Buren St","Mount Vernon","Westchester","NY",10553,"914-855-2115","914-796-3775","nana@aol.com","http://www.raymilbernd.com"
"Layla","Springe","Chadds Ford Winery","229 N Forty Driv","New York","New York","NY",10011,"212-260-3151","212-253-7448","layla.springe@cox.net","http://www.chaddsfordwinery.com"
"Joesph","Degonia","A R Packaging","2887 Knowlton St #5435","Berkeley","Alameda","CA",94710,"510-677-9785","510-942-5916","joesph_degonia@degonia.org","http://www.arpackaging.com"
"Annabelle","Boord","Corn Popper","523 Marquette Ave","Concord","Middlesex","MA","01742","978-697-6263","978-289-7717","annabelle.boord@cox.net","http://www.cornpopper.com"
"Stephaine","Vinning","Birite Foodservice Distr","3717 Hamann Industrial Pky","San Francisco","San Francisco","CA",94104,"415-767-6596","415-712-9530","stephaine@cox.net","http://www.biritefoodservicedistr.com"
"Neli
 da","Sawchuk","Anchorage Museum Of Hist & Art","3 State Route 35 S","Paramus","Bergen","NJ","07652","201-971-1638","201-247-8925","nelida@gmail.com","http://www.anchoragemuseumofhistart.com"
"Marguerita","Hiatt","Haber, George D Md","82 N Highway 67","Oakley","Contra Costa","CA",94561,"925-634-7158","925-541-8521","marguerita.hiatt@gmail.com","http://www.habergeorgedmd.com"
"Carmela","Cookey","Royal Pontiac Olds Inc","9 Murfreesboro Rd","Chicago","Cook","IL",60623,"773-494-4195","773-297-9391","ccookey@cookey.org","http://www.royalpontiacoldsinc.com"
"Junita","Brideau","Leonards Antiques Inc","6 S Broadway St","Cedar Grove","Essex","NJ","07009","973-943-3423","973-582-5469","jbrideau@aol.com","http://www.leonardsantiquesinc.com"
"Claribel","Varriano","Meca","6 Harry L Dr #6327","Perrysburg","Wood","OH",43551,"419-544-4900","419-573-2033","claribel_varriano@cox.net","http://www.meca.com"
"Benton","Skursky","Nercon Engineering & Mfg Inc","47939 Porter Ave","Gardena","Los Angeles","CA"
 ,90248,"310-579-2907","310-694-8466","benton.skursky@aol.com","http://www.nerconengineeringmfginc.com"
"Hillary","Skulski","Replica I","9 Wales Rd Ne #914","Homosassa","Citrus","FL",34448,"352-242-2570","352-990-5946","hillary.skulski@aol.com","http://www.replicai.com"
"Merilyn","Bayless","20 20 Printing Inc","195 13n N","Santa Clara","Santa Clara","CA",95054,"408-758-5015","408-346-2180","merilyn_bayless@cox.net","http://www.printinginc.com"
"Teri","Ennaco","Publishers Group West","99 Tank Farm Rd","Hazleton","Luzerne","PA",18201,"570-889-5187","570-355-1665","tennaco@gmail.com","http://www.publishersgroupwest.com"
"Merlyn","Lawler","Nischwitz, Jeffrey L Esq","4671 Alemany Blvd","Jersey City","Hudson","NJ","07304","201-588-7810","201-858-9960","merlyn_lawler@hotmail.com","http://www.nischwitzjeffreylesq.com"
"Georgene","Montezuma","Payne Blades & Wellborn Pa","98 University Dr","San Ramon","Contra Costa","CA",94583,"925-615-5185","925-943-3449","gmontezuma@cox.net","http://www.payn
 ebladeswellbornpa.com"
"Jettie","Mconnell","Coldwell Bnkr Wright Real Est","50 E Wacker Dr","Bridgewater","Somerset","NJ","08807","908-802-3564","908-602-5258","jmconnell@hotmail.com","http://www.coldwellbnkrwrightrealest.com"
"Lemuel","Latzke","Computer Repair Service","70 Euclid Ave #722","Bohemia","Suffolk","NY",11716,"631-748-6479","631-291-4976","lemuel.latzke@gmail.com","http://www.computerrepairservice.com"
"Melodie","Knipp","Fleetwood Building Block Inc","326 E Main St #6496","Thousand Oaks","Ventura","CA",91362,"805-690-1682","805-810-8964","mknipp@gmail.com","http://www.fleetwoodbuildingblockinc.com"
"Candida","Corbley","Colts Neck Medical Assocs Inc","406 Main St","Somerville","Somerset","NJ","08876","908-275-8357","908-943-6103","candida_corbley@hotmail.com","http://www.coltsneckmedicalassocsinc.com"
"Karan","Karpin","New England Taxidermy","3 Elmwood Dr","Beaverton","Washington","OR",97005,"503-940-8327","503-707-5812","karan_karpin@gmail.com","http://www.newenglandtaxi
 dermy.com"
"Andra","Scheyer","Ludcke, George O Esq","9 Church St","Salem","Marion","OR",97302,"503-516-2189","503-950-3068","andra@gmail.com","http://www.ludckegeorgeoesq.com"
"Felicidad","Poullion","Mccorkle, Tom S Esq","9939 N 14th St","Riverton","Burlington","NJ","08077","856-305-9731","856-828-6021","fpoullion@poullion.com","http://www.mccorkletomsesq.com"
"Belen","Strassner","Eagle Software Inc","5384 Southwyck Blvd","Douglasville","Douglas","GA",30135,"770-507-8791","770-802-4003","belen_strassner@aol.com","http://www.eaglesoftwareinc.com"
"Gracia","Melnyk","Juvenile & Adult Super","97 Airport Loop Dr","Jacksonville","Duval","FL",32216,"904-235-3633","904-627-4341","gracia@melnyk.com","http://www.juvenileadultsuper.com"
"Jolanda","Hanafan","Perez, Joseph J Esq","37855 Nolan Rd","Bangor","Penobscot","ME","04401","207-458-9196","207-233-6185","jhanafan@gmail.com","http://www.perezjosephjesq.com"
"Barrett","Toyama","Case Foundation Co","4252 N Washington Ave #9","Kennedale","Tarr
 ant","TX",76060,"817-765-5781","817-577-6151","barrett.toyama@toyama.org","http://www.casefoundationco.com"
"Helga","Fredicks","Eis Environmental Engrs Inc","42754 S Ash Ave","Buffalo","Erie","NY",14228,"716-752-4114","716-854-9845","helga_fredicks@yahoo.com","http://www.eisenvironmentalengrsinc.com"
"Ashlyn","Pinilla","Art Crafters","703 Beville Rd","Opa Locka","Miami-Dade","FL",33054,"305-670-9628","305-857-5489","apinilla@cox.net","http://www.artcrafters.com"
"Fausto","Agramonte","Marriott Hotels Resorts Suites","5 Harrison Rd","New York","New York","NY",10038,"212-313-1783","212-778-3063","fausto_agramonte@yahoo.com","http://www.marriotthotelsresortssuites.com"
"Ronny","Caiafa","Remaco Inc","73 Southern Blvd","Philadelphia","Philadelphia","PA",19103,"215-605-7570","215-511-3531","ronny.caiafa@caiafa.org","http://www.remacoinc.com"
"Marge","Limmel","Bjork, Robert D Jr","189 Village Park Rd","Crestview","Okaloosa","FL",32536,"850-430-1663","850-330-8079","marge@gmail.com","http://
 www.bjorkrobertdjr.com"
"Norah","Waymire","Carmichael, Jeffery L Esq","6 Middlegate Rd #106","San Francisco","San Francisco","CA",94107,"415-306-7897","415-874-2984","norah.waymire@gmail.com","http://www.carmichaeljefferylesq.com"
"Aliza","Baltimore","Andrews, J Robert Esq","1128 Delaware St","San Jose","Santa Clara","CA",95132,"408-504-3552","408-425-1994","aliza@aol.com","http://www.andrewsjrobertesq.com"
"Mozell","Pelkowski","Winship & Byrne","577 Parade St","South San Francisco","San Mateo","CA",94080,"650-947-1215","650-960-1069","mpelkowski@pelkowski.org","http://www.winshipbyrne.com"
"Viola","Bitsuie","Burton & Davis","70 Mechanic St","Northridge","Los Angeles","CA",91325,"818-864-4875","818-481-5787","viola@gmail.com","http://www.burtondavis.com"
"Franklyn","Emard","Olympic Graphic Arts","4379 Highway 116","Philadelphia","Philadelphia","PA",19103,"215-558-8189","215-483-3003","femard@emard.com","http://www.olympicgraphicarts.com"
"Willodean","Konopacki","Magnuson","55 Hawtho
 rne Blvd","Lafayette","Lafayette","LA",70506,"337-253-8384","337-774-7564","willodean_konopacki@konopacki.org","http://www.magnuson.com"
"Beckie","Silvestrini","A All American Travel Inc","7116 Western Ave","Dearborn","Wayne","MI",48126,"313-533-4884","313-390-7855","beckie.silvestrini@silvestrini.com","http://www.aallamericantravelinc.com"
"Rebecka","Gesick","Polykote Inc","2026 N Plankinton Ave #3","Austin","Travis","TX",78754,"512-213-8574","512-693-8345","rgesick@gesick.org","http://www.polykoteinc.com"
"Frederica","Blunk","Jets Cybernetics","99586 Main St","Dallas","Dallas","TX",75207,"214-428-2285","214-529-1949","frederica_blunk@gmail.com","http://www.jetscybernetics.com"
"Glen","Bartolet","Metlab Testing Services","8739 Hudson St","Vashon","King","WA",98070,"206-697-5796","206-389-1482","glen_bartolet@hotmail.com","http://www.metlabtestingservices.com"
"Freeman","Gochal","Kellermann, William T Esq","383 Gunderman Rd #197","Coatesville","Chester","PA",19320,"610-476-3501","61
 0-752-2683","freeman_gochal@aol.com","http://www.kellermannwilliamtesq.com"
"Vincent","Meinerding","Arturi, Peter D Esq","4441 Point Term Mkt","Philadelphia","Philadelphia","PA",19143,"215-372-1718","215-829-4221","vincent.meinerding@hotmail.com","http://www.arturipeterdesq.com"
"Rima","Bevelacqua","Mcauley Mfg Co","2972 Lafayette Ave","Gardena","Los Angeles","CA",90248,"310-858-5079","310-499-4200","rima@cox.net","http://www.mcauleymfgco.com"
"Glendora","Sarbacher","Defur Voran Hanley Radcliff","2140 Diamond Blvd","Rohnert Park","Sonoma","CA",94928,"707-653-8214","707-881-3154","gsarbacher@gmail.com","http://www.defurvoranhanleyradcliff.com"
"Avery","Steier","Dill Dill Carr & Stonbraker Pc","93 Redmond Rd #492","Orlando","Orange","FL",32803,"407-808-9439","407-945-8566","avery@cox.net","http://www.dilldillcarrstonbrakerpc.com"
"Cristy","Lother","Kleensteel","3989 Portage Tr","Escondido","San Diego","CA",92025,"760-971-4322","760-465-4762","cristy@lother.com","http://www.kleensteel.
 com"
"Nicolette","Brossart","Goulds Pumps Inc Slurry Pump","1 Midway Rd","Westborough","Worcester","MA","01581","508-837-9230","508-504-6388","nicolette_brossart@brossart.com","http://www.gouldspumpsincslurrypump.com"
"Tracey","Modzelewski","Kansas City Insurance Report","77132 Coon Rapids Blvd Nw","Conroe","Montgomery","TX",77301,"936-264-9294","936-988-8171","tracey@hotmail.com","http://www.kansascityinsurancereport.com"
"Virgina","Tegarden","Berhanu International Foods","755 Harbor Way","Milwaukee","Milwaukee","WI",53226,"414-214-8697","414-411-5744","virgina_tegarden@tegarden.com","http://www.berhanuinternationalfoods.com"
"Tiera","Frankel","Roland Ashcroft","87 Sierra Rd","El Monte","Los Angeles","CA",91731,"626-636-4117","626-638-4241","tfrankel@aol.com","http://www.rolandashcroft.com"
"Alaine","Bergesen","Hispanic Magazine","7667 S Hulen St #42","Yonkers","Westchester","NY",10701,"914-300-9193","914-654-1426","alaine_bergesen@cox.net","http://www.hispanicmagazine.com"
"Earlee
 n","Mai","Little Sheet Metal Co","75684 S Withlapopka Dr #32","Dallas","Dallas","TX",75227,"214-289-1973","214-785-6750","earleen_mai@cox.net","http://www.littlesheetmetalco.com"
"Leonida","Gobern","Holmes, Armstead J Esq","5 Elmwood Park Blvd","Biloxi","Harrison","MS",39530,"228-235-5615","228-432-4635","leonida@gobern.org","http://www.holmesarmsteadjesq.com"
"Ressie","Auffrey","Faw, James C Cpa","23 Palo Alto Sq","Miami","Miami-Dade","FL",33134,"305-604-8981","305-287-4743","ressie.auffrey@yahoo.com","http://www.fawjamesccpa.com"
"Justine","Mugnolo","Evans Rule Company","38062 E Main St","New York","New York","NY",10048,"212-304-9225","212-311-6377","jmugnolo@yahoo.com","http://www.evansrulecompany.com"
"Eladia","Saulter","Tyee Productions Inc","3958 S Dupont Hwy #7","Ramsey","Bergen","NJ","07446","201-474-4924","201-365-8698","eladia@saulter.com","http://www.tyeeproductionsinc.com"
"Chaya","Malvin","Dunnells & Duvall","560 Civic Center Dr","Ann Arbor","Washtenaw","MI",48103,"734-
 928-5182","734-408-8174","chaya@malvin.com","http://www.dunnellsduvall.com"
"Gwenn","Suffield","Deltam Systems Inc","3270 Dequindre Rd","Deer Park","Suffolk","NY",11729,"631-258-6558","631-295-9879","gwenn_suffield@suffield.org","http://www.deltamsystemsinc.com"
"Salena","Karpel","Hammill Mfg Co","1 Garfield Ave #7","Canton","Stark","OH",44707,"330-791-8557","330-618-2579","skarpel@cox.net","http://www.hammillmfgco.com"
"Yoko","Fishburne","Sams Corner Store","9122 Carpenter Ave","New Haven","New Haven","CT","06511","203-506-4706","203-840-8634","yoko@fishburne.com","http://www.samscornerstore.com"
"Taryn","Moyd","Siskin, Mark J Esq","48 Lenox St","Fairfax","Fairfax City","VA",22030,"703-322-4041","703-938-7939","taryn.moyd@hotmail.com","http://www.siskinmarkjesq.com"
"Katina","Polidori","Cape & Associates Real Estate","5 Little River Tpke","Wilmington","Middlesex","MA","01887","978-626-2978","978-679-7429","katina_polidori@aol.com","http://www.capeassociatesrealestate.com"
"Rickie",
 "Plumer","Merrill Lynch","3 N Groesbeck Hwy","Toledo","Lucas","OH",43613,"419-693-1334","419-313-5571","rickie.plumer@aol.com","http://www.merrilllynch.com"
"Alex","Loader","Sublett, Scott Esq","37 N Elm St #916","Tacoma","Pierce","WA",98409,"253-660-7821","253-875-9222","alex@loader.com","http://www.sublettscottesq.com"
"Lashon","Vizarro","Sentry Signs","433 Westminster Blvd #590","Roseville","Placer","CA",95661,"916-741-7884","916-289-4526","lashon@aol.com","http://www.sentrysigns.com"
"Lauran","Burnard","Professionals Unlimited","66697 Park Pl #3224","Riverton","Fremont","WY",82501,"307-342-7795","307-453-7589","lburnard@burnard.com","http://www.professionalsunlimited.com"
"Ceola","Setter","Southern Steel Shelving Co","96263 Greenwood Pl","Warren","Knox","ME","04864","207-627-7565","207-297-5029","ceola.setter@setter.org","http://www.southernsteelshelvingco.com"
"My","Rantanen","Bosco, Paul J","8 Mcarthur Ln","Richboro","Bucks","PA",18954,"215-491-5633","215-647-2158","my@hotmail
 .com","http://www.boscopaulj.com"
"Lorrine","Worlds","Longo, Nicholas J Esq","8 Fair Lawn Ave","Tampa","Hillsborough","FL",33614,"813-769-2939","813-863-6467","lorrine.worlds@worlds.com","http://www.longonicholasjesq.com"
"Peggie","Sturiale","Henry County Middle School","9 N 14th St","El Cajon","San Diego","CA",92020,"619-608-1763","619-695-8086","peggie@cox.net","http://www.henrycountymiddleschool.com"
"Marvel","Raymo","Edison Supply & Equipment Co","9 Vanowen St","College Station","Brazos","TX",77840,"979-718-8968","979-809-5770","mraymo@yahoo.com","http://www.edisonsupplyequipmentco.com"
"Daron","Dinos","Wolf, Warren R Esq","18 Waterloo Geneva Rd","Highland Park","Lake","IL",60035,"847-233-3075","847-265-6609","daron_dinos@cox.net","http://www.wolfwarrenresq.com"
"An","Fritz","Linguistic Systems Inc","506 S Hacienda Dr","Atlantic City","Atlantic","NJ","08401","609-228-5265","609-854-7156","an_fritz@hotmail.com","http://www.linguisticsystemsinc.com"
"Portia","Stimmel","Peace Chris
 tian Center","3732 Sherman Ave","Bridgewater","Somerset","NJ","08807","908-722-7128","908-670-4712","portia.stimmel@aol.com","http://www.peacechristiancenter.com"
"Rhea","Aredondo","Double B Foods Inc","25657 Live Oak St","Brooklyn","Kings","NY",11226,"718-560-9537","718-280-4183","rhea_aredondo@cox.net","http://www.doublebfoodsinc.com"
"Benedict","Sama","Alexander & Alexander Inc","4923 Carey Ave","Saint Louis","Saint Louis City","MO",63104,"314-787-1588","314-858-4832","bsama@cox.net","http://www.alexanderalexanderinc.com"
"Alyce","Arias","Fairbanks Scales","3196 S Rider Trl","Stockton","San Joaquin","CA",95207,"209-317-1801","209-242-7022","alyce@arias.org","http://www.fairbanksscales.com"
"Heike","Berganza","Cali Sportswear Cutting Dept","3 Railway Ave #75","Little Falls","Passaic","NJ","07424","973-936-5095","973-822-8827","heike@gmail.com","http://www.calisportswearcuttingdept.com"
"Carey","Dopico","Garofani, John Esq","87393 E Highland Rd","Indianapolis","Marion","IN",46220,"
 317-578-2453","317-441-5848","carey_dopico@dopico.org","http://www.garofanijohnesq.com"
"Dottie","Hellickson","Thompson Fabricating Co","67 E Chestnut Hill Rd","Seattle","King","WA",98133,"206-540-6076","206-295-5631","dottie@hellickson.org","http://www.thompsonfabricatingco.com"
"Deandrea","Hughey","Century 21 Krall Real Estate","33 Lewis Rd #46","Burlington","Alamance","NC",27215,"336-822-7652","336-467-3095","deandrea@yahoo.com","http://www.centurykrallrealestate.com"
"Kimberlie","Duenas","Mid Contntl Rlty & Prop Mgmt","8100 Jacksonville Rd #7","Hays","Ellis","KS",67601,"785-629-8542","785-616-1685","kimberlie_duenas@yahoo.com","http://www.midcontntlrltypropmgmt.com"
"Martina","Staback","Ace Signs Inc","7 W Wabansia Ave #227","Orlando","Orange","FL",32822,"407-471-6908","407-429-2145","martina_staback@staback.com","http://www.acesignsinc.com"
"Skye","Fillingim","Rodeway Inn","25 Minters Chapel Rd #9","Minneapolis","Hennepin","MN",55401,"612-508-2655","612-664-6304","skye_fillingi
 m@yahoo.com","http://www.rodewayinn.com"
"Jade","Farrar","Bonnet & Daughter","6882 Torresdale Ave","Columbia","Richland","SC",29201,"803-352-5387","803-975-3405","jade.farrar@yahoo.com","http://www.bonnetdaughter.com"
"Charlene","Hamilton","Oshins & Gibbons","985 E 6th Ave","Santa Rosa","Sonoma","CA",95407,"707-300-1771","707-821-8037","charlene.hamilton@hotmail.com","http://www.oshinsgibbons.com"
"Geoffrey","Acey","Price Business Services","7 West Ave #1","Palatine","Cook","IL",60067,"847-222-1734","847-556-2909","geoffrey@gmail.com","http://www.pricebusinessservices.com"
"Stevie","Westerbeck","Wise, Dennis W Md","26659 N 13th St","Costa Mesa","Orange","CA",92626,"949-867-4077","949-903-3898","stevie.westerbeck@yahoo.com","http://www.wisedenniswmd.com"
"Pamella","Fortino","Super 8 Motel","669 Packerland Dr #1438","Denver","Denver","CO",80212,"303-404-2210","303-794-1341","pamella@fortino.com","http://www.supermotel.com"
"Harrison","Haufler","John Wagner Associates","759 Eldora St",
 "New Haven","New Haven","CT","06515","203-801-6193","203-801-8497","hhaufler@hotmail.com","http://www.johnwagnerassociates.com"
"Johnna","Engelberg","Thrifty Oil Co","5 S Colorado Blvd #449","Bothell","Snohomish","WA",98021,"425-986-7573","425-700-3751","jengelberg@engelberg.org","http://www.thriftyoilco.com"
"Buddy","Cloney","Larkfield Photo","944 Gaither Dr","Strongsville","Cuyahoga","OH",44136,"440-989-5826","440-327-2093","buddy.cloney@yahoo.com","http://www.larkfieldphoto.com"
"Dalene","Riden","Silverman Planetarium","66552 Malone Rd","Plaistow","Rockingham","NH","03865","603-315-6839","603-745-7497","dalene.riden@aol.com","http://www.silvermanplanetarium.com"
"Jerry","Zurcher","J & F Lumber","77 Massillon Rd #822","Satellite Beach","Brevard","FL",32937,"321-518-5938","321-597-2159","jzurcher@zurcher.org","http://www.jflumber.com"
"Haydee","Denooyer","Cleaning Station Inc","25346 New Rd","New York","New York","NY",10016,"212-792-8658","212-782-3493","hdenooyer@denooyer.org","ht
 tp://www.cleaningstationinc.com"
"Joseph","Cryer","Ames Stationers","60 Fillmore Ave","Huntington Beach","Orange","CA",92647,"714-584-2237","714-698-2170","joseph_cryer@cox.net","http://www.amesstationers.com"
"Deonna","Kippley","Midas Muffler Shops","57 Haven Ave #90","Southfield","Oakland","MI",48075,"248-913-4677","248-793-4966","deonna_kippley@hotmail.com","http://www.midasmufflershops.com"
"Raymon","Calvaresi","Seaboard Securities Inc","6538 E Pomona St #60","Indianapolis","Marion","IN",46222,"317-825-4724","317-342-1532","raymon.calvaresi@gmail.com","http://www.seaboardsecuritiesinc.com"
"Alecia","Bubash","Petersen, James E Esq","6535 Joyce St","Wichita Falls","Wichita","TX",76301,"940-276-7922","940-302-3036","alecia@aol.com","http://www.petersenjameseesq.com"
"Ma","Layous","Development Authority","78112 Morris Ave","North Haven","New Haven","CT","06473","203-721-3388","203-564-1543","mlayous@hotmail.com","http://www.developmentauthority.com"
"Detra","Coyier","Schott Fiber Op
 tics Inc","96950 Hidden Ln","Aberdeen","Harford","MD",21001,"410-739-9277","410-259-2118","detra@aol.com","http://www.schottfiberopticsinc.com"
"Terrilyn","Rodeigues","Stuart J Agins","3718 S Main St","New Orleans","Orleans","LA",70130,"504-463-4384","504-635-8518","terrilyn.rodeigues@cox.net","http://www.stuartjagins.com"
"Salome","Lacovara","Mitsumi Electronics Corp","9677 Commerce Dr","Richmond","Richmond City","VA",23219,"804-550-5097","804-858-1011","slacovara@gmail.com","http://www.mitsumielectronicscorp.com"
"Garry","Keetch","Italian Express Franchise Corp","5 Green Pond Rd #4","Southampton","Bucks","PA",18966,"215-979-8776","215-846-9046","garry_keetch@hotmail.com","http://www.italianexpressfranchisecorp.com"
"Matthew","Neither","American Council On Sci & Hlth","636 Commerce Dr #42","Shakopee","Scott","MN",55379,"952-651-7597","952-906-4597","mneither@yahoo.com","http://www.americancouncilonscihlth.com"
"Theodora","Restrepo","Kleri, Patricia S Esq","42744 Hamann Industrial P
 ky #82","Miami","Miami-Dade","FL",33136,"305-936-8226","305-573-1085","theodora.restrepo@restrepo.com","http://www.kleripatriciasesq.com"
"Noah","Kalafatis","Twiggs Abrams Blanchard","1950 5th Ave","Milwaukee","Milwaukee","WI",53209,"414-263-5287","414-660-9766","noah.kalafatis@aol.com","http://www.twiggsabramsblanchard.com"
"Carmen","Sweigard","Maui Research & Technology Pk","61304 N French Rd","Somerset","Somerset","NJ","08873","732-941-2621","732-445-6940","csweigard@sweigard.com","http://www.mauiresearchtechnologypk.com"
"Lavonda","Hengel","Bradley Nameplate Corp","87 Imperial Ct #79","Fargo","Cass","ND",58102,"701-898-2154","701-421-7080","lavonda@cox.net","http://www.bradleynameplatecorp.com"
"Junita","Stoltzman","Geonex Martel Inc","94 W Dodge Rd","Carson City","Carson City","NV",89701,"775-638-9963","775-578-1214","junita@aol.com","http://www.geonexmartelinc.com"
"Herminia","Nicolozakes","Sea Island Div Of Fstr Ind Inc","4 58th St #3519","Scottsdale","Maricopa","AZ",85254,"6
 02-954-5141","602-304-6433","herminia@nicolozakes.org","http://www.seaislanddivoffstrindinc.com"
"Casie","Good","Papay, Debbie J Esq","5221 Bear Valley Rd","Nashville","Davidson","TN",37211,"615-390-2251","615-825-4297","casie.good@aol.com","http://www.papaydebbiejesq.com"
"Reena","Maisto","Lane Promotions","9648 S Main","Salisbury","Wicomico","MD",21801,"410-351-1863","410-951-2667","reena@hotmail.com","http://www.lanepromotions.com"
"Mirta","Mallett","Stephen Kennerly Archts Inc Pc","7 S San Marcos Rd","New York","New York","NY",10004,"212-870-1286","212-745-6948","mirta_mallett@gmail.com","http://www.stephenkennerlyarchtsincpc.com"
"Cathrine","Pontoriero","Business Systems Of Wis Inc","812 S Haven St","Amarillo","Randall","TX",79109,"806-703-1435","806-558-5848","cathrine.pontoriero@pontoriero.com","http://www.businesssystemsofwisinc.com"
"Filiberto","Tawil","Flash, Elena Salerno Esq","3882 W Congress St #799","Los Angeles","Los Angeles","CA",90016,"323-765-2528","323-842-8226","
 ftawil@hotmail.com","http://www.flashelenasalernoesq.com"
"Raul","Upthegrove","Neeley, Gregory W Esq","4 E Colonial Dr","La Mesa","San Diego","CA",91942,"619-509-5282","619-666-4765","rupthegrove@yahoo.com","http://www.neeleygregorywesq.com"
"Sarah","Candlish","Alabama Educational Tv Comm","45 2nd Ave #9759","Atlanta","Fulton","GA",30328,"770-732-1194","770-531-2842","sarah.candlish@gmail.com","http://www.alabamaeducationaltvcomm.com"
"Lucy","Treston","Franz Inc","57254 Brickell Ave #372","Worcester","Worcester","MA","01602","508-769-5250","508-502-5634","lucy@cox.net","http://www.franzinc.com"
"Judy","Aquas","Plantation Restaurant","8977 Connecticut Ave Nw #3","Niles","Berrien","MI",49120,"269-756-7222","269-431-9464","jaquas@aquas.com","http://www.plantationrestaurant.com"
"Yvonne","Tjepkema","Radio Communications Co","9 Waydell St","Fairfield","Essex","NJ","07004","973-714-1721","973-976-8627","yvonne.tjepkema@hotmail.com","http://www.radiocommunicationsco.com"
"Kayleigh","Lace",
 "Dentalaw Divsn Hlth Care","43 Huey P Long Ave","Lafayette","Lafayette","LA",70508,"337-740-9323","337-751-2326","kayleigh.lace@yahoo.com","http://www.dentalawdivsnhlthcare.com"
"Felix","Hirpara","American Speedy Printing Ctrs","7563 Cornwall Rd #4462","Denver","Lancaster","PA",17517,"717-491-5643","717-583-1497","felix_hirpara@cox.net","http://www.americanspeedyprintingctrs.com"
"Tresa","Sweely","Grayson, Grant S Esq","22 Bridle Ln","Valley Park","Saint Louis","MO",63088,"314-359-9566","314-231-3514","tresa_sweely@hotmail.com","http://www.graysongrantsesq.com"
"Kristeen","Turinetti","Jeanerette Middle School","70099 E North Ave","Arlington","Tarrant","TX",76013,"817-213-8851","817-947-9480","kristeen@gmail.com","http://www.jeanerettemiddleschool.com"
"Jenelle","Regusters","Haavisto, Brian F Esq","3211 E Northeast Loop","Tampa","Hillsborough","FL",33619,"813-932-8715","813-357-7296","jregusters@regusters.com","http://www.haavistobrianfesq.com"
"Renea","Monterrubio","Wmmt Radio Stati
 on","26 Montgomery St","Atlanta","Fulton","GA",30328,"770-679-4752","770-930-9967","renea@hotmail.com","http://www.wmmtradiostation.com"
"Olive","Matuszak","Colony Paints Sales Ofc & Plnt","13252 Lighthouse Ave","Cathedral City","Riverside","CA",92234,"760-938-6069","760-745-2649","olive@aol.com","http://www.colonypaintssalesofcplnt.com"
"Ligia","Reiber","Floral Expressions","206 Main St #2804","Lansing","Ingham","MI",48933,"517-906-1108","517-747-7664","lreiber@cox.net","http://www.floralexpressions.com"
"Christiane","Eschberger","Casco Services Inc","96541 W Central Blvd","Phoenix","Maricopa","AZ",85034,"602-390-4944","602-330-6894","christiane.eschberger@yahoo.com","http://www.cascoservicesinc.com"
"Goldie","Schirpke","Reuter, Arthur C Jr","34 Saint George Ave #2","Bangor","Penobscot","ME","04401","207-295-7569","207-748-3722","goldie.schirpke@yahoo.com","http://www.reuterarthurcjr.com"
"Loreta","Timenez","Kaminski, Katherine Andritsaki","47857 Coney Island Ave","Clinton","Prince
  Georges","MD",20735,"301-696-6420","301-392-6698","loreta.timenez@hotmail.com","http://www.kaminskikatherineandritsaki.com"
"Fabiola","Hauenstein","Sidewinder Products Corp","8573 Lincoln Blvd","York","York","PA",17404,"717-809-3119","717-344-2804","fabiola.hauenstein@hauenstein.org","http://www.sidewinderproductscorp.com"
"Amie","Perigo","General Foam Corporation","596 Santa Maria Ave #7913","Mesquite","Dallas","TX",75150,"972-419-7946","972-898-1033","amie.perigo@yahoo.com","http://www.generalfoamcorporation.com"
"Raina","Brachle","Ikg Borden Divsn Harsco Corp","3829 Ventura Blvd","Butte","Silver Bow","MT",59701,"406-318-1515","406-374-7752","raina.brachle@brachle.org","http://www.ikgbordendivsnharscocorp.com"
"Erinn","Canlas","Anchor Computer Inc","13 S Hacienda Dr","Livingston","Essex","NJ","07039","973-767-3008","973-563-9502","erinn.canlas@canlas.com","http://www.anchorcomputerinc.com"
"Cherry","Lietz","Sebring & Co","40 9th Ave Sw #91","Waterford","Oakland","MI",48329,"248-9
 80-6904","248-697-7722","cherry@lietz.com","http://www.sebringco.com"
"Kattie","Vonasek","H A C Farm Lines Co Optv Assoc","2845 Boulder Crescent St","Cleveland","Cuyahoga","OH",44103,"216-923-3715","216-270-9653","kattie@vonasek.org","http://www.hacfarmlinescooptvassoc.com"
"Lilli","Scriven","Hunter, John J Esq","33 State St","Abilene","Taylor","TX",79601,"325-631-1560","325-667-7868","lilli@aol.com","http://www.hunterjohnjesq.com"
"Whitley","Tomasulo","Freehold Fence Co","2 S 15th St","Fort Worth","Tarrant","TX",76107,"817-526-4408","817-819-7799","whitley.tomasulo@aol.com","http://www.freeholdfenceco.com"
"Barbra","Adkin","Binswanger","4 Kohler Memorial Dr","Brooklyn","Kings","NY",11230,"718-201-3751","718-732-9475","badkin@hotmail.com","http://www.binswanger.com"
"Hermila","Thyberg","Chilton Malting Co","1 Rancho Del Mar Shopping C","Providence","Providence","RI","02903","401-893-4882","401-885-7681","hermila_thyberg@hotmail.com","http://www.chiltonmaltingco.com"
"Jesusita","Flis
 ter","Schoen, Edward J Jr","3943 N Highland Ave","Lancaster","Lancaster","PA",17601,"717-885-9118","717-686-7564","jesusita.flister@hotmail.com","http://www.schoenedwardjjr.com"
"Caitlin","Julia","Helderman, Seymour Cpa","5 Williams St","Johnston","Providence","RI","02919","401-948-4982","401-552-9059","caitlin.julia@julia.org","http://www.heldermanseymourcpa.com"
"Roosevelt","Hoffis","Denbrook, Myron","60 Old Dover Rd","Hialeah","Miami-Dade","FL",33014,"305-622-4739","305-302-1135","roosevelt.hoffis@aol.com","http://www.denbrookmyron.com"
"Helaine","Halter","Lippitt, Mike","8 Sheridan Rd","Jersey City","Hudson","NJ","07304","201-832-4168","201-412-3040","hhalter@yahoo.com","http://www.lippittmike.com"
"Lorean","Martabano","Hiram, Hogg P Esq","85092 Southern Blvd","San Antonio","Bexar","TX",78204,"210-856-4979","210-634-2447","lorean.martabano@hotmail.com","http://www.hiramhoggpesq.com"
"France","Buzick","In Travel Agency","64 Newman Springs Rd E","Brooklyn","Kings","NY",11219,"718-
 478-8504","718-853-3740","france.buzick@yahoo.com","http://www.intravelagency.com"
"Justine","Ferrario","Newhart Foods Inc","48 Stratford Ave","Pomona","Los Angeles","CA",91768,"909-993-3242","909-631-5703","jferrario@hotmail.com","http://www.newhartfoodsinc.com"
"Adelina","Nabours","Courtyard By Marriott","80 Pittsford Victor Rd #9","Cleveland","Cuyahoga","OH",44103,"216-230-4892","216-937-5320","adelina_nabours@gmail.com","http://www.courtyardbymarriott.com"
"Derick","Dhamer","Studer, Eugene A Esq","87163 N Main Ave","New York","New York","NY",10013,"212-304-4515","212-225-9676","ddhamer@cox.net","http://www.studereugeneaesq.com"
"Jerry","Dallen","Seashore Supply Co Waretown","393 Lafayette Ave","Richmond","Richmond City","VA",23219,"804-762-9576","804-808-9574","jerry.dallen@yahoo.com","http://www.seashoresupplycowaretown.com"
"Leota","Ragel","Mayar Silk Inc","99 5th Ave #33","Trion","Chattooga","GA",30753,"706-221-4243","706-616-5131","leota.ragel@gmail.com","http://www.mayarsil
 kinc.com"
"Jutta","Amyot","National Medical Excess Corp","49 N Mays St","Broussard","Lafayette","LA",70518,"337-515-1438","337-991-8070","jamyot@hotmail.com","http://www.nationalmedicalexcesscorp.com"
"Aja","Gehrett","Stero Company","993 Washington Ave","Nutley","Essex","NJ","07110","973-544-2677","973-986-4456","aja_gehrett@hotmail.com","http://www.sterocompany.com"
"Kirk","Herritt","Hasting, H Duane Esq","88 15th Ave Ne","Vestal","Broome","NY",13850,"607-407-3716","607-350-7690","kirk.herritt@aol.com","http://www.hastinghduaneesq.com"
"Leonora","Mauson","Insty Prints","3381 E 40th Ave","Passaic","Passaic","NJ","07055","973-412-2995","973-355-2120","leonora@yahoo.com","http://www.instyprints.com"
"Winfred","Brucato","Glenridge Manor Mobile Home Pk","201 Ridgewood Rd","Moscow","Latah","ID",83843,"208-252-4552","208-793-4108","winfred_brucato@hotmail.com","http://www.glenridgemanormobilehomepk.com"
"Tarra","Nachor","Circuit Solution Inc","39 Moccasin Dr","San Francisco","San Francisc
 o","CA",94104,"415-411-1775","415-284-2730","tarra.nachor@cox.net","http://www.circuitsolutioninc.com"
"Corinne","Loder","Local Office","4 Carroll St","North Attleboro","Bristol","MA","02760","508-942-4186","508-618-7826","corinne@loder.org","http://www.localoffice.com"
"Dulce","Labreche","Lee Kilkelly Paulson & Kabaker","9581 E Arapahoe Rd","Rochester","Oakland","MI",48307,"248-357-8718","248-811-5696","dulce_labreche@yahoo.com","http://www.leekilkellypaulsonkabaker.com"
"Kate","Keneipp","Davis, Maxon R Esq","33 N Michigan Ave","Green Bay","Brown","WI",54301,"920-353-6377","920-355-1610","kate_keneipp@yahoo.com","http://www.davismaxonresq.com"
"Kaitlyn","Ogg","Garrison, Paul E Esq","2 S Biscayne Blvd","Baltimore","Baltimore City","MD",21230,"410-665-4903","410-773-3862","kaitlyn.ogg@gmail.com","http://www.garrisonpauleesq.com"
"Sherita","Saras","Black History Resource Center","8 Us Highway 22","Colorado Springs","El Paso","CO",80937,"719-669-1664","719-547-9543","sherita.saras@cox.
 net","http://www.blackhistoryresourcecenter.com"
"Lashawnda","Stuer","Rodriguez, J Christopher Esq","7422 Martin Ave #8","Toledo","Lucas","OH",43607,"419-588-8719","419-399-1744","lstuer@cox.net","http://www.rodriguezjchristopheresq.com"
"Ernest","Syrop","Grant Family Health Center","94 Chase Rd","Hyattsville","Prince Georges","MD",20785,"301-998-9644","301-257-4883","ernest@cox.net","http://www.grantfamilyhealthcenter.com"
"Nobuko","Halsey","Goeman Wood Products Inc","8139 I Hwy 10 #92","New Bedford","Bristol","MA","02745","508-855-9887","508-897-7916","nobuko.halsey@yahoo.com","http://www.goemanwoodproductsinc.com"
"Lavonna","Wolny","Linhares, Kenneth A Esq","5 Cabot Rd","Mc Lean","Fairfax","VA",22102,"703-483-1970","703-892-2914","lavonna.wolny@hotmail.com","http://www.linhareskennethaesq.com"
"Lashaunda","Lizama","Earnhardt Printing","3387 Ryan Dr","Hanover","Anne Arundel","MD",21076,"410-678-2473","410-912-6032","llizama@cox.net","http://www.earnhardtprinting.com"
"Mariann","Bi
 lden","H P G Industrys Inc","3125 Packer Ave #9851","Austin","Travis","TX",78753,"512-223-4791","512-742-1149","mariann.bilden@aol.com","http://www.hpgindustrysinc.com"
"Helene","Rodenberger","Bailey Transportation Prod Inc","347 Chestnut St","Peoria","Maricopa","AZ",85381,"623-461-8551","623-426-4907","helene@aol.com","http://www.baileytransportationprodinc.com"
"Roselle","Estell","Mcglynn Bliss Pc","8116 Mount Vernon Ave","Bucyrus","Crawford","OH",44820,"419-571-5920","419-488-6648","roselle.estell@hotmail.com","http://www.mcglynnblisspc.com"
"Samira","Heintzman","Mutual Fish Co","8772 Old County Rd #5410","Kent","King","WA",98032,"206-311-4137","206-923-6042","sheintzman@hotmail.com","http://www.mutualfishco.com"
"Margart","Meisel","Yeates, Arthur L Aia","868 State St #38","Cincinnati","Hamilton","OH",45251,"513-617-2362","513-747-9603","margart_meisel@yahoo.com","http://www.yeatesarthurlaia.com"
"Kristofer","Bennick","Logan, Ronald J Esq","772 W River Dr","Bloomington","Monroe",
 "IN",47404,"812-368-1511","812-442-8544","kristofer.bennick@yahoo.com","http://www.loganronaldjesq.com"
"Weldon","Acuff","Advantage Martgage Company","73 W Barstow Ave","Arlington Heights","Cook","IL",60004,"847-353-2156","847-613-5866","wacuff@gmail.com","http://www.advantagemartgagecompany.com"
"Shalon","Shadrick","Germer And Gertz Llp","61047 Mayfield Ave","Brooklyn","Kings","NY",11223,"718-232-2337","718-394-4974","shalon@cox.net","http://www.germerandgertzllp.com"
"Denise","Patak","Spence Law Offices","2139 Santa Rosa Ave","Orlando","Orange","FL",32801,"407-446-4358","407-808-3254","denise@patak.org","http://www.spencelawoffices.com"
"Louvenia","Beech","John Ortiz Nts Therapy Center","598 43rd St","Beverly Hills","Los Angeles","CA",90210,"310-820-2117","310-652-2379","louvenia.beech@beech.com","http://www.johnortizntstherapycenter.com"
"Audry","Yaw","Mike Uchrin Htg & Air Cond Inc","70295 Pioneer Ct","Brandon","Hillsborough","FL",33511,"813-797-4816","813-744-7100","audry.yaw@y
 aw.org","http://www.mikeuchrinhtgaircondinc.com"
"Kristel","Ehmann","Mccoy, Joy Reynolds Esq","92899 Kalakaua Ave","El Paso","El Paso","TX",79925,"915-452-1290","915-300-6100","kristel.ehmann@aol.com","http://www.mccoyjoyreynoldsesq.com"
"Vincenza","Zepp","Kbor 1600 Am","395 S 6th St #2","El Cajon","San Diego","CA",92020,"619-603-5125","619-935-6661","vzepp@gmail.com","http://www.kboram.com"
"Elouise","Gwalthney","Quality Inn Northwest","9506 Edgemore Ave","Bladensburg","Prince Georges","MD",20710,"301-841-5012","301-591-3034","egwalthney@yahoo.com","http://www.qualityinnnorthwest.com"
"Venita","Maillard","Wallace Church Assoc Inc","72119 S Walker Ave #63","Anaheim","Orange","CA",92801,"714-523-6653","714-663-9740","venita_maillard@gmail.com","http://www.wallacechurchassocinc.com"
"Kasandra","Semidey","Can Tron","369 Latham St #500","Saint Louis","Saint Louis City","MO",63102,"314-732-9131","314-697-3652","kasandra_semidey@semidey.com","http://www.cantron.com"
"Xochitl","Discipio","
 Ravaal Enterprises Inc","3158 Runamuck Pl","Round Rock","Williamson","TX",78664,"512-233-1831","512-942-3411","xdiscipio@gmail.com","http://www.ravaalenterprisesinc.com"
"Maile","Linahan","Thompson Steel Company Inc","9 Plainsboro Rd #598","Greensboro","Guilford","NC",27409,"336-670-2640","336-364-6037","mlinahan@yahoo.com","http://www.thompsonsteelcompanyinc.com"
"Krissy","Rauser","Anderson, Mark A Esq","8728 S Broad St","Coram","Suffolk","NY",11727,"631-443-4710","631-288-2866","krauser@cox.net","http://www.andersonmarkaesq.com"
"Pete","Dubaldi","Womack & Galich","2215 Prosperity Dr","Lyndhurst","Bergen","NJ","07071","201-825-2514","201-749-8866","pdubaldi@hotmail.com","http://www.womackgalich.com"
"Linn","Paa","Valerie & Company","1 S Pine St","Memphis","Shelby","TN",38112,"901-412-4381","901-573-9024","linn_paa@paa.com","http://www.valeriecompany.com"
"Paris","Wide","Gehring Pumps Inc","187 Market St","Atlanta","Fulton","GA",30342,"404-505-4445","404-607-8435","paris@hotmail.com
 ","http://www.gehringpumpsinc.com"
"Wynell","Dorshorst","Haehnel, Craig W Esq","94290 S Buchanan St","Pacifica","San Mateo","CA",94044,"650-473-1262","650-749-9879","wynell_dorshorst@dorshorst.org","http://www.haehnelcraigwesq.com"
"Quentin","Birkner","Spoor Behrins Campbell & Young","7061 N 2nd St","Burnsville","Dakota","MN",55337,"952-702-7993","952-314-5871","qbirkner@aol.com","http://www.spoorbehrinscampbellyoung.com"
"Regenia","Kannady","Ken Jeter Store Equipment Inc","10759 Main St","Scottsdale","Maricopa","AZ",85260,"480-726-1280","480-205-5121","regenia.kannady@cox.net","http://www.kenjeterstoreequipmentinc.com"
"Sheron","Louissant","Potter, Brenda J Cpa","97 E 3rd St #9","Long Island City","Queens","NY",11101,"718-976-8610","718-613-9994","sheron@aol.com","http://www.potterbrendajcpa.com"
"Izetta","Funnell","Baird Kurtz & Dobson","82 Winsor St #54","Atlanta","Dekalb","GA",30340,"770-844-3447","770-584-4119","izetta.funnell@hotmail.com","http://www.bairdkurtzdobson.com"
"Rod
 olfo","Butzen","Minor, Cynthia A Esq","41 Steel Ct","Northfield","Rice","MN",55057,"507-210-3510","507-590-5237","rodolfo@hotmail.com","http://www.minorcynthiaaesq.com"
"Zona","Colla","Solove, Robert A Esq","49440 Dearborn St","Norwalk","Fairfield","CT","06854","203-461-1949","203-938-2557","zona@hotmail.com","http://www.soloverobertaesq.com"
"Serina","Zagen","Mark Ii Imports Inc","7 S Beverly Dr","Fort Wayne","Allen","IN",46802,"260-273-3725","260-382-4869","szagen@aol.com","http://www.markiiimportsinc.com"
"Paz","Sahagun","White Sign Div Ctrl Equip Co","919 Wall Blvd","Meridian","Lauderdale","MS",39307,"601-927-8287","601-249-4511","paz_sahagun@cox.net","http://www.whitesigndivctrlequipco.com"
"Markus","Lukasik","M & M Store Fixtures Co Inc","89 20th St E #779","Sterling Heights","Macomb","MI",48310,"586-970-7380","586-247-1614","markus@yahoo.com","http://www.mmstorefixturescoinc.com"
"Jaclyn","Bachman","Judah Caster & Wheel Co","721 Interstate 45 S","Colorado Springs","El Paso","
 CO",80919,"719-853-3600","719-223-2074","jaclyn@aol.com","http://www.judahcasterwheelco.com"
"Cyril","Daufeldt","Galaxy International Inc","3 Lawton St","New York","New York","NY",10013,"212-745-8484","212-422-5427","cyril_daufeldt@daufeldt.com","http://www.galaxyinternationalinc.com"
"Gayla","Schnitzler","Sigma Corp Of America","38 Pleasant Hill Rd","Hayward","Alameda","CA",94545,"510-686-3407","510-441-4055","gschnitzler@gmail.com","http://www.sigmacorpofamerica.com"
"Erick","Nievas","Soward, Anne Esq","45 E Acacia Ct","Chicago","Cook","IL",60624,"773-704-9903","773-359-6109","erick_nievas@aol.com","http://www.sowardanneesq.com"
"Jennie","Drymon","Osborne, Michelle M Esq","63728 Poway Rd #1","Scranton","Lackawanna","PA",18509,"570-218-4831","570-868-8688","jennie@cox.net","http://www.osbornemichellemesq.com"
"Mitsue","Scipione","Students In Free Entrprs Natl","77 222 Dr","Oroville","Butte","CA",95965,"530-986-9272","530-399-3254","mscipione@scipione.com","http://www.studentsinfree
 entrprsnatl.com"
"Ciara","Ventura","Johnson, Robert M Esq","53 W Carey St","Port Jervis","Orange","NY",12771,"845-823-8877","845-694-7919","cventura@yahoo.com","http://www.johnsonrobertmesq.com"
"Galen","Cantres","Del Charro Apartments","617 Nw 36th Ave","Brook Park","Cuyahoga","OH",44142,"216-600-6111","216-871-6876","galen@yahoo.com","http://www.delcharroapartments.com"
"Truman","Feichtner","Legal Search Inc","539 Coldwater Canyon Ave","Bloomfield","Essex","NJ","07003","973-852-2736","973-473-5108","tfeichtner@yahoo.com","http://www.legalsearchinc.com"
"Gail","Kitty","Service Supply Co Inc","735 Crawford Dr","Anchorage","Anchorage","AK",99501,"907-435-9166","907-770-3542","gail@kitty.com","http://www.servicesupplycoinc.com"
"Dalene","Schoeneck","Sameshima, Douglas J Esq","910 Rahway Ave","Philadelphia","Philadelphia","PA",19102,"215-268-1275","215-380-8820","dalene@schoeneck.org","http://www.sameshimadouglasjesq.com"
"Gertude","Witten","Thompson, John Randolph Jr","7 Tarrytown Rd"
 ,"Cincinnati","Hamilton","OH",45217,"513-977-7043","513-863-9471","gertude.witten@gmail.com","http://www.thompsonjohnrandolphjr.com"
"Lizbeth","Kohl","E T Balancing Co Inc","35433 Blake St #588","Gardena","Los Angeles","CA",90248,"310-699-1222","310-955-5788","lizbeth@yahoo.com","http://www.etbalancingcoinc.com"
"Glenn","Berray","Griswold, John E Esq","29 Cherry St #7073","Des Moines","Polk","IA",50315,"515-370-7348","515-372-1738","gberray@gmail.com","http://www.griswoldjohneesq.com"
"Lashandra","Klang","Acqua Group","810 N La Brea Ave","King of Prussia","Montgomery","PA",19406,"610-809-1818","610-378-7332","lashandra@yahoo.com","http://www.acquagroup.com"
"Lenna","Newville","Brooks, Morris J Jr","987 Main St","Raleigh","Wake","NC",27601,"919-623-2524","919-254-5987","lnewville@newville.com","http://www.brooksmorrisjjr.com"
"Laurel","Pagliuca","Printing Images Corp","36 Enterprise St Se","Richland","Benton","WA",99352,"509-695-5199","509-595-6485","laurel@yahoo.com","http://www.pri
 ntingimagescorp.com"
"Mireya","Frerking","Roberts Supply Co Inc","8429 Miller Rd","Pelham","Westchester","NY",10803,"914-868-5965","914-883-3061","mireya.frerking@hotmail.com","http://www.robertssupplycoinc.com"
"Annelle","Tagala","Vico Products Mfg Co","5 W 7th St","Parkville","Baltimore","MD",21234,"410-757-1035","410-234-2267","annelle@yahoo.com","http://www.vicoproductsmfgco.com"
"Dean","Ketelsen","J M Custom Design Millwork","2 Flynn Rd","Hicksville","Nassau","NY",11801,"516-847-4418","516-732-6649","dean_ketelsen@gmail.com","http://www.jmcustomdesignmillwork.com"
"Levi","Munis","Farrell & Johnson Office Equip","2094 Ne 36th Ave","Worcester","Worcester","MA","01603","508-456-4907","508-658-7802","levi.munis@gmail.com","http://www.farrelljohnsonofficeequip.com"
"Sylvie","Ryser","Millers Market & Deli","649 Tulane Ave","Tulsa","Tulsa","OK",74105,"918-644-9555","918-565-1706","sylvie@aol.com","http://www.millersmarketdeli.com"
"Sharee","Maile","Holiday Inn Naperville","2094 Montou
 r Blvd","Muskegon","Muskegon","MI",49442,"231-467-9978","231-265-6940","sharee_maile@aol.com","http://www.holidayinnnaperville.com"
"Cordelia","Storment","Burrows, Jon H Esq","393 Hammond Dr","Lafayette","Lafayette","LA",70506,"337-566-6001","337-255-3427","cordelia_storment@aol.com","http://www.burrowsjonhesq.com"
"Mollie","Mcdoniel","Dock Seal Specialty","8590 Lake Lizzie Dr","Bowling Green","Wood","OH",43402,"419-975-3182","419-417-4674","mollie_mcdoniel@yahoo.com","http://www.docksealspecialty.com"
"Brett","Mccullan","Five Star Limousines Of Tx Inc","87895 Concord Rd","La Mesa","San Diego","CA",91942,"619-461-9984","619-727-3892","brett.mccullan@mccullan.com","http://www.fivestarlimousinesoftxinc.com"
"Teddy","Pedrozo","Barkan, Neal J Esq","46314 Route 130","Bridgeport","Fairfield","CT","06610","203-892-3863","203-918-3939","teddy_pedrozo@aol.com","http://www.barkannealjesq.com"
"Tasia","Andreason","Campbell, Robert A","4 Cowesett Ave","Kearny","Hudson","NJ","07032","201-920-900
 2","201-969-7063","tasia_andreason@yahoo.com","http://www.campbellroberta.com"
"Hubert","Walthall","Dee, Deanna","95 Main Ave #2","Barberton","Summit","OH",44203,"330-903-1345","330-566-8898","hubert@walthall.org","http://www.deedeanna.com"
"Arthur","Farrow","Young, Timothy L Esq","28 S 7th St #2824","Englewood","Bergen","NJ","07631","201-238-5688","201-772-4377","arthur.farrow@yahoo.com","http://www.youngtimothylesq.com"
"Vilma","Berlanga","Wells, D Fred Esq","79 S Howell Ave","Grand Rapids","Kent","MI",49546,"616-737-3085","616-568-4113","vberlanga@berlanga.com","http://www.wellsdfredesq.com"
"Billye","Miro","Gray, Francine H Esq","36 Lancaster Dr Se","Pearl","Rankin","MS",39208,"601-567-5386","601-637-5479","billye_miro@cox.net","http://www.grayfrancinehesq.com"
"Glenna","Slayton","Toledo Iv Care","2759 Livingston Ave","Memphis","Shelby","TN",38118,"901-640-9178","901-869-4314","glenna_slayton@cox.net","http://www.toledoivcare.com"
"Mitzie","Hudnall","Cangro Transmission Co","17 
 Jersey Ave","Englewood","Arapahoe","CO",80110,"303-402-1940","303-997-7760","mitzie_hudnall@yahoo.com","http://www.cangrotransmissionco.com"
"Bernardine","Rodefer","Sat Poly Inc","2 W Grand Ave","Memphis","Shelby","TN",38112,"901-901-4726","901-739-5892","bernardine_rodefer@yahoo.com","http://www.satpolyinc.com"
"Staci","Schmaltz","Midwest Contracting & Mfg Inc","18 Coronado Ave #563","Pasadena","Los Angeles","CA",91106,"626-866-2339","626-293-7678","staci_schmaltz@aol.com","http://www.midwestcontractingmfginc.com"
"Nichelle","Meteer","Print Doctor","72 Beechwood Ter","Chicago","Cook","IL",60657,"773-225-9985","773-857-2231","nichelle_meteer@meteer.com","http://www.printdoctor.com"
"Janine","Rhoden","Nordic Group Inc","92 Broadway","Astoria","Queens","NY",11103,"718-228-5894","718-728-5051","jrhoden@yahoo.com","http://www.nordicgroupinc.com"
"Ettie","Hoopengardner","Jackson Millwork Co","39 Franklin Ave","Richland","Benton","WA",99352,"509-755-5393","509-847-3352","ettie.hoopengardn
 er@hotmail.com","http://www.jacksonmillworkco.com"
"Eden","Jayson","Harris Corporation","4 Iwaena St","Baltimore","Baltimore City","MD",21202,"410-890-7866","410-429-4888","eden_jayson@yahoo.com","http://www.harriscorporation.com"
"Lynelle","Auber","United Cerebral Palsy Of Ne Pa","32820 Corkwood Rd","Newark","Essex","NJ","07104","973-860-8610","973-605-6492","lynelle_auber@gmail.com","http://www.unitedcerebralpalsyofnepa.com"
"Merissa","Tomblin","One Day Surgery Center Inc","34 Raritan Center Pky","Bellflower","Los Angeles","CA",90706,"562-579-6900","562-719-7922","merissa.tomblin@gmail.com","http://www.onedaysurgerycenterinc.com"
"Golda","Kaniecki","Calaveras Prospect","6201 S Nevada Ave","Toms River","Ocean","NJ","08755","732-628-9909","732-617-5310","golda_kaniecki@yahoo.com","http://www.calaverasprospect.com"
"Catarina","Gleich","Terk, Robert E Esq","78 Maryland Dr #146","Denville","Morris","NJ","07834","973-210-3994","973-491-8723","catarina_gleich@hotmail.com","http://www.ter
 kroberteesq.com"
"Virgie","Kiel","Cullen, Terrence P Esq","76598 Rd  I 95 #1","Denver","Denver","CO",80216,"303-776-7548","303-845-5408","vkiel@hotmail.com","http://www.cullenterrencepesq.com"
"Jolene","Ostolaza","Central Die Casting Mfg Co Inc","1610 14th St Nw","Newport News","Newport News City","VA",23608,"757-682-7116","757-940-1741","jolene@yahoo.com","http://www.centraldiecastingmfgcoinc.com"
"Keneth","Borgman","Centerline Engineering","86350 Roszel Rd","Phoenix","Maricopa","AZ",85012,"602-919-4211","602-442-3092","keneth@yahoo.com","http://www.centerlineengineering.com"
"Rikki","Nayar","Targan & Kievit Pa","1644 Clove Rd","Miami","Miami-Dade","FL",33155,"305-968-9487","305-978-2069","rikki@nayar.com","http://www.targankievitpa.com"
"Elke","Sengbusch","Riley Riper Hollin & Colagreco","9 W Central Ave","Phoenix","Maricopa","AZ",85013,"602-896-2993","602-575-3457","elke_sengbusch@yahoo.com","http://www.rileyriperhollincolagreco.com"
"Hoa","Sarao","Kaplan, Joel S Esq","27846 Lafa
 yette Ave","Oak Hill","Volusia","FL",32759,"386-526-7800","386-599-7296","hoa@sarao.org","http://www.kaplanjoelsesq.com"
"Trinidad","Mcrae","Water Office","10276 Brooks St","San Francisco","San Francisco","CA",94105,"415-331-9634","415-419-1597","trinidad_mcrae@yahoo.com","http://www.wateroffice.com"
"Mari","Lueckenbach","Westbrooks, Nelson E Jr","1 Century Park E","San Diego","San Diego","CA",92110,"858-793-9684","858-228-5683","mari_lueckenbach@yahoo.com","http://www.westbrooksnelsonejr.com"
"Selma","Husser","Armon Communications","9 State Highway 57 #22","Jersey City","Hudson","NJ","07306","201-991-8369","201-772-7699","selma.husser@cox.net","http://www.armoncommunications.com"
"Antione","Onofrio","Jacobs & Gerber Inc","4 S Washington Ave","San Bernardino","San Bernardino","CA",92410,"909-430-7765","909-665-3223","aonofrio@onofrio.com","http://www.jacobsgerberinc.com"
"Luisa","Jurney","Forest Fire Laboratory","25 Se 176th Pl","Cambridge","Middlesex","MA","02138","617-365-2134","6
 17-544-2541","ljurney@hotmail.com","http://www.forestfirelaboratory.com"
"Clorinda","Heimann","Haughey, Charles Jr","105 Richmond Valley Rd","Escondido","San Diego","CA",92025,"760-291-5497","760-261-4786","clorinda.heimann@hotmail.com","http://www.haugheycharlesjr.com"
"Dick","Wenzinger","Wheaton Plastic Products","22 Spruce St #595","Gardena","Los Angeles","CA",90248,"310-510-9713","310-936-2258","dick@yahoo.com","http://www.wheatonplasticproducts.com"
"Ahmed","Angalich","Reese Plastics","2 W Beverly Blvd","Harrisburg","Dauphin","PA",17110,"717-528-8996","717-632-5831","ahmed.angalich@angalich.com","http://www.reeseplastics.com"
"Iluminada","Ohms","Nazette Marner Good Wendt","72 Southern Blvd","Mesa","Maricopa","AZ",85204,"480-293-2882","480-866-6544","iluminada.ohms@yahoo.com","http://www.nazettemarnergoodwendt.com"
"Joanna","Leinenbach","Levinson Axelrod Wheaton","1 Washington St","Lake Worth","Palm Beach","FL",33461,"561-470-4574","561-951-9734","joanna_leinenbach@hotmail.com",
 "http://www.levinsonaxelrodwheaton.com"
"Caprice","Suell","Egnor, W Dan Esq","90177 N 55th Ave","Nashville","Davidson","TN",37211,"615-246-1824","615-726-4537","caprice@aol.com","http://www.egnorwdanesq.com"
"Stephane","Myricks","Portland Central Thriftlodge","9 Tower Ave","Burlington","Boone","KY",41005,"859-717-7638","859-308-4286","stephane_myricks@cox.net","http://www.portlandcentralthriftlodge.com"
"Quentin","Swayze","Ulbrich Trucking","278 Bayview Ave","Milan","Monroe","MI",48160,"734-561-6170","734-851-8571","quentin_swayze@yahoo.com","http://www.ulbrichtrucking.com"
"Annmarie","Castros","Tipiak Inc","80312 W 32nd St","Conroe","Montgomery","TX",77301,"936-751-7961","936-937-2334","annmarie_castros@gmail.com","http://www.tipiakinc.com"
"Shonda","Greenbush","Saint George Well Drilling","82 Us Highway 46","Clifton","Passaic","NJ","07011","973-482-2430","973-644-2974","shonda_greenbush@cox.net","http://www.saintgeorgewelldrilling.com"
"Cecil","Lapage","Hawkes, Douglas D","4 Stova
 ll St #72","Union City","Hudson","NJ","07087","201-693-3967","201-856-2720","clapage@lapage.com","http://www.hawkesdouglasd.com"
"Jeanice","Claucherty","Accurel Systems Intrntl Corp","19 Amboy Ave","Miami","Miami-Dade","FL",33142,"305-988-4162","305-306-7834","jeanice.claucherty@yahoo.com","http://www.accurelsystemsintrntlcorp.com"
"Josphine","Villanueva","Santa Cruz Community Internet","63 Smith Ln #8343","Moss","Clay","TN",38575,"931-553-9774","931-486-6946","josphine_villanueva@villanueva.com","http://www.santacruzcommunityinternet.com"
"Daniel","Perruzza","Gersh & Danielson","11360 S Halsted St","Santa Ana","Orange","CA",92705,"714-771-3880","714-531-1391","dperruzza@perruzza.com","http://www.gershdanielson.com"
"Cassi","Wildfong","Cobb, James O Esq","26849 Jefferson Hwy","Rolling Meadows","Cook","IL",60008,"847-633-3216","847-755-9041","cassi.wildfong@aol.com","http://www.cobbjamesoesq.com"
"Britt","Galam","Wheatley Trucking Company","2500 Pringle Rd Se #508","Hatfield","Montgo
 mery","PA",19440,"215-888-3304","215-351-8523","britt@galam.org","http://www.wheatleytruckingcompany.com"
"Adell","Lipkin","Systems Graph Inc Ab Dick Dlr","65 Mountain View Dr","Whippany","Morris","NJ","07981","973-654-1561","973-662-8988","adell.lipkin@lipkin.com","http://www.systemsgraphincabdickdlr.com"
"Jacqueline","Rowling","John Hancock Mutl Life Ins Co","1 N San Saba","Erie","Erie","PA",16501,"814-865-8113","814-481-1700","jacqueline.rowling@yahoo.com","http://www.johnhancockmutllifeinsco.com"
"Lonny","Weglarz","History Division Of State","51120 State Route 18","Salt Lake City","Salt Lake","UT",84115,"801-293-9853","801-892-8781","lonny_weglarz@gmail.com","http://www.historydivisionofstate.com"
"Lonna","Diestel","Dimmock, Thomas J Esq","1482 College Ave","Fayetteville","Cumberland","NC",28301,"910-922-3672","910-200-7912","lonna_diestel@gmail.com","http://www.dimmockthomasjesq.com"
"Cristal","Samara","Intermed Inc","4119 Metropolitan Dr","Los Angeles","Los Angeles","CA",90021
 ,"213-975-8026","213-696-8004","cristal@cox.net","http://www.intermedinc.com"
"Kenneth","Grenet","Bank Of New York","2167 Sierra Rd","East Lansing","Ingham","MI",48823,"517-499-2322","517-867-8077","kenneth.grenet@grenet.org","http://www.bankofnewyork.com"
"Elli","Mclaird","Sportmaster Intrnatl","6 Sunrise Ave","Utica","Oneida","NY",13501,"315-818-2638","315-474-5570","emclaird@mclaird.com","http://www.sportmasterintrnatl.com"
"Alline","Jeanty","W W John Holden Inc","55713 Lake City Hwy","South Bend","St Joseph","IN",46601,"574-656-2800","574-405-1983","ajeanty@gmail.com","http://www.wwjohnholdeninc.com"
"Sharika","Eanes","Maccani & Delp","75698 N Fiesta Blvd","Orlando","Orange","FL",32806,"407-312-1691","407-472-1332","sharika.eanes@aol.com","http://www.maccanidelp.com"
"Nu","Mcnease","Amazonia Film Project","88 Sw 28th Ter","Harrison","Hudson","NJ","07029","973-751-9003","973-903-4175","nu@gmail.com","http://www.amazoniafilmproject.com"
"Daniela","Comnick","Water & Sewer Departmen
 t","7 Flowers Rd #403","Trenton","Mercer","NJ","08611","609-200-8577","609-398-2805","dcomnick@cox.net","http://www.watersewerdepartment.com"
"Cecilia","Colaizzo","Switchcraft Inc","4 Nw 12th St #3849","Madison","Dane","WI",53717,"608-382-4541","608-302-3387","cecilia_colaizzo@colaizzo.com","http://www.switchcraftinc.com"
"Leslie","Threets","C W D C Metal Fabricators","2 A Kelley Dr","Katonah","Westchester","NY",10536,"914-861-9748","914-396-2615","leslie@cox.net","http://www.cwdcmetalfabricators.com"
"Nan","Koppinger","Shimotani, Grace T","88827 Frankford Ave","Greensboro","Guilford","NC",27401,"336-370-5333","336-564-1492","nan@koppinger.com","http://www.shimotanigracet.com"
"Izetta","Dewar","Lisatoni, Jean Esq","2 W Scyene Rd #3","Baltimore","Baltimore City","MD",21217,"410-473-1708","410-522-7621","idewar@dewar.com","http://www.lisatonijeanesq.com"
"Tegan","Arceo","Ceramic Tile Sales Inc","62260 Park Stre","Monroe Township","Middlesex","NJ","08831","732-730-2692","732-705-6719",
 "tegan.arceo@arceo.org","http://www.ceramictilesalesinc.com"
"Ruthann","Keener","Maiden Craft Inc","3424 29th St Se","Kerrville","Kerr","TX",78028,"830-258-2769","830-919-5991","ruthann@hotmail.com","http://www.maidencraftinc.com"
"Joni","Breland","Carriage House Cllsn Rpr Inc","35 E Main St #43","Elk Grove Village","Cook","IL",60007,"847-519-5906","847-740-5304","joni_breland@cox.net","http://www.carriagehousecllsnrprinc.com"
"Vi","Rentfro","Video Workshop","7163 W Clark Rd","Freehold","Monmouth","NJ","07728","732-605-4781","732-724-7251","vrentfro@cox.net","http://www.videoworkshop.com"
"Colette","Kardas","Fresno Tile Center Inc","21575 S Apple Creek Rd","Omaha","Douglas","NE",68124,"402-896-5943","402-707-1602","colette.kardas@yahoo.com","http://www.fresnotilecenterinc.com"
"Malcolm","Tromblay","Versatile Sash & Woodwork","747 Leonis Blvd","Annandale","Fairfax","VA",22003,"703-221-5602","703-874-4248","malcolm_tromblay@cox.net","http://www.versatilesashwoodwork.com"
"Ryan","Harno
 s","Warner Electric Brk & Cltch Co","13 Gunnison St","Plano","Collin","TX",75075,"972-558-1665","972-961-4968","ryan@cox.net","http://www.warnerelectricbrkcltchco.com"
"Jess","Chaffins","New York Public Library","18 3rd Ave","New York","New York","NY",10016,"212-510-4633","212-428-9538","jess.chaffins@chaffins.org","http://www.newyorkpubliclibrary.com"
"Sharen","Bourbon","Mccaleb, John A Esq","62 W Austin St","Syosset","Nassau","NY",11791,"516-816-1541","516-749-3188","sbourbon@yahoo.com","http://www.mccalebjohnaesq.com"
"Nickolas","Juvera","United Oil Co Inc","177 S Rider Trl #52","Crystal River","Citrus","FL",34429,"352-598-8301","352-947-6152","nickolas_juvera@cox.net","http://www.unitedoilcoinc.com"
"Gary","Nunlee","Irving Foot Center","2 W Mount Royal Ave","Fortville","Hancock","IN",46040,"317-542-6023","317-887-8486","gary_nunlee@nunlee.org","http://www.irvingfootcenter.com"
"Diane","Devreese","Acme Supply Co","1953 Telegraph Rd","Saint Joseph","Buchanan","MO",64504,"816-557-9
 673","816-329-5565","diane@cox.net","http://www.acmesupplyco.com"
"Roslyn","Chavous","Mcrae, James L","63517 Dupont St","Jackson","Hinds","MS",39211,"601-234-9632","601-973-5754","roslyn.chavous@chavous.org","http://www.mcraejamesl.com"
"Glory","Schieler","Mcgraths Seafood","5 E Truman Rd","Abilene","Taylor","TX",79602,"325-869-2649","325-740-3778","glory@yahoo.com","http://www.mcgrathsseafood.com"
"Rasheeda","Sayaphon","Kummerer, J Michael Esq","251 Park Ave #979","Saratoga","Santa Clara","CA",95070,"408-805-4309","408-997-7490","rasheeda@aol.com","http://www.kummererjmichaelesq.com"
"Alpha","Palaia","Stoffer, James M Jr","43496 Commercial Dr #29","Cherry Hill","Camden","NJ","08003","856-312-2629","856-513-7024","alpha@yahoo.com","http://www.stofferjamesmjr.com"
"Refugia","Jacobos","North Central Fl Sfty Cncl","2184 Worth St","Hayward","Alameda","CA",94545,"510-974-8671","510-509-3496","refugia.jacobos@jacobos.com","http://www.northcentralflsftycncl.com"
"Shawnda","Yori","Fiorucci 
 Foods Usa Inc","50126 N Plankinton Ave","Longwood","Seminole","FL",32750,"40

<TRUNCATED>

[05/19] nifi git commit: NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Servic

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/pom.xml
new file mode 100644
index 0000000..d7d5605
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/pom.xml
@@ -0,0 +1,31 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      Licensed to the Apache Software Foundation (ASF) under one or more
+      contributor license agreements.  See the NOTICE file distributed with
+      this work for additional information regarding copyright ownership.
+      The ASF licenses this file to You under the Apache License, Version 2.0
+      (the "License"); you may not use this file except in compliance with
+      the License.  You may obtain a copy of the License at
+          http://www.apache.org/licenses/LICENSE-2.0
+      Unless required by applicable law or agreed to in writing, software
+      distributed under the License is distributed on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+      See the License for the specific language governing permissions and
+      limitations under the License.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-standard-services</artifactId>
+        <version>1.1.0-SNAPSHOT</version>
+    </parent>
+    
+    <artifactId>nifi-record-serialization-service-api</artifactId>
+    <packaging>jar</packaging>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/DataTypeValidator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/DataTypeValidator.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/DataTypeValidator.java
new file mode 100644
index 0000000..5a71086
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/DataTypeValidator.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+
+public class DataTypeValidator implements Validator {
+    private static final Set<String> validValues;
+    private static final Set<String> allowsFormatting;
+
+    static {
+        final Set<String> values = new HashSet<>();
+        values.add("string");
+        values.add("boolean");
+        values.add("byte");
+        values.add("char");
+        values.add("int");
+        values.add("long");
+        values.add("float");
+        values.add("double");
+        values.add("time");
+        values.add("date");
+        values.add("timestamp");
+        validValues = Collections.unmodifiableSet(values);
+
+        final Set<String> formattable = new HashSet<>();
+        formattable.add("date");
+        formattable.add("time");
+        formattable.add("timestmap");
+        allowsFormatting = Collections.unmodifiableSet(formattable);
+    }
+
+    @Override
+    public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+        final String[] splits = input.split("\\:");
+
+        final boolean valid;
+        if (splits.length == 2) {
+            final String type = splits[0].trim();
+            if (validValues.contains(type)) {
+                if (allowsFormatting.contains(splits[0].trim())) {
+                    valid = true;
+                } else {
+                    valid = false;
+                }
+            } else {
+                valid = false;
+            }
+        } else {
+            valid = validValues.contains(input.trim());
+        }
+
+        return new ValidationResult.Builder()
+            .subject(subject)
+            .input(input)
+            .valid(valid)
+            .explanation("Valid values for this property are: " + validValues
+                + ", where date, time, and timestamp may optionally contain a format (e.g., date:MM-dd-yyyy)")
+            .build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/MalformedRecordException.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/MalformedRecordException.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/MalformedRecordException.java
new file mode 100644
index 0000000..d45a850
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/MalformedRecordException.java
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+/**
+ * An Exception that can be thrown to indicate that data was read but could not properly be parsed
+ */
+public class MalformedRecordException extends Exception {
+    public MalformedRecordException(final String message) {
+        super(message);
+    }
+
+    public MalformedRecordException(final String message, final Throwable cause) {
+        super(message, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReader.java
new file mode 100644
index 0000000..a0cfc79
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReader.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+/**
+ * <p>
+ * A RowRecordReader is responsible for parsing data and returning a record at a time
+ * in order to allow the caller to iterate over the records individually.
+ * </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 RecordReader extends Closeable {
+
+    /**
+     * Returns the next record in the stream or <code>null</code> if no more records are available.
+     *
+     * @param schema the schema to use in order to determine how to interprets the fields in a record
+     * @return the next record in the stream or <code>null</code> if no more records are available.
+     *
+     * @throws IOException if unable to read from the underlying data
+     * @throws MalformedRecordException if an unrecoverable failure occurs when trying to parse a record
+     */
+    Record nextRecord() throws IOException, MalformedRecordException;
+
+    /**
+     * @return a RecordSchema that is appropriate for the records in the stream
+     * @throws MalformedRecordException if an unrecoverable failure occurs when trying to parse the underlying data
+     */
+    RecordSchema getSchema() throws MalformedRecordException;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriter.java
new file mode 100644
index 0000000..7d6fa1c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriter.java
@@ -0,0 +1,45 @@
+/*
+ * 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.OutputStream;
+
+import org.apache.nifi.serialization.record.RecordSet;
+
+/**
+ * <p>
+ * A ResultSetWriter is responsible for writing a ResultSet to a given {@link OutputStream}.
+ * </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 RecordSetWriter extends RecordWriter {
+    /**
+     * Writes the given result set to the given output stream
+     *
+     * @param recordSet the record set to serialize
+     * @param out the OutputStream to write to
+     * @return the results of writing the data
+     * @throws IOException if unable to write to the given OutputStream
+     */
+    WriteResult write(RecordSet recordSet, OutputStream out) throws IOException;
+}

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordWriter.java
new file mode 100644
index 0000000..eef8d82
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordWriter.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.nifi.serialization.record.Record;
+
+public interface RecordWriter {
+    /**
+     * Writes the given result set to the given output stream
+     *
+     * @param recordSet the record set to serialize
+     * @param out the OutputStream to write to
+     * @return the results of writing the data
+     * @throws IOException if unable to write to the given OutputStream
+     */
+    WriteResult write(Record record, OutputStream out) throws IOException;
+
+    /**
+     * @return the MIME Type that the Result Set Writer produces. This will be added to FlowFiles using
+     *         the mime.type attribute.
+     */
+    String getMimeType();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..5ef4c7c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RowRecordReaderFactory.java
@@ -0,0 +1,33 @@
+/*
+ * 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.logging.ComponentLog;
+
+/**
+ * <p>
+ * A Controller Service that is responsible for creating a {@link RecordReader}.
+ * </p>
+ */
+public interface RowRecordReaderFactory extends ControllerService {
+    RecordReader createRecordReader(InputStream in, ComponentLog logger) throws MalformedRecordException, IOException;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..246e0af
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
@@ -0,0 +1,126 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+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;
+
+public class SimpleRecordSchema implements RecordSchema {
+    private final List<RecordField> fields;
+    private final Map<String, Integer> fieldIndices;
+
+    public SimpleRecordSchema(final List<RecordField> fields) {
+        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++);
+        }
+    }
+
+    @Override
+    public List<RecordField> getFields() {
+        return fields;
+    }
+
+    @Override
+    public int getFieldCount() {
+        return fields.size();
+    }
+
+    @Override
+    public RecordField getField(final int index) {
+        return fields.get(index);
+    }
+
+    @Override
+    public List<DataType> getDataTypes() {
+        return getFields().stream().map(recordField -> recordField.getDataType())
+            .collect(Collectors.toList());
+    }
+
+    @Override
+    public List<String> getFieldNames() {
+        return getFields().stream().map(recordField -> recordField.getFieldName())
+            .collect(Collectors.toList());
+    }
+
+    @Override
+    public Optional<DataType> getDataType(final String fieldName) {
+        final OptionalInt idx = getFieldIndex(fieldName);
+        return idx.isPresent() ? Optional.of(fields.get(idx.getAsInt()).getDataType()) : Optional.empty();
+    }
+
+    private OptionalInt getFieldIndex(final String fieldName) {
+        final Integer index = fieldIndices.get(fieldName);
+        return index == null ? OptionalInt.empty() : OptionalInt.of(index);
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (obj == null) {
+            return false;
+        }
+        if (obj == this) {
+            return true;
+        }
+        if (!(obj instanceof RecordSchema)) {
+            return false;
+        }
+
+        final RecordSchema other = (RecordSchema) obj;
+        return fields.equals(other.getFields());
+    }
+
+    @Override
+    public int hashCode() {
+        return 143 + 3 * fields.hashCode();
+    }
+
+    @Override
+    public String toString() {
+        final StringBuilder sb = new StringBuilder("[");
+
+        for (int i = 0; i < fields.size(); i++) {
+            final RecordField field = fields.get(i);
+
+            sb.append("\"");
+            sb.append(field.getFieldName());
+            sb.append("\" : \"");
+            sb.append(field.getDataType());
+            sb.append("\"");
+
+            if (i < fields.size() - 1) {
+                sb.append(", ");
+            }
+        }
+        sb.append("]");
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/WriteResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/WriteResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/WriteResult.java
new file mode 100644
index 0000000..3fb2741
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/WriteResult.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.serialization;
+
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * <p>
+ * Provides information about what was written to an OutputStream by a {@link RecordSetWriter}.
+ * Instances of WriteResult are typically instantiated by calling the static method {@link WriteResult#of(int, Map)}
+ * or using {@link WriteResult#EMPTY}.
+ * </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 WriteResult {
+
+    /**
+     * @return the number of records written
+     */
+    int getRecordCount();
+
+    /**
+     * @return values that should be added to the FlowFile as attributes
+     */
+    Map<String, String> getAttributes();
+
+    /**
+     * Creates a WriteResult with the given record count and attributes
+     *
+     * @param recordCount the number of records written
+     * @param attributes the attributes to add to the FlowFile
+     * @return A {@link WriteResult} representing the given parameters
+     */
+    public static WriteResult of(final int recordCount, final Map<String, String> attributes) {
+        return new WriteResult() {
+            @Override
+            public int getRecordCount() {
+                return recordCount;
+            }
+
+            @Override
+            public Map<String, String> getAttributes() {
+                return attributes;
+            }
+        };
+    }
+
+    public static final WriteResult EMPTY = of(0, Collections.emptyMap());
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..0c187f1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/DataType.java
@@ -0,0 +1,95 @@
+/*
+ * 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.Collections;
+import java.util.List;
+import java.util.Optional;
+
+public class DataType {
+    private final RecordFieldType fieldType;
+    private final String format;
+
+    private final RecordSchema childSchema;
+    private final List<DataType> childTypes;
+
+    DataType(final RecordFieldType fieldType, final String format) {
+        this(fieldType, format, (RecordSchema) null);
+    }
+
+    DataType(final RecordFieldType fieldType, final String format, final RecordSchema childSchema) {
+        this.fieldType = fieldType;
+        this.format = format;
+        this.childSchema = childSchema;
+        this.childTypes = Collections.emptyList();
+    }
+
+    DataType(final RecordFieldType fieldType, final String format, final List<DataType> childTypes) {
+        this.fieldType = fieldType;
+        this.format = format;
+        this.childSchema = null;
+        this.childTypes = Collections.unmodifiableList(childTypes);
+    }
+
+
+    public String getFormat() {
+        return format;
+    }
+
+    public RecordFieldType getFieldType() {
+        return fieldType;
+    }
+
+    public Optional<RecordSchema> getChildRecordSchema() {
+        return Optional.ofNullable(childSchema);
+    }
+
+    public List<DataType> getPossibleTypes() {
+        return childTypes;
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 + 41 * fieldType.hashCode() + 41 * (format == null ? 0 : format.hashCode());
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (obj == this) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (!(obj instanceof DataType)) {
+            return false;
+        }
+
+        final DataType other = (DataType) obj;
+        return fieldType.equals(other.fieldType) && ((format == null && other.format == null) || (format != null && format.equals(other.format)));
+    }
+
+    @Override
+    public String toString() {
+        if (format == null) {
+            return fieldType.toString();
+        } else {
+            return fieldType.toString() + ":" + format;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ListRecordSet.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ListRecordSet.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ListRecordSet.java
new file mode 100644
index 0000000..3968f50
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ListRecordSet.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization.record;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class ListRecordSet implements RecordSet {
+    private final Iterator<Record> recordItr;
+    private final RecordSchema schema;
+
+    public ListRecordSet(final RecordSchema schema, final List<Record> records) {
+        this.schema = schema;
+
+        final List<Record> copy = new ArrayList<>(records);
+        recordItr = copy.iterator();
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return schema;
+    }
+
+    @Override
+    public Record next() {
+        return recordItr.hasNext() ? recordItr.next() : null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..f3f9024
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
@@ -0,0 +1,322 @@
+/*
+ * 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.sql.Time;
+import java.sql.Timestamp;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+public class MapRecord implements Record {
+    private final RecordSchema schema;
+    private final Map<String, Object> values;
+
+    public MapRecord(final RecordSchema schema, final Map<String, Object> values) {
+        this.schema = Objects.requireNonNull(schema);
+        this.values = Objects.requireNonNull(values);
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return schema;
+    }
+
+    @Override
+    public Object[] getValues() {
+        final Object[] values = new Object[schema.getFieldCount()];
+        int i = 0;
+        for (final String fieldName : schema.getFieldNames()) {
+            values[i++] = getValue(fieldName);
+        }
+        return values;
+    }
+
+    @Override
+    public Object getValue(final String fieldName) {
+        return values.get(fieldName);
+    }
+
+    @Override
+    public String getAsString(final String fieldName) {
+        final Optional<DataType> dataTypeOption = schema.getDataType(fieldName);
+        if (!dataTypeOption.isPresent()) {
+            return null;
+        }
+
+        return convertToString(getValue(fieldName), dataTypeOption.get().getFormat());
+    }
+
+    @Override
+    public String getAsString(final String fieldName, final String format) {
+        return convertToString(getValue(fieldName), format);
+    }
+
+    private String getFormat(final String optionalFormat, final RecordFieldType fieldType) {
+        return (optionalFormat == null) ? fieldType.getDefaultFormat() : optionalFormat;
+    }
+
+    private String convertToString(final Object value, final String format) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof java.sql.Date) {
+            java.sql.Date date = (java.sql.Date) value;
+            final long time = date.getTime();
+            return new SimpleDateFormat(getFormat(format, RecordFieldType.DATE)).format(new java.util.Date(time));
+        }
+        if (value instanceof java.util.Date) {
+            return new SimpleDateFormat(getFormat(format, RecordFieldType.DATE)).format((java.util.Date) value);
+        }
+        if (value instanceof Timestamp) {
+            java.sql.Timestamp date = (java.sql.Timestamp) value;
+            final long time = date.getTime();
+            return new SimpleDateFormat(getFormat(format, RecordFieldType.TIMESTAMP)).format(new java.util.Date(time));
+        }
+        if (value instanceof Time) {
+            java.sql.Time date = (java.sql.Time) value;
+            final long time = date.getTime();
+            return new SimpleDateFormat(getFormat(format, RecordFieldType.TIME)).format(new java.util.Date(time));
+        }
+
+        return value.toString();
+    }
+
+    @Override
+    public Long getAsLong(final String fieldName) {
+        return convertToLong(getValue(fieldName), fieldName);
+    }
+
+    private Long convertToLong(final Object value, final Object fieldDesc) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).longValue();
+        }
+        if (value instanceof String) {
+            return Long.parseLong((String) value);
+        }
+        if (value instanceof Date) {
+            return ((Date) value).getTime();
+        }
+
+        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Long for field " + fieldDesc);
+    }
+
+    @Override
+    public Integer getAsInt(final String fieldName) {
+        return convertToInt(getValue(fieldName), fieldName);
+    }
+
+    private Integer convertToInt(final Object value, final Object fieldDesc) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).intValue();
+        }
+        if (value instanceof String) {
+            return Integer.parseInt((String) value);
+        }
+
+        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Integer for field " + fieldDesc);
+    }
+
+
+    @Override
+    public Double getAsDouble(final String fieldName) {
+        return convertToDouble(getValue(fieldName), fieldName);
+    }
+
+    private Double convertToDouble(final Object value, final Object fieldDesc) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).doubleValue();
+        }
+        if (value instanceof String) {
+            return Double.parseDouble((String) value);
+        }
+
+        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Double for field " + fieldDesc);
+    }
+
+    @Override
+    public Float getAsFloat(final String fieldName) {
+        return convertToFloat(getValue(fieldName), fieldName);
+    }
+
+    private Float convertToFloat(final Object value, final Object fieldDesc) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Number) {
+            return ((Number) value).floatValue();
+        }
+        if (value instanceof String) {
+            return Float.parseFloat((String) value);
+        }
+
+        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Float for field " + fieldDesc);
+    }
+
+    @Override
+    public Record getAsRecord(String fieldName) {
+        return convertToRecord(getValue(fieldName), fieldName);
+    }
+
+    private Record convertToRecord(final Object value, final Object fieldDesc) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Record) {
+            return (Record) value;
+        }
+
+        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Record for field " + fieldDesc);
+    }
+
+
+    @Override
+    public Boolean getAsBoolean(final String fieldName) {
+        return convertToBoolean(getValue(fieldName), fieldName);
+    }
+
+    private Boolean convertToBoolean(final Object value, final Object fieldDesc) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Boolean) {
+            return (Boolean) value;
+        }
+        if (value instanceof String) {
+            final String string = (String) value;
+            if (string.equalsIgnoreCase("true") || string.equalsIgnoreCase("t")) {
+                return Boolean.TRUE;
+            }
+
+            if (string.equalsIgnoreCase("false") || string.equals("f")) {
+                return Boolean.FALSE;
+            }
+
+            throw new TypeMismatchException("Cannot convert String value to Boolean for field " + fieldDesc + " because it is not a valid boolean value");
+        }
+
+        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Boolean for field " + fieldDesc);
+    }
+
+    @Override
+    public Date getAsDate(final String fieldName) {
+        final Optional<DataType> dataTypeOption = schema.getDataType(fieldName);
+        if (!dataTypeOption.isPresent()) {
+            return null;
+        }
+
+        return convertToDate(getValue(fieldName), fieldName, dataTypeOption.get().getFormat());
+    }
+
+    @Override
+    public Date getAsDate(final String fieldName, final String format) {
+        return convertToDate(getValue(fieldName), fieldName, format);
+    }
+
+    private Date convertToDate(final Object value, final Object fieldDesc, final String format) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Date) {
+            return (Date) value;
+        }
+        if (value instanceof Number) {
+            final Long time = ((Number) value).longValue();
+            return new Date(time);
+        }
+        if (value instanceof java.sql.Date) {
+            return new Date(((java.sql.Date) value).getTime());
+        }
+        if (value instanceof String) {
+            try {
+                return new SimpleDateFormat(getFormat(format, RecordFieldType.DATE)).parse((String) value);
+            } catch (final ParseException e) {
+                throw new TypeMismatchException("Cannot convert String value to date for field " + fieldDesc + " because it is not in the correct format of: " + format, e);
+            }
+        }
+
+        throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Boolean for field " + fieldDesc);
+    }
+
+    @Override
+    public Object[] getAsArray(final String fieldName) {
+        return convertToArray(getValue(fieldName));
+    }
+
+    private Object[] convertToArray(final Object value) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof Object[]) {
+            return (Object[]) value;
+        }
+
+        if (value instanceof List) {
+            return ((List<?>) value).toArray();
+        }
+
+        return new Object[] {value};
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 + 41 * values.hashCode() + 7 * schema.hashCode();
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (obj == this) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (!(obj instanceof MapRecord)) {
+            return false;
+        }
+        final MapRecord other = (MapRecord) obj;
+        return schema.equals(other.schema) && values.equals(other.values);
+    }
+
+    @Override
+    public String toString() {
+        return "MapRecord[values=" + values + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..ca85741
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/Record.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.serialization.record;
+
+import java.util.Date;
+
+public interface Record {
+
+    RecordSchema getSchema();
+
+    /**
+     * <p>
+     * Returns a view of the the values of the fields in this Record.
+     * </p>
+     *
+     * <b>NOTE:</b> The array that is returned may be an underlying array that is backing
+     * the contents of the Record. As such, modifying the array in any way may result in
+     * modifying the record.
+     *
+     * @return a view of the values of the fields in this Record
+     */
+    Object[] getValues();
+
+    Object getValue(String fieldName);
+
+    String getAsString(String fieldName);
+
+    String getAsString(String fieldName, String format);
+
+    Long getAsLong(String fieldName);
+
+    Integer getAsInt(String fieldName);
+
+    Double getAsDouble(String fieldName);
+
+    Float getAsFloat(String fieldName);
+
+    Record getAsRecord(String fieldName);
+
+    Boolean getAsBoolean(String fieldName);
+
+    Date getAsDate(String fieldName);
+
+    Date getAsDate(String fieldName, String format);
+
+    Object[] getAsArray(String fieldName);
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..135ae66
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordField.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.serialization.record;
+
+public class RecordField {
+    private final String fieldName;
+    private final DataType dataType;
+
+    public RecordField(final String fieldName, final DataType dataType) {
+        this.fieldName = fieldName;
+        this.dataType = dataType;
+    }
+
+    public String getFieldName() {
+        return fieldName;
+    }
+
+    public DataType getDataType() {
+        return dataType;
+    }
+
+    @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());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+
+        RecordField other = (RecordField) obj;
+        return dataType.equals(other.getDataType()) && fieldName.equals(other.getFieldName());
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..8ad212b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
@@ -0,0 +1,114 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public enum RecordFieldType {
+    STRING("string"),
+    BOOLEAN("boolean"),
+    BYTE("byte"),
+    CHAR("char"),
+    SHORT("short"),
+    INT("int"),
+    BIGINT("bigint"),
+    LONG("long"),
+    FLOAT("float"),
+    DOUBLE("double"),
+    DATE("date", "yyyy-MM-dd"),
+    TIME("time", "HH:mm:ss"),
+    TIMESTAMP("timestamp", "yyyy-MM-dd HH:mm:ss"),
+    RECORD("record"),
+    CHOICE("choice"),
+    ARRAY("array");
+
+
+    private static final Map<String, RecordFieldType> SIMPLE_NAME_MAP = new HashMap<String, RecordFieldType>();
+
+    static {
+      for (RecordFieldType value : values()) {
+        SIMPLE_NAME_MAP.put(value.simpleName, value);
+      }
+    }
+
+    private final String simpleName;
+    private final String defaultFormat;
+    private final DataType defaultDataType;
+
+    private RecordFieldType(final String simpleName) {
+        this(simpleName, null);
+    }
+
+    private RecordFieldType(final String simpleName, final String defaultFormat) {
+        this.simpleName = simpleName;
+        this.defaultFormat = defaultFormat;
+        this.defaultDataType = new DataType(this, defaultFormat);
+    }
+
+    public String getDefaultFormat() {
+        return defaultFormat;
+    }
+
+    /**
+     * @return the DataType with the default format
+     */
+    public DataType getDataType() {
+        return defaultDataType;
+    }
+
+    public DataType getDataType(final String format) {
+        return new DataType(this, format);
+    }
+
+    /**
+     * Returns a Data Type that represents a "RECORD" type with the given schema.
+     *
+     * @param childSchema the Schema for the Record
+     * @return a DataType that represents a Record with the given schema, or <code>null</code> if this RecordFieldType
+     *         is not the RECORD type.
+     */
+    public DataType getDataType(final RecordSchema childSchema) {
+        if (this != RECORD) {
+            return null;
+        }
+
+        return new DataType(this, getDefaultFormat(), childSchema);
+    }
+
+    /**
+     * Returns a Data Type that represents a "CHOICE" of multiple possible types. This method is
+     * only applicable for a RecordFieldType of {@link #CHOICE}.
+     *
+     * @param possibleChildTypes the possible types that are allowable
+     * @return a DataType that represents a "CHOICE" of multiple possible types, or <code>null</code> if this RecordFieldType
+     *         is not the CHOICE type.
+     */
+    public DataType getDataType(final List<DataType> possibleChildTypes) {
+        if (this != CHOICE) {
+            return null;
+        }
+
+        return new DataType(this, getDefaultFormat(), possibleChildTypes);
+    }
+
+    public static RecordFieldType of(final String typeString) {
+      return SIMPLE_NAME_MAP.get(typeString);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..115fb51
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java
@@ -0,0 +1,58 @@
+/*
+ * 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.List;
+import java.util.Optional;
+
+public interface RecordSchema {
+    /**
+     * @return the list of fields that are present in the schema
+     */
+    List<RecordField> getFields();
+
+    /**
+     * @return the number of fields in the schema
+     */
+    int getFieldCount();
+
+    /**
+     * @param index the 0-based index of which field to return
+     * @return the index'th field
+     *
+     * @throws IndexOutOfBoundsException if the index is < 0 or >= the number of fields (determined by {@link #getFieldCount()}).
+     */
+    RecordField getField(int index);
+
+    /**
+     * @return the data types of the fields
+     */
+    List<DataType> getDataTypes();
+
+    /**
+     * @return the names of the fields
+     */
+    List<String> getFieldNames();
+
+    /**
+     * @param fieldName the name of the field whose type is desired
+     * @return the RecordFieldType associated with the field that has the given name, or
+     *         <code>null</code> if the schema does not contain a field with the given name
+     */
+    Optional<DataType> getDataType(String fieldName);
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSet.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSet.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSet.java
new file mode 100644
index 0000000..25bbcdc
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSet.java
@@ -0,0 +1,53 @@
+/*
+ * 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.io.IOException;
+
+public interface RecordSet {
+
+    /**
+     * @return the {@link RecordSchema} that applies to the records in this RecordSet
+     */
+    RecordSchema getSchema() throws IOException;
+
+    /**
+     * @return the next {@link Record} in the set or <code>null</code> if there are no more records
+     */
+    Record next() throws IOException;
+
+    public static RecordSet of(final RecordSchema schema, final Record... records) {
+        return new RecordSet() {
+            private int index = 0;
+
+            @Override
+            public RecordSchema getSchema() {
+                return schema;
+            }
+
+            @Override
+            public Record next() {
+                if (index >= records.length) {
+                    return null;
+                }
+
+                return records[index++];
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..e166918
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java
@@ -0,0 +1,169 @@
+/*
+ * 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.io.Closeable;
+import java.io.IOException;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Types;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ResultSetRecordSet implements RecordSet, Closeable {
+    private static final Logger logger = LoggerFactory.getLogger(ResultSetRecordSet.class);
+    private final ResultSet rs;
+    private final RecordSchema schema;
+    private final Set<String> rsColumnNames;
+
+    public ResultSetRecordSet(final ResultSet rs) throws SQLException {
+        this.rs = rs;
+        this.schema = createSchema(rs);
+
+        rsColumnNames = new HashSet<>();
+        final ResultSetMetaData metadata = rs.getMetaData();
+        for (int i = 0; i < metadata.getColumnCount(); i++) {
+            rsColumnNames.add(metadata.getColumnLabel(i + 1));
+        }
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return schema;
+    }
+
+    @Override
+    public Record next() throws IOException {
+        try {
+            if (rs.next()) {
+                return createRecord(rs);
+            }
+        } catch (final SQLException e) {
+            throw new IOException("Could not obtain next record from ResultSet", e);
+        }
+
+        return null;
+    }
+
+    @Override
+    public void close() {
+        try {
+            rs.close();
+        } catch (final SQLException e) {
+            logger.error("Failed to close ResultSet", e);
+        }
+    }
+
+    private Record createRecord(final ResultSet rs) throws SQLException {
+        final Map<String, Object> values = new HashMap<>(schema.getFieldCount());
+
+        for (final RecordField field : schema.getFields()) {
+            final String fieldName = field.getFieldName();
+
+            final Object value;
+            if (rsColumnNames.contains(fieldName)) {
+                value = rs.getObject(field.getFieldName());
+            } else {
+                value = null;
+            }
+
+            values.put(fieldName, value);
+        }
+
+        return new MapRecord(schema, values);
+    }
+
+    private static RecordSchema createSchema(final ResultSet rs) throws SQLException {
+        final ResultSetMetaData metadata = rs.getMetaData();
+        final int numCols = metadata.getColumnCount();
+        final List<RecordField> fields = new ArrayList<>(numCols);
+
+        for (int i = 0; i < numCols; i++) {
+            final int column = i + 1;
+            final int sqlType = metadata.getColumnType(column);
+
+            final RecordFieldType fieldType = getFieldType(sqlType);
+            final String fieldName = metadata.getColumnLabel(column);
+            final RecordField field = new RecordField(fieldName, fieldType.getDataType());
+            fields.add(field);
+        }
+
+        return new SimpleRecordSchema(fields);
+    }
+
+    private static RecordFieldType getFieldType(final int sqlType) {
+        switch (sqlType) {
+            case Types.ARRAY:
+                return RecordFieldType.ARRAY;
+            case Types.BIGINT:
+            case Types.ROWID:
+                return RecordFieldType.LONG;
+            case Types.BINARY:
+            case Types.LONGVARBINARY:
+            case Types.VARBINARY:
+                return RecordFieldType.ARRAY;
+            case Types.BIT:
+            case Types.BOOLEAN:
+                return RecordFieldType.BOOLEAN;
+            case Types.CHAR:
+                return RecordFieldType.CHAR;
+            case Types.DATE:
+                return RecordFieldType.DATE;
+            case Types.DECIMAL:
+            case Types.DOUBLE:
+            case Types.NUMERIC:
+            case Types.REAL:
+                return RecordFieldType.DOUBLE;
+            case Types.FLOAT:
+                return RecordFieldType.FLOAT;
+            case Types.INTEGER:
+                return RecordFieldType.INT;
+            case Types.SMALLINT:
+                return RecordFieldType.SHORT;
+            case Types.TINYINT:
+                return RecordFieldType.BYTE;
+            case Types.LONGNVARCHAR:
+            case Types.LONGVARCHAR:
+            case Types.NCHAR:
+            case Types.NULL:
+            case Types.NVARCHAR:
+            case Types.VARCHAR:
+                return RecordFieldType.STRING;
+            case Types.OTHER:
+            case Types.JAVA_OBJECT:
+                return RecordFieldType.RECORD;
+            case Types.TIME:
+            case Types.TIME_WITH_TIMEZONE:
+                return RecordFieldType.TIME;
+            case Types.TIMESTAMP:
+            case Types.TIMESTAMP_WITH_TIMEZONE:
+                return RecordFieldType.TIMESTAMP;
+        }
+
+        return RecordFieldType.STRING;
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/pom.xml
new file mode 100644
index 0000000..2a6f240
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/pom.xml
@@ -0,0 +1,41 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      Licensed to the Apache Software Foundation (ASF) under one or more
+      contributor license agreements.  See the NOTICE file distributed with
+      this work for additional information regarding copyright ownership.
+      The ASF licenses this file to You under the Apache License, Version 2.0
+      (the "License"); you may not use this file except in compliance with
+      the License.  You may obtain a copy of the License at
+          http://www.apache.org/licenses/LICENSE-2.0
+      Unless required by applicable law or agreed to in writing, software
+      distributed under the License is distributed on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+      See the License for the specific language governing permissions and
+      limitations under the License.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-record-serialization-services-bundle</artifactId>
+        <version>1.2.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-record-serialization-services-nar</artifactId>
+    <packaging>nar</packaging>
+    <properties>
+        <maven.javadoc.skip>true</maven.javadoc.skip>
+        <source.skip>true</source.skip>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services</artifactId>
+            <version>1.2.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..581eac4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,269 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+APACHE NIFI SUBCOMPONENTS:
+
+The Apache NiFi project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses. 
+
+The binary distribution of this product bundles 'Hamcrest' which is available
+under a BSD license.  More details found here: http://hamcrest.org.
+
+	Copyright (c) 2000-2006, www.hamcrest.org
+	All rights reserved.
+	
+	Redistribution and use in source and binary forms, with or without
+	modification, are permitted provided that the following conditions are met:
+	
+	Redistributions of source code must retain the above copyright notice, this list of
+	conditions and the following disclaimer. Redistributions in binary form must reproduce
+	the above copyright notice, this list of conditions and the following disclaimer in
+	the documentation and/or other materials provided with the distribution.
+	
+	Neither the name of Hamcrest nor the names of its contributors may be used to endorse
+	or promote products derived from this software without specific prior written
+	permission.
+	
+	THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
+	EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+	OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT
+	SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
+	INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED
+	TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
+	BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+	CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY
+	WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
+	DAMAGE.
+
+This product bundles 'asm' which is available under a 3-Clause BSD style license.
+For details see http://asm.ow2.org/asmdex-license.html
+
+    Copyright (c) 2012 France T�l�com
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions
+    are met:
+    1. Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+    2. Redistributions in binary form must reproduce the above copyright
+       notice, this list of conditions and the following disclaimer in the
+       documentation and/or other materials provided with the distribution.
+    3. Neither the name of the copyright holders nor the names of its
+       contributors may be used to endorse or promote products derived from
+       this software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+    THE POSSIBILITY OF SUCH DAMAGE.
+


[09/19] nifi git commit: NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Servic

Posted by ma...@apache.org.
NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Services for reading and writing specific data types

Signed-off-by: Matt Burgess <ma...@apache.org>


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

Branch: refs/heads/master
Commit: a88d3bfa3c53d9cbe375f2b89eaa9248eb92df29
Parents: 4d5872a
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Jul 11 14:57:00 2016 -0400
Committer: Matt Burgess <ma...@apache.org>
Committed: Tue Apr 11 19:29:04 2017 -0400

----------------------------------------------------------------------
 nifi-assembly/pom.xml                           |  59 +-
 .../apache/nifi/util/MockProcessSession.java    |   4 +-
 .../nifi/cluster/manager/NodeResponse.java      |   2 +-
 .../repository/StandardProcessSession.java      |   8 +-
 .../nifi/processor/SimpleProcessLogger.java     |  75 ++-
 .../src/main/resources/META-INF/NOTICE          |  16 +
 .../nifi-standard-processors/pom.xml            |  27 +-
 .../calcite/adapter/csv/CsvEnumerator2.java     | 303 -----------
 .../apache/calcite/adapter/csv/CsvSchema2.java  |  98 ----
 .../calcite/adapter/csv/CsvSchemaFactory2.java  |  53 --
 .../calcite/adapter/csv/CsvTableScan2.java      | 104 ----
 .../adapter/csv/CsvTranslatableTable2.java      | 121 -----
 .../processors/standard/FilterCSVColumns.java   | 258 ---------
 .../nifi/processors/standard/QueryFlowFile.java | 541 +++++++++++++++++++
 .../nifi/queryflowfile/FlowFileEnumerator.java  | 150 +++++
 .../FlowFileProjectTableScanRule.java           |  76 +++
 .../nifi/queryflowfile/FlowFileTable.java       | 203 +++++++
 .../nifi/queryflowfile/FlowFileTableScan.java   |  91 ++++
 .../org.apache.nifi.processor.Processor         |   1 +
 .../additionalDetails.html                      |  47 ++
 .../standard/TestFilterCSVColumns.java          | 117 ----
 .../processors/standard/TestQueryFlowFile.java  | 379 +++++++++++++
 .../resources/TestFilterCSVColumns/Numeric.csv  |   5 -
 .../resources/TestFilterCSVColumns/US500.csv    |   1 -
 .../TestFilterCSVColumns/US500_typeless.csv     |   1 -
 .../pom.xml                                     |  31 ++
 .../nifi/serialization/DataTypeValidator.java   |  82 +++
 .../serialization/MalformedRecordException.java |  31 ++
 .../apache/nifi/serialization/RecordReader.java |  55 ++
 .../nifi/serialization/RecordSetWriter.java     |  45 ++
 .../serialization/RecordSetWriterFactory.java   |  30 +
 .../apache/nifi/serialization/RecordWriter.java |  41 ++
 .../serialization/RowRecordReaderFactory.java   |  33 ++
 .../nifi/serialization/SimpleRecordSchema.java  | 126 +++++
 .../apache/nifi/serialization/WriteResult.java  |  69 +++
 .../nifi/serialization/record/DataType.java     |  95 ++++
 .../serialization/record/ListRecordSet.java     |  44 ++
 .../nifi/serialization/record/MapRecord.java    | 322 +++++++++++
 .../nifi/serialization/record/Record.java       |  62 +++
 .../nifi/serialization/record/RecordField.java  |  64 +++
 .../serialization/record/RecordFieldType.java   | 114 ++++
 .../nifi/serialization/record/RecordSchema.java |  58 ++
 .../nifi/serialization/record/RecordSet.java    |  53 ++
 .../record/ResultSetRecordSet.java              | 169 ++++++
 .../record/TypeMismatchException.java           |  28 +
 .../pom.xml                                     |  41 ++
 .../src/main/resources/META-INF/LICENSE         | 269 +++++++++
 .../src/main/resources/META-INF/NOTICE          |  77 +++
 .../.gitignore                                  |   1 +
 .../nifi-record-serialization-services/pom.xml  |  94 ++++
 .../java/org/apache/nifi/avro/AvroReader.java   |  40 ++
 .../org/apache/nifi/avro/AvroRecordReader.java  | 254 +++++++++
 .../apache/nifi/avro/AvroRecordSetWriter.java   |  67 +++
 .../apache/nifi/avro/AvroSchemaValidator.java   |  45 ++
 .../org/apache/nifi/avro/WriteAvroResult.java   | 286 ++++++++++
 .../java/org/apache/nifi/csv/CSVReader.java     |  49 ++
 .../org/apache/nifi/csv/CSVRecordReader.java    | 216 ++++++++
 .../org/apache/nifi/csv/CSVRecordSetWriter.java |  37 ++
 .../org/apache/nifi/csv/WriteCSVResult.java     | 127 +++++
 .../nifi/grok/GrokExpressionValidator.java      |  48 ++
 .../java/org/apache/nifi/grok/GrokReader.java   |  99 ++++
 .../org/apache/nifi/grok/GrokRecordReader.java  | 323 +++++++++++
 .../nifi/json/AbstractJsonRowRecordReader.java  | 307 +++++++++++
 .../org/apache/nifi/json/JsonPathReader.java    | 126 +++++
 .../nifi/json/JsonPathRowRecordReader.java      | 280 ++++++++++
 .../org/apache/nifi/json/JsonPathValidator.java |  60 ++
 .../apache/nifi/json/JsonRecordSetWriter.java   |  66 +++
 .../org/apache/nifi/json/JsonTreeReader.java    |  56 ++
 .../nifi/json/JsonTreeRowRecordReader.java      | 115 ++++
 .../org/apache/nifi/json/PropertyNameUtil.java  |  88 +++
 .../org/apache/nifi/json/WriteJsonResult.java   | 309 +++++++++++
 .../serialization/AbstractRecordSetWriter.java  |  84 +++
 .../nifi/serialization/DataTypeUtils.java       | 165 ++++++
 .../SimpleDateFormatValidator.java              |  48 ++
 .../UserTypeOverrideRowReader.java              |  78 +++
 .../nifi/text/FreeFormTextRecordSetWriter.java  |  80 +++
 .../apache/nifi/text/FreeFormTextWriter.java    |  99 ++++
 ...org.apache.nifi.controller.ControllerService |  28 +
 .../main/resources/default-grok-patterns.txt    | 115 ++++
 .../additionalDetails.html                      | 185 +++++++
 .../additionalDetails.html                      | 396 ++++++++++++++
 .../additionalDetails.html                      | 227 ++++++++
 .../additionalDetails.html                      | 102 ++++
 .../apache/nifi/avro/TestAvroRecordReader.java  | 221 ++++++++
 .../apache/nifi/csv/TestCSVRecordReader.java    | 122 +++++
 .../org/apache/nifi/csv/TestWriteCSVResult.java | 121 +++++
 .../apache/nifi/grok/TestGrokRecordReader.java  | 190 +++++++
 .../nifi/json/TestJsonPathRowRecordReader.java  | 292 ++++++++++
 .../nifi/json/TestJsonTreeRowRecordReader.java  | 266 +++++++++
 .../apache/nifi/json/TestWriteJsonResult.java   | 102 ++++
 .../test/resources/csv/extra-white-space.csv    |   9 +
 .../test/resources/csv/multi-bank-account.csv   |   3 +
 .../test/resources/csv/single-bank-account.csv  |   2 +
 .../resources/grok/error-with-stack-trace.log   |  25 +
 ...ifi-log-sample-multiline-with-stacktrace.log |  29 +
 .../src/test/resources/grok/nifi-log-sample.log |   5 +
 .../resources/grok/single-line-log-messages.txt |   5 +
 .../bank-account-array-different-schemas.json   |  30 +
 .../bank-account-array-optional-balance.json    |  29 +
 .../test/resources/json/bank-account-array.json |  21 +
 .../test/resources/json/json-with-unicode.json  |   9 +
 .../test/resources/json/output/dataTypes.json   |  18 +
 .../resources/json/primitive-type-array.json    |  13 +
 .../resources/json/single-bank-account.json     |  10 +
 .../json/single-element-nested-array.json       |  16 +
 .../resources/json/single-element-nested.json   |  13 +
 .../pom.xml                                     |  30 +
 .../nifi-standard-services-api-nar/pom.xml      |   5 +
 nifi-nar-bundles/nifi-standard-services/pom.xml |   2 +
 pom.xml                                         |  11 +
 110 files changed, 9838 insertions(+), 1140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index 070beba..4a2babb 100755
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -1,15 +1,16 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!-- 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. -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor 
+    license agreements. See the NOTICE file distributed with this work for additional 
+    information regarding copyright ownership. The ASF licenses this file to 
+    You under the Apache License, Version 2.0 (the "License"); you may not use 
+    this file except in compliance with the License. You may obtain a copy of 
+    the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required 
+    by applicable law or agreed to in writing, software distributed under the 
+    License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 
+    OF ANY KIND, either express or implied. See the License for the specific 
+    language governing permissions and limitations under the License. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.nifi</groupId>
@@ -405,6 +406,11 @@ language governing permissions and limitations under the License. -->
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mqtt-nar</artifactId>
             <type>nar</type>
         </dependency>
@@ -513,13 +519,17 @@ language governing permissions and limitations under the License. -->
                         <configuration>
                             <name>nifi</name>
                             <summary>Apache NiFi</summary>
-                            <description>Apache NiFi is dataflow system based on the Flow-Based Programming concepts.</description>
-                            <license>Apache License, Version 2.0 and others (see included LICENSE file)</license>
+                            <description>Apache NiFi is dataflow system
+                                based on the Flow-Based Programming
+                                concepts.</description>
+                            <license>Apache License, Version 2.0 and
+                                others (see included LICENSE file)</license>
                             <url>http://nifi.apache.org</url>
                             <group>Utilities</group>
                             <prefix>/opt/nifi</prefix>
                             <defineStatements>
-                                <defineStatement>_use_internal_dependency_generator 0</defineStatement>
+                                <defineStatement>_use_internal_dependency_generator
+                                    0</defineStatement>
                             </defineStatements>
                             <defaultDirmode>750</defaultDirmode>
                             <defaultFilemode>640</defaultFilemode>
@@ -536,7 +546,13 @@ language governing permissions and limitations under the License. -->
                             </installScriptlet>
                             <preinstallScriptlet>
                                 <script>
-                                    /usr/bin/getent group nifi &gt;/dev/null || /usr/sbin/groupadd -r nifi; /usr/bin/getent passwd nifi &gt;/dev/null || /usr/sbin/useradd -r -g nifi -d /opt/nifi -s /sbin/nologin -c "NiFi System User" nifi
+                                    /usr/bin/getent group nifi
+                                    &gt;/dev/null || /usr/sbin/groupadd
+                                    -r nifi; /usr/bin/getent passwd nifi
+                                    &gt;/dev/null || /usr/sbin/useradd
+                                    -r -g nifi -d /opt/nifi -s
+                                    /sbin/nologin -c "NiFi System User"
+                                    nifi
                                 </script>
                             </preinstallScriptlet>
                         </configuration>
@@ -602,10 +618,12 @@ language governing permissions and limitations under the License. -->
                                         <mapping>
                                             <directory>/opt/nifi/nifi-${project.version}/lib</directory>
                                         </mapping>
-                                        <!-- The lib excludes and lib/bootstrap includes are computed by looking at the desired contents of
-                                        lib vs the desired contents of bootstrap directories.  The bootstrap directory should be comprised of explicitly
-                                        included items as found from the lib/bootstrap of a non rpm build and the lib folder should be specific excludes
-                                        being those which we want in bootstrap and NOT in lib. -->
+                                        <!-- The lib excludes and lib/bootstrap 
+                                            includes are computed by looking at the desired contents of lib vs the desired 
+                                            contents of bootstrap directories. The bootstrap directory should be comprised 
+                                            of explicitly included items as found from the lib/bootstrap of a non rpm 
+                                            build and the lib folder should be specific excludes being those which we 
+                                            want in bootstrap and NOT in lib. -->
                                         <mapping>
                                             <directory>/opt/nifi/nifi-${project.version}/lib</directory>
                                             <dependency>
@@ -636,7 +654,8 @@ language governing permissions and limitations under the License. -->
                                                     <!-- must be in lib <exclude>ch.qos.logback:logback-core</exclude> -->
                                                     <exclude>org.apache.nifi:nifi-security-utils</exclude>
                                                     <exclude>org.apache.nifi:nifi-utils</exclude>
-                                                    <!-- Items to not include which are also not in bootstrap -->
+                                                    <!-- Items to not include 
+                                                        which are also not in bootstrap -->
                                                     <exclude>org.apache.nifi:nifi-resources</exclude>
                                                     <exclude>org.apache.nifi:nifi-docs</exclude>
                                                 </excludes>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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..7dd9714 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
@@ -218,8 +218,8 @@ 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);
+            //            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/a88d3bfa/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java
index 7c911b8..73dd92f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java
@@ -239,7 +239,7 @@ public class NodeResponse {
 
         // if no client response was created, then generate a 500 response
         if (hasThrowable()) {
-            return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+            return Response.status(Status.INTERNAL_SERVER_ERROR).entity(getThrowable().toString()).build();
         }
 
         // set the status

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index fe99fb3..3a51816 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@ -2157,10 +2157,11 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
 
         final InputStream rawIn = getInputStream(source, record.getCurrentClaim(), record.getCurrentClaimOffset(), false);
         final InputStream limitedIn = new LimitedInputStream(rawIn, source.getSize());
-        final ByteCountingInputStream countingStream = new ByteCountingInputStream(limitedIn, this.bytesRead);
+        final ByteCountingInputStream countingStream = new ByteCountingInputStream(limitedIn);
         final FlowFileAccessInputStream ffais = new FlowFileAccessInputStream(countingStream, source, record.getCurrentClaim());
 
         final InputStream errorHandlingStream = new InputStream() {
+            private boolean closed = false;
 
             @Override
             public int read() throws IOException {
@@ -2201,7 +2202,10 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
 
             @Override
             public void close() throws IOException {
-                StandardProcessSession.this.bytesRead += countingStream.getBytesRead();
+                if (!closed) {
+                    StandardProcessSession.this.bytesRead += countingStream.getBytesRead();
+                    closed = true;
+                }
 
                 ffais.close();
                 openInputStreams.remove(source);

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
index cc17abc..8e92604 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
@@ -16,11 +16,13 @@
  */
 package org.apache.nifi.processor;
 
+import java.util.Arrays;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.logging.LogRepository;
 import org.apache.nifi.logging.LogRepositoryFactory;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,16 +49,6 @@ public class SimpleProcessLogger implements ComponentLog {
         return newArgs;
     }
 
-    private Object[] translateException(final Object[] os) {
-        if (os != null && os.length > 0 && (os[os.length - 1] instanceof Throwable)) {
-            final Object[] osCopy = new Object[os.length];
-            osCopy[osCopy.length - 1] = os[os.length - 1].toString();
-            System.arraycopy(os, 0, osCopy, 0, os.length - 1);
-            return osCopy;
-        }
-        return os;
-    }
-
     private boolean lastArgIsException(final Object[] os) {
         return (os != null && os.length > 0 && (os[os.length - 1] instanceof Throwable));
     }
@@ -80,7 +72,7 @@ public class SimpleProcessLogger implements ComponentLog {
         }
 
         if (lastArgIsException(os)) {
-            warn(msg, translateException(os), (Throwable) os[os.length - 1]);
+            warn(msg, Arrays.copyOfRange(os, 0, os.length - 1), (Throwable) os[os.length - 1]);
         } else {
             msg = "{} " + msg;
             os = addProcessor(os);
@@ -95,13 +87,9 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        os = addProcessorAndThrowable(os, t);
+        os = addProcessorAndThrowable(os, t, logger.isDebugEnabled());
         msg = "{} " + msg + ": {}";
-
         logger.warn(msg, os);
-        if (logger.isDebugEnabled()) {
-            logger.warn("", t);
-        }
         logRepository.addLogMessage(LogLevel.WARN, msg, os, t);
     }
 
@@ -159,11 +147,10 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        os = addProcessorAndThrowable(os, t);
+        os = addProcessorAndThrowable(os, t, true);
         msg = "{} " + msg + ": {}";
 
         logger.trace(msg, os);
-        logger.trace("", t);
         logRepository.addLogMessage(LogLevel.TRACE, msg, os, t);
     }
 
@@ -240,13 +227,10 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        os = addProcessorAndThrowable(os, t);
+        os = addProcessorAndThrowable(os, t, logger.isDebugEnabled());
         msg = "{} " + msg + ": {}";
 
         logger.info(msg, os);
-        if (logger.isDebugEnabled()) {
-            logger.info("", t);
-        }
         logRepository.addLogMessage(LogLevel.INFO, msg, os, t);
     }
 
@@ -261,14 +245,16 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        msg = "{} " + msg;
-        Object[] os = t == null ? new Object[]{component} : new Object[]{component, t.toString()};
-        logger.error(msg, os);
-        if (t != null){
-            logger.error("", t);
-            logRepository.addLogMessage(LogLevel.ERROR, msg, os, t);
-        } else {
+        if (t == null) {
+            msg = "{} " + msg;
+            final Object[] os = new Object[] {component};
+            logger.error(msg, os);
             logRepository.addLogMessage(LogLevel.ERROR, msg, os);
+        } else {
+            msg = "{} " + msg + ": {}";
+            final Object[] os = new Object[] {component, t.toString(), t};
+            logger.error(msg, os);
+            logRepository.addLogMessage(LogLevel.ERROR, msg, os, t);
         }
     }
 
@@ -279,7 +265,7 @@ public class SimpleProcessLogger implements ComponentLog {
         }
 
         if (lastArgIsException(os)) {
-            error(msg, translateException(os), (Throwable) os[os.length - 1]);
+            error(msg, Arrays.copyOfRange(os, 0, os.length - 1), (Throwable) os[os.length - 1]);
         } else {
             os = addProcessor(os);
             msg = "{} " + msg;
@@ -299,21 +285,27 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        os = addProcessorAndThrowable(os, t);
+        os = addProcessorAndThrowable(os, t, true);
         msg = "{} " + msg + ": {}";
 
         logger.error(msg, os);
-        logger.error("", t);
         logRepository.addLogMessage(LogLevel.ERROR, msg, os, t);
     }
 
-    private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t) {
-        final Object[] modifiedArgs = new Object[os.length + 2];
-        modifiedArgs[0] = component.toString();
-        for (int i = 0; i < os.length; i++) {
-            modifiedArgs[i + 1] = os[i];
+    private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t, final boolean includeStackTrace) {
+        final Object[] modifiedArgs;
+        if (t == null || !includeStackTrace) {
+            modifiedArgs = new Object[os.length + 2];
+            modifiedArgs[0] = component.toString();
+            System.arraycopy(os, 0, modifiedArgs, 1, os.length);
+            modifiedArgs[modifiedArgs.length - 1] = StringUtils.EMPTY;
+        } else {
+            modifiedArgs = new Object[os.length + 3];
+            modifiedArgs[0] = component.toString();
+            System.arraycopy(os, 0, modifiedArgs, 1, os.length);
+            modifiedArgs[modifiedArgs.length - 2] = t.toString();
+            modifiedArgs[modifiedArgs.length - 1] = t;
         }
-        modifiedArgs[modifiedArgs.length - 1] = (t == null) ? "" : t.toString();
 
         return modifiedArgs;
     }
@@ -350,13 +342,10 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        os = addProcessorAndThrowable(os, t);
+        os = addProcessorAndThrowable(os, t, true);
         msg = "{} " + msg + ": {}";
 
         logger.debug(msg, os);
-        if (logger.isDebugEnabled()) {
-            logger.debug("", t);
-        }
         logRepository.addLogMessage(LogLevel.DEBUG, msg, os, t);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
index e0d1300..51c6080 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
@@ -178,6 +178,22 @@ The following binary components are provided under the Apache Software License v
       Grok
       Copyright 2014 Anthony Corbacho, and contributors.
 
+  (ASLv2) Apache Calcite
+    The following NOTICE information applies:
+		Apache Calcite
+		Copyright 2012-2017 The Apache Software Foundation
+		
+		This product includes software developed at
+		The Apache Software Foundation (http://www.apache.org/).
+		
+		This product is based on source code originally developed
+		by DynamoBI Corporation, LucidEra Inc., SQLstream Inc. and others
+		under the auspices of the Eigenbase Foundation
+		and released as the LucidDB project.
+		
+		The web site includes files generated by Jekyll.
+
+
 ************************
 Common Development and Distribution License 1.1
 ************************

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index d410f43..e390097 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -1,15 +1,16 @@
 <?xml version="1.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. -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor 
+    license agreements. See the NOTICE file distributed with this work for additional 
+    information regarding copyright ownership. The ASF licenses this file to 
+    You under the Apache License, Version 2.0 (the "License"); you may not use 
+    this file except in compliance with the License. You may obtain a copy of 
+    the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required 
+    by applicable law or agreed to in writing, software distributed under the 
+    License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 
+    OF ANY KIND, either express or implied. See the License for the specific 
+    language governing permissions and limitations under the License. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.nifi</groupId>
@@ -49,6 +50,10 @@ language governing permissions and limitations under the License. -->
             <artifactId>nifi-http-context-map-api</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+        </dependency>
+        <dependency>
             <groupId>commons-io</groupId>
             <artifactId>commons-io</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator2.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator2.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator2.java
deleted file mode 100644
index 0f928ce..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator2.java
+++ /dev/null
@@ -1,303 +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.calcite.adapter.csv;
-
-import java.io.IOException;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import java.util.TimeZone;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.linq4j.Enumerator;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.util.Pair;
-import org.apache.commons.lang3.time.FastDateFormat;
-
-import au.com.bytecode.opencsv.CSVReader;
-
-
-/** Enumerator that reads from a CSV stream.
- *
- * @param <E> Row type
- */
-class CsvEnumerator2<E> implements Enumerator<E> {
-  private final CSVReader reader;
-  private final String[] filterValues;
-  private final RowConverter<E> rowConverter;
-  private E current;
-
-  private static final FastDateFormat TIME_FORMAT_DATE;
-  private static final FastDateFormat TIME_FORMAT_TIME;
-  private static final FastDateFormat TIME_FORMAT_TIMESTAMP;
-
-  static {
-    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 CsvEnumerator2(CSVReader csvReader, List<CsvFieldType> fieldTypes) {
-    this(verifyNotNullReader(csvReader), fieldTypes, identityList(fieldTypes.size()));
-  }
-
-  public CsvEnumerator2(CSVReader csvReader, List<CsvFieldType> fieldTypes, int[] fields) {
-    //noinspection unchecked
-    this(csvReader, null, (RowConverter<E>) converter(fieldTypes, fields));
-  }
-
-  public CsvEnumerator2(CSVReader csvReader, String[] filterValues, RowConverter<E> rowConverter) {
-    this.rowConverter = rowConverter;
-    this.filterValues = filterValues;
-    this.reader = csvReader;
-  }
-
-  static public CSVReader verifyNotNullReader(CSVReader csvReader) {
-    if (csvReader==null)
-      throw new IllegalArgumentException("csvReader cannot be null");
-    return csvReader;
-  }
-
-  private static RowConverter<?> converter(List<CsvFieldType> fieldTypes,
-      int[] fields) {
-    if (fields.length == 1) {
-      final int field = fields[0];
-      return new SingleColumnRowConverter(fieldTypes.get(field), field);
-    } else {
-      return new ArrayRowConverter(fieldTypes, fields);
-    }
-  }
-
-  /** Deduces the names and types of a table's columns by reading the first line
-   * of a CSV stream. */
-  static public RelDataType deduceRowType(JavaTypeFactory typeFactory, String[] firstLine,
-      List<CsvFieldType> fieldTypes) {
-    final List<RelDataType> types = new ArrayList<>();
-    final List<String> names = new ArrayList<>();
-      for (String string : firstLine) {
-        final String name;
-        final CsvFieldType fieldType;
-        final int colon = string.indexOf(':');
-        if (colon >= 0) {
-          name = string.substring(0, colon);
-          String typeString = string.substring(colon + 1);
-          typeString = typeString.trim();
-          fieldType = CsvFieldType.of(typeString);
-          if (fieldType == null) {
-            System.out.println("WARNING: Found unknown type: "
-              + typeString + " in first line: "
-              + " for column: " + name
-              + ". Will assume the type of column is string");
-          }
-        } else {
-          name = string;
-          fieldType = null;
-        }
-        final RelDataType type;
-        if (fieldType == null) {
-          type = typeFactory.createJavaType(String.class);
-        } else {
-          type = fieldType.toType(typeFactory);
-        }
-        names.add(name);
-        types.add(type);
-        if (fieldTypes != null) {
-          fieldTypes.add(fieldType);
-        }
-      }
-
-    if (names.isEmpty()) {
-      names.add("line");
-      types.add(typeFactory.createJavaType(String.class));
-    }
-    return typeFactory.createStructType(Pair.zip(names, types));
-  }
-
-  public E current() {
-    return current;
-  }
-
-  public boolean moveNext() {
-    try {
-    outer:
-      for (;;) {
-        final String[] strings = reader.readNext();
-        if (strings == null) {
-          current = null;
-          reader.close();
-          return false;
-        }
-        if (filterValues != null) {
-          for (int i = 0; i < strings.length; i++) {
-            String filterValue = filterValues[i];
-            if (filterValue != null) {
-              if (!filterValue.equals(strings[i])) {
-                continue outer;
-              }
-            }
-          }
-        }
-        current = rowConverter.convertRow(strings);
-        return true;
-      }
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public void reset() {
-    throw new UnsupportedOperationException();
-  }
-
-  public void close() {
-    try {
-      reader.close();
-    } catch (IOException e) {
-      throw new RuntimeException("Error closing CSV reader", e);
-    }
-  }
-
-  /** Returns an array of integers {0, ..., n - 1}. */
-  static int[] identityList(int n) {
-    int[] integers = new int[n];
-    for (int i = 0; i < n; i++) {
-      integers[i] = i;
-    }
-    return integers;
-  }
-
-  /** Row converter. */
-  abstract static class RowConverter<E> {
-    abstract E convertRow(String[] rows);
-
-    protected Object convert(CsvFieldType fieldType, String string) {
-      if (fieldType == null) {
-        return string;
-      }
-      switch (fieldType) {
-      case BOOLEAN:
-        if (string.length() == 0) {
-          return null;
-        }
-        return Boolean.parseBoolean(string);
-      case BYTE:
-        if (string.length() == 0) {
-          return null;
-        }
-        return Byte.parseByte(string);
-      case SHORT:
-        if (string.length() == 0) {
-          return null;
-        }
-        return Short.parseShort(string);
-      case INT:
-        if (string.length() == 0) {
-          return null;
-        }
-        return Integer.parseInt(string);
-      case LONG:
-        if (string.length() == 0) {
-          return null;
-        }
-        return Long.parseLong(string);
-      case FLOAT:
-        if (string.length() == 0) {
-          return null;
-        }
-        return Float.parseFloat(string);
-      case DOUBLE:
-        if (string.length() == 0) {
-          return null;
-        }
-        return Double.parseDouble(string);
-      case DATE:
-        if (string.length() == 0) {
-          return null;
-        }
-        try {
-          Date date = TIME_FORMAT_DATE.parse(string);
-          return new java.sql.Date(date.getTime());
-        } catch (ParseException e) {
-          return null;
-        }
-      case TIME:
-        if (string.length() == 0) {
-          return null;
-        }
-        try {
-          Date date = TIME_FORMAT_TIME.parse(string);
-          return new java.sql.Time(date.getTime());
-        } catch (ParseException e) {
-          return null;
-        }
-      case TIMESTAMP:
-        if (string.length() == 0) {
-          return null;
-        }
-        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;
-      }
-    }
-  }
-
-  /** Array row converter. */
-  static class ArrayRowConverter extends RowConverter<Object[]> {
-    private final CsvFieldType[] fieldTypes;
-    private final int[] fields;
-
-    ArrayRowConverter(List<CsvFieldType> fieldTypes, int[] fields) {
-      this.fieldTypes = fieldTypes.toArray(new CsvFieldType[fieldTypes.size()]);
-      this.fields = fields;
-    }
-
-    public Object[] convertRow(String[] strings) {
-      final Object[] objects = new Object[fields.length];
-      for (int i = 0; i < fields.length; i++) {
-        int field = fields[i];
-        objects[i] = convert(fieldTypes[field], strings[field]);
-      }
-      return objects;
-    }
-  }
-
-  /** Single column row converter. */
-  private static class SingleColumnRowConverter extends RowConverter {
-    private final CsvFieldType fieldType;
-    private final int fieldIndex;
-
-    private SingleColumnRowConverter(CsvFieldType fieldType, int fieldIndex) {
-      this.fieldType = fieldType;
-      this.fieldIndex = fieldIndex;
-    }
-
-    public Object convertRow(String[] strings) {
-      return convert(fieldType, strings[fieldIndex]);
-    }
-  }
-}
-
-// End CsvEnumerator2.java

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchema2.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchema2.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchema2.java
deleted file mode 100644
index f724f79..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchema2.java
+++ /dev/null
@@ -1,98 +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.calcite.adapter.csv;
-
-import java.io.Reader;
-import java.util.Map;
-
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.impl.AbstractSchema;
-
-import com.google.common.collect.ImmutableMap;
-
-/**
- * Schema mapped onto a directory of CSV files. Each table in the schema
- * is a CSV file in that directory.
- */
-public class CsvSchema2 extends AbstractSchema {
-  final private Map<String, Reader> inputs;
-  private final CsvTable.Flavor flavor;
-  private Map<String, Table> tableMap;
-
-  /**
-   * Creates a CSV schema.
-   *
-   * @param inputs     Inputs map
-   * @param flavor     Whether to instantiate flavor tables that undergo
-   *                   query optimization
-   */
-  public CsvSchema2(Map<String, Reader> inputs, CsvTable.Flavor flavor) {
-    super();
-    this.inputs = inputs;
-    this.flavor = flavor;
-  }
-
-  /** Looks for a suffix on a string and returns
-   * either the string with the suffix removed
-   * or the original string. */
-  private static String trim(String s, String suffix) {
-    String trimmed = trimOrNull(s, suffix);
-    return trimmed != null ? trimmed : s;
-  }
-
-  /** Looks for a suffix on a string and returns
-   * either the string with the suffix removed
-   * or null. */
-  private static String trimOrNull(String s, String suffix) {
-    return s.endsWith(suffix)
-        ? s.substring(0, s.length() - suffix.length())
-        : null;
-  }
-
-  @Override protected Map<String, Table> getTableMap() {
-
-    if (tableMap!=null)
-      return tableMap;
-
-    // Build a map from table name to table; each file becomes a table.
-    final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
-
-    for (Map.Entry<String, Reader> entry : inputs.entrySet()) {
-      final Table table = createTable(entry.getValue());
-      builder.put(entry.getKey(), table);
-    }
-
-    tableMap = builder.build();
-    return tableMap;
-  }
-
-  /** Creates different sub-type of table based on the "flavor" attribute. */
-  private Table createTable(Reader readerx) {
-    switch (flavor) {
-    case TRANSLATABLE:
-      return new CsvTranslatableTable2(readerx, null);
-//    case SCANNABLE:
-//      return new CsvScannableTable(file, null);
-//    case FILTERABLE:
-//      return new CsvFilterableTable(file, null);
-    default:
-      throw new AssertionError("Unknown flavor " + flavor);
-    }
-  }
-}
-
-// End CsvSchema2.java

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory2.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory2.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory2.java
deleted file mode 100644
index f8ec576..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory2.java
+++ /dev/null
@@ -1,53 +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.calcite.adapter.csv;
-
-import java.io.Reader;
-import java.util.Map;
-
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaFactory;
-import org.apache.calcite.schema.SchemaPlus;
-
-/**
- * Factory that creates a {@link CsvSchema}.
- *
- * <p>Allows a custom schema to be included in a <code><i>model</i>.json</code>
- * file.</p>
- */
-@SuppressWarnings("UnusedDeclaration")
-public class CsvSchemaFactory2 implements SchemaFactory {
-  final private Map<String, Reader> inputs;
-  // public constructor, per factory contract
-  public CsvSchemaFactory2(Map<String, Reader> inputs) {
-      this.inputs = inputs;
-  }
-
-  public Schema create(SchemaPlus parentSchema, String name, Map<String, Object> operand) {
-    String flavorName = (String) operand.get("flavor");
-    CsvTable.Flavor flavor;
-    if (flavorName == null) {
-      flavor = CsvTable.Flavor.SCANNABLE;
-    } else {
-      flavor = CsvTable.Flavor.valueOf(flavorName.toUpperCase());
-    }
-
-    return new CsvSchema2(inputs, flavor);
-  }
-}
-
-// End CsvSchemaFactory2.java

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan2.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan2.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan2.java
deleted file mode 100644
index 75f013c..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan2.java
+++ /dev/null
@@ -1,104 +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.calcite.adapter.csv;
-
-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;
-
-import java.util.List;
-
-/**
- * Relational expression representing a scan of a CSV stream.
- *
- * <p>Like any table scan, it serves as a leaf node of a query tree.</p>
- */
-public class CsvTableScan2 extends TableScan implements EnumerableRel {
-  final CsvTranslatableTable2 csvTable;
-  final int[] fields;
-
-  protected CsvTableScan2(RelOptCluster cluster, RelOptTable table,
-      CsvTranslatableTable2 csvTable, int[] fields) {
-    super(cluster, cluster.traitSetOf(EnumerableConvention.INSTANCE), table);
-    this.csvTable = csvTable;
-    this.fields = fields;
-
-    assert csvTable != null;
-  }
-
-  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    assert inputs.isEmpty();
-    return new CsvTableScan2(getCluster(), table, csvTable, fields);
-  }
-
-  @Override public RelWriter explainTerms(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(CsvProjectTableScanRule.INSTANCE);
-  }
-
-  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-    PhysType physType =
-        PhysTypeImpl.of(
-            implementor.getTypeFactory(),
-            getRowType(),
-            pref.preferArray());
-
-    if (table instanceof JsonTable) {
-      return implementor.result(
-          physType,
-          Blocks.toBlock(
-              Expressions.call(table.getExpression(JsonTable.class),
-                  "enumerable")));
-    }
-    return implementor.result(
-        physType,
-        Blocks.toBlock(
-            Expressions.call(table.getExpression(CsvTranslatableTable2.class),
-                "project", Expressions.constant(fields))));
-  }
-}
-
-// End CsvTableScan.java

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable2.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable2.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable2.java
deleted file mode 100644
index bc28fdd..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable2.java
+++ /dev/null
@@ -1,121 +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.calcite.adapter.csv;
-
-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.rel.type.RelProtoDataType;
-import org.apache.calcite.schema.QueryableTable;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Schemas;
-import org.apache.calcite.schema.TranslatableTable;
-
-import au.com.bytecode.opencsv.CSVReader;
-
-import java.io.IOException;
-import java.io.Reader;
-import java.lang.reflect.Type;
-import java.util.ArrayList;
-
-/**
- * Table based on a CSV stream.
- */
-public class CsvTranslatableTable2 extends CsvTable
-    implements QueryableTable, TranslatableTable {
-
-  final private CSVReader csvReader;
-  private CsvEnumerator2<Object> csvEnumerator2;
-  final private String[] firstLine;
-
-  /** Creates a CsvTable.
-   */
-  CsvTranslatableTable2(Reader readerx, RelProtoDataType protoRowType) {
-    super(null, protoRowType);
-    this.csvReader = new CSVReader(readerx);
-    try {
-        this.firstLine = csvReader.readNext();
-    } catch (IOException e) {
-        throw new RuntimeException("csvReader.readNext() failed ", e);
-    }
-  }
-
-  public String toString() {
-    return "CsvTranslatableTable2";
-  }
-
-  /** 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>() {
-      public Enumerator<Object> enumerator() {
-        return csvEnumerator2;
-      }
-    };
-  }
-
-  public Expression getExpression(SchemaPlus schema, String tableName,
-      Class clazz) {
-    return Schemas.tableExpression(schema, getElementType(), tableName, clazz);
-  }
-
-  public Type getElementType() {
-    return Object[].class;
-  }
-
-  public <T> Queryable<T> asQueryable(QueryProvider queryProvider,
-      SchemaPlus schema, String tableName) {
-    throw new UnsupportedOperationException();
-  }
-
-  public RelNode toRel(
-      RelOptTable.ToRelContext context,
-      RelOptTable relOptTable) {
-    // Request all fields.
-    final int fieldCount = relOptTable.getRowType().getFieldCount();
-    final int[] fields = CsvEnumerator.identityList(fieldCount);
-    return new CsvTableScan2(context.getCluster(), relOptTable, this, fields);
-  }
-
-  @Override
-  public RelDataType getRowType(RelDataTypeFactory typeFactory) {
-      RelDataType rowType = null;
-
-      if (fieldTypes == null) {
-          fieldTypes = new ArrayList<CsvFieldType>();
-          rowType =  CsvEnumerator2.deduceRowType((JavaTypeFactory) typeFactory, firstLine, fieldTypes);
-      } else {
-          rowType = CsvEnumerator2.deduceRowType((JavaTypeFactory) typeFactory, firstLine, null);
-      }
-
-      if (csvEnumerator2==null)
-          csvEnumerator2 = new CsvEnumerator2<Object>(csvReader, fieldTypes);
-
-          return rowType;
-      }
-}
-
-// End CsvTranslatableTable2.java

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterCSVColumns.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterCSVColumns.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterCSVColumns.java
deleted file mode 100644
index 718f462..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterCSVColumns.java
+++ /dev/null
@@ -1,258 +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 static java.sql.Types.CHAR;
-import static java.sql.Types.LONGNVARCHAR;
-import static java.sql.Types.LONGVARCHAR;
-import static java.sql.Types.NCHAR;
-import static java.sql.Types.NVARCHAR;
-import static java.sql.Types.VARCHAR;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.Reader;
-import java.nio.charset.StandardCharsets;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.calcite.adapter.csv.CsvSchemaFactory2;
-import org.apache.calcite.jdbc.CalciteConnection;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.commons.lang3.StringEscapeUtils;
-import org.apache.commons.lang3.StringUtils;
-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.components.PropertyDescriptor;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.logging.ProcessorLog;
-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.StreamCallback;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.stream.io.BufferedInputStream;
-import org.apache.nifi.util.StopWatch;
-
-import com.google.common.collect.ImmutableMap;
-
-@EventDriven
-@SideEffectFree
-@SupportsBatching
-@Tags({"xml", "xslt", "transform"})
-@InputRequirement(Requirement.INPUT_REQUIRED)
-@CapabilityDescription("Filter out specific columns from CSV data. Some other transformations are also supported."
-        + "Columns can be renamed, simple calculations performed, aggregations, etc."
-        + "SQL select statement is used to specify how CSV data should be transformed."
-        + "SQL statement follows standard SQL, some restrictions may apply."
-        + "Successfully transformed CSV data is routed to the 'success' relationship."
-        + "If transform fails, the original FlowFile is routed to the 'failure' relationship")
-public class FilterCSVColumns  extends AbstractProcessor {
-
-    public static final PropertyDescriptor SQL_SELECT = new PropertyDescriptor.Builder()
-            .name("SQL select statement")
-            .description("SQL select statement specifies how CSV data should be transformed. "
-                       + "Sql select should select from CSV.A table")
-            .required(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    public static final Relationship REL_SUCCESS = new Relationship.Builder()
-            .name("success")
-            .description("The FlowFile with transformed content will be 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 CSV), it will be routed to this relationship")
-            .build();
-
-    private List<PropertyDescriptor> properties;
-    private Set<Relationship> relationships;
-
-    @Override
-    protected void init(final ProcessorInitializationContext context) {
-        final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(SQL_SELECT);
-        this.properties = Collections.unmodifiableList(properties);
-
-        final Set<Relationship> relationships = new HashSet<>();
-        relationships.add(REL_SUCCESS);
-        relationships.add(REL_FAILURE);
-        this.relationships = Collections.unmodifiableSet(relationships);
-    }
-
-    @Override
-    public Set<Relationship> getRelationships() {
-        return relationships;
-    }
-
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return properties;
-    }
-
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSession session) {
-        final FlowFile original = session.get();
-        if (original == null) {
-            return;
-        }
-
-        final ProcessorLog logger = getLogger();
-        final StopWatch stopWatch = new StopWatch(true);
-
-        try {
-            FlowFile transformed = session.write(original, new StreamCallback() {
-                @Override
-                public void process(final InputStream rawIn, final OutputStream out) throws IOException {
-                    try (final InputStream in = new BufferedInputStream(rawIn)) {
-
-                        String sql = context.getProperty(SQL_SELECT).getValue();
-                        final ResultSet resultSet = transform(rawIn, sql);
-                        convertToCSV(resultSet, out);
-
-                    } catch (final Exception e) {
-                        throw new IOException(e);
-                    }
-                }
-            });
-            session.transfer(transformed, REL_SUCCESS);
-            session.getProvenanceReporter().modifyContent(transformed, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
-            logger.info("Transformed {}", new Object[]{original});
-        } catch (ProcessException e) {
-            logger.error("Unable to transform {} due to {}", new Object[]{original, e});
-            session.transfer(original, REL_FAILURE);
-        }
-    }
-
-    static protected ResultSet transform(InputStream rawIn, String sql) throws SQLException {
-
-        Reader readerx = new InputStreamReader(rawIn);
-        HashMap<String, Reader> inputs = new HashMap<>();
-        inputs.put("A", readerx);
-
-        Statement statement = null;
-        final Properties properties = new Properties();
-//      properties.setProperty("caseSensitive", "true");
-        try (final Connection connection = DriverManager.getConnection("jdbc:calcite:", properties)) {
-            final CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class);
-
-            final SchemaPlus rootSchema = calciteConnection.getRootSchema();
-            final Schema schema =
-              new CsvSchemaFactory2(inputs)
-                  .create(rootSchema, "CSV", ImmutableMap.<String, Object>of("flavor", "TRANSLATABLE"));
-
-            calciteConnection.getRootSchema().add("CSV", schema);
-            rootSchema.add("default", schema);
-
-            statement = connection.createStatement();
-            final ResultSet resultSet = statement.executeQuery(sql);
-            return resultSet;
-        }
-    }
-
-    static protected void convertToCSV(ResultSet resultSet, OutputStream out) throws SQLException, IOException {
-
-        convertToCsvStream(resultSet, out);
-    }
-
-    public static long convertToCsvStream(final ResultSet rs, final OutputStream outStream) throws SQLException, IOException {
-        return convertToCsvStream(rs, outStream, null, null);
-    }
-
-    public static long convertToCsvStream(final ResultSet rs, final OutputStream outStream, String recordName, ResultSetRowCallback callback)
-            throws SQLException, IOException {
-
-        final ResultSetMetaData meta = rs.getMetaData();
-        final int nrOfColumns = meta.getColumnCount();
-        List<String> columnNames = new ArrayList<>(nrOfColumns);
-
-        for (int i = 1; i <= nrOfColumns; i++) {
-            String columnNameFromMeta = meta.getColumnName(i);
-            // Hive returns table.column for column name. Grab the column name as the string after the last period
-            int columnNameDelimiter = columnNameFromMeta.lastIndexOf(".");
-            columnNames.add(columnNameFromMeta.substring(columnNameDelimiter + 1));
-        }
-
-        // Write column names as header row
-        outStream.write(StringUtils.join(columnNames, ",").getBytes(StandardCharsets.UTF_8));
-        outStream.write("\n".getBytes(StandardCharsets.UTF_8));
-
-        // Iterate over the rows
-        long nrOfRows = 0;
-        while (rs.next()) {
-            if (callback != null) {
-                callback.processRow(rs);
-            }
-            List<String> rowValues = new ArrayList<>(nrOfColumns);
-            for (int i = 1; i <= nrOfColumns; i++) {
-                final int javaSqlType = meta.getColumnType(i);
-                final Object value = rs.getObject(i);
-
-                switch (javaSqlType) {
-                    case CHAR:
-                    case LONGNVARCHAR:
-                    case LONGVARCHAR:
-                    case NCHAR:
-                    case NVARCHAR:
-                    case VARCHAR:
-                        rowValues.add("\"" + StringEscapeUtils.escapeCsv(rs.getString(i)) + "\"");
-                        break;
-                    default:
-                        rowValues.add(value.toString());
-                }
-            }
-            // Write row values
-            outStream.write(StringUtils.join(rowValues, ",").getBytes(StandardCharsets.UTF_8));
-            outStream.write("\n".getBytes(StandardCharsets.UTF_8));
-            nrOfRows++;
-        }
-        return nrOfRows;
-    }
-
-    /**
-     * An interface for callback methods which allows processing of a row during the convertToXYZStream() processing.
-     * <b>IMPORTANT:</b> This method should only work on the row pointed at by the current ResultSet reference.
-     * Advancing the cursor (e.g.) can cause rows to be skipped during Avro transformation.
-     */
-    public interface ResultSetRowCallback {
-        void processRow(ResultSet resultSet) throws IOException;
-    }
-}


[10/19] nifi git commit: NIFI-1280 Create FilterCSVColumns Processor.

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/4d5872a3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500_typeless.csv
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500_typeless.csv b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500_typeless.csv
new file mode 100644
index 0000000..61ce4bd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500_typeless.csv
@@ -0,0 +1 @@
+FIRST_NAME:string,LAST_NAME,COMPANY_NAME,ADDRESS,CITY,COUNTY,STATE,zip,phone1,phone2,email,web
"James","Butt","Benton, John B Jr","6649 N Blue Gum St","New Orleans","Orleans","LA",70116,"504-621-8927","504-845-1427","jbutt@gmail.com","http://www.bentonjohnbjr.com"
"Josephine","Darakjy","Chanay, Jeffrey A Esq","4 B Blue Ridge Blvd","Brighton","Livingston","MI",48116,"810-292-9388","810-374-9840","josephine_darakjy@darakjy.org","http://www.chanayjeffreyaesq.com"
"Art","Venere","Chemel, James L Cpa","8 W Cerritos Ave #54","Bridgeport","Gloucester","NJ","08014","856-636-8749","856-264-4130","art@venere.org","http://www.chemeljameslcpa.com"
"Lenna","Paprocki","Feltz Printing Service","639 Main St","Anchorage","Anchorage","AK",99501,"907-385-4412","907-921-2010","lpaprocki@hotmail.com","http://www.feltzprintingservice.com"
"Donette","Foller","Printing Dimensions","34 Center St","Hamilton","Butler","OH",45011,"513-570-1893","513-549-4561","donette.foller@cox.net","http://www.printingdimens
 ions.com"
"Simona","Morasca","Chapman, Ross E Esq","3 Mcauley Dr","Ashland","Ashland","OH",44805,"419-503-2484","419-800-6759","simona@morasca.com","http://www.chapmanrosseesq.com"
"Mitsue","Tollner","Morlong Associates","7 Eads St","Chicago","Cook","IL",60632,"773-573-6914","773-924-8565","mitsue_tollner@yahoo.com","http://www.morlongassociates.com"
"Leota","Dilliard","Commercial Press","7 W Jackson Blvd","San Jose","Santa Clara","CA",95111,"408-752-3500","408-813-1105","leota@hotmail.com","http://www.commercialpress.com"
"Sage","Wieser","Truhlar And Truhlar Attys","5 Boston Ave #88","Sioux Falls","Minnehaha","SD",57105,"605-414-2147","605-794-4895","sage_wieser@cox.net","http://www.truhlarandtruhlarattys.com"
"Kris","Marrier","King, Christopher A Esq","228 Runamuck Pl #2808","Baltimore","Baltimore City","MD",21224,"410-655-8723","410-804-4694","kris@gmail.com","http://www.kingchristopheraesq.com"
"Minna","Amigon","Dorl, James J Esq","2371 Jerrold Ave","Kulpsville","Montgomery","PA
 ",19443,"215-874-1229","215-422-8694","minna_amigon@yahoo.com","http://www.dorljamesjesq.com"
"Abel","Maclead","Rangoni Of Florence","37275 St  Rt 17m M","Middle Island","Suffolk","NY",11953,"631-335-3414","631-677-3675","amaclead@gmail.com","http://www.rangoniofflorence.com"
"Kiley","Caldarera","Feiner Bros","25 E 75th St #69","Los Angeles","Los Angeles","CA",90034,"310-498-5651","310-254-3084","kiley.caldarera@aol.com","http://www.feinerbros.com"
"Graciela","Ruta","Buckley Miller & Wright","98 Connecticut Ave Nw","Chagrin Falls","Geauga","OH",44023,"440-780-8425","440-579-7763","gruta@cox.net","http://www.buckleymillerwright.com"
"Cammy","Albares","Rousseaux, Michael Esq","56 E Morehead St","Laredo","Webb","TX",78045,"956-537-6195","956-841-7216","calbares@gmail.com","http://www.rousseauxmichaelesq.com"
"Mattie","Poquette","Century Communications","73 State Road 434 E","Phoenix","Maricopa","AZ",85013,"602-277-4385","602-953-6360","mattie@aol.com","http://www.centurycommunications.
 com"
"Meaghan","Garufi","Bolton, Wilbur Esq","69734 E Carrillo St","Mc Minnville","Warren","TN",37110,"931-313-9635","931-235-7959","meaghan@hotmail.com","http://www.boltonwilburesq.com"
"Gladys","Rim","T M Byxbee Company Pc","322 New Horizon Blvd","Milwaukee","Milwaukee","WI",53207,"414-661-9598","414-377-2880","gladys.rim@rim.org","http://www.tmbyxbeecompanypc.com"
"Yuki","Whobrey","Farmers Insurance Group","1 State Route 27","Taylor","Wayne","MI",48180,"313-288-7937","313-341-4470","yuki_whobrey@aol.com","http://www.farmersinsurancegroup.com"
"Fletcher","Flosi","Post Box Services Plus","394 Manchester Blvd","Rockford","Winnebago","IL",61109,"815-828-2147","815-426-5657","fletcher.flosi@yahoo.com","http://www.postboxservicesplus.com"
"Bette","Nicka","Sport En Art","6 S 33rd St","Aston","Delaware","PA",19014,"610-545-3615","610-492-4643","bette_nicka@cox.net","http://www.sportenart.com"
"Veronika","Inouye","C 4 Network Inc","6 Greenleaf Ave","San Jose","Santa Clara","CA",95111,"408
 -540-1785","408-813-4592","vinouye@aol.com","http://www.cnetworkinc.com"
"Willard","Kolmetz","Ingalls, Donald R Esq","618 W Yakima Ave","Irving","Dallas","TX",75062,"972-303-9197","972-896-4882","willard@hotmail.com","http://www.ingallsdonaldresq.com"
"Maryann","Royster","Franklin, Peter L Esq","74 S Westgate St","Albany","Albany","NY",12204,"518-966-7987","518-448-8982","mroyster@royster.com","http://www.franklinpeterlesq.com"
"Alisha","Slusarski","Wtlz Power 107 Fm","3273 State St","Middlesex","Middlesex","NJ","08846","732-658-3154","732-635-3453","alisha@slusarski.com","http://www.wtlzpowerfm.com"
"Allene","Iturbide","Ledecky, David Esq","1 Central Ave","Stevens Point","Portage","WI",54481,"715-662-6764","715-530-9863","allene_iturbide@cox.net","http://www.ledeckydavidesq.com"
"Chanel","Caudy","Professional Image Inc","86 Nw 66th St #8673","Shawnee","Johnson","KS",66218,"913-388-2079","913-899-1103","chanel.caudy@caudy.org","http://www.professionalimageinc.com"
"Ezekiel","Chui","
 Sider, Donald C Esq","2 Cedar Ave #84","Easton","Talbot","MD",21601,"410-669-1642","410-235-8738","ezekiel@chui.com","http://www.siderdonaldcesq.com"
"Willow","Kusko","U Pull It","90991 Thorburn Ave","New York","New York","NY",10011,"212-582-4976","212-934-5167","wkusko@yahoo.com","http://www.upullit.com"
"Bernardo","Figeroa","Clark, Richard Cpa","386 9th Ave N","Conroe","Montgomery","TX",77301,"936-336-3951","936-597-3614","bfigeroa@aol.com","http://www.clarkrichardcpa.com"
"Ammie","Corrio","Moskowitz, Barry S","74874 Atlantic Ave","Columbus","Franklin","OH",43215,"614-801-9788","614-648-3265","ammie@corrio.com","http://www.moskowitzbarrys.com"
"Francine","Vocelka","Cascade Realty Advisors Inc","366 South Dr","Las Cruces","Dona Ana","NM",88011,"505-977-3911","505-335-5293","francine_vocelka@vocelka.com","http://www.cascaderealtyadvisorsinc.com"
"Ernie","Stenseth","Knwz Newsradio","45 E Liberty St","Ridgefield Park","Bergen","NJ","07660","201-709-6245","201-387-9093","ernie_stenseth
 @aol.com","http://www.knwznewsradio.com"
"Albina","Glick","Giampetro, Anthony D","4 Ralph Ct","Dunellen","Middlesex","NJ","08812","732-924-7882","732-782-6701","albina@glick.com","http://www.giampetroanthonyd.com"
"Alishia","Sergi","Milford Enterprises Inc","2742 Distribution Way","New York","New York","NY",10025,"212-860-1579","212-753-2740","asergi@gmail.com","http://www.milfordenterprisesinc.com"
"Solange","Shinko","Mosocco, Ronald A","426 Wolf St","Metairie","Jefferson","LA",70002,"504-979-9175","504-265-8174","solange@shinko.com","http://www.mosoccoronalda.com"
"Jose","Stockham","Tri State Refueler Co","128 Bransten Rd","New York","New York","NY",10011,"212-675-8570","212-569-4233","jose@yahoo.com","http://www.tristaterefuelerco.com"
"Rozella","Ostrosky","Parkway Company","17 Morena Blvd","Camarillo","Ventura","CA",93012,"805-832-6163","805-609-1531","rozella.ostrosky@ostrosky.com","http://www.parkwaycompany.com"
"Valentine","Gillian","Fbs Business Finance","775 W 17th St","San
  Antonio","Bexar","TX",78204,"210-812-9597","210-300-6244","valentine_gillian@gmail.com","http://www.fbsbusinessfinance.com"
"Kati","Rulapaugh","Eder Assocs Consltng Engrs Pc","6980 Dorsett Rd","Abilene","Dickinson","KS",67410,"785-463-7829","785-219-7724","kati.rulapaugh@hotmail.com","http://www.ederassocsconsltngengrspc.com"
"Youlanda","Schemmer","Tri M Tool Inc","2881 Lewis Rd","Prineville","Crook","OR",97754,"541-548-8197","541-993-2611","youlanda@aol.com","http://www.trimtoolinc.com"
"Dyan","Oldroyd","International Eyelets Inc","7219 Woodfield Rd","Overland Park","Johnson","KS",66204,"913-413-4604","913-645-8918","doldroyd@aol.com","http://www.internationaleyeletsinc.com"
"Roxane","Campain","Rapid Trading Intl","1048 Main St","Fairbanks","Fairbanks North Star","AK",99708,"907-231-4722","907-335-6568","roxane@hotmail.com","http://www.rapidtradingintl.com"
"Lavera","Perin","Abc Enterprises Inc","678 3rd Ave","Miami","Miami-Dade","FL",33196,"305-606-7291","305-995-2078","lperin@pe
 rin.org","http://www.abcenterprisesinc.com"
"Erick","Ferencz","Cindy Turner Associates","20 S Babcock St","Fairbanks","Fairbanks North Star","AK",99712,"907-741-1044","907-227-6777","erick.ferencz@aol.com","http://www.cindyturnerassociates.com"
"Fatima","Saylors","Stanton, James D Esq","2 Lighthouse Ave","Hopkins","Hennepin","MN",55343,"952-768-2416","952-479-2375","fsaylors@saylors.org","http://www.stantonjamesdesq.com"
"Jina","Briddick","Grace Pastries Inc","38938 Park Blvd","Boston","Suffolk","MA","02128","617-399-5124","617-997-5771","jina_briddick@briddick.com","http://www.gracepastriesinc.com"
"Kanisha","Waycott","Schroer, Gene E Esq","5 Tomahawk Dr","Los Angeles","Los Angeles","CA",90006,"323-453-2780","323-315-7314","kanisha_waycott@yahoo.com","http://www.schroergeneeesq.com"
"Emerson","Bowley","Knights Inn","762 S Main St","Madison","Dane","WI",53711,"608-336-7444","608-658-7940","emerson.bowley@bowley.org","http://www.knightsinn.com"
"Blair","Malet","Bollinger Mach Shp & S
 hipyard","209 Decker Dr","Philadelphia","Philadelphia","PA",19132,"215-907-9111","215-794-4519","bmalet@yahoo.com","http://www.bollingermachshpshipyard.com"
"Brock","Bolognia","Orinda News","4486 W O St #1","New York","New York","NY",10003,"212-402-9216","212-617-5063","bbolognia@yahoo.com","http://www.orindanews.com"
"Lorrie","Nestle","Ballard Spahr Andrews","39 S 7th St","Tullahoma","Coffee","TN",37388,"931-875-6644","931-303-6041","lnestle@hotmail.com","http://www.ballardspahrandrews.com"
"Sabra","Uyetake","Lowy Limousine Service","98839 Hawthorne Blvd #6101","Columbia","Richland","SC",29201,"803-925-5213","803-681-3678","sabra@uyetake.org","http://www.lowylimousineservice.com"
"Marjory","Mastella","Vicon Corporation","71 San Mateo Ave","Wayne","Delaware","PA",19087,"610-814-5533","610-379-7125","mmastella@mastella.com","http://www.viconcorporation.com"
"Karl","Klonowski","Rossi, Michael M","76 Brooks St #9","Flemington","Hunterdon","NJ","08822","908-877-6135","908-470-4661","kar
 l_klonowski@yahoo.com","http://www.rossimichaelm.com"
"Tonette","Wenner","Northwest Publishing","4545 Courthouse Rd","Westbury","Nassau","NY",11590,"516-968-6051","516-333-4861","twenner@aol.com","http://www.northwestpublishing.com"
"Amber","Monarrez","Branford Wire & Mfg Co","14288 Foster Ave #4121","Jenkintown","Montgomery","PA",19046,"215-934-8655","215-329-6386","amber_monarrez@monarrez.org","http://www.branfordwiremfgco.com"
"Shenika","Seewald","East Coast Marketing","4 Otis St","Van Nuys","Los Angeles","CA",91405,"818-423-4007","818-749-8650","shenika@gmail.com","http://www.eastcoastmarketing.com"
"Delmy","Ahle","Wye Technologies Inc","65895 S 16th St","Providence","Providence","RI","02909","401-458-2547","401-559-8961","delmy.ahle@hotmail.com","http://www.wyetechnologiesinc.com"
"Deeanna","Juhas","Healy, George W Iv","14302 Pennsylvania Ave","Huntingdon Valley","Montgomery","PA",19006,"215-211-9589","215-417-9563","deeanna_juhas@gmail.com","http://www.healygeorgewiv.com"
"Blo
 ndell","Pugh","Alpenlite Inc","201 Hawk Ct","Providence","Providence","RI","02904","401-960-8259","401-300-8122","bpugh@aol.com","http://www.alpenliteinc.com"
"Jamal","Vanausdal","Hubbard, Bruce Esq","53075 Sw 152nd Ter #615","Monroe Township","Middlesex","NJ","08831","732-234-1546","732-904-2931","jamal@vanausdal.org","http://www.hubbardbruceesq.com"
"Cecily","Hollack","Arthur A Oliver & Son Inc","59 N Groesbeck Hwy","Austin","Travis","TX",78731,"512-486-3817","512-861-3814","cecily@hollack.org","http://www.arthuraoliversoninc.com"
"Carmelina","Lindall","George Jessop Carter Jewelers","2664 Lewis Rd","Littleton","Douglas","CO",80126,"303-724-7371","303-874-5160","carmelina_lindall@lindall.com","http://www.georgejessopcarterjewelers.com"
"Maurine","Yglesias","Schultz, Thomas C Md","59 Shady Ln #53","Milwaukee","Milwaukee","WI",53214,"414-748-1374","414-573-7719","maurine_yglesias@yglesias.com","http://www.schultzthomascmd.com"
"Tawna","Buvens","H H H Enterprises Inc","3305 Nabell Av
 e #679","New York","New York","NY",10009,"212-674-9610","212-462-9157","tawna@gmail.com","http://www.hhhenterprisesinc.com"
"Penney","Weight","Hawaiian King Hotel","18 Fountain St","Anchorage","Anchorage","AK",99515,"907-797-9628","907-873-2882","penney_weight@aol.com","http://www.hawaiiankinghotel.com"
"Elly","Morocco","Killion Industries","7 W 32nd St","Erie","Erie","PA",16502,"814-393-5571","814-420-3553","elly_morocco@gmail.com","http://www.killionindustries.com"
"Ilene","Eroman","Robinson, William J Esq","2853 S Central Expy","Glen Burnie","Anne Arundel","MD",21061,"410-914-9018","410-937-4543","ilene.eroman@hotmail.com","http://www.robinsonwilliamjesq.com"
"Vallie","Mondella","Private Properties","74 W College St","Boise","Ada","ID",83707,"208-862-5339","208-737-8439","vmondella@mondella.com","http://www.privateproperties.com"
"Kallie","Blackwood","Rowley Schlimgen Inc","701 S Harrison Rd","San Francisco","San Francisco","CA",94104,"415-315-2761","415-604-7609","kallie.blackwo
 od@gmail.com","http://www.rowleyschlimgeninc.com"
"Johnetta","Abdallah","Forging Specialties","1088 Pinehurst St","Chapel Hill","Orange","NC",27514,"919-225-9345","919-715-3791","johnetta_abdallah@aol.com","http://www.forgingspecialties.com"
"Bobbye","Rhym","Smits, Patricia Garity","30 W 80th St #1995","San Carlos","San Mateo","CA",94070,"650-528-5783","650-811-9032","brhym@rhym.com","http://www.smitspatriciagarity.com"
"Micaela","Rhymes","H Lee Leonard Attorney At Law","20932 Hedley St","Concord","Contra Costa","CA",94520,"925-647-3298","925-522-7798","micaela_rhymes@gmail.com","http://www.hleeleonardattorneyatlaw.com"
"Tamar","Hoogland","A K Construction Co","2737 Pistorio Rd #9230","London","Madison","OH",43140,"740-343-8575","740-526-5410","tamar@hotmail.com","http://www.akconstructionco.com"
"Moon","Parlato","Ambelang, Jessica M Md","74989 Brandon St","Wellsville","Allegany","NY",14895,"585-866-8313","585-498-4278","moon@yahoo.com","http://www.ambelangjessicammd.com"
"Laurel","
 Reitler","Q A Service","6 Kains Ave","Baltimore","Baltimore City","MD",21215,"410-520-4832","410-957-6903","laurel_reitler@reitler.com","http://www.qaservice.com"
"Delisa","Crupi","Wood & Whitacre Contractors","47565 W Grand Ave","Newark","Essex","NJ","07105","973-354-2040","973-847-9611","delisa.crupi@crupi.com","http://www.woodwhitacrecontractors.com"
"Viva","Toelkes","Mark Iv Press Ltd","4284 Dorigo Ln","Chicago","Cook","IL",60647,"773-446-5569","773-352-3437","viva.toelkes@gmail.com","http://www.markivpressltd.com"
"Elza","Lipke","Museum Of Science & Industry","6794 Lake Dr E","Newark","Essex","NJ","07104","973-927-3447","973-796-3667","elza@yahoo.com","http://www.museumofscienceindustry.com"
"Devorah","Chickering","Garrison Ind","31 Douglas Blvd #950","Clovis","Curry","NM",88101,"505-975-8559","505-950-1763","devorah@hotmail.com","http://www.garrisonind.com"
"Timothy","Mulqueen","Saronix Nymph Products","44 W 4th St","Staten Island","Richmond","NY",10309,"718-332-6527","718-654
 -7063","timothy_mulqueen@mulqueen.org","http://www.saronixnymphproducts.com"
"Arlette","Honeywell","Smc Inc","11279 Loytan St","Jacksonville","Duval","FL",32254,"904-775-4480","904-514-9918","ahoneywell@honeywell.com","http://www.smcinc.com"
"Dominque","Dickerson","E A I Electronic Assocs Inc","69 Marquette Ave","Hayward","Alameda","CA",94545,"510-993-3758","510-901-7640","dominque.dickerson@dickerson.org","http://www.eaielectronicassocsinc.com"
"Lettie","Isenhower","Conte, Christopher A Esq","70 W Main St","Beachwood","Cuyahoga","OH",44122,"216-657-7668","216-733-8494","lettie_isenhower@yahoo.com","http://www.contechristopheraesq.com"
"Myra","Munns","Anker Law Office","461 Prospect Pl #316","Euless","Tarrant","TX",76040,"817-914-7518","817-451-3518","mmunns@cox.net","http://www.ankerlawoffice.com"
"Stephaine","Barfield","Beutelschies & Company","47154 Whipple Ave Nw","Gardena","Los Angeles","CA",90247,"310-774-7643","310-968-1219","stephaine@barfield.com","http://www.beutelschiesco
 mpany.com"
"Lai","Gato","Fligg, Kenneth I Jr","37 Alabama Ave","Evanston","Cook","IL",60201,"847-728-7286","847-957-4614","lai.gato@gato.org","http://www.fliggkennethijr.com"
"Stephen","Emigh","Sharp, J Daniel Esq","3777 E Richmond St #900","Akron","Summit","OH",44302,"330-537-5358","330-700-2312","stephen_emigh@hotmail.com","http://www.sharpjdanielesq.com"
"Tyra","Shields","Assink, Anne H Esq","3 Fort Worth Ave","Philadelphia","Philadelphia","PA",19106,"215-255-1641","215-228-8264","tshields@gmail.com","http://www.assinkannehesq.com"
"Tammara","Wardrip","Jewel My Shop Inc","4800 Black Horse Pike","Burlingame","San Mateo","CA",94010,"650-803-1936","650-216-5075","twardrip@cox.net","http://www.jewelmyshopinc.com"
"Cory","Gibes","Chinese Translation Resources","83649 W Belmont Ave","San Gabriel","Los Angeles","CA",91776,"626-572-1096","626-696-2777","cory.gibes@gmail.com","http://www.chinesetranslationresources.com"
"Danica","Bruschke","Stevens, Charles T","840 15th Ave","Waco","McLen
 nan","TX",76708,"254-782-8569","254-205-1422","danica_bruschke@gmail.com","http://www.stevenscharlest.com"
"Wilda","Giguere","Mclaughlin, Luther W Cpa","1747 Calle Amanecer #2","Anchorage","Anchorage","AK",99501,"907-870-5536","907-914-9482","wilda@cox.net","http://www.mclaughlinlutherwcpa.com"
"Elvera","Benimadho","Tree Musketeers","99385 Charity St #840","San Jose","Santa Clara","CA",95110,"408-703-8505","408-440-8447","elvera.benimadho@cox.net","http://www.treemusketeers.com"
"Carma","Vanheusen","Springfield Div Oh Edison Co","68556 Central Hwy","San Leandro","Alameda","CA",94577,"510-503-7169","510-452-4835","carma@cox.net","http://www.springfielddivohedisonco.com"
"Malinda","Hochard","Logan Memorial Hospital","55 Riverside Ave","Indianapolis","Marion","IN",46202,"317-722-5066","317-472-2412","malinda.hochard@yahoo.com","http://www.loganmemorialhospital.com"
"Natalie","Fern","Kelly, Charles G Esq","7140 University Ave","Rock Springs","Sweetwater","WY",82901,"307-704-8713","307-2
 79-3793","natalie.fern@hotmail.com","http://www.kellycharlesgesq.com"
"Lisha","Centini","Industrial Paper Shredders Inc","64 5th Ave #1153","Mc Lean","Fairfax","VA",22102,"703-235-3937","703-475-7568","lisha@centini.org","http://www.industrialpapershreddersinc.com"
"Arlene","Klusman","Beck Horizon Builders","3 Secor Rd","New Orleans","Orleans","LA",70112,"504-710-5840","504-946-1807","arlene_klusman@gmail.com","http://www.beckhorizonbuilders.com"
"Alease","Buemi","Porto Cayo At Hawks Cay","4 Webbs Chapel Rd","Boulder","Boulder","CO",80303,"303-301-4946","303-521-9860","alease@buemi.com","http://www.portocayoathawkscay.com"
"Louisa","Cronauer","Pacific Grove Museum Ntrl Hist","524 Louisiana Ave Nw","San Leandro","Alameda","CA",94577,"510-828-7047","510-472-7758","louisa@cronauer.com","http://www.pacificgrovemuseumntrlhist.com"
"Angella","Cetta","Bender & Hatley Pc","185 Blackstone Bldge","Honolulu","Honolulu","HI",96817,"808-892-7943","808-475-2310","angella.cetta@hotmail.com","http:
 //www.benderhatleypc.com"
"Cyndy","Goldammer","Di Cristina J & Son","170 Wyoming Ave","Burnsville","Dakota","MN",55337,"952-334-9408","952-938-9457","cgoldammer@cox.net","http://www.dicristinajson.com"
"Rosio","Cork","Green Goddess","4 10th St W","High Point","Guilford","NC",27263,"336-243-5659","336-497-4407","rosio.cork@gmail.com","http://www.greengoddess.com"
"Celeste","Korando","American Arts & Graphics","7 W Pinhook Rd","Lynbrook","Nassau","NY",11563,"516-509-2347","516-365-7266","ckorando@hotmail.com","http://www.americanartsgraphics.com"
"Twana","Felger","Opryland Hotel","1 Commerce Way","Portland","Washington","OR",97224,"503-939-3153","503-909-7167","twana.felger@felger.org","http://www.oprylandhotel.com"
"Estrella","Samu","Marking Devices Pubg Co","64 Lakeview Ave","Beloit","Rock","WI",53511,"608-976-7199","608-942-8836","estrella@aol.com","http://www.markingdevicespubgco.com"
"Donte","Kines","W Tc Industries Inc","3 Aspen St","Worcester","Worcester","MA","01602","508-429-
 8576","508-843-1426","dkines@hotmail.com","http://www.wtcindustriesinc.com"
"Tiffiny","Steffensmeier","Whitehall Robbins Labs Divsn","32860 Sierra Rd","Miami","Miami-Dade","FL",33133,"305-385-9695","305-304-6573","tiffiny_steffensmeier@cox.net","http://www.whitehallrobbinslabsdivsn.com"
"Edna","Miceli","Sampler","555 Main St","Erie","Erie","PA",16502,"814-460-2655","814-299-2877","emiceli@miceli.org","http://www.sampler.com"
"Sue","Kownacki","Juno Chefs Incorporated","2 Se 3rd Ave","Mesquite","Dallas","TX",75149,"972-666-3413","972-742-4000","sue@aol.com","http://www.junochefsincorporated.com"
"Jesusa","Shin","Carroccio, A Thomas Esq","2239 Shawnee Mission Pky","Tullahoma","Coffee","TN",37388,"931-273-8709","931-739-1551","jshin@shin.com","http://www.carroccioathomasesq.com"
"Rolland","Francescon","Stanley, Richard L Esq","2726 Charcot Ave","Paterson","Passaic","NJ","07501","973-649-2922","973-284-4048","rolland@cox.net","http://www.stanleyrichardlesq.com"
"Pamella","Schmierer","K C
 s Cstm Mouldings Windows","5161 Dorsett Rd","Homestead","Miami-Dade","FL",33030,"305-420-8970","305-575-8481","pamella.schmierer@schmierer.org","http://www.kcscstmmouldingswindows.com"
"Glory","Kulzer","Comfort Inn","55892 Jacksonville Rd","Owings Mills","Baltimore","MD",21117,"410-224-9462","410-916-8015","gkulzer@kulzer.org","http://www.comfortinn.com"
"Shawna","Palaspas","Windsor, James L Esq","5 N Cleveland Massillon Rd","Thousand Oaks","Ventura","CA",91362,"805-275-3566","805-638-6617","shawna_palaspas@palaspas.org","http://www.windsorjameslesq.com"
"Brandon","Callaro","Jackson Shields Yeiser","7 Benton Dr","Honolulu","Honolulu","HI",96819,"808-215-6832","808-240-5168","brandon_callaro@hotmail.com","http://www.jacksonshieldsyeiser.com"
"Scarlet","Cartan","Box, J Calvin Esq","9390 S Howell Ave","Albany","Dougherty","GA",31701,"229-735-3378","229-365-9658","scarlet.cartan@yahoo.com","http://www.boxjcalvinesq.com"
"Oretha","Menter","Custom Engineering Inc","8 County Center Dr #647
 ","Boston","Suffolk","MA","02210","617-418-5043","617-697-6024","oretha_menter@yahoo.com","http://www.customengineeringinc.com"
"Ty","Smith","Bresler Eitel Framg Gllry Ltd","4646 Kaahumanu St","Hackensack","Bergen","NJ","07601","201-672-1553","201-995-3149","tsmith@aol.com","http://www.breslereitelframggllryltd.com"
"Xuan","Rochin","Carol, Drake Sparks Esq","2 Monroe St","San Mateo","San Mateo","CA",94403,"650-933-5072","650-247-2625","xuan@gmail.com","http://www.caroldrakesparksesq.com"
"Lindsey","Dilello","Biltmore Investors Bank","52777 Leaders Heights Rd","Ontario","San Bernardino","CA",91761,"909-639-9887","909-589-1693","lindsey.dilello@hotmail.com","http://www.biltmoreinvestorsbank.com"
"Devora","Perez","Desco Equipment Corp","72868 Blackington Ave","Oakland","Alameda","CA",94606,"510-955-3016","510-755-9274","devora_perez@perez.org","http://www.descoequipmentcorp.com"
"Herman","Demesa","Merlin Electric Co","9 Norristown Rd","Troy","Rensselaer","NY",12180,"518-497-2940","518-
 931-7852","hdemesa@cox.net","http://www.merlinelectricco.com"
"Rory","Papasergi","Bailey Cntl Co Div Babcock","83 County Road 437 #8581","Clarks Summit","Lackawanna","PA",18411,"570-867-7489","570-469-8401","rpapasergi@cox.net","http://www.baileycntlcodivbabcock.com"
"Talia","Riopelle","Ford Brothers Wholesale Inc","1 N Harlem Ave #9","Orange","Essex","NJ","07050","973-245-2133","973-818-9788","talia_riopelle@aol.com","http://www.fordbrotherswholesaleinc.com"
"Van","Shire","Cambridge Inn","90131 J St","Pittstown","Hunterdon","NJ","08867","908-409-2890","908-448-1209","van.shire@shire.com","http://www.cambridgeinn.com"
"Lucina","Lary","Matricciani, Albert J Jr","8597 W National Ave","Cocoa","Brevard","FL",32922,"321-749-4981","321-632-4668","lucina_lary@cox.net","http://www.matriccianialbertjjr.com"
"Bok","Isaacs","Nelson Hawaiian Ltd","6 Gilson St","Bronx","Bronx","NY",10468,"718-809-3762","718-478-8568","bok.isaacs@aol.com","http://www.nelsonhawaiianltd.com"
"Rolande","Spickerman",
 "Neland Travel Agency","65 W Maple Ave","Pearl City","Honolulu","HI",96782,"808-315-3077","808-526-5863","rolande.spickerman@spickerman.com","http://www.nelandtravelagency.com"
"Howard","Paulas","Asendorf, J Alan Esq","866 34th Ave","Denver","Denver","CO",80231,"303-623-4241","303-692-3118","hpaulas@gmail.com","http://www.asendorfjalanesq.com"
"Kimbery","Madarang","Silberman, Arthur L Esq","798 Lund Farm Way","Rockaway","Morris","NJ","07866","973-310-1634","973-225-6259","kimbery_madarang@cox.net","http://www.silbermanarthurlesq.com"
"Thurman","Manno","Honey Bee Breeding Genetics &","9387 Charcot Ave","Absecon","Atlantic","NJ","08201","609-524-3586","609-234-8376","thurman.manno@yahoo.com","http://www.honeybeebreedinggenetics.com"
"Becky","Mirafuentes","Wells Kravitz Schnitzer","30553 Washington Rd","Plainfield","Union","NJ","07062","908-877-8409","908-426-8272","becky.mirafuentes@mirafuentes.com","http://www.wellskravitzschnitzer.com"
"Beatriz","Corrington","Prohab Rehabilitation S
 ervs","481 W Lemon St","Middleboro","Plymouth","MA","02346","508-584-4279","508-315-3867","beatriz@yahoo.com","http://www.prohabrehabilitationservs.com"
"Marti","Maybury","Eldridge, Kristin K Esq","4 Warehouse Point Rd #7","Chicago","Cook","IL",60638,"773-775-4522","773-539-1058","marti.maybury@yahoo.com","http://www.eldridgekristinkesq.com"
"Nieves","Gotter","Vlahos, John J Esq","4940 Pulaski Park Dr","Portland","Multnomah","OR",97202,"503-527-5274","503-455-3094","nieves_gotter@gmail.com","http://www.vlahosjohnjesq.com"
"Leatha","Hagele","Ninas Indian Grs & Videos","627 Walford Ave","Dallas","Dallas","TX",75227,"214-339-1809","214-225-5850","lhagele@cox.net","http://www.ninasindiangrsvideos.com"
"Valentin","Klimek","Schmid, Gayanne K Esq","137 Pioneer Way","Chicago","Cook","IL",60604,"312-303-5453","312-512-2338","vklimek@klimek.org","http://www.schmidgayannekesq.com"
"Melissa","Wiklund","Moapa Valley Federal Credit Un","61 13 Stoneridge #835","Findlay","Hancock","OH",45840,"419-9
 39-3613","419-254-4591","melissa@cox.net","http://www.moapavalleyfederalcreditun.com"
"Sheridan","Zane","Kentucky Tennessee Clay Co","2409 Alabama Rd","Riverside","Riverside","CA",92501,"951-645-3605","951-248-6822","sheridan.zane@zane.com","http://www.kentuckytennesseeclayco.com"
"Bulah","Padilla","Admiral Party Rentals & Sales","8927 Vandever Ave","Waco","McLennan","TX",76707,"254-463-4368","254-816-8417","bulah_padilla@hotmail.com","http://www.admiralpartyrentalssales.com"
"Audra","Kohnert","Nelson, Karolyn King Esq","134 Lewis Rd","Nashville","Davidson","TN",37211,"615-406-7854","615-448-9249","audra@kohnert.com","http://www.nelsonkarolynkingesq.com"
"Daren","Weirather","Panasystems","9 N College Ave #3","Milwaukee","Milwaukee","WI",53216,"414-959-2540","414-838-3151","dweirather@aol.com","http://www.panasystems.com"
"Fernanda","Jillson","Shank, Edward L Esq","60480 Old Us Highway 51","Preston","Caroline","MD",21655,"410-387-5260","410-724-6472","fjillson@aol.com","http://www.sh
 ankedwardlesq.com"
"Gearldine","Gellinger","Megibow & Edwards","4 Bloomfield Ave","Irving","Dallas","TX",75061,"972-934-6914","972-821-7118","gearldine_gellinger@gellinger.com","http://www.megibowedwards.com"
"Chau","Kitzman","Benoff, Edward Esq","429 Tiger Ln","Beverly Hills","Los Angeles","CA",90212,"310-560-8022","310-969-7230","chau@gmail.com","http://www.benoffedwardesq.com"
"Theola","Frey","Woodbridge Free Public Library","54169 N Main St","Massapequa","Nassau","NY",11758,"516-948-5768","516-357-3362","theola_frey@frey.com","http://www.woodbridgefreepubliclibrary.com"
"Cheryl","Haroldson","New York Life John Thune","92 Main St","Atlantic City","Atlantic","NJ","08401","609-518-7697","609-263-9243","cheryl@haroldson.org","http://www.newyorklifejohnthune.com"
"Laticia","Merced","Alinabal Inc","72 Mannix Dr","Cincinnati","Hamilton","OH",45203,"513-508-7371","513-418-1566","lmerced@gmail.com","http://www.alinabalinc.com"
"Carissa","Batman","Poletto, Kim David Esq","12270 Caton Cent
 er Dr","Eugene","Lane","OR",97401,"541-326-4074","541-801-5717","carissa.batman@yahoo.com","http://www.polettokimdavidesq.com"
"Lezlie","Craghead","Chang, Carolyn Esq","749 W 18th St #45","Smithfield","Johnston","NC",27577,"919-533-3762","919-885-2453","lezlie.craghead@craghead.org","http://www.changcarolynesq.com"
"Ozell","Shealy","Silver Bros Inc","8 Industry Ln","New York","New York","NY",10002,"212-332-8435","212-880-8865","oshealy@hotmail.com","http://www.silverbrosinc.com"
"Arminda","Parvis","Newtec Inc","1 Huntwood Ave","Phoenix","Maricopa","AZ",85017,"602-906-9419","602-277-3025","arminda@parvis.com","http://www.newtecinc.com"
"Reita","Leto","Creative Business Systems","55262 N French Rd","Indianapolis","Marion","IN",46240,"317-234-1135","317-787-5514","reita.leto@gmail.com","http://www.creativebusinesssystems.com"
"Yolando","Luczki","Dal Tile Corporation","422 E 21st St","Syracuse","Onondaga","NY",13214,"315-304-4759","315-640-6357","yolando@cox.net","http://www.daltilecorp
 oration.com"
"Lizette","Stem","Edward S Katz","501 N 19th Ave","Cherry Hill","Camden","NJ","08002","856-487-5412","856-702-3676","lizette.stem@aol.com","http://www.edwardskatz.com"
"Gregoria","Pawlowicz","Oh My Goodknits Inc","455 N Main Ave","Garden City","Nassau","NY",11530,"516-212-1915","516-376-4230","gpawlowicz@yahoo.com","http://www.ohmygoodknitsinc.com"
"Carin","Deleo","Redeker, Debbie","1844 Southern Blvd","Little Rock","Pulaski","AR",72202,"501-308-1040","501-409-6072","cdeleo@deleo.com","http://www.redekerdebbie.com"
"Chantell","Maynerich","Desert Sands Motel","2023 Greg St","Saint Paul","Ramsey","MN",55101,"651-591-2583","651-776-9688","chantell@yahoo.com","http://www.desertsandsmotel.com"
"Dierdre","Yum","Cummins Southern Plains Inc","63381 Jenks Ave","Philadelphia","Philadelphia","PA",19134,"215-325-3042","215-346-4666","dyum@yahoo.com","http://www.cumminssouthernplainsinc.com"
"Larae","Gudroe","Lehigh Furn Divsn Lehigh","6651 Municipal Rd","Houma","Terrebonne","LA",70
 360,"985-890-7262","985-261-5783","larae_gudroe@gmail.com","http://www.lehighfurndivsnlehigh.com"
"Latrice","Tolfree","United Van Lines Agent","81 Norris Ave #525","Ronkonkoma","Suffolk","NY",11779,"631-957-7624","631-998-2102","latrice.tolfree@hotmail.com","http://www.unitedvanlinesagent.com"
"Kerry","Theodorov","Capitol Reporters","6916 W Main St","Sacramento","Sacramento","CA",95827,"916-591-3277","916-770-7448","kerry.theodorov@gmail.com","http://www.capitolreporters.com"
"Dorthy","Hidvegi","Kwik Kopy Printing","9635 S Main St","Boise","Ada","ID",83704,"208-649-2373","208-690-3315","dhidvegi@yahoo.com","http://www.kwikkopyprinting.com"
"Fannie","Lungren","Centro Inc","17 Us Highway 111","Round Rock","Williamson","TX",78664,"512-587-5746","512-528-9933","fannie.lungren@yahoo.com","http://www.centroinc.com"
"Evangelina","Radde","Campbell, Jan Esq","992 Civic Center Dr","Philadelphia","Philadelphia","PA",19123,"215-964-3284","215-417-5612","evangelina@aol.com","http://www.campbellj
 anesq.com"
"Novella","Degroot","Evans, C Kelly Esq","303 N Radcliffe St","Hilo","Hawaii","HI",96720,"808-477-4775","808-746-1865","novella_degroot@degroot.org","http://www.evansckellyesq.com"
"Clay","Hoa","Scat Enterprises","73 Saint Ann St #86","Reno","Washoe","NV",89502,"775-501-8109","775-848-9135","choa@hoa.org","http://www.scatenterprises.com"
"Jennifer","Fallick","Nagle, Daniel J Esq","44 58th St","Wheeling","Cook","IL",60090,"847-979-9545","847-800-3054","jfallick@yahoo.com","http://www.nagledanieljesq.com"
"Irma","Wolfgramm","Serendiquity Bed & Breakfast","9745 W Main St","Randolph","Morris","NJ","07869","973-545-7355","973-868-8660","irma.wolfgramm@hotmail.com","http://www.serendiquitybedbreakfast.com"
"Eun","Coody","Ray Carolyne Realty","84 Bloomfield Ave","Spartanburg","Spartanburg","SC",29301,"864-256-3620","864-594-4578","eun@yahoo.com","http://www.raycarolynerealty.com"
"Sylvia","Cousey","Berg, Charles E","287 Youngstown Warren Rd","Hampstead","Carroll","MD",21074,"410
 -209-9545","410-863-8263","sylvia_cousey@cousey.org","http://www.bergcharlese.com"
"Nana","Wrinkles","Ray, Milbern D","6 Van Buren St","Mount Vernon","Westchester","NY",10553,"914-855-2115","914-796-3775","nana@aol.com","http://www.raymilbernd.com"
"Layla","Springe","Chadds Ford Winery","229 N Forty Driv","New York","New York","NY",10011,"212-260-3151","212-253-7448","layla.springe@cox.net","http://www.chaddsfordwinery.com"
"Joesph","Degonia","A R Packaging","2887 Knowlton St #5435","Berkeley","Alameda","CA",94710,"510-677-9785","510-942-5916","joesph_degonia@degonia.org","http://www.arpackaging.com"
"Annabelle","Boord","Corn Popper","523 Marquette Ave","Concord","Middlesex","MA","01742","978-697-6263","978-289-7717","annabelle.boord@cox.net","http://www.cornpopper.com"
"Stephaine","Vinning","Birite Foodservice Distr","3717 Hamann Industrial Pky","San Francisco","San Francisco","CA",94104,"415-767-6596","415-712-9530","stephaine@cox.net","http://www.biritefoodservicedistr.com"
"Neli
 da","Sawchuk","Anchorage Museum Of Hist & Art","3 State Route 35 S","Paramus","Bergen","NJ","07652","201-971-1638","201-247-8925","nelida@gmail.com","http://www.anchoragemuseumofhistart.com"
"Marguerita","Hiatt","Haber, George D Md","82 N Highway 67","Oakley","Contra Costa","CA",94561,"925-634-7158","925-541-8521","marguerita.hiatt@gmail.com","http://www.habergeorgedmd.com"
"Carmela","Cookey","Royal Pontiac Olds Inc","9 Murfreesboro Rd","Chicago","Cook","IL",60623,"773-494-4195","773-297-9391","ccookey@cookey.org","http://www.royalpontiacoldsinc.com"
"Junita","Brideau","Leonards Antiques Inc","6 S Broadway St","Cedar Grove","Essex","NJ","07009","973-943-3423","973-582-5469","jbrideau@aol.com","http://www.leonardsantiquesinc.com"
"Claribel","Varriano","Meca","6 Harry L Dr #6327","Perrysburg","Wood","OH",43551,"419-544-4900","419-573-2033","claribel_varriano@cox.net","http://www.meca.com"
"Benton","Skursky","Nercon Engineering & Mfg Inc","47939 Porter Ave","Gardena","Los Angeles","CA"
 ,90248,"310-579-2907","310-694-8466","benton.skursky@aol.com","http://www.nerconengineeringmfginc.com"
"Hillary","Skulski","Replica I","9 Wales Rd Ne #914","Homosassa","Citrus","FL",34448,"352-242-2570","352-990-5946","hillary.skulski@aol.com","http://www.replicai.com"
"Merilyn","Bayless","20 20 Printing Inc","195 13n N","Santa Clara","Santa Clara","CA",95054,"408-758-5015","408-346-2180","merilyn_bayless@cox.net","http://www.printinginc.com"
"Teri","Ennaco","Publishers Group West","99 Tank Farm Rd","Hazleton","Luzerne","PA",18201,"570-889-5187","570-355-1665","tennaco@gmail.com","http://www.publishersgroupwest.com"
"Merlyn","Lawler","Nischwitz, Jeffrey L Esq","4671 Alemany Blvd","Jersey City","Hudson","NJ","07304","201-588-7810","201-858-9960","merlyn_lawler@hotmail.com","http://www.nischwitzjeffreylesq.com"
"Georgene","Montezuma","Payne Blades & Wellborn Pa","98 University Dr","San Ramon","Contra Costa","CA",94583,"925-615-5185","925-943-3449","gmontezuma@cox.net","http://www.payn
 ebladeswellbornpa.com"
"Jettie","Mconnell","Coldwell Bnkr Wright Real Est","50 E Wacker Dr","Bridgewater","Somerset","NJ","08807","908-802-3564","908-602-5258","jmconnell@hotmail.com","http://www.coldwellbnkrwrightrealest.com"
"Lemuel","Latzke","Computer Repair Service","70 Euclid Ave #722","Bohemia","Suffolk","NY",11716,"631-748-6479","631-291-4976","lemuel.latzke@gmail.com","http://www.computerrepairservice.com"
"Melodie","Knipp","Fleetwood Building Block Inc","326 E Main St #6496","Thousand Oaks","Ventura","CA",91362,"805-690-1682","805-810-8964","mknipp@gmail.com","http://www.fleetwoodbuildingblockinc.com"
"Candida","Corbley","Colts Neck Medical Assocs Inc","406 Main St","Somerville","Somerset","NJ","08876","908-275-8357","908-943-6103","candida_corbley@hotmail.com","http://www.coltsneckmedicalassocsinc.com"
"Karan","Karpin","New England Taxidermy","3 Elmwood Dr","Beaverton","Washington","OR",97005,"503-940-8327","503-707-5812","karan_karpin@gmail.com","http://www.newenglandtaxi
 dermy.com"
"Andra","Scheyer","Ludcke, George O Esq","9 Church St","Salem","Marion","OR",97302,"503-516-2189","503-950-3068","andra@gmail.com","http://www.ludckegeorgeoesq.com"
"Felicidad","Poullion","Mccorkle, Tom S Esq","9939 N 14th St","Riverton","Burlington","NJ","08077","856-305-9731","856-828-6021","fpoullion@poullion.com","http://www.mccorkletomsesq.com"
"Belen","Strassner","Eagle Software Inc","5384 Southwyck Blvd","Douglasville","Douglas","GA",30135,"770-507-8791","770-802-4003","belen_strassner@aol.com","http://www.eaglesoftwareinc.com"
"Gracia","Melnyk","Juvenile & Adult Super","97 Airport Loop Dr","Jacksonville","Duval","FL",32216,"904-235-3633","904-627-4341","gracia@melnyk.com","http://www.juvenileadultsuper.com"
"Jolanda","Hanafan","Perez, Joseph J Esq","37855 Nolan Rd","Bangor","Penobscot","ME","04401","207-458-9196","207-233-6185","jhanafan@gmail.com","http://www.perezjosephjesq.com"
"Barrett","Toyama","Case Foundation Co","4252 N Washington Ave #9","Kennedale","Tarr
 ant","TX",76060,"817-765-5781","817-577-6151","barrett.toyama@toyama.org","http://www.casefoundationco.com"
"Helga","Fredicks","Eis Environmental Engrs Inc","42754 S Ash Ave","Buffalo","Erie","NY",14228,"716-752-4114","716-854-9845","helga_fredicks@yahoo.com","http://www.eisenvironmentalengrsinc.com"
"Ashlyn","Pinilla","Art Crafters","703 Beville Rd","Opa Locka","Miami-Dade","FL",33054,"305-670-9628","305-857-5489","apinilla@cox.net","http://www.artcrafters.com"
"Fausto","Agramonte","Marriott Hotels Resorts Suites","5 Harrison Rd","New York","New York","NY",10038,"212-313-1783","212-778-3063","fausto_agramonte@yahoo.com","http://www.marriotthotelsresortssuites.com"
"Ronny","Caiafa","Remaco Inc","73 Southern Blvd","Philadelphia","Philadelphia","PA",19103,"215-605-7570","215-511-3531","ronny.caiafa@caiafa.org","http://www.remacoinc.com"
"Marge","Limmel","Bjork, Robert D Jr","189 Village Park Rd","Crestview","Okaloosa","FL",32536,"850-430-1663","850-330-8079","marge@gmail.com","http://
 www.bjorkrobertdjr.com"
"Norah","Waymire","Carmichael, Jeffery L Esq","6 Middlegate Rd #106","San Francisco","San Francisco","CA",94107,"415-306-7897","415-874-2984","norah.waymire@gmail.com","http://www.carmichaeljefferylesq.com"
"Aliza","Baltimore","Andrews, J Robert Esq","1128 Delaware St","San Jose","Santa Clara","CA",95132,"408-504-3552","408-425-1994","aliza@aol.com","http://www.andrewsjrobertesq.com"
"Mozell","Pelkowski","Winship & Byrne","577 Parade St","South San Francisco","San Mateo","CA",94080,"650-947-1215","650-960-1069","mpelkowski@pelkowski.org","http://www.winshipbyrne.com"
"Viola","Bitsuie","Burton & Davis","70 Mechanic St","Northridge","Los Angeles","CA",91325,"818-864-4875","818-481-5787","viola@gmail.com","http://www.burtondavis.com"
"Franklyn","Emard","Olympic Graphic Arts","4379 Highway 116","Philadelphia","Philadelphia","PA",19103,"215-558-8189","215-483-3003","femard@emard.com","http://www.olympicgraphicarts.com"
"Willodean","Konopacki","Magnuson","55 Hawtho
 rne Blvd","Lafayette","Lafayette","LA",70506,"337-253-8384","337-774-7564","willodean_konopacki@konopacki.org","http://www.magnuson.com"
"Beckie","Silvestrini","A All American Travel Inc","7116 Western Ave","Dearborn","Wayne","MI",48126,"313-533-4884","313-390-7855","beckie.silvestrini@silvestrini.com","http://www.aallamericantravelinc.com"
"Rebecka","Gesick","Polykote Inc","2026 N Plankinton Ave #3","Austin","Travis","TX",78754,"512-213-8574","512-693-8345","rgesick@gesick.org","http://www.polykoteinc.com"
"Frederica","Blunk","Jets Cybernetics","99586 Main St","Dallas","Dallas","TX",75207,"214-428-2285","214-529-1949","frederica_blunk@gmail.com","http://www.jetscybernetics.com"
"Glen","Bartolet","Metlab Testing Services","8739 Hudson St","Vashon","King","WA",98070,"206-697-5796","206-389-1482","glen_bartolet@hotmail.com","http://www.metlabtestingservices.com"
"Freeman","Gochal","Kellermann, William T Esq","383 Gunderman Rd #197","Coatesville","Chester","PA",19320,"610-476-3501","61
 0-752-2683","freeman_gochal@aol.com","http://www.kellermannwilliamtesq.com"
"Vincent","Meinerding","Arturi, Peter D Esq","4441 Point Term Mkt","Philadelphia","Philadelphia","PA",19143,"215-372-1718","215-829-4221","vincent.meinerding@hotmail.com","http://www.arturipeterdesq.com"
"Rima","Bevelacqua","Mcauley Mfg Co","2972 Lafayette Ave","Gardena","Los Angeles","CA",90248,"310-858-5079","310-499-4200","rima@cox.net","http://www.mcauleymfgco.com"
"Glendora","Sarbacher","Defur Voran Hanley Radcliff","2140 Diamond Blvd","Rohnert Park","Sonoma","CA",94928,"707-653-8214","707-881-3154","gsarbacher@gmail.com","http://www.defurvoranhanleyradcliff.com"
"Avery","Steier","Dill Dill Carr & Stonbraker Pc","93 Redmond Rd #492","Orlando","Orange","FL",32803,"407-808-9439","407-945-8566","avery@cox.net","http://www.dilldillcarrstonbrakerpc.com"
"Cristy","Lother","Kleensteel","3989 Portage Tr","Escondido","San Diego","CA",92025,"760-971-4322","760-465-4762","cristy@lother.com","http://www.kleensteel.
 com"
"Nicolette","Brossart","Goulds Pumps Inc Slurry Pump","1 Midway Rd","Westborough","Worcester","MA","01581","508-837-9230","508-504-6388","nicolette_brossart@brossart.com","http://www.gouldspumpsincslurrypump.com"
"Tracey","Modzelewski","Kansas City Insurance Report","77132 Coon Rapids Blvd Nw","Conroe","Montgomery","TX",77301,"936-264-9294","936-988-8171","tracey@hotmail.com","http://www.kansascityinsurancereport.com"
"Virgina","Tegarden","Berhanu International Foods","755 Harbor Way","Milwaukee","Milwaukee","WI",53226,"414-214-8697","414-411-5744","virgina_tegarden@tegarden.com","http://www.berhanuinternationalfoods.com"
"Tiera","Frankel","Roland Ashcroft","87 Sierra Rd","El Monte","Los Angeles","CA",91731,"626-636-4117","626-638-4241","tfrankel@aol.com","http://www.rolandashcroft.com"
"Alaine","Bergesen","Hispanic Magazine","7667 S Hulen St #42","Yonkers","Westchester","NY",10701,"914-300-9193","914-654-1426","alaine_bergesen@cox.net","http://www.hispanicmagazine.com"
"Earlee
 n","Mai","Little Sheet Metal Co","75684 S Withlapopka Dr #32","Dallas","Dallas","TX",75227,"214-289-1973","214-785-6750","earleen_mai@cox.net","http://www.littlesheetmetalco.com"
"Leonida","Gobern","Holmes, Armstead J Esq","5 Elmwood Park Blvd","Biloxi","Harrison","MS",39530,"228-235-5615","228-432-4635","leonida@gobern.org","http://www.holmesarmsteadjesq.com"
"Ressie","Auffrey","Faw, James C Cpa","23 Palo Alto Sq","Miami","Miami-Dade","FL",33134,"305-604-8981","305-287-4743","ressie.auffrey@yahoo.com","http://www.fawjamesccpa.com"
"Justine","Mugnolo","Evans Rule Company","38062 E Main St","New York","New York","NY",10048,"212-304-9225","212-311-6377","jmugnolo@yahoo.com","http://www.evansrulecompany.com"
"Eladia","Saulter","Tyee Productions Inc","3958 S Dupont Hwy #7","Ramsey","Bergen","NJ","07446","201-474-4924","201-365-8698","eladia@saulter.com","http://www.tyeeproductionsinc.com"
"Chaya","Malvin","Dunnells & Duvall","560 Civic Center Dr","Ann Arbor","Washtenaw","MI",48103,"734-
 928-5182","734-408-8174","chaya@malvin.com","http://www.dunnellsduvall.com"
"Gwenn","Suffield","Deltam Systems Inc","3270 Dequindre Rd","Deer Park","Suffolk","NY",11729,"631-258-6558","631-295-9879","gwenn_suffield@suffield.org","http://www.deltamsystemsinc.com"
"Salena","Karpel","Hammill Mfg Co","1 Garfield Ave #7","Canton","Stark","OH",44707,"330-791-8557","330-618-2579","skarpel@cox.net","http://www.hammillmfgco.com"
"Yoko","Fishburne","Sams Corner Store","9122 Carpenter Ave","New Haven","New Haven","CT","06511","203-506-4706","203-840-8634","yoko@fishburne.com","http://www.samscornerstore.com"
"Taryn","Moyd","Siskin, Mark J Esq","48 Lenox St","Fairfax","Fairfax City","VA",22030,"703-322-4041","703-938-7939","taryn.moyd@hotmail.com","http://www.siskinmarkjesq.com"
"Katina","Polidori","Cape & Associates Real Estate","5 Little River Tpke","Wilmington","Middlesex","MA","01887","978-626-2978","978-679-7429","katina_polidori@aol.com","http://www.capeassociatesrealestate.com"
"Rickie",
 "Plumer","Merrill Lynch","3 N Groesbeck Hwy","Toledo","Lucas","OH",43613,"419-693-1334","419-313-5571","rickie.plumer@aol.com","http://www.merrilllynch.com"
"Alex","Loader","Sublett, Scott Esq","37 N Elm St #916","Tacoma","Pierce","WA",98409,"253-660-7821","253-875-9222","alex@loader.com","http://www.sublettscottesq.com"
"Lashon","Vizarro","Sentry Signs","433 Westminster Blvd #590","Roseville","Placer","CA",95661,"916-741-7884","916-289-4526","lashon@aol.com","http://www.sentrysigns.com"
"Lauran","Burnard","Professionals Unlimited","66697 Park Pl #3224","Riverton","Fremont","WY",82501,"307-342-7795","307-453-7589","lburnard@burnard.com","http://www.professionalsunlimited.com"
"Ceola","Setter","Southern Steel Shelving Co","96263 Greenwood Pl","Warren","Knox","ME","04864","207-627-7565","207-297-5029","ceola.setter@setter.org","http://www.southernsteelshelvingco.com"
"My","Rantanen","Bosco, Paul J","8 Mcarthur Ln","Richboro","Bucks","PA",18954,"215-491-5633","215-647-2158","my@hotmail
 .com","http://www.boscopaulj.com"
"Lorrine","Worlds","Longo, Nicholas J Esq","8 Fair Lawn Ave","Tampa","Hillsborough","FL",33614,"813-769-2939","813-863-6467","lorrine.worlds@worlds.com","http://www.longonicholasjesq.com"
"Peggie","Sturiale","Henry County Middle School","9 N 14th St","El Cajon","San Diego","CA",92020,"619-608-1763","619-695-8086","peggie@cox.net","http://www.henrycountymiddleschool.com"
"Marvel","Raymo","Edison Supply & Equipment Co","9 Vanowen St","College Station","Brazos","TX",77840,"979-718-8968","979-809-5770","mraymo@yahoo.com","http://www.edisonsupplyequipmentco.com"
"Daron","Dinos","Wolf, Warren R Esq","18 Waterloo Geneva Rd","Highland Park","Lake","IL",60035,"847-233-3075","847-265-6609","daron_dinos@cox.net","http://www.wolfwarrenresq.com"
"An","Fritz","Linguistic Systems Inc","506 S Hacienda Dr","Atlantic City","Atlantic","NJ","08401","609-228-5265","609-854-7156","an_fritz@hotmail.com","http://www.linguisticsystemsinc.com"
"Portia","Stimmel","Peace Chris
 tian Center","3732 Sherman Ave","Bridgewater","Somerset","NJ","08807","908-722-7128","908-670-4712","portia.stimmel@aol.com","http://www.peacechristiancenter.com"
"Rhea","Aredondo","Double B Foods Inc","25657 Live Oak St","Brooklyn","Kings","NY",11226,"718-560-9537","718-280-4183","rhea_aredondo@cox.net","http://www.doublebfoodsinc.com"
"Benedict","Sama","Alexander & Alexander Inc","4923 Carey Ave","Saint Louis","Saint Louis City","MO",63104,"314-787-1588","314-858-4832","bsama@cox.net","http://www.alexanderalexanderinc.com"
"Alyce","Arias","Fairbanks Scales","3196 S Rider Trl","Stockton","San Joaquin","CA",95207,"209-317-1801","209-242-7022","alyce@arias.org","http://www.fairbanksscales.com"
"Heike","Berganza","Cali Sportswear Cutting Dept","3 Railway Ave #75","Little Falls","Passaic","NJ","07424","973-936-5095","973-822-8827","heike@gmail.com","http://www.calisportswearcuttingdept.com"
"Carey","Dopico","Garofani, John Esq","87393 E Highland Rd","Indianapolis","Marion","IN",46220,"
 317-578-2453","317-441-5848","carey_dopico@dopico.org","http://www.garofanijohnesq.com"
"Dottie","Hellickson","Thompson Fabricating Co","67 E Chestnut Hill Rd","Seattle","King","WA",98133,"206-540-6076","206-295-5631","dottie@hellickson.org","http://www.thompsonfabricatingco.com"
"Deandrea","Hughey","Century 21 Krall Real Estate","33 Lewis Rd #46","Burlington","Alamance","NC",27215,"336-822-7652","336-467-3095","deandrea@yahoo.com","http://www.centurykrallrealestate.com"
"Kimberlie","Duenas","Mid Contntl Rlty & Prop Mgmt","8100 Jacksonville Rd #7","Hays","Ellis","KS",67601,"785-629-8542","785-616-1685","kimberlie_duenas@yahoo.com","http://www.midcontntlrltypropmgmt.com"
"Martina","Staback","Ace Signs Inc","7 W Wabansia Ave #227","Orlando","Orange","FL",32822,"407-471-6908","407-429-2145","martina_staback@staback.com","http://www.acesignsinc.com"
"Skye","Fillingim","Rodeway Inn","25 Minters Chapel Rd #9","Minneapolis","Hennepin","MN",55401,"612-508-2655","612-664-6304","skye_fillingi
 m@yahoo.com","http://www.rodewayinn.com"
"Jade","Farrar","Bonnet & Daughter","6882 Torresdale Ave","Columbia","Richland","SC",29201,"803-352-5387","803-975-3405","jade.farrar@yahoo.com","http://www.bonnetdaughter.com"
"Charlene","Hamilton","Oshins & Gibbons","985 E 6th Ave","Santa Rosa","Sonoma","CA",95407,"707-300-1771","707-821-8037","charlene.hamilton@hotmail.com","http://www.oshinsgibbons.com"
"Geoffrey","Acey","Price Business Services","7 West Ave #1","Palatine","Cook","IL",60067,"847-222-1734","847-556-2909","geoffrey@gmail.com","http://www.pricebusinessservices.com"
"Stevie","Westerbeck","Wise, Dennis W Md","26659 N 13th St","Costa Mesa","Orange","CA",92626,"949-867-4077","949-903-3898","stevie.westerbeck@yahoo.com","http://www.wisedenniswmd.com"
"Pamella","Fortino","Super 8 Motel","669 Packerland Dr #1438","Denver","Denver","CO",80212,"303-404-2210","303-794-1341","pamella@fortino.com","http://www.supermotel.com"
"Harrison","Haufler","John Wagner Associates","759 Eldora St",
 "New Haven","New Haven","CT","06515","203-801-6193","203-801-8497","hhaufler@hotmail.com","http://www.johnwagnerassociates.com"
"Johnna","Engelberg","Thrifty Oil Co","5 S Colorado Blvd #449","Bothell","Snohomish","WA",98021,"425-986-7573","425-700-3751","jengelberg@engelberg.org","http://www.thriftyoilco.com"
"Buddy","Cloney","Larkfield Photo","944 Gaither Dr","Strongsville","Cuyahoga","OH",44136,"440-989-5826","440-327-2093","buddy.cloney@yahoo.com","http://www.larkfieldphoto.com"
"Dalene","Riden","Silverman Planetarium","66552 Malone Rd","Plaistow","Rockingham","NH","03865","603-315-6839","603-745-7497","dalene.riden@aol.com","http://www.silvermanplanetarium.com"
"Jerry","Zurcher","J & F Lumber","77 Massillon Rd #822","Satellite Beach","Brevard","FL",32937,"321-518-5938","321-597-2159","jzurcher@zurcher.org","http://www.jflumber.com"
"Haydee","Denooyer","Cleaning Station Inc","25346 New Rd","New York","New York","NY",10016,"212-792-8658","212-782-3493","hdenooyer@denooyer.org","ht
 tp://www.cleaningstationinc.com"
"Joseph","Cryer","Ames Stationers","60 Fillmore Ave","Huntington Beach","Orange","CA",92647,"714-584-2237","714-698-2170","joseph_cryer@cox.net","http://www.amesstationers.com"
"Deonna","Kippley","Midas Muffler Shops","57 Haven Ave #90","Southfield","Oakland","MI",48075,"248-913-4677","248-793-4966","deonna_kippley@hotmail.com","http://www.midasmufflershops.com"
"Raymon","Calvaresi","Seaboard Securities Inc","6538 E Pomona St #60","Indianapolis","Marion","IN",46222,"317-825-4724","317-342-1532","raymon.calvaresi@gmail.com","http://www.seaboardsecuritiesinc.com"
"Alecia","Bubash","Petersen, James E Esq","6535 Joyce St","Wichita Falls","Wichita","TX",76301,"940-276-7922","940-302-3036","alecia@aol.com","http://www.petersenjameseesq.com"
"Ma","Layous","Development Authority","78112 Morris Ave","North Haven","New Haven","CT","06473","203-721-3388","203-564-1543","mlayous@hotmail.com","http://www.developmentauthority.com"
"Detra","Coyier","Schott Fiber Op
 tics Inc","96950 Hidden Ln","Aberdeen","Harford","MD",21001,"410-739-9277","410-259-2118","detra@aol.com","http://www.schottfiberopticsinc.com"
"Terrilyn","Rodeigues","Stuart J Agins","3718 S Main St","New Orleans","Orleans","LA",70130,"504-463-4384","504-635-8518","terrilyn.rodeigues@cox.net","http://www.stuartjagins.com"
"Salome","Lacovara","Mitsumi Electronics Corp","9677 Commerce Dr","Richmond","Richmond City","VA",23219,"804-550-5097","804-858-1011","slacovara@gmail.com","http://www.mitsumielectronicscorp.com"
"Garry","Keetch","Italian Express Franchise Corp","5 Green Pond Rd #4","Southampton","Bucks","PA",18966,"215-979-8776","215-846-9046","garry_keetch@hotmail.com","http://www.italianexpressfranchisecorp.com"
"Matthew","Neither","American Council On Sci & Hlth","636 Commerce Dr #42","Shakopee","Scott","MN",55379,"952-651-7597","952-906-4597","mneither@yahoo.com","http://www.americancouncilonscihlth.com"
"Theodora","Restrepo","Kleri, Patricia S Esq","42744 Hamann Industrial P
 ky #82","Miami","Miami-Dade","FL",33136,"305-936-8226","305-573-1085","theodora.restrepo@restrepo.com","http://www.kleripatriciasesq.com"
"Noah","Kalafatis","Twiggs Abrams Blanchard","1950 5th Ave","Milwaukee","Milwaukee","WI",53209,"414-263-5287","414-660-9766","noah.kalafatis@aol.com","http://www.twiggsabramsblanchard.com"
"Carmen","Sweigard","Maui Research & Technology Pk","61304 N French Rd","Somerset","Somerset","NJ","08873","732-941-2621","732-445-6940","csweigard@sweigard.com","http://www.mauiresearchtechnologypk.com"
"Lavonda","Hengel","Bradley Nameplate Corp","87 Imperial Ct #79","Fargo","Cass","ND",58102,"701-898-2154","701-421-7080","lavonda@cox.net","http://www.bradleynameplatecorp.com"
"Junita","Stoltzman","Geonex Martel Inc","94 W Dodge Rd","Carson City","Carson City","NV",89701,"775-638-9963","775-578-1214","junita@aol.com","http://www.geonexmartelinc.com"
"Herminia","Nicolozakes","Sea Island Div Of Fstr Ind Inc","4 58th St #3519","Scottsdale","Maricopa","AZ",85254,"6
 02-954-5141","602-304-6433","herminia@nicolozakes.org","http://www.seaislanddivoffstrindinc.com"
"Casie","Good","Papay, Debbie J Esq","5221 Bear Valley Rd","Nashville","Davidson","TN",37211,"615-390-2251","615-825-4297","casie.good@aol.com","http://www.papaydebbiejesq.com"
"Reena","Maisto","Lane Promotions","9648 S Main","Salisbury","Wicomico","MD",21801,"410-351-1863","410-951-2667","reena@hotmail.com","http://www.lanepromotions.com"
"Mirta","Mallett","Stephen Kennerly Archts Inc Pc","7 S San Marcos Rd","New York","New York","NY",10004,"212-870-1286","212-745-6948","mirta_mallett@gmail.com","http://www.stephenkennerlyarchtsincpc.com"
"Cathrine","Pontoriero","Business Systems Of Wis Inc","812 S Haven St","Amarillo","Randall","TX",79109,"806-703-1435","806-558-5848","cathrine.pontoriero@pontoriero.com","http://www.businesssystemsofwisinc.com"
"Filiberto","Tawil","Flash, Elena Salerno Esq","3882 W Congress St #799","Los Angeles","Los Angeles","CA",90016,"323-765-2528","323-842-8226","
 ftawil@hotmail.com","http://www.flashelenasalernoesq.com"
"Raul","Upthegrove","Neeley, Gregory W Esq","4 E Colonial Dr","La Mesa","San Diego","CA",91942,"619-509-5282","619-666-4765","rupthegrove@yahoo.com","http://www.neeleygregorywesq.com"
"Sarah","Candlish","Alabama Educational Tv Comm","45 2nd Ave #9759","Atlanta","Fulton","GA",30328,"770-732-1194","770-531-2842","sarah.candlish@gmail.com","http://www.alabamaeducationaltvcomm.com"
"Lucy","Treston","Franz Inc","57254 Brickell Ave #372","Worcester","Worcester","MA","01602","508-769-5250","508-502-5634","lucy@cox.net","http://www.franzinc.com"
"Judy","Aquas","Plantation Restaurant","8977 Connecticut Ave Nw #3","Niles","Berrien","MI",49120,"269-756-7222","269-431-9464","jaquas@aquas.com","http://www.plantationrestaurant.com"
"Yvonne","Tjepkema","Radio Communications Co","9 Waydell St","Fairfield","Essex","NJ","07004","973-714-1721","973-976-8627","yvonne.tjepkema@hotmail.com","http://www.radiocommunicationsco.com"
"Kayleigh","Lace",
 "Dentalaw Divsn Hlth Care","43 Huey P Long Ave","Lafayette","Lafayette","LA",70508,"337-740-9323","337-751-2326","kayleigh.lace@yahoo.com","http://www.dentalawdivsnhlthcare.com"
"Felix","Hirpara","American Speedy Printing Ctrs","7563 Cornwall Rd #4462","Denver","Lancaster","PA",17517,"717-491-5643","717-583-1497","felix_hirpara@cox.net","http://www.americanspeedyprintingctrs.com"
"Tresa","Sweely","Grayson, Grant S Esq","22 Bridle Ln","Valley Park","Saint Louis","MO",63088,"314-359-9566","314-231-3514","tresa_sweely@hotmail.com","http://www.graysongrantsesq.com"
"Kristeen","Turinetti","Jeanerette Middle School","70099 E North Ave","Arlington","Tarrant","TX",76013,"817-213-8851","817-947-9480","kristeen@gmail.com","http://www.jeanerettemiddleschool.com"
"Jenelle","Regusters","Haavisto, Brian F Esq","3211 E Northeast Loop","Tampa","Hillsborough","FL",33619,"813-932-8715","813-357-7296","jregusters@regusters.com","http://www.haavistobrianfesq.com"
"Renea","Monterrubio","Wmmt Radio Stati
 on","26 Montgomery St","Atlanta","Fulton","GA",30328,"770-679-4752","770-930-9967","renea@hotmail.com","http://www.wmmtradiostation.com"
"Olive","Matuszak","Colony Paints Sales Ofc & Plnt","13252 Lighthouse Ave","Cathedral City","Riverside","CA",92234,"760-938-6069","760-745-2649","olive@aol.com","http://www.colonypaintssalesofcplnt.com"
"Ligia","Reiber","Floral Expressions","206 Main St #2804","Lansing","Ingham","MI",48933,"517-906-1108","517-747-7664","lreiber@cox.net","http://www.floralexpressions.com"
"Christiane","Eschberger","Casco Services Inc","96541 W Central Blvd","Phoenix","Maricopa","AZ",85034,"602-390-4944","602-330-6894","christiane.eschberger@yahoo.com","http://www.cascoservicesinc.com"
"Goldie","Schirpke","Reuter, Arthur C Jr","34 Saint George Ave #2","Bangor","Penobscot","ME","04401","207-295-7569","207-748-3722","goldie.schirpke@yahoo.com","http://www.reuterarthurcjr.com"
"Loreta","Timenez","Kaminski, Katherine Andritsaki","47857 Coney Island Ave","Clinton","Prince
  Georges","MD",20735,"301-696-6420","301-392-6698","loreta.timenez@hotmail.com","http://www.kaminskikatherineandritsaki.com"
"Fabiola","Hauenstein","Sidewinder Products Corp","8573 Lincoln Blvd","York","York","PA",17404,"717-809-3119","717-344-2804","fabiola.hauenstein@hauenstein.org","http://www.sidewinderproductscorp.com"
"Amie","Perigo","General Foam Corporation","596 Santa Maria Ave #7913","Mesquite","Dallas","TX",75150,"972-419-7946","972-898-1033","amie.perigo@yahoo.com","http://www.generalfoamcorporation.com"
"Raina","Brachle","Ikg Borden Divsn Harsco Corp","3829 Ventura Blvd","Butte","Silver Bow","MT",59701,"406-318-1515","406-374-7752","raina.brachle@brachle.org","http://www.ikgbordendivsnharscocorp.com"
"Erinn","Canlas","Anchor Computer Inc","13 S Hacienda Dr","Livingston","Essex","NJ","07039","973-767-3008","973-563-9502","erinn.canlas@canlas.com","http://www.anchorcomputerinc.com"
"Cherry","Lietz","Sebring & Co","40 9th Ave Sw #91","Waterford","Oakland","MI",48329,"248-9
 80-6904","248-697-7722","cherry@lietz.com","http://www.sebringco.com"
"Kattie","Vonasek","H A C Farm Lines Co Optv Assoc","2845 Boulder Crescent St","Cleveland","Cuyahoga","OH",44103,"216-923-3715","216-270-9653","kattie@vonasek.org","http://www.hacfarmlinescooptvassoc.com"
"Lilli","Scriven","Hunter, John J Esq","33 State St","Abilene","Taylor","TX",79601,"325-631-1560","325-667-7868","lilli@aol.com","http://www.hunterjohnjesq.com"
"Whitley","Tomasulo","Freehold Fence Co","2 S 15th St","Fort Worth","Tarrant","TX",76107,"817-526-4408","817-819-7799","whitley.tomasulo@aol.com","http://www.freeholdfenceco.com"
"Barbra","Adkin","Binswanger","4 Kohler Memorial Dr","Brooklyn","Kings","NY",11230,"718-201-3751","718-732-9475","badkin@hotmail.com","http://www.binswanger.com"
"Hermila","Thyberg","Chilton Malting Co","1 Rancho Del Mar Shopping C","Providence","Providence","RI","02903","401-893-4882","401-885-7681","hermila_thyberg@hotmail.com","http://www.chiltonmaltingco.com"
"Jesusita","Flis
 ter","Schoen, Edward J Jr","3943 N Highland Ave","Lancaster","Lancaster","PA",17601,"717-885-9118","717-686-7564","jesusita.flister@hotmail.com","http://www.schoenedwardjjr.com"
"Caitlin","Julia","Helderman, Seymour Cpa","5 Williams St","Johnston","Providence","RI","02919","401-948-4982","401-552-9059","caitlin.julia@julia.org","http://www.heldermanseymourcpa.com"
"Roosevelt","Hoffis","Denbrook, Myron","60 Old Dover Rd","Hialeah","Miami-Dade","FL",33014,"305-622-4739","305-302-1135","roosevelt.hoffis@aol.com","http://www.denbrookmyron.com"
"Helaine","Halter","Lippitt, Mike","8 Sheridan Rd","Jersey City","Hudson","NJ","07304","201-832-4168","201-412-3040","hhalter@yahoo.com","http://www.lippittmike.com"
"Lorean","Martabano","Hiram, Hogg P Esq","85092 Southern Blvd","San Antonio","Bexar","TX",78204,"210-856-4979","210-634-2447","lorean.martabano@hotmail.com","http://www.hiramhoggpesq.com"
"France","Buzick","In Travel Agency","64 Newman Springs Rd E","Brooklyn","Kings","NY",11219,"718-
 478-8504","718-853-3740","france.buzick@yahoo.com","http://www.intravelagency.com"
"Justine","Ferrario","Newhart Foods Inc","48 Stratford Ave","Pomona","Los Angeles","CA",91768,"909-993-3242","909-631-5703","jferrario@hotmail.com","http://www.newhartfoodsinc.com"
"Adelina","Nabours","Courtyard By Marriott","80 Pittsford Victor Rd #9","Cleveland","Cuyahoga","OH",44103,"216-230-4892","216-937-5320","adelina_nabours@gmail.com","http://www.courtyardbymarriott.com"
"Derick","Dhamer","Studer, Eugene A Esq","87163 N Main Ave","New York","New York","NY",10013,"212-304-4515","212-225-9676","ddhamer@cox.net","http://www.studereugeneaesq.com"
"Jerry","Dallen","Seashore Supply Co Waretown","393 Lafayette Ave","Richmond","Richmond City","VA",23219,"804-762-9576","804-808-9574","jerry.dallen@yahoo.com","http://www.seashoresupplycowaretown.com"
"Leota","Ragel","Mayar Silk Inc","99 5th Ave #33","Trion","Chattooga","GA",30753,"706-221-4243","706-616-5131","leota.ragel@gmail.com","http://www.mayarsil
 kinc.com"
"Jutta","Amyot","National Medical Excess Corp","49 N Mays St","Broussard","Lafayette","LA",70518,"337-515-1438","337-991-8070","jamyot@hotmail.com","http://www.nationalmedicalexcesscorp.com"
"Aja","Gehrett","Stero Company","993 Washington Ave","Nutley","Essex","NJ","07110","973-544-2677","973-986-4456","aja_gehrett@hotmail.com","http://www.sterocompany.com"
"Kirk","Herritt","Hasting, H Duane Esq","88 15th Ave Ne","Vestal","Broome","NY",13850,"607-407-3716","607-350-7690","kirk.herritt@aol.com","http://www.hastinghduaneesq.com"
"Leonora","Mauson","Insty Prints","3381 E 40th Ave","Passaic","Passaic","NJ","07055","973-412-2995","973-355-2120","leonora@yahoo.com","http://www.instyprints.com"
"Winfred","Brucato","Glenridge Manor Mobile Home Pk","201 Ridgewood Rd","Moscow","Latah","ID",83843,"208-252-4552","208-793-4108","winfred_brucato@hotmail.com","http://www.glenridgemanormobilehomepk.com"
"Tarra","Nachor","Circuit Solution Inc","39 Moccasin Dr","San Francisco","San Francisc
 o","CA",94104,"415-411-1775","415-284-2730","tarra.nachor@cox.net","http://www.circuitsolutioninc.com"
"Corinne","Loder","Local Office","4 Carroll St","North Attleboro","Bristol","MA","02760","508-942-4186","508-618-7826","corinne@loder.org","http://www.localoffice.com"
"Dulce","Labreche","Lee Kilkelly Paulson & Kabaker","9581 E Arapahoe Rd","Rochester","Oakland","MI",48307,"248-357-8718","248-811-5696","dulce_labreche@yahoo.com","http://www.leekilkellypaulsonkabaker.com"
"Kate","Keneipp","Davis, Maxon R Esq","33 N Michigan Ave","Green Bay","Brown","WI",54301,"920-353-6377","920-355-1610","kate_keneipp@yahoo.com","http://www.davismaxonresq.com"
"Kaitlyn","Ogg","Garrison, Paul E Esq","2 S Biscayne Blvd","Baltimore","Baltimore City","MD",21230,"410-665-4903","410-773-3862","kaitlyn.ogg@gmail.com","http://www.garrisonpauleesq.com"
"Sherita","Saras","Black History Resource Center","8 Us Highway 22","Colorado Springs","El Paso","CO",80937,"719-669-1664","719-547-9543","sherita.saras@cox.
 net","http://www.blackhistoryresourcecenter.com"
"Lashawnda","Stuer","Rodriguez, J Christopher Esq","7422 Martin Ave #8","Toledo","Lucas","OH",43607,"419-588-8719","419-399-1744","lstuer@cox.net","http://www.rodriguezjchristopheresq.com"
"Ernest","Syrop","Grant Family Health Center","94 Chase Rd","Hyattsville","Prince Georges","MD",20785,"301-998-9644","301-257-4883","ernest@cox.net","http://www.grantfamilyhealthcenter.com"
"Nobuko","Halsey","Goeman Wood Products Inc","8139 I Hwy 10 #92","New Bedford","Bristol","MA","02745","508-855-9887","508-897-7916","nobuko.halsey@yahoo.com","http://www.goemanwoodproductsinc.com"
"Lavonna","Wolny","Linhares, Kenneth A Esq","5 Cabot Rd","Mc Lean","Fairfax","VA",22102,"703-483-1970","703-892-2914","lavonna.wolny@hotmail.com","http://www.linhareskennethaesq.com"
"Lashaunda","Lizama","Earnhardt Printing","3387 Ryan Dr","Hanover","Anne Arundel","MD",21076,"410-678-2473","410-912-6032","llizama@cox.net","http://www.earnhardtprinting.com"
"Mariann","Bi
 lden","H P G Industrys Inc","3125 Packer Ave #9851","Austin","Travis","TX",78753,"512-223-4791","512-742-1149","mariann.bilden@aol.com","http://www.hpgindustrysinc.com"
"Helene","Rodenberger","Bailey Transportation Prod Inc","347 Chestnut St","Peoria","Maricopa","AZ",85381,"623-461-8551","623-426-4907","helene@aol.com","http://www.baileytransportationprodinc.com"
"Roselle","Estell","Mcglynn Bliss Pc","8116 Mount Vernon Ave","Bucyrus","Crawford","OH",44820,"419-571-5920","419-488-6648","roselle.estell@hotmail.com","http://www.mcglynnblisspc.com"
"Samira","Heintzman","Mutual Fish Co","8772 Old County Rd #5410","Kent","King","WA",98032,"206-311-4137","206-923-6042","sheintzman@hotmail.com","http://www.mutualfishco.com"
"Margart","Meisel","Yeates, Arthur L Aia","868 State St #38","Cincinnati","Hamilton","OH",45251,"513-617-2362","513-747-9603","margart_meisel@yahoo.com","http://www.yeatesarthurlaia.com"
"Kristofer","Bennick","Logan, Ronald J Esq","772 W River Dr","Bloomington","Monroe",
 "IN",47404,"812-368-1511","812-442-8544","kristofer.bennick@yahoo.com","http://www.loganronaldjesq.com"
"Weldon","Acuff","Advantage Martgage Company","73 W Barstow Ave","Arlington Heights","Cook","IL",60004,"847-353-2156","847-613-5866","wacuff@gmail.com","http://www.advantagemartgagecompany.com"
"Shalon","Shadrick","Germer And Gertz Llp","61047 Mayfield Ave","Brooklyn","Kings","NY",11223,"718-232-2337","718-394-4974","shalon@cox.net","http://www.germerandgertzllp.com"
"Denise","Patak","Spence Law Offices","2139 Santa Rosa Ave","Orlando","Orange","FL",32801,"407-446-4358","407-808-3254","denise@patak.org","http://www.spencelawoffices.com"
"Louvenia","Beech","John Ortiz Nts Therapy Center","598 43rd St","Beverly Hills","Los Angeles","CA",90210,"310-820-2117","310-652-2379","louvenia.beech@beech.com","http://www.johnortizntstherapycenter.com"
"Audry","Yaw","Mike Uchrin Htg & Air Cond Inc","70295 Pioneer Ct","Brandon","Hillsborough","FL",33511,"813-797-4816","813-744-7100","audry.yaw@y
 aw.org","http://www.mikeuchrinhtgaircondinc.com"
"Kristel","Ehmann","Mccoy, Joy Reynolds Esq","92899 Kalakaua Ave","El Paso","El Paso","TX",79925,"915-452-1290","915-300-6100","kristel.ehmann@aol.com","http://www.mccoyjoyreynoldsesq.com"
"Vincenza","Zepp","Kbor 1600 Am","395 S 6th St #2","El Cajon","San Diego","CA",92020,"619-603-5125","619-935-6661","vzepp@gmail.com","http://www.kboram.com"
"Elouise","Gwalthney","Quality Inn Northwest","9506 Edgemore Ave","Bladensburg","Prince Georges","MD",20710,"301-841-5012","301-591-3034","egwalthney@yahoo.com","http://www.qualityinnnorthwest.com"
"Venita","Maillard","Wallace Church Assoc Inc","72119 S Walker Ave #63","Anaheim","Orange","CA",92801,"714-523-6653","714-663-9740","venita_maillard@gmail.com","http://www.wallacechurchassocinc.com"
"Kasandra","Semidey","Can Tron","369 Latham St #500","Saint Louis","Saint Louis City","MO",63102,"314-732-9131","314-697-3652","kasandra_semidey@semidey.com","http://www.cantron.com"
"Xochitl","Discipio","
 Ravaal Enterprises Inc","3158 Runamuck Pl","Round Rock","Williamson","TX",78664,"512-233-1831","512-942-3411","xdiscipio@gmail.com","http://www.ravaalenterprisesinc.com"
"Maile","Linahan","Thompson Steel Company Inc","9 Plainsboro Rd #598","Greensboro","Guilford","NC",27409,"336-670-2640","336-364-6037","mlinahan@yahoo.com","http://www.thompsonsteelcompanyinc.com"
"Krissy","Rauser","Anderson, Mark A Esq","8728 S Broad St","Coram","Suffolk","NY",11727,"631-443-4710","631-288-2866","krauser@cox.net","http://www.andersonmarkaesq.com"
"Pete","Dubaldi","Womack & Galich","2215 Prosperity Dr","Lyndhurst","Bergen","NJ","07071","201-825-2514","201-749-8866","pdubaldi@hotmail.com","http://www.womackgalich.com"
"Linn","Paa","Valerie & Company","1 S Pine St","Memphis","Shelby","TN",38112,"901-412-4381","901-573-9024","linn_paa@paa.com","http://www.valeriecompany.com"
"Paris","Wide","Gehring Pumps Inc","187 Market St","Atlanta","Fulton","GA",30342,"404-505-4445","404-607-8435","paris@hotmail.com
 ","http://www.gehringpumpsinc.com"
"Wynell","Dorshorst","Haehnel, Craig W Esq","94290 S Buchanan St","Pacifica","San Mateo","CA",94044,"650-473-1262","650-749-9879","wynell_dorshorst@dorshorst.org","http://www.haehnelcraigwesq.com"
"Quentin","Birkner","Spoor Behrins Campbell & Young","7061 N 2nd St","Burnsville","Dakota","MN",55337,"952-702-7993","952-314-5871","qbirkner@aol.com","http://www.spoorbehrinscampbellyoung.com"
"Regenia","Kannady","Ken Jeter Store Equipment Inc","10759 Main St","Scottsdale","Maricopa","AZ",85260,"480-726-1280","480-205-5121","regenia.kannady@cox.net","http://www.kenjeterstoreequipmentinc.com"
"Sheron","Louissant","Potter, Brenda J Cpa","97 E 3rd St #9","Long Island City","Queens","NY",11101,"718-976-8610","718-613-9994","sheron@aol.com","http://www.potterbrendajcpa.com"
"Izetta","Funnell","Baird Kurtz & Dobson","82 Winsor St #54","Atlanta","Dekalb","GA",30340,"770-844-3447","770-584-4119","izetta.funnell@hotmail.com","http://www.bairdkurtzdobson.com"
"Rod
 olfo","Butzen","Minor, Cynthia A Esq","41 Steel Ct","Northfield","Rice","MN",55057,"507-210-3510","507-590-5237","rodolfo@hotmail.com","http://www.minorcynthiaaesq.com"
"Zona","Colla","Solove, Robert A Esq","49440 Dearborn St","Norwalk","Fairfield","CT","06854","203-461-1949","203-938-2557","zona@hotmail.com","http://www.soloverobertaesq.com"
"Serina","Zagen","Mark Ii Imports Inc","7 S Beverly Dr","Fort Wayne","Allen","IN",46802,"260-273-3725","260-382-4869","szagen@aol.com","http://www.markiiimportsinc.com"
"Paz","Sahagun","White Sign Div Ctrl Equip Co","919 Wall Blvd","Meridian","Lauderdale","MS",39307,"601-927-8287","601-249-4511","paz_sahagun@cox.net","http://www.whitesigndivctrlequipco.com"
"Markus","Lukasik","M & M Store Fixtures Co Inc","89 20th St E #779","Sterling Heights","Macomb","MI",48310,"586-970-7380","586-247-1614","markus@yahoo.com","http://www.mmstorefixturescoinc.com"
"Jaclyn","Bachman","Judah Caster & Wheel Co","721 Interstate 45 S","Colorado Springs","El Paso","
 CO",80919,"719-853-3600","719-223-2074","jaclyn@aol.com","http://www.judahcasterwheelco.com"
"Cyril","Daufeldt","Galaxy International Inc","3 Lawton St","New York","New York","NY",10013,"212-745-8484","212-422-5427","cyril_daufeldt@daufeldt.com","http://www.galaxyinternationalinc.com"
"Gayla","Schnitzler","Sigma Corp Of America","38 Pleasant Hill Rd","Hayward","Alameda","CA",94545,"510-686-3407","510-441-4055","gschnitzler@gmail.com","http://www.sigmacorpofamerica.com"
"Erick","Nievas","Soward, Anne Esq","45 E Acacia Ct","Chicago","Cook","IL",60624,"773-704-9903","773-359-6109","erick_nievas@aol.com","http://www.sowardanneesq.com"
"Jennie","Drymon","Osborne, Michelle M Esq","63728 Poway Rd #1","Scranton","Lackawanna","PA",18509,"570-218-4831","570-868-8688","jennie@cox.net","http://www.osbornemichellemesq.com"
"Mitsue","Scipione","Students In Free Entrprs Natl","77 222 Dr","Oroville","Butte","CA",95965,"530-986-9272","530-399-3254","mscipione@scipione.com","http://www.studentsinfree
 entrprsnatl.com"
"Ciara","Ventura","Johnson, Robert M Esq","53 W Carey St","Port Jervis","Orange","NY",12771,"845-823-8877","845-694-7919","cventura@yahoo.com","http://www.johnsonrobertmesq.com"
"Galen","Cantres","Del Charro Apartments","617 Nw 36th Ave","Brook Park","Cuyahoga","OH",44142,"216-600-6111","216-871-6876","galen@yahoo.com","http://www.delcharroapartments.com"
"Truman","Feichtner","Legal Search Inc","539 Coldwater Canyon Ave","Bloomfield","Essex","NJ","07003","973-852-2736","973-473-5108","tfeichtner@yahoo.com","http://www.legalsearchinc.com"
"Gail","Kitty","Service Supply Co Inc","735 Crawford Dr","Anchorage","Anchorage","AK",99501,"907-435-9166","907-770-3542","gail@kitty.com","http://www.servicesupplycoinc.com"
"Dalene","Schoeneck","Sameshima, Douglas J Esq","910 Rahway Ave","Philadelphia","Philadelphia","PA",19102,"215-268-1275","215-380-8820","dalene@schoeneck.org","http://www.sameshimadouglasjesq.com"
"Gertude","Witten","Thompson, John Randolph Jr","7 Tarrytown Rd"
 ,"Cincinnati","Hamilton","OH",45217,"513-977-7043","513-863-9471","gertude.witten@gmail.com","http://www.thompsonjohnrandolphjr.com"
"Lizbeth","Kohl","E T Balancing Co Inc","35433 Blake St #588","Gardena","Los Angeles","CA",90248,"310-699-1222","310-955-5788","lizbeth@yahoo.com","http://www.etbalancingcoinc.com"
"Glenn","Berray","Griswold, John E Esq","29 Cherry St #7073","Des Moines","Polk","IA",50315,"515-370-7348","515-372-1738","gberray@gmail.com","http://www.griswoldjohneesq.com"
"Lashandra","Klang","Acqua Group","810 N La Brea Ave","King of Prussia","Montgomery","PA",19406,"610-809-1818","610-378-7332","lashandra@yahoo.com","http://www.acquagroup.com"
"Lenna","Newville","Brooks, Morris J Jr","987 Main St","Raleigh","Wake","NC",27601,"919-623-2524","919-254-5987","lnewville@newville.com","http://www.brooksmorrisjjr.com"
"Laurel","Pagliuca","Printing Images Corp","36 Enterprise St Se","Richland","Benton","WA",99352,"509-695-5199","509-595-6485","laurel@yahoo.com","http://www.pri
 ntingimagescorp.com"
"Mireya","Frerking","Roberts Supply Co Inc","8429 Miller Rd","Pelham","Westchester","NY",10803,"914-868-5965","914-883-3061","mireya.frerking@hotmail.com","http://www.robertssupplycoinc.com"
"Annelle","Tagala","Vico Products Mfg Co","5 W 7th St","Parkville","Baltimore","MD",21234,"410-757-1035","410-234-2267","annelle@yahoo.com","http://www.vicoproductsmfgco.com"
"Dean","Ketelsen","J M Custom Design Millwork","2 Flynn Rd","Hicksville","Nassau","NY",11801,"516-847-4418","516-732-6649","dean_ketelsen@gmail.com","http://www.jmcustomdesignmillwork.com"
"Levi","Munis","Farrell & Johnson Office Equip","2094 Ne 36th Ave","Worcester","Worcester","MA","01603","508-456-4907","508-658-7802","levi.munis@gmail.com","http://www.farrelljohnsonofficeequip.com"
"Sylvie","Ryser","Millers Market & Deli","649 Tulane Ave","Tulsa","Tulsa","OK",74105,"918-644-9555","918-565-1706","sylvie@aol.com","http://www.millersmarketdeli.com"
"Sharee","Maile","Holiday Inn Naperville","2094 Montou
 r Blvd","Muskegon","Muskegon","MI",49442,"231-467-9978","231-265-6940","sharee_maile@aol.com","http://www.holidayinnnaperville.com"
"Cordelia","Storment","Burrows, Jon H Esq","393 Hammond Dr","Lafayette","Lafayette","LA",70506,"337-566-6001","337-255-3427","cordelia_storment@aol.com","http://www.burrowsjonhesq.com"
"Mollie","Mcdoniel","Dock Seal Specialty","8590 Lake Lizzie Dr","Bowling Green","Wood","OH",43402,"419-975-3182","419-417-4674","mollie_mcdoniel@yahoo.com","http://www.docksealspecialty.com"
"Brett","Mccullan","Five Star Limousines Of Tx Inc","87895 Concord Rd","La Mesa","San Diego","CA",91942,"619-461-9984","619-727-3892","brett.mccullan@mccullan.com","http://www.fivestarlimousinesoftxinc.com"
"Teddy","Pedrozo","Barkan, Neal J Esq","46314 Route 130","Bridgeport","Fairfield","CT","06610","203-892-3863","203-918-3939","teddy_pedrozo@aol.com","http://www.barkannealjesq.com"
"Tasia","Andreason","Campbell, Robert A","4 Cowesett Ave","Kearny","Hudson","NJ","07032","201-920-900
 2","201-969-7063","tasia_andreason@yahoo.com","http://www.campbellroberta.com"
"Hubert","Walthall","Dee, Deanna","95 Main Ave #2","Barberton","Summit","OH",44203,"330-903-1345","330-566-8898","hubert@walthall.org","http://www.deedeanna.com"
"Arthur","Farrow","Young, Timothy L Esq","28 S 7th St #2824","Englewood","Bergen","NJ","07631","201-238-5688","201-772-4377","arthur.farrow@yahoo.com","http://www.youngtimothylesq.com"
"Vilma","Berlanga","Wells, D Fred Esq","79 S Howell Ave","Grand Rapids","Kent","MI",49546,"616-737-3085","616-568-4113","vberlanga@berlanga.com","http://www.wellsdfredesq.com"
"Billye","Miro","Gray, Francine H Esq","36 Lancaster Dr Se","Pearl","Rankin","MS",39208,"601-567-5386","601-637-5479","billye_miro@cox.net","http://www.grayfrancinehesq.com"
"Glenna","Slayton","Toledo Iv Care","2759 Livingston Ave","Memphis","Shelby","TN",38118,"901-640-9178","901-869-4314","glenna_slayton@cox.net","http://www.toledoivcare.com"
"Mitzie","Hudnall","Cangro Transmission Co","17 
 Jersey Ave","Englewood","Arapahoe","CO",80110,"303-402-1940","303-997-7760","mitzie_hudnall@yahoo.com","http://www.cangrotransmissionco.com"
"Bernardine","Rodefer","Sat Poly Inc","2 W Grand Ave","Memphis","Shelby","TN",38112,"901-901-4726","901-739-5892","bernardine_rodefer@yahoo.com","http://www.satpolyinc.com"
"Staci","Schmaltz","Midwest Contracting & Mfg Inc","18 Coronado Ave #563","Pasadena","Los Angeles","CA",91106,"626-866-2339","626-293-7678","staci_schmaltz@aol.com","http://www.midwestcontractingmfginc.com"
"Nichelle","Meteer","Print Doctor","72 Beechwood Ter","Chicago","Cook","IL",60657,"773-225-9985","773-857-2231","nichelle_meteer@meteer.com","http://www.printdoctor.com"
"Janine","Rhoden","Nordic Group Inc","92 Broadway","Astoria","Queens","NY",11103,"718-228-5894","718-728-5051","jrhoden@yahoo.com","http://www.nordicgroupinc.com"
"Ettie","Hoopengardner","Jackson Millwork Co","39 Franklin Ave","Richland","Benton","WA",99352,"509-755-5393","509-847-3352","ettie.hoopengardn
 er@hotmail.com","http://www.jacksonmillworkco.com"
"Eden","Jayson","Harris Corporation","4 Iwaena St","Baltimore","Baltimore City","MD",21202,"410-890-7866","410-429-4888","eden_jayson@yahoo.com","http://www.harriscorporation.com"
"Lynelle","Auber","United Cerebral Palsy Of Ne Pa","32820 Corkwood Rd","Newark","Essex","NJ","07104","973-860-8610","973-605-6492","lynelle_auber@gmail.com","http://www.unitedcerebralpalsyofnepa.com"
"Merissa","Tomblin","One Day Surgery Center Inc","34 Raritan Center Pky","Bellflower","Los Angeles","CA",90706,"562-579-6900","562-719-7922","merissa.tomblin@gmail.com","http://www.onedaysurgerycenterinc.com"
"Golda","Kaniecki","Calaveras Prospect","6201 S Nevada Ave","Toms River","Ocean","NJ","08755","732-628-9909","732-617-5310","golda_kaniecki@yahoo.com","http://www.calaverasprospect.com"
"Catarina","Gleich","Terk, Robert E Esq","78 Maryland Dr #146","Denville","Morris","NJ","07834","973-210-3994","973-491-8723","catarina_gleich@hotmail.com","http://www.ter
 kroberteesq.com"
"Virgie","Kiel","Cullen, Terrence P Esq","76598 Rd  I 95 #1","Denver","Denver","CO",80216,"303-776-7548","303-845-5408","vkiel@hotmail.com","http://www.cullenterrencepesq.com"
"Jolene","Ostolaza","Central Die Casting Mfg Co Inc","1610 14th St Nw","Newport News","Newport News City","VA",23608,"757-682-7116","757-940-1741","jolene@yahoo.com","http://www.centraldiecastingmfgcoinc.com"
"Keneth","Borgman","Centerline Engineering","86350 Roszel Rd","Phoenix","Maricopa","AZ",85012,"602-919-4211","602-442-3092","keneth@yahoo.com","http://www.centerlineengineering.com"
"Rikki","Nayar","Targan & Kievit Pa","1644 Clove Rd","Miami","Miami-Dade","FL",33155,"305-968-9487","305-978-2069","rikki@nayar.com","http://www.targankievitpa.com"
"Elke","Sengbusch","Riley Riper Hollin & Colagreco","9 W Central Ave","Phoenix","Maricopa","AZ",85013,"602-896-2993","602-575-3457","elke_sengbusch@yahoo.com","http://www.rileyriperhollincolagreco.com"
"Hoa","Sarao","Kaplan, Joel S Esq","27846 Lafa
 yette Ave","Oak Hill","Volusia","FL",32759,"386-526-7800","386-599-7296","hoa@sarao.org","http://www.kaplanjoelsesq.com"
"Trinidad","Mcrae","Water Office","10276 Brooks St","San Francisco","San Francisco","CA",94105,"415-331-9634","415-419-1597","trinidad_mcrae@yahoo.com","http://www.wateroffice.com"
"Mari","Lueckenbach","Westbrooks, Nelson E Jr","1 Century Park E","San Diego","San Diego","CA",92110,"858-793-9684","858-228-5683","mari_lueckenbach@yahoo.com","http://www.westbrooksnelsonejr.com"
"Selma","Husser","Armon Communications","9 State Highway 57 #22","Jersey City","Hudson","NJ","07306","201-991-8369","201-772-7699","selma.husser@cox.net","http://www.armoncommunications.com"
"Antione","Onofrio","Jacobs & Gerber Inc","4 S Washington Ave","San Bernardino","San Bernardino","CA",92410,"909-430-7765","909-665-3223","aonofrio@onofrio.com","http://www.jacobsgerberinc.com"
"Luisa","Jurney","Forest Fire Laboratory","25 Se 176th Pl","Cambridge","Middlesex","MA","02138","617-365-2134","6
 17-544-2541","ljurney@hotmail.com","http://www.forestfirelaboratory.com"
"Clorinda","Heimann","Haughey, Charles Jr","105 Richmond Valley Rd","Escondido","San Diego","CA",92025,"760-291-5497","760-261-4786","clorinda.heimann@hotmail.com","http://www.haugheycharlesjr.com"
"Dick","Wenzinger","Wheaton Plastic Products","22 Spruce St #595","Gardena","Los Angeles","CA",90248,"310-510-9713","310-936-2258","dick@yahoo.com","http://www.wheatonplasticproducts.com"
"Ahmed","Angalich","Reese Plastics","2 W Beverly Blvd","Harrisburg","Dauphin","PA",17110,"717-528-8996","717-632-5831","ahmed.angalich@angalich.com","http://www.reeseplastics.com"
"Iluminada","Ohms","Nazette Marner Good Wendt","72 Southern Blvd","Mesa","Maricopa","AZ",85204,"480-293-2882","480-866-6544","iluminada.ohms@yahoo.com","http://www.nazettemarnergoodwendt.com"
"Joanna","Leinenbach","Levinson Axelrod Wheaton","1 Washington St","Lake Worth","Palm Beach","FL",33461,"561-470-4574","561-951-9734","joanna_leinenbach@hotmail.com",
 "http://www.levinsonaxelrodwheaton.com"
"Caprice","Suell","Egnor, W Dan Esq","90177 N 55th Ave","Nashville","Davidson","TN",37211,"615-246-1824","615-726-4537","caprice@aol.com","http://www.egnorwdanesq.com"
"Stephane","Myricks","Portland Central Thriftlodge","9 Tower Ave","Burlington","Boone","KY",41005,"859-717-7638","859-308-4286","stephane_myricks@cox.net","http://www.portlandcentralthriftlodge.com"
"Quentin","Swayze","Ulbrich Trucking","278 Bayview Ave","Milan","Monroe","MI",48160,"734-561-6170","734-851-8571","quentin_swayze@yahoo.com","http://www.ulbrichtrucking.com"
"Annmarie","Castros","Tipiak Inc","80312 W 32nd St","Conroe","Montgomery","TX",77301,"936-751-7961","936-937-2334","annmarie_castros@gmail.com","http://www.tipiakinc.com"
"Shonda","Greenbush","Saint George Well Drilling","82 Us Highway 46","Clifton","Passaic","NJ","07011","973-482-2430","973-644-2974","shonda_greenbush@cox.net","http://www.saintgeorgewelldrilling.com"
"Cecil","Lapage","Hawkes, Douglas D","4 Stova
 ll St #72","Union City","Hudson","NJ","07087","201-693-3967","201-856-2720","clapage@lapage.com","http://www.hawkesdouglasd.com"
"Jeanice","Claucherty","Accurel Systems Intrntl Corp","19 Amboy Ave","Miami","Miami-Dade","FL",33142,"305-988-4162","305-306-7834","jeanice.claucherty@yahoo.com","http://www.accurelsystemsintrntlcorp.com"
"Josphine","Villanueva","Santa Cruz Community Internet","63 Smith Ln #8343","Moss","Clay","TN",38575,"931-553-9774","931-486-6946","josphine_villanueva@villanueva.com","http://www.santacruzcommunityinternet.com"
"Daniel","Perruzza","Gersh & Danielson","11360 S Halsted St","Santa Ana","Orange","CA",92705,"714-771-3880","714-531-1391","dperruzza@perruzza.com","http://www.gershdanielson.com"
"Cassi","Wildfong","Cobb, James O Esq","26849 Jefferson Hwy","Rolling Meadows","Cook","IL",60008,"847-633-3216","847-755-9041","cassi.wildfong@aol.com","http://www.cobbjamesoesq.com"
"Britt","Galam","Wheatley Trucking Company","2500 Pringle Rd Se #508","Hatfield","Montgo
 mery","PA",19440,"215-888-3304","215-351-8523","britt@galam.org","http://www.wheatleytruckingcompany.com"
"Adell","Lipkin","Systems Graph Inc Ab Dick Dlr","65 Mountain View Dr","Whippany","Morris","NJ","07981","973-654-1561","973-662-8988","adell.lipkin@lipkin.com","http://www.systemsgraphincabdickdlr.com"
"Jacqueline","Rowling","John Hancock Mutl Life Ins Co","1 N San Saba","Erie","Erie","PA",16501,"814-865-8113","814-481-1700","jacqueline.rowling@yahoo.com","http://www.johnhancockmutllifeinsco.com"
"Lonny","Weglarz","History Division Of State","51120 State Route 18","Salt Lake City","Salt Lake","UT",84115,"801-293-9853","801-892-8781","lonny_weglarz@gmail.com","http://www.historydivisionofstate.com"
"Lonna","Diestel","Dimmock, Thomas J Esq","1482 College Ave","Fayetteville","Cumberland","NC",28301,"910-922-3672","910-200-7912","lonna_diestel@gmail.com","http://www.dimmockthomasjesq.com"
"Cristal","Samara","Intermed Inc","4119 Metropolitan Dr","Los Angeles","Los Angeles","CA",90021
 ,"213-975-8026","213-696-8004","cristal@cox.net","http://www.intermedinc.com"
"Kenneth","Grenet","Bank Of New York","2167 Sierra Rd","East Lansing","Ingham","MI",48823,"517-499-2322","517-867-8077","kenneth.grenet@grenet.org","http://www.bankofnewyork.com"
"Elli","Mclaird","Sportmaster Intrnatl","6 Sunrise Ave","Utica","Oneida","NY",13501,"315-818-2638","315-474-5570","emclaird@mclaird.com","http://www.sportmasterintrnatl.com"
"Alline","Jeanty","W W John Holden Inc","55713 Lake City Hwy","South Bend","St Joseph","IN",46601,"574-656-2800","574-405-1983","ajeanty@gmail.com","http://www.wwjohnholdeninc.com"
"Sharika","Eanes","Maccani & Delp","75698 N Fiesta Blvd","Orlando","Orange","FL",32806,"407-312-1691","407-472-1332","sharika.eanes@aol.com","http://www.maccanidelp.com"
"Nu","Mcnease","Amazonia Film Project","88 Sw 28th Ter","Harrison","Hudson","NJ","07029","973-751-9003","973-903-4175","nu@gmail.com","http://www.amazoniafilmproject.com"
"Daniela","Comnick","Water & Sewer Departmen
 t","7 Flowers Rd #403","Trenton","Mercer","NJ","08611","609-200-8577","609-398-2805","dcomnick@cox.net","http://www.watersewerdepartment.com"
"Cecilia","Colaizzo","Switchcraft Inc","4 Nw 12th St #3849","Madison","Dane","WI",53717,"608-382-4541","608-302-3387","cecilia_colaizzo@colaizzo.com","http://www.switchcraftinc.com"
"Leslie","Threets","C W D C Metal Fabricators","2 A Kelley Dr","Katonah","Westchester","NY",10536,"914-861-9748","914-396-2615","leslie@cox.net","http://www.cwdcmetalfabricators.com"
"Nan","Koppinger","Shimotani, Grace T","88827 Frankford Ave","Greensboro","Guilford","NC",27401,"336-370-5333","336-564-1492","nan@koppinger.com","http://www.shimotanigracet.com"
"Izetta","Dewar","Lisatoni, Jean Esq","2 W Scyene Rd #3","Baltimore","Baltimore City","MD",21217,"410-473-1708","410-522-7621","idewar@dewar.com","http://www.lisatonijeanesq.com"
"Tegan","Arceo","Ceramic Tile Sales Inc","62260 Park Stre","Monroe Township","Middlesex","NJ","08831","732-730-2692","732-705-6719",
 "tegan.arceo@arceo.org","http://www.ceramictilesalesinc.com"
"Ruthann","Keener","Maiden Craft Inc","3424 29th St Se","Kerrville","Kerr","TX",78028,"830-258-2769","830-919-5991","ruthann@hotmail.com","http://www.maidencraftinc.com"
"Joni","Breland","Carriage House Cllsn Rpr Inc","35 E Main St #43","Elk Grove Village","Cook","IL",60007,"847-519-5906","847-740-5304","joni_breland@cox.net","http://www.carriagehousecllsnrprinc.com"
"Vi","Rentfro","Video Workshop","7163 W Clark Rd","Freehold","Monmouth","NJ","07728","732-605-4781","732-724-7251","vrentfro@cox.net","http://www.videoworkshop.com"
"Colette","Kardas","Fresno Tile Center Inc","21575 S Apple Creek Rd","Omaha","Douglas","NE",68124,"402-896-5943","402-707-1602","colette.kardas@yahoo.com","http://www.fresnotilecenterinc.com"
"Malcolm","Tromblay","Versatile Sash & Woodwork","747 Leonis Blvd","Annandale","Fairfax","VA",22003,"703-221-5602","703-874-4248","malcolm_tromblay@cox.net","http://www.versatilesashwoodwork.com"
"Ryan","Harno
 s","Warner Electric Brk & Cltch Co","13 Gunnison St","Plano","Collin","TX",75075,"972-558-1665","972-961-4968","ryan@cox.net","http://www.warnerelectricbrkcltchco.com"
"Jess","Chaffins","New York Public Library","18 3rd Ave","New York","New York","NY",10016,"212-510-4633","212-428-9538","jess.chaffins@chaffins.org","http://www.newyorkpubliclibrary.com"
"Sharen","Bourbon","Mccaleb, John A Esq","62 W Austin St","Syosset","Nassau","NY",11791,"516-816-1541","516-749-3188","sbourbon@yahoo.com","http://www.mccalebjohnaesq.com"
"Nickolas","Juvera","United Oil Co Inc","177 S Rider Trl #52","Crystal River","Citrus","FL",34429,"352-598-8301","352-947-6152","nickolas_juvera@cox.net","http://www.unitedoilcoinc.com"
"Gary","Nunlee","Irving Foot Center","2 W Mount Royal Ave","Fortville","Hancock","IN",46040,"317-542-6023","317-887-8486","gary_nunlee@nunlee.org","http://www.irvingfootcenter.com"
"Diane","Devreese","Acme Supply Co","1953 Telegraph Rd","Saint Joseph","Buchanan","MO",64504,"816-557-9
 673","816-329-5565","diane@cox.net","http://www.acmesupplyco.com"
"Roslyn","Chavous","Mcrae, James L","63517 Dupont St","Jackson","Hinds","MS",39211,"601-234-9632","601-973-5754","roslyn.chavous@chavous.org","http://www.mcraejamesl.com"
"Glory","Schieler","Mcgraths Seafood","5 E Truman Rd","Abilene","Taylor","TX",79602,"325-869-2649","325-740-3778","glory@yahoo.com","http://www.mcgrathsseafood.com"
"Rasheeda","Sayaphon","Kummerer, J Michael Esq","251 Park Ave #979","Saratoga","Santa Clara","CA",95070,"408-805-4309","408-997-7490","rasheeda@aol.com","http://www.kummererjmichaelesq.com"
"Alpha","Palaia","Stoffer, James M Jr","43496 Commercial Dr #29","Cherry Hill","Camden","NJ","08003","856-312-2629","856-513-7024","alpha@yahoo.com","http://www.stofferjamesmjr.com"
"Refugia","Jacobos","North Central Fl Sfty Cncl","2184 Worth St","Hayward","Alameda","CA",94545,"510-974-8671","510-509-3496","refugia.jacobos@jacobos.com","http://www.northcentralflsftycncl.com"
"Shawnda","Yori","Fiorucci 
 Foods Usa Inc","50126 N Plankinton Ave","Lon

<TRUNCATED>

[08/19] nifi git commit: NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Servic

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..833a5d6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryFlowFile.java
@@ -0,0 +1,541 @@
+/*
+ * 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")
+        .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")
+        .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")
+        .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")
+        .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.JAVA.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) {
+            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;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..1a62d14
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileEnumerator.java
@@ -0,0 +1,150 @@
+/*
+ * 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.
+        if (row.length <= fields.length) {
+            return row;
+        }
+
+        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(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/a88d3bfa/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
new file mode 100644
index 0000000..c5179c9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/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.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/a88d3bfa/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
new file mode 100644
index 0000000..a23dcfa
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTable.java
@@ -0,0 +1,203 @@
+/*
+ * 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(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 OBJECT:
+                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/a88d3bfa/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
new file mode 100644
index 0000000..ad3a1c3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/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.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/a88d3bfa/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 9de5ab6..2f2b0cb 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
@@ -75,6 +75,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.ReplaceText
 org.apache.nifi.processors.standard.RouteText
 org.apache.nifi.processors.standard.ReplaceTextWithMapping

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..1cc7923
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryFlowFile/additionalDetails.html
@@ -0,0 +1,47 @@
+<!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.
+    	</p>
+	</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFilterCSVColumns.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFilterCSVColumns.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFilterCSVColumns.java
deleted file mode 100644
index 421da98..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFilterCSVColumns.java
+++ /dev/null
@@ -1,117 +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 static org.junit.Assert.assertEquals;
-
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.List;
-
-import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TestFilterCSVColumns {
-
-    private static final Logger LOGGER;
-
-    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.nifi.processors.standard.FilterCSVColumns", "debug");
-        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestFilterCSVColumns", "debug");
-        LOGGER = LoggerFactory.getLogger(TestFilterCSVColumns.class);
-    }
-
-    @Test
-    public void testTransformSimple() throws InitializationException, IOException, SQLException {
-        String sql = "select first_name, last_name, company_name, address, city from CSV.A where city='New York'";
-
-        Path inpath = Paths.get("src/test/resources/TestFilterCSVColumns/US500.csv");
-        InputStream in = new FileInputStream(inpath.toFile());
-
-        ResultSet resultSet = FilterCSVColumns.transform(in, sql);
-
-        int nrofColumns = resultSet.getMetaData().getColumnCount();
-
-        for (int i = 1; i <= nrofColumns; i++) {
-            System.out.print(resultSet.getMetaData().getColumnLabel(i) + "      ");
-        }
-        System.out.println();
-
-        while (resultSet.next()) {
-            for (int i = 1; i <= nrofColumns; i++) {
-                System.out.print(resultSet.getString(i)+ "  ");
-            }
-            System.out.println();
-        }
-    }
-
-    @Test
-    public void testTransformCalc() throws InitializationException, IOException, SQLException {
-        String sql = "select ID, AMOUNT1+AMOUNT2+AMOUNT3 as TOTAL from CSV.A where ID=100";
-
-        Path inpath = Paths.get("src/test/resources/TestFilterCSVColumns/Numeric.csv");
-        InputStream in = new FileInputStream(inpath.toFile());
-
-        ResultSet resultSet = FilterCSVColumns.transform(in, sql);
-
-        int nrofColumns = resultSet.getMetaData().getColumnCount();
-
-        for (int i = 1; i <= nrofColumns; i++) {
-            System.out.print(resultSet.getMetaData().getColumnLabel(i) + "      ");
-        }
-        System.out.println();
-
-        while (resultSet.next()) {
-            for (int i = 1; i <= nrofColumns; i++) {
-                System.out.print(resultSet.getString(i)+ "  ");
-            }
-            double total = resultSet.getDouble("TOTAL");
-            System.out.println();
-            assertEquals(90.75, total, 0.0001);
-        }
-    }
-
-    @Test
-    public void testSimpleTypeless() throws InitializationException, IOException {
-        final TestRunner runner = TestRunners.newTestRunner(FilterCSVColumns.class);
-        String sql = "select first_name, last_name, company_name, address, city from CSV.A where city='New York'";
-        runner.setProperty(FilterCSVColumns.SQL_SELECT, sql);
-
-        runner.enqueue(Paths.get("src/test/resources/TestFilterCSVColumns/US500_typeless.csv"));
-        runner.run();
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ExecuteProcess.REL_SUCCESS);
-        for (final MockFlowFile flowFile : flowFiles) {
-            System.out.println(flowFile);
-            System.out.println(new String(flowFile.toByteArray()));
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/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
new file mode 100644
index 0000000..41469ba
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryFlowFile.java
@@ -0,0 +1,379 @@
+/*
+ * 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.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.RowRecordReaderFactory;
+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;
+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.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);
+
+                    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;
+                }
+            };
+        }
+
+    }
+
+    private static class MockRecordWriter extends AbstractControllerService implements RecordSetWriterFactory {
+        private final String header;
+
+        public MockRecordWriter(final String header) {
+            this.header = header;
+        }
+
+        @Override
+        public RecordSetWriter createWriter(final ComponentLog logger) {
+            return new RecordSetWriter() {
+                @Override
+                public WriteResult write(final RecordSet rs, final OutputStream out) throws IOException {
+                    out.write(header.getBytes());
+                    out.write("\n".getBytes());
+
+                    int recordCount = 0;
+                    final int numCols = rs.getSchema().getFieldCount();
+                    Record record = null;
+                    while ((record = rs.next()) != null) {
+                        recordCount++;
+                        int i = 0;
+                        for (final String fieldName : record.getSchema().getFieldNames()) {
+                            final String val = record.getAsString(fieldName);
+                            out.write("\"".getBytes());
+                            out.write(val.getBytes());
+                            out.write("\"".getBytes());
+
+                            if (i++ < numCols - 1) {
+                                out.write(",".getBytes());
+                            }
+                        }
+                        out.write("\n".getBytes());
+                    }
+
+                    return WriteResult.of(recordCount, Collections.emptyMap());
+                }
+
+                @Override
+                public String getMimeType() {
+                    return "text/plain";
+                }
+
+                @Override
+                public WriteResult write(Record record, OutputStream out) throws IOException {
+                    return null;
+                }
+            };
+        }
+    }
+
+    private static class MockRecordParser extends AbstractControllerService implements RowRecordReaderFactory {
+        private final List<Object[]> records = new ArrayList<>();
+        private final List<RecordField> fields = new ArrayList<>();
+        private final int failAfterN;
+
+        public MockRecordParser() {
+            this(-1);
+        }
+
+        public MockRecordParser(final int failAfterN) {
+            this.failAfterN = failAfterN;
+        }
+
+
+        public void addSchemaField(final String fieldName, final RecordFieldType type) {
+            fields.add(new RecordField(fieldName, type.getDataType()));
+        }
+
+        public void addRecord(Object... values) {
+            records.add(values);
+        }
+
+        @Override
+        public RecordReader createRecordReader(InputStream in, ComponentLog logger) throws IOException {
+            final Iterator<Object[]> itr = records.iterator();
+
+            return new RecordReader() {
+                private int recordCount = 0;
+
+                @Override
+                public void close() throws IOException {
+                }
+
+                @Override
+                public Record nextRecord() throws IOException, MalformedRecordException {
+                    if (failAfterN >= recordCount) {
+                        throw new MalformedRecordException("Intentional Unit Test Exception because " + recordCount + " records have been read");
+                    }
+                    recordCount++;
+
+                    if (!itr.hasNext()) {
+                        return null;
+                    }
+
+                    final Object[] values = itr.next();
+                    final Map<String, Object> valueMap = new HashMap<>();
+                    int i = 0;
+                    for (final RecordField field : fields) {
+                        final String fieldName = field.getFieldName();
+                        valueMap.put(fieldName, values[i++]);
+                    }
+
+                    return new MapRecord(new SimpleRecordSchema(fields), valueMap);
+                }
+
+                @Override
+                public RecordSchema getSchema() {
+                    return new SimpleRecordSchema(fields);
+                }
+            };
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/Numeric.csv
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/Numeric.csv b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/Numeric.csv
deleted file mode 100644
index 2d56bb7..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/Numeric.csv
+++ /dev/null
@@ -1,5 +0,0 @@
-ID:int,AMOUNT1: float,AMOUNT2:float,AMOUNT3:float
-008, 10.05, 15.45, 89.99
-100, 20.25, 25.25, 45.25
-105, 20.05, 25.05, 45.05
-200, 34.05, 25.05, 75.05
\ No newline at end of file


[06/19] nifi git commit: NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Servic

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/a88d3bfa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500_typeless.csv
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500_typeless.csv b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500_typeless.csv
deleted file mode 100644
index 61ce4bd..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestFilterCSVColumns/US500_typeless.csv
+++ /dev/null
@@ -1 +0,0 @@
-FIRST_NAME:string,LAST_NAME,COMPANY_NAME,ADDRESS,CITY,COUNTY,STATE,zip,phone1,phone2,email,web
"James","Butt","Benton, John B Jr","6649 N Blue Gum St","New Orleans","Orleans","LA",70116,"504-621-8927","504-845-1427","jbutt@gmail.com","http://www.bentonjohnbjr.com"
"Josephine","Darakjy","Chanay, Jeffrey A Esq","4 B Blue Ridge Blvd","Brighton","Livingston","MI",48116,"810-292-9388","810-374-9840","josephine_darakjy@darakjy.org","http://www.chanayjeffreyaesq.com"
"Art","Venere","Chemel, James L Cpa","8 W Cerritos Ave #54","Bridgeport","Gloucester","NJ","08014","856-636-8749","856-264-4130","art@venere.org","http://www.chemeljameslcpa.com"
"Lenna","Paprocki","Feltz Printing Service","639 Main St","Anchorage","Anchorage","AK",99501,"907-385-4412","907-921-2010","lpaprocki@hotmail.com","http://www.feltzprintingservice.com"
"Donette","Foller","Printing Dimensions","34 Center St","Hamilton","Butler","OH",45011,"513-570-1893","513-549-4561","donette.foller@cox.net","http://www.printingdimens
 ions.com"
"Simona","Morasca","Chapman, Ross E Esq","3 Mcauley Dr","Ashland","Ashland","OH",44805,"419-503-2484","419-800-6759","simona@morasca.com","http://www.chapmanrosseesq.com"
"Mitsue","Tollner","Morlong Associates","7 Eads St","Chicago","Cook","IL",60632,"773-573-6914","773-924-8565","mitsue_tollner@yahoo.com","http://www.morlongassociates.com"
"Leota","Dilliard","Commercial Press","7 W Jackson Blvd","San Jose","Santa Clara","CA",95111,"408-752-3500","408-813-1105","leota@hotmail.com","http://www.commercialpress.com"
"Sage","Wieser","Truhlar And Truhlar Attys","5 Boston Ave #88","Sioux Falls","Minnehaha","SD",57105,"605-414-2147","605-794-4895","sage_wieser@cox.net","http://www.truhlarandtruhlarattys.com"
"Kris","Marrier","King, Christopher A Esq","228 Runamuck Pl #2808","Baltimore","Baltimore City","MD",21224,"410-655-8723","410-804-4694","kris@gmail.com","http://www.kingchristopheraesq.com"
"Minna","Amigon","Dorl, James J Esq","2371 Jerrold Ave","Kulpsville","Montgomery","PA
 ",19443,"215-874-1229","215-422-8694","minna_amigon@yahoo.com","http://www.dorljamesjesq.com"
"Abel","Maclead","Rangoni Of Florence","37275 St  Rt 17m M","Middle Island","Suffolk","NY",11953,"631-335-3414","631-677-3675","amaclead@gmail.com","http://www.rangoniofflorence.com"
"Kiley","Caldarera","Feiner Bros","25 E 75th St #69","Los Angeles","Los Angeles","CA",90034,"310-498-5651","310-254-3084","kiley.caldarera@aol.com","http://www.feinerbros.com"
"Graciela","Ruta","Buckley Miller & Wright","98 Connecticut Ave Nw","Chagrin Falls","Geauga","OH",44023,"440-780-8425","440-579-7763","gruta@cox.net","http://www.buckleymillerwright.com"
"Cammy","Albares","Rousseaux, Michael Esq","56 E Morehead St","Laredo","Webb","TX",78045,"956-537-6195","956-841-7216","calbares@gmail.com","http://www.rousseauxmichaelesq.com"
"Mattie","Poquette","Century Communications","73 State Road 434 E","Phoenix","Maricopa","AZ",85013,"602-277-4385","602-953-6360","mattie@aol.com","http://www.centurycommunications.
 com"
"Meaghan","Garufi","Bolton, Wilbur Esq","69734 E Carrillo St","Mc Minnville","Warren","TN",37110,"931-313-9635","931-235-7959","meaghan@hotmail.com","http://www.boltonwilburesq.com"
"Gladys","Rim","T M Byxbee Company Pc","322 New Horizon Blvd","Milwaukee","Milwaukee","WI",53207,"414-661-9598","414-377-2880","gladys.rim@rim.org","http://www.tmbyxbeecompanypc.com"
"Yuki","Whobrey","Farmers Insurance Group","1 State Route 27","Taylor","Wayne","MI",48180,"313-288-7937","313-341-4470","yuki_whobrey@aol.com","http://www.farmersinsurancegroup.com"
"Fletcher","Flosi","Post Box Services Plus","394 Manchester Blvd","Rockford","Winnebago","IL",61109,"815-828-2147","815-426-5657","fletcher.flosi@yahoo.com","http://www.postboxservicesplus.com"
"Bette","Nicka","Sport En Art","6 S 33rd St","Aston","Delaware","PA",19014,"610-545-3615","610-492-4643","bette_nicka@cox.net","http://www.sportenart.com"
"Veronika","Inouye","C 4 Network Inc","6 Greenleaf Ave","San Jose","Santa Clara","CA",95111,"408
 -540-1785","408-813-4592","vinouye@aol.com","http://www.cnetworkinc.com"
"Willard","Kolmetz","Ingalls, Donald R Esq","618 W Yakima Ave","Irving","Dallas","TX",75062,"972-303-9197","972-896-4882","willard@hotmail.com","http://www.ingallsdonaldresq.com"
"Maryann","Royster","Franklin, Peter L Esq","74 S Westgate St","Albany","Albany","NY",12204,"518-966-7987","518-448-8982","mroyster@royster.com","http://www.franklinpeterlesq.com"
"Alisha","Slusarski","Wtlz Power 107 Fm","3273 State St","Middlesex","Middlesex","NJ","08846","732-658-3154","732-635-3453","alisha@slusarski.com","http://www.wtlzpowerfm.com"
"Allene","Iturbide","Ledecky, David Esq","1 Central Ave","Stevens Point","Portage","WI",54481,"715-662-6764","715-530-9863","allene_iturbide@cox.net","http://www.ledeckydavidesq.com"
"Chanel","Caudy","Professional Image Inc","86 Nw 66th St #8673","Shawnee","Johnson","KS",66218,"913-388-2079","913-899-1103","chanel.caudy@caudy.org","http://www.professionalimageinc.com"
"Ezekiel","Chui","
 Sider, Donald C Esq","2 Cedar Ave #84","Easton","Talbot","MD",21601,"410-669-1642","410-235-8738","ezekiel@chui.com","http://www.siderdonaldcesq.com"
"Willow","Kusko","U Pull It","90991 Thorburn Ave","New York","New York","NY",10011,"212-582-4976","212-934-5167","wkusko@yahoo.com","http://www.upullit.com"
"Bernardo","Figeroa","Clark, Richard Cpa","386 9th Ave N","Conroe","Montgomery","TX",77301,"936-336-3951","936-597-3614","bfigeroa@aol.com","http://www.clarkrichardcpa.com"
"Ammie","Corrio","Moskowitz, Barry S","74874 Atlantic Ave","Columbus","Franklin","OH",43215,"614-801-9788","614-648-3265","ammie@corrio.com","http://www.moskowitzbarrys.com"
"Francine","Vocelka","Cascade Realty Advisors Inc","366 South Dr","Las Cruces","Dona Ana","NM",88011,"505-977-3911","505-335-5293","francine_vocelka@vocelka.com","http://www.cascaderealtyadvisorsinc.com"
"Ernie","Stenseth","Knwz Newsradio","45 E Liberty St","Ridgefield Park","Bergen","NJ","07660","201-709-6245","201-387-9093","ernie_stenseth
 @aol.com","http://www.knwznewsradio.com"
"Albina","Glick","Giampetro, Anthony D","4 Ralph Ct","Dunellen","Middlesex","NJ","08812","732-924-7882","732-782-6701","albina@glick.com","http://www.giampetroanthonyd.com"
"Alishia","Sergi","Milford Enterprises Inc","2742 Distribution Way","New York","New York","NY",10025,"212-860-1579","212-753-2740","asergi@gmail.com","http://www.milfordenterprisesinc.com"
"Solange","Shinko","Mosocco, Ronald A","426 Wolf St","Metairie","Jefferson","LA",70002,"504-979-9175","504-265-8174","solange@shinko.com","http://www.mosoccoronalda.com"
"Jose","Stockham","Tri State Refueler Co","128 Bransten Rd","New York","New York","NY",10011,"212-675-8570","212-569-4233","jose@yahoo.com","http://www.tristaterefuelerco.com"
"Rozella","Ostrosky","Parkway Company","17 Morena Blvd","Camarillo","Ventura","CA",93012,"805-832-6163","805-609-1531","rozella.ostrosky@ostrosky.com","http://www.parkwaycompany.com"
"Valentine","Gillian","Fbs Business Finance","775 W 17th St","San
  Antonio","Bexar","TX",78204,"210-812-9597","210-300-6244","valentine_gillian@gmail.com","http://www.fbsbusinessfinance.com"
"Kati","Rulapaugh","Eder Assocs Consltng Engrs Pc","6980 Dorsett Rd","Abilene","Dickinson","KS",67410,"785-463-7829","785-219-7724","kati.rulapaugh@hotmail.com","http://www.ederassocsconsltngengrspc.com"
"Youlanda","Schemmer","Tri M Tool Inc","2881 Lewis Rd","Prineville","Crook","OR",97754,"541-548-8197","541-993-2611","youlanda@aol.com","http://www.trimtoolinc.com"
"Dyan","Oldroyd","International Eyelets Inc","7219 Woodfield Rd","Overland Park","Johnson","KS",66204,"913-413-4604","913-645-8918","doldroyd@aol.com","http://www.internationaleyeletsinc.com"
"Roxane","Campain","Rapid Trading Intl","1048 Main St","Fairbanks","Fairbanks North Star","AK",99708,"907-231-4722","907-335-6568","roxane@hotmail.com","http://www.rapidtradingintl.com"
"Lavera","Perin","Abc Enterprises Inc","678 3rd Ave","Miami","Miami-Dade","FL",33196,"305-606-7291","305-995-2078","lperin@pe
 rin.org","http://www.abcenterprisesinc.com"
"Erick","Ferencz","Cindy Turner Associates","20 S Babcock St","Fairbanks","Fairbanks North Star","AK",99712,"907-741-1044","907-227-6777","erick.ferencz@aol.com","http://www.cindyturnerassociates.com"
"Fatima","Saylors","Stanton, James D Esq","2 Lighthouse Ave","Hopkins","Hennepin","MN",55343,"952-768-2416","952-479-2375","fsaylors@saylors.org","http://www.stantonjamesdesq.com"
"Jina","Briddick","Grace Pastries Inc","38938 Park Blvd","Boston","Suffolk","MA","02128","617-399-5124","617-997-5771","jina_briddick@briddick.com","http://www.gracepastriesinc.com"
"Kanisha","Waycott","Schroer, Gene E Esq","5 Tomahawk Dr","Los Angeles","Los Angeles","CA",90006,"323-453-2780","323-315-7314","kanisha_waycott@yahoo.com","http://www.schroergeneeesq.com"
"Emerson","Bowley","Knights Inn","762 S Main St","Madison","Dane","WI",53711,"608-336-7444","608-658-7940","emerson.bowley@bowley.org","http://www.knightsinn.com"
"Blair","Malet","Bollinger Mach Shp & S
 hipyard","209 Decker Dr","Philadelphia","Philadelphia","PA",19132,"215-907-9111","215-794-4519","bmalet@yahoo.com","http://www.bollingermachshpshipyard.com"
"Brock","Bolognia","Orinda News","4486 W O St #1","New York","New York","NY",10003,"212-402-9216","212-617-5063","bbolognia@yahoo.com","http://www.orindanews.com"
"Lorrie","Nestle","Ballard Spahr Andrews","39 S 7th St","Tullahoma","Coffee","TN",37388,"931-875-6644","931-303-6041","lnestle@hotmail.com","http://www.ballardspahrandrews.com"
"Sabra","Uyetake","Lowy Limousine Service","98839 Hawthorne Blvd #6101","Columbia","Richland","SC",29201,"803-925-5213","803-681-3678","sabra@uyetake.org","http://www.lowylimousineservice.com"
"Marjory","Mastella","Vicon Corporation","71 San Mateo Ave","Wayne","Delaware","PA",19087,"610-814-5533","610-379-7125","mmastella@mastella.com","http://www.viconcorporation.com"
"Karl","Klonowski","Rossi, Michael M","76 Brooks St #9","Flemington","Hunterdon","NJ","08822","908-877-6135","908-470-4661","kar
 l_klonowski@yahoo.com","http://www.rossimichaelm.com"
"Tonette","Wenner","Northwest Publishing","4545 Courthouse Rd","Westbury","Nassau","NY",11590,"516-968-6051","516-333-4861","twenner@aol.com","http://www.northwestpublishing.com"
"Amber","Monarrez","Branford Wire & Mfg Co","14288 Foster Ave #4121","Jenkintown","Montgomery","PA",19046,"215-934-8655","215-329-6386","amber_monarrez@monarrez.org","http://www.branfordwiremfgco.com"
"Shenika","Seewald","East Coast Marketing","4 Otis St","Van Nuys","Los Angeles","CA",91405,"818-423-4007","818-749-8650","shenika@gmail.com","http://www.eastcoastmarketing.com"
"Delmy","Ahle","Wye Technologies Inc","65895 S 16th St","Providence","Providence","RI","02909","401-458-2547","401-559-8961","delmy.ahle@hotmail.com","http://www.wyetechnologiesinc.com"
"Deeanna","Juhas","Healy, George W Iv","14302 Pennsylvania Ave","Huntingdon Valley","Montgomery","PA",19006,"215-211-9589","215-417-9563","deeanna_juhas@gmail.com","http://www.healygeorgewiv.com"
"Blo
 ndell","Pugh","Alpenlite Inc","201 Hawk Ct","Providence","Providence","RI","02904","401-960-8259","401-300-8122","bpugh@aol.com","http://www.alpenliteinc.com"
"Jamal","Vanausdal","Hubbard, Bruce Esq","53075 Sw 152nd Ter #615","Monroe Township","Middlesex","NJ","08831","732-234-1546","732-904-2931","jamal@vanausdal.org","http://www.hubbardbruceesq.com"
"Cecily","Hollack","Arthur A Oliver & Son Inc","59 N Groesbeck Hwy","Austin","Travis","TX",78731,"512-486-3817","512-861-3814","cecily@hollack.org","http://www.arthuraoliversoninc.com"
"Carmelina","Lindall","George Jessop Carter Jewelers","2664 Lewis Rd","Littleton","Douglas","CO",80126,"303-724-7371","303-874-5160","carmelina_lindall@lindall.com","http://www.georgejessopcarterjewelers.com"
"Maurine","Yglesias","Schultz, Thomas C Md","59 Shady Ln #53","Milwaukee","Milwaukee","WI",53214,"414-748-1374","414-573-7719","maurine_yglesias@yglesias.com","http://www.schultzthomascmd.com"
"Tawna","Buvens","H H H Enterprises Inc","3305 Nabell Av
 e #679","New York","New York","NY",10009,"212-674-9610","212-462-9157","tawna@gmail.com","http://www.hhhenterprisesinc.com"
"Penney","Weight","Hawaiian King Hotel","18 Fountain St","Anchorage","Anchorage","AK",99515,"907-797-9628","907-873-2882","penney_weight@aol.com","http://www.hawaiiankinghotel.com"
"Elly","Morocco","Killion Industries","7 W 32nd St","Erie","Erie","PA",16502,"814-393-5571","814-420-3553","elly_morocco@gmail.com","http://www.killionindustries.com"
"Ilene","Eroman","Robinson, William J Esq","2853 S Central Expy","Glen Burnie","Anne Arundel","MD",21061,"410-914-9018","410-937-4543","ilene.eroman@hotmail.com","http://www.robinsonwilliamjesq.com"
"Vallie","Mondella","Private Properties","74 W College St","Boise","Ada","ID",83707,"208-862-5339","208-737-8439","vmondella@mondella.com","http://www.privateproperties.com"
"Kallie","Blackwood","Rowley Schlimgen Inc","701 S Harrison Rd","San Francisco","San Francisco","CA",94104,"415-315-2761","415-604-7609","kallie.blackwo
 od@gmail.com","http://www.rowleyschlimgeninc.com"
"Johnetta","Abdallah","Forging Specialties","1088 Pinehurst St","Chapel Hill","Orange","NC",27514,"919-225-9345","919-715-3791","johnetta_abdallah@aol.com","http://www.forgingspecialties.com"
"Bobbye","Rhym","Smits, Patricia Garity","30 W 80th St #1995","San Carlos","San Mateo","CA",94070,"650-528-5783","650-811-9032","brhym@rhym.com","http://www.smitspatriciagarity.com"
"Micaela","Rhymes","H Lee Leonard Attorney At Law","20932 Hedley St","Concord","Contra Costa","CA",94520,"925-647-3298","925-522-7798","micaela_rhymes@gmail.com","http://www.hleeleonardattorneyatlaw.com"
"Tamar","Hoogland","A K Construction Co","2737 Pistorio Rd #9230","London","Madison","OH",43140,"740-343-8575","740-526-5410","tamar@hotmail.com","http://www.akconstructionco.com"
"Moon","Parlato","Ambelang, Jessica M Md","74989 Brandon St","Wellsville","Allegany","NY",14895,"585-866-8313","585-498-4278","moon@yahoo.com","http://www.ambelangjessicammd.com"
"Laurel","
 Reitler","Q A Service","6 Kains Ave","Baltimore","Baltimore City","MD",21215,"410-520-4832","410-957-6903","laurel_reitler@reitler.com","http://www.qaservice.com"
"Delisa","Crupi","Wood & Whitacre Contractors","47565 W Grand Ave","Newark","Essex","NJ","07105","973-354-2040","973-847-9611","delisa.crupi@crupi.com","http://www.woodwhitacrecontractors.com"
"Viva","Toelkes","Mark Iv Press Ltd","4284 Dorigo Ln","Chicago","Cook","IL",60647,"773-446-5569","773-352-3437","viva.toelkes@gmail.com","http://www.markivpressltd.com"
"Elza","Lipke","Museum Of Science & Industry","6794 Lake Dr E","Newark","Essex","NJ","07104","973-927-3447","973-796-3667","elza@yahoo.com","http://www.museumofscienceindustry.com"
"Devorah","Chickering","Garrison Ind","31 Douglas Blvd #950","Clovis","Curry","NM",88101,"505-975-8559","505-950-1763","devorah@hotmail.com","http://www.garrisonind.com"
"Timothy","Mulqueen","Saronix Nymph Products","44 W 4th St","Staten Island","Richmond","NY",10309,"718-332-6527","718-654
 -7063","timothy_mulqueen@mulqueen.org","http://www.saronixnymphproducts.com"
"Arlette","Honeywell","Smc Inc","11279 Loytan St","Jacksonville","Duval","FL",32254,"904-775-4480","904-514-9918","ahoneywell@honeywell.com","http://www.smcinc.com"
"Dominque","Dickerson","E A I Electronic Assocs Inc","69 Marquette Ave","Hayward","Alameda","CA",94545,"510-993-3758","510-901-7640","dominque.dickerson@dickerson.org","http://www.eaielectronicassocsinc.com"
"Lettie","Isenhower","Conte, Christopher A Esq","70 W Main St","Beachwood","Cuyahoga","OH",44122,"216-657-7668","216-733-8494","lettie_isenhower@yahoo.com","http://www.contechristopheraesq.com"
"Myra","Munns","Anker Law Office","461 Prospect Pl #316","Euless","Tarrant","TX",76040,"817-914-7518","817-451-3518","mmunns@cox.net","http://www.ankerlawoffice.com"
"Stephaine","Barfield","Beutelschies & Company","47154 Whipple Ave Nw","Gardena","Los Angeles","CA",90247,"310-774-7643","310-968-1219","stephaine@barfield.com","http://www.beutelschiesco
 mpany.com"
"Lai","Gato","Fligg, Kenneth I Jr","37 Alabama Ave","Evanston","Cook","IL",60201,"847-728-7286","847-957-4614","lai.gato@gato.org","http://www.fliggkennethijr.com"
"Stephen","Emigh","Sharp, J Daniel Esq","3777 E Richmond St #900","Akron","Summit","OH",44302,"330-537-5358","330-700-2312","stephen_emigh@hotmail.com","http://www.sharpjdanielesq.com"
"Tyra","Shields","Assink, Anne H Esq","3 Fort Worth Ave","Philadelphia","Philadelphia","PA",19106,"215-255-1641","215-228-8264","tshields@gmail.com","http://www.assinkannehesq.com"
"Tammara","Wardrip","Jewel My Shop Inc","4800 Black Horse Pike","Burlingame","San Mateo","CA",94010,"650-803-1936","650-216-5075","twardrip@cox.net","http://www.jewelmyshopinc.com"
"Cory","Gibes","Chinese Translation Resources","83649 W Belmont Ave","San Gabriel","Los Angeles","CA",91776,"626-572-1096","626-696-2777","cory.gibes@gmail.com","http://www.chinesetranslationresources.com"
"Danica","Bruschke","Stevens, Charles T","840 15th Ave","Waco","McLen
 nan","TX",76708,"254-782-8569","254-205-1422","danica_bruschke@gmail.com","http://www.stevenscharlest.com"
"Wilda","Giguere","Mclaughlin, Luther W Cpa","1747 Calle Amanecer #2","Anchorage","Anchorage","AK",99501,"907-870-5536","907-914-9482","wilda@cox.net","http://www.mclaughlinlutherwcpa.com"
"Elvera","Benimadho","Tree Musketeers","99385 Charity St #840","San Jose","Santa Clara","CA",95110,"408-703-8505","408-440-8447","elvera.benimadho@cox.net","http://www.treemusketeers.com"
"Carma","Vanheusen","Springfield Div Oh Edison Co","68556 Central Hwy","San Leandro","Alameda","CA",94577,"510-503-7169","510-452-4835","carma@cox.net","http://www.springfielddivohedisonco.com"
"Malinda","Hochard","Logan Memorial Hospital","55 Riverside Ave","Indianapolis","Marion","IN",46202,"317-722-5066","317-472-2412","malinda.hochard@yahoo.com","http://www.loganmemorialhospital.com"
"Natalie","Fern","Kelly, Charles G Esq","7140 University Ave","Rock Springs","Sweetwater","WY",82901,"307-704-8713","307-2
 79-3793","natalie.fern@hotmail.com","http://www.kellycharlesgesq.com"
"Lisha","Centini","Industrial Paper Shredders Inc","64 5th Ave #1153","Mc Lean","Fairfax","VA",22102,"703-235-3937","703-475-7568","lisha@centini.org","http://www.industrialpapershreddersinc.com"
"Arlene","Klusman","Beck Horizon Builders","3 Secor Rd","New Orleans","Orleans","LA",70112,"504-710-5840","504-946-1807","arlene_klusman@gmail.com","http://www.beckhorizonbuilders.com"
"Alease","Buemi","Porto Cayo At Hawks Cay","4 Webbs Chapel Rd","Boulder","Boulder","CO",80303,"303-301-4946","303-521-9860","alease@buemi.com","http://www.portocayoathawkscay.com"
"Louisa","Cronauer","Pacific Grove Museum Ntrl Hist","524 Louisiana Ave Nw","San Leandro","Alameda","CA",94577,"510-828-7047","510-472-7758","louisa@cronauer.com","http://www.pacificgrovemuseumntrlhist.com"
"Angella","Cetta","Bender & Hatley Pc","185 Blackstone Bldge","Honolulu","Honolulu","HI",96817,"808-892-7943","808-475-2310","angella.cetta@hotmail.com","http:
 //www.benderhatleypc.com"
"Cyndy","Goldammer","Di Cristina J & Son","170 Wyoming Ave","Burnsville","Dakota","MN",55337,"952-334-9408","952-938-9457","cgoldammer@cox.net","http://www.dicristinajson.com"
"Rosio","Cork","Green Goddess","4 10th St W","High Point","Guilford","NC",27263,"336-243-5659","336-497-4407","rosio.cork@gmail.com","http://www.greengoddess.com"
"Celeste","Korando","American Arts & Graphics","7 W Pinhook Rd","Lynbrook","Nassau","NY",11563,"516-509-2347","516-365-7266","ckorando@hotmail.com","http://www.americanartsgraphics.com"
"Twana","Felger","Opryland Hotel","1 Commerce Way","Portland","Washington","OR",97224,"503-939-3153","503-909-7167","twana.felger@felger.org","http://www.oprylandhotel.com"
"Estrella","Samu","Marking Devices Pubg Co","64 Lakeview Ave","Beloit","Rock","WI",53511,"608-976-7199","608-942-8836","estrella@aol.com","http://www.markingdevicespubgco.com"
"Donte","Kines","W Tc Industries Inc","3 Aspen St","Worcester","Worcester","MA","01602","508-429-
 8576","508-843-1426","dkines@hotmail.com","http://www.wtcindustriesinc.com"
"Tiffiny","Steffensmeier","Whitehall Robbins Labs Divsn","32860 Sierra Rd","Miami","Miami-Dade","FL",33133,"305-385-9695","305-304-6573","tiffiny_steffensmeier@cox.net","http://www.whitehallrobbinslabsdivsn.com"
"Edna","Miceli","Sampler","555 Main St","Erie","Erie","PA",16502,"814-460-2655","814-299-2877","emiceli@miceli.org","http://www.sampler.com"
"Sue","Kownacki","Juno Chefs Incorporated","2 Se 3rd Ave","Mesquite","Dallas","TX",75149,"972-666-3413","972-742-4000","sue@aol.com","http://www.junochefsincorporated.com"
"Jesusa","Shin","Carroccio, A Thomas Esq","2239 Shawnee Mission Pky","Tullahoma","Coffee","TN",37388,"931-273-8709","931-739-1551","jshin@shin.com","http://www.carroccioathomasesq.com"
"Rolland","Francescon","Stanley, Richard L Esq","2726 Charcot Ave","Paterson","Passaic","NJ","07501","973-649-2922","973-284-4048","rolland@cox.net","http://www.stanleyrichardlesq.com"
"Pamella","Schmierer","K C
 s Cstm Mouldings Windows","5161 Dorsett Rd","Homestead","Miami-Dade","FL",33030,"305-420-8970","305-575-8481","pamella.schmierer@schmierer.org","http://www.kcscstmmouldingswindows.com"
"Glory","Kulzer","Comfort Inn","55892 Jacksonville Rd","Owings Mills","Baltimore","MD",21117,"410-224-9462","410-916-8015","gkulzer@kulzer.org","http://www.comfortinn.com"
"Shawna","Palaspas","Windsor, James L Esq","5 N Cleveland Massillon Rd","Thousand Oaks","Ventura","CA",91362,"805-275-3566","805-638-6617","shawna_palaspas@palaspas.org","http://www.windsorjameslesq.com"
"Brandon","Callaro","Jackson Shields Yeiser","7 Benton Dr","Honolulu","Honolulu","HI",96819,"808-215-6832","808-240-5168","brandon_callaro@hotmail.com","http://www.jacksonshieldsyeiser.com"
"Scarlet","Cartan","Box, J Calvin Esq","9390 S Howell Ave","Albany","Dougherty","GA",31701,"229-735-3378","229-365-9658","scarlet.cartan@yahoo.com","http://www.boxjcalvinesq.com"
"Oretha","Menter","Custom Engineering Inc","8 County Center Dr #647
 ","Boston","Suffolk","MA","02210","617-418-5043","617-697-6024","oretha_menter@yahoo.com","http://www.customengineeringinc.com"
"Ty","Smith","Bresler Eitel Framg Gllry Ltd","4646 Kaahumanu St","Hackensack","Bergen","NJ","07601","201-672-1553","201-995-3149","tsmith@aol.com","http://www.breslereitelframggllryltd.com"
"Xuan","Rochin","Carol, Drake Sparks Esq","2 Monroe St","San Mateo","San Mateo","CA",94403,"650-933-5072","650-247-2625","xuan@gmail.com","http://www.caroldrakesparksesq.com"
"Lindsey","Dilello","Biltmore Investors Bank","52777 Leaders Heights Rd","Ontario","San Bernardino","CA",91761,"909-639-9887","909-589-1693","lindsey.dilello@hotmail.com","http://www.biltmoreinvestorsbank.com"
"Devora","Perez","Desco Equipment Corp","72868 Blackington Ave","Oakland","Alameda","CA",94606,"510-955-3016","510-755-9274","devora_perez@perez.org","http://www.descoequipmentcorp.com"
"Herman","Demesa","Merlin Electric Co","9 Norristown Rd","Troy","Rensselaer","NY",12180,"518-497-2940","518-
 931-7852","hdemesa@cox.net","http://www.merlinelectricco.com"
"Rory","Papasergi","Bailey Cntl Co Div Babcock","83 County Road 437 #8581","Clarks Summit","Lackawanna","PA",18411,"570-867-7489","570-469-8401","rpapasergi@cox.net","http://www.baileycntlcodivbabcock.com"
"Talia","Riopelle","Ford Brothers Wholesale Inc","1 N Harlem Ave #9","Orange","Essex","NJ","07050","973-245-2133","973-818-9788","talia_riopelle@aol.com","http://www.fordbrotherswholesaleinc.com"
"Van","Shire","Cambridge Inn","90131 J St","Pittstown","Hunterdon","NJ","08867","908-409-2890","908-448-1209","van.shire@shire.com","http://www.cambridgeinn.com"
"Lucina","Lary","Matricciani, Albert J Jr","8597 W National Ave","Cocoa","Brevard","FL",32922,"321-749-4981","321-632-4668","lucina_lary@cox.net","http://www.matriccianialbertjjr.com"
"Bok","Isaacs","Nelson Hawaiian Ltd","6 Gilson St","Bronx","Bronx","NY",10468,"718-809-3762","718-478-8568","bok.isaacs@aol.com","http://www.nelsonhawaiianltd.com"
"Rolande","Spickerman",
 "Neland Travel Agency","65 W Maple Ave","Pearl City","Honolulu","HI",96782,"808-315-3077","808-526-5863","rolande.spickerman@spickerman.com","http://www.nelandtravelagency.com"
"Howard","Paulas","Asendorf, J Alan Esq","866 34th Ave","Denver","Denver","CO",80231,"303-623-4241","303-692-3118","hpaulas@gmail.com","http://www.asendorfjalanesq.com"
"Kimbery","Madarang","Silberman, Arthur L Esq","798 Lund Farm Way","Rockaway","Morris","NJ","07866","973-310-1634","973-225-6259","kimbery_madarang@cox.net","http://www.silbermanarthurlesq.com"
"Thurman","Manno","Honey Bee Breeding Genetics &","9387 Charcot Ave","Absecon","Atlantic","NJ","08201","609-524-3586","609-234-8376","thurman.manno@yahoo.com","http://www.honeybeebreedinggenetics.com"
"Becky","Mirafuentes","Wells Kravitz Schnitzer","30553 Washington Rd","Plainfield","Union","NJ","07062","908-877-8409","908-426-8272","becky.mirafuentes@mirafuentes.com","http://www.wellskravitzschnitzer.com"
"Beatriz","Corrington","Prohab Rehabilitation S
 ervs","481 W Lemon St","Middleboro","Plymouth","MA","02346","508-584-4279","508-315-3867","beatriz@yahoo.com","http://www.prohabrehabilitationservs.com"
"Marti","Maybury","Eldridge, Kristin K Esq","4 Warehouse Point Rd #7","Chicago","Cook","IL",60638,"773-775-4522","773-539-1058","marti.maybury@yahoo.com","http://www.eldridgekristinkesq.com"
"Nieves","Gotter","Vlahos, John J Esq","4940 Pulaski Park Dr","Portland","Multnomah","OR",97202,"503-527-5274","503-455-3094","nieves_gotter@gmail.com","http://www.vlahosjohnjesq.com"
"Leatha","Hagele","Ninas Indian Grs & Videos","627 Walford Ave","Dallas","Dallas","TX",75227,"214-339-1809","214-225-5850","lhagele@cox.net","http://www.ninasindiangrsvideos.com"
"Valentin","Klimek","Schmid, Gayanne K Esq","137 Pioneer Way","Chicago","Cook","IL",60604,"312-303-5453","312-512-2338","vklimek@klimek.org","http://www.schmidgayannekesq.com"
"Melissa","Wiklund","Moapa Valley Federal Credit Un","61 13 Stoneridge #835","Findlay","Hancock","OH",45840,"419-9
 39-3613","419-254-4591","melissa@cox.net","http://www.moapavalleyfederalcreditun.com"
"Sheridan","Zane","Kentucky Tennessee Clay Co","2409 Alabama Rd","Riverside","Riverside","CA",92501,"951-645-3605","951-248-6822","sheridan.zane@zane.com","http://www.kentuckytennesseeclayco.com"
"Bulah","Padilla","Admiral Party Rentals & Sales","8927 Vandever Ave","Waco","McLennan","TX",76707,"254-463-4368","254-816-8417","bulah_padilla@hotmail.com","http://www.admiralpartyrentalssales.com"
"Audra","Kohnert","Nelson, Karolyn King Esq","134 Lewis Rd","Nashville","Davidson","TN",37211,"615-406-7854","615-448-9249","audra@kohnert.com","http://www.nelsonkarolynkingesq.com"
"Daren","Weirather","Panasystems","9 N College Ave #3","Milwaukee","Milwaukee","WI",53216,"414-959-2540","414-838-3151","dweirather@aol.com","http://www.panasystems.com"
"Fernanda","Jillson","Shank, Edward L Esq","60480 Old Us Highway 51","Preston","Caroline","MD",21655,"410-387-5260","410-724-6472","fjillson@aol.com","http://www.sh
 ankedwardlesq.com"
"Gearldine","Gellinger","Megibow & Edwards","4 Bloomfield Ave","Irving","Dallas","TX",75061,"972-934-6914","972-821-7118","gearldine_gellinger@gellinger.com","http://www.megibowedwards.com"
"Chau","Kitzman","Benoff, Edward Esq","429 Tiger Ln","Beverly Hills","Los Angeles","CA",90212,"310-560-8022","310-969-7230","chau@gmail.com","http://www.benoffedwardesq.com"
"Theola","Frey","Woodbridge Free Public Library","54169 N Main St","Massapequa","Nassau","NY",11758,"516-948-5768","516-357-3362","theola_frey@frey.com","http://www.woodbridgefreepubliclibrary.com"
"Cheryl","Haroldson","New York Life John Thune","92 Main St","Atlantic City","Atlantic","NJ","08401","609-518-7697","609-263-9243","cheryl@haroldson.org","http://www.newyorklifejohnthune.com"
"Laticia","Merced","Alinabal Inc","72 Mannix Dr","Cincinnati","Hamilton","OH",45203,"513-508-7371","513-418-1566","lmerced@gmail.com","http://www.alinabalinc.com"
"Carissa","Batman","Poletto, Kim David Esq","12270 Caton Cent
 er Dr","Eugene","Lane","OR",97401,"541-326-4074","541-801-5717","carissa.batman@yahoo.com","http://www.polettokimdavidesq.com"
"Lezlie","Craghead","Chang, Carolyn Esq","749 W 18th St #45","Smithfield","Johnston","NC",27577,"919-533-3762","919-885-2453","lezlie.craghead@craghead.org","http://www.changcarolynesq.com"
"Ozell","Shealy","Silver Bros Inc","8 Industry Ln","New York","New York","NY",10002,"212-332-8435","212-880-8865","oshealy@hotmail.com","http://www.silverbrosinc.com"
"Arminda","Parvis","Newtec Inc","1 Huntwood Ave","Phoenix","Maricopa","AZ",85017,"602-906-9419","602-277-3025","arminda@parvis.com","http://www.newtecinc.com"
"Reita","Leto","Creative Business Systems","55262 N French Rd","Indianapolis","Marion","IN",46240,"317-234-1135","317-787-5514","reita.leto@gmail.com","http://www.creativebusinesssystems.com"
"Yolando","Luczki","Dal Tile Corporation","422 E 21st St","Syracuse","Onondaga","NY",13214,"315-304-4759","315-640-6357","yolando@cox.net","http://www.daltilecorp
 oration.com"
"Lizette","Stem","Edward S Katz","501 N 19th Ave","Cherry Hill","Camden","NJ","08002","856-487-5412","856-702-3676","lizette.stem@aol.com","http://www.edwardskatz.com"
"Gregoria","Pawlowicz","Oh My Goodknits Inc","455 N Main Ave","Garden City","Nassau","NY",11530,"516-212-1915","516-376-4230","gpawlowicz@yahoo.com","http://www.ohmygoodknitsinc.com"
"Carin","Deleo","Redeker, Debbie","1844 Southern Blvd","Little Rock","Pulaski","AR",72202,"501-308-1040","501-409-6072","cdeleo@deleo.com","http://www.redekerdebbie.com"
"Chantell","Maynerich","Desert Sands Motel","2023 Greg St","Saint Paul","Ramsey","MN",55101,"651-591-2583","651-776-9688","chantell@yahoo.com","http://www.desertsandsmotel.com"
"Dierdre","Yum","Cummins Southern Plains Inc","63381 Jenks Ave","Philadelphia","Philadelphia","PA",19134,"215-325-3042","215-346-4666","dyum@yahoo.com","http://www.cumminssouthernplainsinc.com"
"Larae","Gudroe","Lehigh Furn Divsn Lehigh","6651 Municipal Rd","Houma","Terrebonne","LA",70
 360,"985-890-7262","985-261-5783","larae_gudroe@gmail.com","http://www.lehighfurndivsnlehigh.com"
"Latrice","Tolfree","United Van Lines Agent","81 Norris Ave #525","Ronkonkoma","Suffolk","NY",11779,"631-957-7624","631-998-2102","latrice.tolfree@hotmail.com","http://www.unitedvanlinesagent.com"
"Kerry","Theodorov","Capitol Reporters","6916 W Main St","Sacramento","Sacramento","CA",95827,"916-591-3277","916-770-7448","kerry.theodorov@gmail.com","http://www.capitolreporters.com"
"Dorthy","Hidvegi","Kwik Kopy Printing","9635 S Main St","Boise","Ada","ID",83704,"208-649-2373","208-690-3315","dhidvegi@yahoo.com","http://www.kwikkopyprinting.com"
"Fannie","Lungren","Centro Inc","17 Us Highway 111","Round Rock","Williamson","TX",78664,"512-587-5746","512-528-9933","fannie.lungren@yahoo.com","http://www.centroinc.com"
"Evangelina","Radde","Campbell, Jan Esq","992 Civic Center Dr","Philadelphia","Philadelphia","PA",19123,"215-964-3284","215-417-5612","evangelina@aol.com","http://www.campbellj
 anesq.com"
"Novella","Degroot","Evans, C Kelly Esq","303 N Radcliffe St","Hilo","Hawaii","HI",96720,"808-477-4775","808-746-1865","novella_degroot@degroot.org","http://www.evansckellyesq.com"
"Clay","Hoa","Scat Enterprises","73 Saint Ann St #86","Reno","Washoe","NV",89502,"775-501-8109","775-848-9135","choa@hoa.org","http://www.scatenterprises.com"
"Jennifer","Fallick","Nagle, Daniel J Esq","44 58th St","Wheeling","Cook","IL",60090,"847-979-9545","847-800-3054","jfallick@yahoo.com","http://www.nagledanieljesq.com"
"Irma","Wolfgramm","Serendiquity Bed & Breakfast","9745 W Main St","Randolph","Morris","NJ","07869","973-545-7355","973-868-8660","irma.wolfgramm@hotmail.com","http://www.serendiquitybedbreakfast.com"
"Eun","Coody","Ray Carolyne Realty","84 Bloomfield Ave","Spartanburg","Spartanburg","SC",29301,"864-256-3620","864-594-4578","eun@yahoo.com","http://www.raycarolynerealty.com"
"Sylvia","Cousey","Berg, Charles E","287 Youngstown Warren Rd","Hampstead","Carroll","MD",21074,"410
 -209-9545","410-863-8263","sylvia_cousey@cousey.org","http://www.bergcharlese.com"
"Nana","Wrinkles","Ray, Milbern D","6 Van Buren St","Mount Vernon","Westchester","NY",10553,"914-855-2115","914-796-3775","nana@aol.com","http://www.raymilbernd.com"
"Layla","Springe","Chadds Ford Winery","229 N Forty Driv","New York","New York","NY",10011,"212-260-3151","212-253-7448","layla.springe@cox.net","http://www.chaddsfordwinery.com"
"Joesph","Degonia","A R Packaging","2887 Knowlton St #5435","Berkeley","Alameda","CA",94710,"510-677-9785","510-942-5916","joesph_degonia@degonia.org","http://www.arpackaging.com"
"Annabelle","Boord","Corn Popper","523 Marquette Ave","Concord","Middlesex","MA","01742","978-697-6263","978-289-7717","annabelle.boord@cox.net","http://www.cornpopper.com"
"Stephaine","Vinning","Birite Foodservice Distr","3717 Hamann Industrial Pky","San Francisco","San Francisco","CA",94104,"415-767-6596","415-712-9530","stephaine@cox.net","http://www.biritefoodservicedistr.com"
"Neli
 da","Sawchuk","Anchorage Museum Of Hist & Art","3 State Route 35 S","Paramus","Bergen","NJ","07652","201-971-1638","201-247-8925","nelida@gmail.com","http://www.anchoragemuseumofhistart.com"
"Marguerita","Hiatt","Haber, George D Md","82 N Highway 67","Oakley","Contra Costa","CA",94561,"925-634-7158","925-541-8521","marguerita.hiatt@gmail.com","http://www.habergeorgedmd.com"
"Carmela","Cookey","Royal Pontiac Olds Inc","9 Murfreesboro Rd","Chicago","Cook","IL",60623,"773-494-4195","773-297-9391","ccookey@cookey.org","http://www.royalpontiacoldsinc.com"
"Junita","Brideau","Leonards Antiques Inc","6 S Broadway St","Cedar Grove","Essex","NJ","07009","973-943-3423","973-582-5469","jbrideau@aol.com","http://www.leonardsantiquesinc.com"
"Claribel","Varriano","Meca","6 Harry L Dr #6327","Perrysburg","Wood","OH",43551,"419-544-4900","419-573-2033","claribel_varriano@cox.net","http://www.meca.com"
"Benton","Skursky","Nercon Engineering & Mfg Inc","47939 Porter Ave","Gardena","Los Angeles","CA"
 ,90248,"310-579-2907","310-694-8466","benton.skursky@aol.com","http://www.nerconengineeringmfginc.com"
"Hillary","Skulski","Replica I","9 Wales Rd Ne #914","Homosassa","Citrus","FL",34448,"352-242-2570","352-990-5946","hillary.skulski@aol.com","http://www.replicai.com"
"Merilyn","Bayless","20 20 Printing Inc","195 13n N","Santa Clara","Santa Clara","CA",95054,"408-758-5015","408-346-2180","merilyn_bayless@cox.net","http://www.printinginc.com"
"Teri","Ennaco","Publishers Group West","99 Tank Farm Rd","Hazleton","Luzerne","PA",18201,"570-889-5187","570-355-1665","tennaco@gmail.com","http://www.publishersgroupwest.com"
"Merlyn","Lawler","Nischwitz, Jeffrey L Esq","4671 Alemany Blvd","Jersey City","Hudson","NJ","07304","201-588-7810","201-858-9960","merlyn_lawler@hotmail.com","http://www.nischwitzjeffreylesq.com"
"Georgene","Montezuma","Payne Blades & Wellborn Pa","98 University Dr","San Ramon","Contra Costa","CA",94583,"925-615-5185","925-943-3449","gmontezuma@cox.net","http://www.payn
 ebladeswellbornpa.com"
"Jettie","Mconnell","Coldwell Bnkr Wright Real Est","50 E Wacker Dr","Bridgewater","Somerset","NJ","08807","908-802-3564","908-602-5258","jmconnell@hotmail.com","http://www.coldwellbnkrwrightrealest.com"
"Lemuel","Latzke","Computer Repair Service","70 Euclid Ave #722","Bohemia","Suffolk","NY",11716,"631-748-6479","631-291-4976","lemuel.latzke@gmail.com","http://www.computerrepairservice.com"
"Melodie","Knipp","Fleetwood Building Block Inc","326 E Main St #6496","Thousand Oaks","Ventura","CA",91362,"805-690-1682","805-810-8964","mknipp@gmail.com","http://www.fleetwoodbuildingblockinc.com"
"Candida","Corbley","Colts Neck Medical Assocs Inc","406 Main St","Somerville","Somerset","NJ","08876","908-275-8357","908-943-6103","candida_corbley@hotmail.com","http://www.coltsneckmedicalassocsinc.com"
"Karan","Karpin","New England Taxidermy","3 Elmwood Dr","Beaverton","Washington","OR",97005,"503-940-8327","503-707-5812","karan_karpin@gmail.com","http://www.newenglandtaxi
 dermy.com"
"Andra","Scheyer","Ludcke, George O Esq","9 Church St","Salem","Marion","OR",97302,"503-516-2189","503-950-3068","andra@gmail.com","http://www.ludckegeorgeoesq.com"
"Felicidad","Poullion","Mccorkle, Tom S Esq","9939 N 14th St","Riverton","Burlington","NJ","08077","856-305-9731","856-828-6021","fpoullion@poullion.com","http://www.mccorkletomsesq.com"
"Belen","Strassner","Eagle Software Inc","5384 Southwyck Blvd","Douglasville","Douglas","GA",30135,"770-507-8791","770-802-4003","belen_strassner@aol.com","http://www.eaglesoftwareinc.com"
"Gracia","Melnyk","Juvenile & Adult Super","97 Airport Loop Dr","Jacksonville","Duval","FL",32216,"904-235-3633","904-627-4341","gracia@melnyk.com","http://www.juvenileadultsuper.com"
"Jolanda","Hanafan","Perez, Joseph J Esq","37855 Nolan Rd","Bangor","Penobscot","ME","04401","207-458-9196","207-233-6185","jhanafan@gmail.com","http://www.perezjosephjesq.com"
"Barrett","Toyama","Case Foundation Co","4252 N Washington Ave #9","Kennedale","Tarr
 ant","TX",76060,"817-765-5781","817-577-6151","barrett.toyama@toyama.org","http://www.casefoundationco.com"
"Helga","Fredicks","Eis Environmental Engrs Inc","42754 S Ash Ave","Buffalo","Erie","NY",14228,"716-752-4114","716-854-9845","helga_fredicks@yahoo.com","http://www.eisenvironmentalengrsinc.com"
"Ashlyn","Pinilla","Art Crafters","703 Beville Rd","Opa Locka","Miami-Dade","FL",33054,"305-670-9628","305-857-5489","apinilla@cox.net","http://www.artcrafters.com"
"Fausto","Agramonte","Marriott Hotels Resorts Suites","5 Harrison Rd","New York","New York","NY",10038,"212-313-1783","212-778-3063","fausto_agramonte@yahoo.com","http://www.marriotthotelsresortssuites.com"
"Ronny","Caiafa","Remaco Inc","73 Southern Blvd","Philadelphia","Philadelphia","PA",19103,"215-605-7570","215-511-3531","ronny.caiafa@caiafa.org","http://www.remacoinc.com"
"Marge","Limmel","Bjork, Robert D Jr","189 Village Park Rd","Crestview","Okaloosa","FL",32536,"850-430-1663","850-330-8079","marge@gmail.com","http://
 www.bjorkrobertdjr.com"
"Norah","Waymire","Carmichael, Jeffery L Esq","6 Middlegate Rd #106","San Francisco","San Francisco","CA",94107,"415-306-7897","415-874-2984","norah.waymire@gmail.com","http://www.carmichaeljefferylesq.com"
"Aliza","Baltimore","Andrews, J Robert Esq","1128 Delaware St","San Jose","Santa Clara","CA",95132,"408-504-3552","408-425-1994","aliza@aol.com","http://www.andrewsjrobertesq.com"
"Mozell","Pelkowski","Winship & Byrne","577 Parade St","South San Francisco","San Mateo","CA",94080,"650-947-1215","650-960-1069","mpelkowski@pelkowski.org","http://www.winshipbyrne.com"
"Viola","Bitsuie","Burton & Davis","70 Mechanic St","Northridge","Los Angeles","CA",91325,"818-864-4875","818-481-5787","viola@gmail.com","http://www.burtondavis.com"
"Franklyn","Emard","Olympic Graphic Arts","4379 Highway 116","Philadelphia","Philadelphia","PA",19103,"215-558-8189","215-483-3003","femard@emard.com","http://www.olympicgraphicarts.com"
"Willodean","Konopacki","Magnuson","55 Hawtho
 rne Blvd","Lafayette","Lafayette","LA",70506,"337-253-8384","337-774-7564","willodean_konopacki@konopacki.org","http://www.magnuson.com"
"Beckie","Silvestrini","A All American Travel Inc","7116 Western Ave","Dearborn","Wayne","MI",48126,"313-533-4884","313-390-7855","beckie.silvestrini@silvestrini.com","http://www.aallamericantravelinc.com"
"Rebecka","Gesick","Polykote Inc","2026 N Plankinton Ave #3","Austin","Travis","TX",78754,"512-213-8574","512-693-8345","rgesick@gesick.org","http://www.polykoteinc.com"
"Frederica","Blunk","Jets Cybernetics","99586 Main St","Dallas","Dallas","TX",75207,"214-428-2285","214-529-1949","frederica_blunk@gmail.com","http://www.jetscybernetics.com"
"Glen","Bartolet","Metlab Testing Services","8739 Hudson St","Vashon","King","WA",98070,"206-697-5796","206-389-1482","glen_bartolet@hotmail.com","http://www.metlabtestingservices.com"
"Freeman","Gochal","Kellermann, William T Esq","383 Gunderman Rd #197","Coatesville","Chester","PA",19320,"610-476-3501","61
 0-752-2683","freeman_gochal@aol.com","http://www.kellermannwilliamtesq.com"
"Vincent","Meinerding","Arturi, Peter D Esq","4441 Point Term Mkt","Philadelphia","Philadelphia","PA",19143,"215-372-1718","215-829-4221","vincent.meinerding@hotmail.com","http://www.arturipeterdesq.com"
"Rima","Bevelacqua","Mcauley Mfg Co","2972 Lafayette Ave","Gardena","Los Angeles","CA",90248,"310-858-5079","310-499-4200","rima@cox.net","http://www.mcauleymfgco.com"
"Glendora","Sarbacher","Defur Voran Hanley Radcliff","2140 Diamond Blvd","Rohnert Park","Sonoma","CA",94928,"707-653-8214","707-881-3154","gsarbacher@gmail.com","http://www.defurvoranhanleyradcliff.com"
"Avery","Steier","Dill Dill Carr & Stonbraker Pc","93 Redmond Rd #492","Orlando","Orange","FL",32803,"407-808-9439","407-945-8566","avery@cox.net","http://www.dilldillcarrstonbrakerpc.com"
"Cristy","Lother","Kleensteel","3989 Portage Tr","Escondido","San Diego","CA",92025,"760-971-4322","760-465-4762","cristy@lother.com","http://www.kleensteel.
 com"
"Nicolette","Brossart","Goulds Pumps Inc Slurry Pump","1 Midway Rd","Westborough","Worcester","MA","01581","508-837-9230","508-504-6388","nicolette_brossart@brossart.com","http://www.gouldspumpsincslurrypump.com"
"Tracey","Modzelewski","Kansas City Insurance Report","77132 Coon Rapids Blvd Nw","Conroe","Montgomery","TX",77301,"936-264-9294","936-988-8171","tracey@hotmail.com","http://www.kansascityinsurancereport.com"
"Virgina","Tegarden","Berhanu International Foods","755 Harbor Way","Milwaukee","Milwaukee","WI",53226,"414-214-8697","414-411-5744","virgina_tegarden@tegarden.com","http://www.berhanuinternationalfoods.com"
"Tiera","Frankel","Roland Ashcroft","87 Sierra Rd","El Monte","Los Angeles","CA",91731,"626-636-4117","626-638-4241","tfrankel@aol.com","http://www.rolandashcroft.com"
"Alaine","Bergesen","Hispanic Magazine","7667 S Hulen St #42","Yonkers","Westchester","NY",10701,"914-300-9193","914-654-1426","alaine_bergesen@cox.net","http://www.hispanicmagazine.com"
"Earlee
 n","Mai","Little Sheet Metal Co","75684 S Withlapopka Dr #32","Dallas","Dallas","TX",75227,"214-289-1973","214-785-6750","earleen_mai@cox.net","http://www.littlesheetmetalco.com"
"Leonida","Gobern","Holmes, Armstead J Esq","5 Elmwood Park Blvd","Biloxi","Harrison","MS",39530,"228-235-5615","228-432-4635","leonida@gobern.org","http://www.holmesarmsteadjesq.com"
"Ressie","Auffrey","Faw, James C Cpa","23 Palo Alto Sq","Miami","Miami-Dade","FL",33134,"305-604-8981","305-287-4743","ressie.auffrey@yahoo.com","http://www.fawjamesccpa.com"
"Justine","Mugnolo","Evans Rule Company","38062 E Main St","New York","New York","NY",10048,"212-304-9225","212-311-6377","jmugnolo@yahoo.com","http://www.evansrulecompany.com"
"Eladia","Saulter","Tyee Productions Inc","3958 S Dupont Hwy #7","Ramsey","Bergen","NJ","07446","201-474-4924","201-365-8698","eladia@saulter.com","http://www.tyeeproductionsinc.com"
"Chaya","Malvin","Dunnells & Duvall","560 Civic Center Dr","Ann Arbor","Washtenaw","MI",48103,"734-
 928-5182","734-408-8174","chaya@malvin.com","http://www.dunnellsduvall.com"
"Gwenn","Suffield","Deltam Systems Inc","3270 Dequindre Rd","Deer Park","Suffolk","NY",11729,"631-258-6558","631-295-9879","gwenn_suffield@suffield.org","http://www.deltamsystemsinc.com"
"Salena","Karpel","Hammill Mfg Co","1 Garfield Ave #7","Canton","Stark","OH",44707,"330-791-8557","330-618-2579","skarpel@cox.net","http://www.hammillmfgco.com"
"Yoko","Fishburne","Sams Corner Store","9122 Carpenter Ave","New Haven","New Haven","CT","06511","203-506-4706","203-840-8634","yoko@fishburne.com","http://www.samscornerstore.com"
"Taryn","Moyd","Siskin, Mark J Esq","48 Lenox St","Fairfax","Fairfax City","VA",22030,"703-322-4041","703-938-7939","taryn.moyd@hotmail.com","http://www.siskinmarkjesq.com"
"Katina","Polidori","Cape & Associates Real Estate","5 Little River Tpke","Wilmington","Middlesex","MA","01887","978-626-2978","978-679-7429","katina_polidori@aol.com","http://www.capeassociatesrealestate.com"
"Rickie",
 "Plumer","Merrill Lynch","3 N Groesbeck Hwy","Toledo","Lucas","OH",43613,"419-693-1334","419-313-5571","rickie.plumer@aol.com","http://www.merrilllynch.com"
"Alex","Loader","Sublett, Scott Esq","37 N Elm St #916","Tacoma","Pierce","WA",98409,"253-660-7821","253-875-9222","alex@loader.com","http://www.sublettscottesq.com"
"Lashon","Vizarro","Sentry Signs","433 Westminster Blvd #590","Roseville","Placer","CA",95661,"916-741-7884","916-289-4526","lashon@aol.com","http://www.sentrysigns.com"
"Lauran","Burnard","Professionals Unlimited","66697 Park Pl #3224","Riverton","Fremont","WY",82501,"307-342-7795","307-453-7589","lburnard@burnard.com","http://www.professionalsunlimited.com"
"Ceola","Setter","Southern Steel Shelving Co","96263 Greenwood Pl","Warren","Knox","ME","04864","207-627-7565","207-297-5029","ceola.setter@setter.org","http://www.southernsteelshelvingco.com"
"My","Rantanen","Bosco, Paul J","8 Mcarthur Ln","Richboro","Bucks","PA",18954,"215-491-5633","215-647-2158","my@hotmail
 .com","http://www.boscopaulj.com"
"Lorrine","Worlds","Longo, Nicholas J Esq","8 Fair Lawn Ave","Tampa","Hillsborough","FL",33614,"813-769-2939","813-863-6467","lorrine.worlds@worlds.com","http://www.longonicholasjesq.com"
"Peggie","Sturiale","Henry County Middle School","9 N 14th St","El Cajon","San Diego","CA",92020,"619-608-1763","619-695-8086","peggie@cox.net","http://www.henrycountymiddleschool.com"
"Marvel","Raymo","Edison Supply & Equipment Co","9 Vanowen St","College Station","Brazos","TX",77840,"979-718-8968","979-809-5770","mraymo@yahoo.com","http://www.edisonsupplyequipmentco.com"
"Daron","Dinos","Wolf, Warren R Esq","18 Waterloo Geneva Rd","Highland Park","Lake","IL",60035,"847-233-3075","847-265-6609","daron_dinos@cox.net","http://www.wolfwarrenresq.com"
"An","Fritz","Linguistic Systems Inc","506 S Hacienda Dr","Atlantic City","Atlantic","NJ","08401","609-228-5265","609-854-7156","an_fritz@hotmail.com","http://www.linguisticsystemsinc.com"
"Portia","Stimmel","Peace Chris
 tian Center","3732 Sherman Ave","Bridgewater","Somerset","NJ","08807","908-722-7128","908-670-4712","portia.stimmel@aol.com","http://www.peacechristiancenter.com"
"Rhea","Aredondo","Double B Foods Inc","25657 Live Oak St","Brooklyn","Kings","NY",11226,"718-560-9537","718-280-4183","rhea_aredondo@cox.net","http://www.doublebfoodsinc.com"
"Benedict","Sama","Alexander & Alexander Inc","4923 Carey Ave","Saint Louis","Saint Louis City","MO",63104,"314-787-1588","314-858-4832","bsama@cox.net","http://www.alexanderalexanderinc.com"
"Alyce","Arias","Fairbanks Scales","3196 S Rider Trl","Stockton","San Joaquin","CA",95207,"209-317-1801","209-242-7022","alyce@arias.org","http://www.fairbanksscales.com"
"Heike","Berganza","Cali Sportswear Cutting Dept","3 Railway Ave #75","Little Falls","Passaic","NJ","07424","973-936-5095","973-822-8827","heike@gmail.com","http://www.calisportswearcuttingdept.com"
"Carey","Dopico","Garofani, John Esq","87393 E Highland Rd","Indianapolis","Marion","IN",46220,"
 317-578-2453","317-441-5848","carey_dopico@dopico.org","http://www.garofanijohnesq.com"
"Dottie","Hellickson","Thompson Fabricating Co","67 E Chestnut Hill Rd","Seattle","King","WA",98133,"206-540-6076","206-295-5631","dottie@hellickson.org","http://www.thompsonfabricatingco.com"
"Deandrea","Hughey","Century 21 Krall Real Estate","33 Lewis Rd #46","Burlington","Alamance","NC",27215,"336-822-7652","336-467-3095","deandrea@yahoo.com","http://www.centurykrallrealestate.com"
"Kimberlie","Duenas","Mid Contntl Rlty & Prop Mgmt","8100 Jacksonville Rd #7","Hays","Ellis","KS",67601,"785-629-8542","785-616-1685","kimberlie_duenas@yahoo.com","http://www.midcontntlrltypropmgmt.com"
"Martina","Staback","Ace Signs Inc","7 W Wabansia Ave #227","Orlando","Orange","FL",32822,"407-471-6908","407-429-2145","martina_staback@staback.com","http://www.acesignsinc.com"
"Skye","Fillingim","Rodeway Inn","25 Minters Chapel Rd #9","Minneapolis","Hennepin","MN",55401,"612-508-2655","612-664-6304","skye_fillingi
 m@yahoo.com","http://www.rodewayinn.com"
"Jade","Farrar","Bonnet & Daughter","6882 Torresdale Ave","Columbia","Richland","SC",29201,"803-352-5387","803-975-3405","jade.farrar@yahoo.com","http://www.bonnetdaughter.com"
"Charlene","Hamilton","Oshins & Gibbons","985 E 6th Ave","Santa Rosa","Sonoma","CA",95407,"707-300-1771","707-821-8037","charlene.hamilton@hotmail.com","http://www.oshinsgibbons.com"
"Geoffrey","Acey","Price Business Services","7 West Ave #1","Palatine","Cook","IL",60067,"847-222-1734","847-556-2909","geoffrey@gmail.com","http://www.pricebusinessservices.com"
"Stevie","Westerbeck","Wise, Dennis W Md","26659 N 13th St","Costa Mesa","Orange","CA",92626,"949-867-4077","949-903-3898","stevie.westerbeck@yahoo.com","http://www.wisedenniswmd.com"
"Pamella","Fortino","Super 8 Motel","669 Packerland Dr #1438","Denver","Denver","CO",80212,"303-404-2210","303-794-1341","pamella@fortino.com","http://www.supermotel.com"
"Harrison","Haufler","John Wagner Associates","759 Eldora St",
 "New Haven","New Haven","CT","06515","203-801-6193","203-801-8497","hhaufler@hotmail.com","http://www.johnwagnerassociates.com"
"Johnna","Engelberg","Thrifty Oil Co","5 S Colorado Blvd #449","Bothell","Snohomish","WA",98021,"425-986-7573","425-700-3751","jengelberg@engelberg.org","http://www.thriftyoilco.com"
"Buddy","Cloney","Larkfield Photo","944 Gaither Dr","Strongsville","Cuyahoga","OH",44136,"440-989-5826","440-327-2093","buddy.cloney@yahoo.com","http://www.larkfieldphoto.com"
"Dalene","Riden","Silverman Planetarium","66552 Malone Rd","Plaistow","Rockingham","NH","03865","603-315-6839","603-745-7497","dalene.riden@aol.com","http://www.silvermanplanetarium.com"
"Jerry","Zurcher","J & F Lumber","77 Massillon Rd #822","Satellite Beach","Brevard","FL",32937,"321-518-5938","321-597-2159","jzurcher@zurcher.org","http://www.jflumber.com"
"Haydee","Denooyer","Cleaning Station Inc","25346 New Rd","New York","New York","NY",10016,"212-792-8658","212-782-3493","hdenooyer@denooyer.org","ht
 tp://www.cleaningstationinc.com"
"Joseph","Cryer","Ames Stationers","60 Fillmore Ave","Huntington Beach","Orange","CA",92647,"714-584-2237","714-698-2170","joseph_cryer@cox.net","http://www.amesstationers.com"
"Deonna","Kippley","Midas Muffler Shops","57 Haven Ave #90","Southfield","Oakland","MI",48075,"248-913-4677","248-793-4966","deonna_kippley@hotmail.com","http://www.midasmufflershops.com"
"Raymon","Calvaresi","Seaboard Securities Inc","6538 E Pomona St #60","Indianapolis","Marion","IN",46222,"317-825-4724","317-342-1532","raymon.calvaresi@gmail.com","http://www.seaboardsecuritiesinc.com"
"Alecia","Bubash","Petersen, James E Esq","6535 Joyce St","Wichita Falls","Wichita","TX",76301,"940-276-7922","940-302-3036","alecia@aol.com","http://www.petersenjameseesq.com"
"Ma","Layous","Development Authority","78112 Morris Ave","North Haven","New Haven","CT","06473","203-721-3388","203-564-1543","mlayous@hotmail.com","http://www.developmentauthority.com"
"Detra","Coyier","Schott Fiber Op
 tics Inc","96950 Hidden Ln","Aberdeen","Harford","MD",21001,"410-739-9277","410-259-2118","detra@aol.com","http://www.schottfiberopticsinc.com"
"Terrilyn","Rodeigues","Stuart J Agins","3718 S Main St","New Orleans","Orleans","LA",70130,"504-463-4384","504-635-8518","terrilyn.rodeigues@cox.net","http://www.stuartjagins.com"
"Salome","Lacovara","Mitsumi Electronics Corp","9677 Commerce Dr","Richmond","Richmond City","VA",23219,"804-550-5097","804-858-1011","slacovara@gmail.com","http://www.mitsumielectronicscorp.com"
"Garry","Keetch","Italian Express Franchise Corp","5 Green Pond Rd #4","Southampton","Bucks","PA",18966,"215-979-8776","215-846-9046","garry_keetch@hotmail.com","http://www.italianexpressfranchisecorp.com"
"Matthew","Neither","American Council On Sci & Hlth","636 Commerce Dr #42","Shakopee","Scott","MN",55379,"952-651-7597","952-906-4597","mneither@yahoo.com","http://www.americancouncilonscihlth.com"
"Theodora","Restrepo","Kleri, Patricia S Esq","42744 Hamann Industrial P
 ky #82","Miami","Miami-Dade","FL",33136,"305-936-8226","305-573-1085","theodora.restrepo@restrepo.com","http://www.kleripatriciasesq.com"
"Noah","Kalafatis","Twiggs Abrams Blanchard","1950 5th Ave","Milwaukee","Milwaukee","WI",53209,"414-263-5287","414-660-9766","noah.kalafatis@aol.com","http://www.twiggsabramsblanchard.com"
"Carmen","Sweigard","Maui Research & Technology Pk","61304 N French Rd","Somerset","Somerset","NJ","08873","732-941-2621","732-445-6940","csweigard@sweigard.com","http://www.mauiresearchtechnologypk.com"
"Lavonda","Hengel","Bradley Nameplate Corp","87 Imperial Ct #79","Fargo","Cass","ND",58102,"701-898-2154","701-421-7080","lavonda@cox.net","http://www.bradleynameplatecorp.com"
"Junita","Stoltzman","Geonex Martel Inc","94 W Dodge Rd","Carson City","Carson City","NV",89701,"775-638-9963","775-578-1214","junita@aol.com","http://www.geonexmartelinc.com"
"Herminia","Nicolozakes","Sea Island Div Of Fstr Ind Inc","4 58th St #3519","Scottsdale","Maricopa","AZ",85254,"6
 02-954-5141","602-304-6433","herminia@nicolozakes.org","http://www.seaislanddivoffstrindinc.com"
"Casie","Good","Papay, Debbie J Esq","5221 Bear Valley Rd","Nashville","Davidson","TN",37211,"615-390-2251","615-825-4297","casie.good@aol.com","http://www.papaydebbiejesq.com"
"Reena","Maisto","Lane Promotions","9648 S Main","Salisbury","Wicomico","MD",21801,"410-351-1863","410-951-2667","reena@hotmail.com","http://www.lanepromotions.com"
"Mirta","Mallett","Stephen Kennerly Archts Inc Pc","7 S San Marcos Rd","New York","New York","NY",10004,"212-870-1286","212-745-6948","mirta_mallett@gmail.com","http://www.stephenkennerlyarchtsincpc.com"
"Cathrine","Pontoriero","Business Systems Of Wis Inc","812 S Haven St","Amarillo","Randall","TX",79109,"806-703-1435","806-558-5848","cathrine.pontoriero@pontoriero.com","http://www.businesssystemsofwisinc.com"
"Filiberto","Tawil","Flash, Elena Salerno Esq","3882 W Congress St #799","Los Angeles","Los Angeles","CA",90016,"323-765-2528","323-842-8226","
 ftawil@hotmail.com","http://www.flashelenasalernoesq.com"
"Raul","Upthegrove","Neeley, Gregory W Esq","4 E Colonial Dr","La Mesa","San Diego","CA",91942,"619-509-5282","619-666-4765","rupthegrove@yahoo.com","http://www.neeleygregorywesq.com"
"Sarah","Candlish","Alabama Educational Tv Comm","45 2nd Ave #9759","Atlanta","Fulton","GA",30328,"770-732-1194","770-531-2842","sarah.candlish@gmail.com","http://www.alabamaeducationaltvcomm.com"
"Lucy","Treston","Franz Inc","57254 Brickell Ave #372","Worcester","Worcester","MA","01602","508-769-5250","508-502-5634","lucy@cox.net","http://www.franzinc.com"
"Judy","Aquas","Plantation Restaurant","8977 Connecticut Ave Nw #3","Niles","Berrien","MI",49120,"269-756-7222","269-431-9464","jaquas@aquas.com","http://www.plantationrestaurant.com"
"Yvonne","Tjepkema","Radio Communications Co","9 Waydell St","Fairfield","Essex","NJ","07004","973-714-1721","973-976-8627","yvonne.tjepkema@hotmail.com","http://www.radiocommunicationsco.com"
"Kayleigh","Lace",
 "Dentalaw Divsn Hlth Care","43 Huey P Long Ave","Lafayette","Lafayette","LA",70508,"337-740-9323","337-751-2326","kayleigh.lace@yahoo.com","http://www.dentalawdivsnhlthcare.com"
"Felix","Hirpara","American Speedy Printing Ctrs","7563 Cornwall Rd #4462","Denver","Lancaster","PA",17517,"717-491-5643","717-583-1497","felix_hirpara@cox.net","http://www.americanspeedyprintingctrs.com"
"Tresa","Sweely","Grayson, Grant S Esq","22 Bridle Ln","Valley Park","Saint Louis","MO",63088,"314-359-9566","314-231-3514","tresa_sweely@hotmail.com","http://www.graysongrantsesq.com"
"Kristeen","Turinetti","Jeanerette Middle School","70099 E North Ave","Arlington","Tarrant","TX",76013,"817-213-8851","817-947-9480","kristeen@gmail.com","http://www.jeanerettemiddleschool.com"
"Jenelle","Regusters","Haavisto, Brian F Esq","3211 E Northeast Loop","Tampa","Hillsborough","FL",33619,"813-932-8715","813-357-7296","jregusters@regusters.com","http://www.haavistobrianfesq.com"
"Renea","Monterrubio","Wmmt Radio Stati
 on","26 Montgomery St","Atlanta","Fulton","GA",30328,"770-679-4752","770-930-9967","renea@hotmail.com","http://www.wmmtradiostation.com"
"Olive","Matuszak","Colony Paints Sales Ofc & Plnt","13252 Lighthouse Ave","Cathedral City","Riverside","CA",92234,"760-938-6069","760-745-2649","olive@aol.com","http://www.colonypaintssalesofcplnt.com"
"Ligia","Reiber","Floral Expressions","206 Main St #2804","Lansing","Ingham","MI",48933,"517-906-1108","517-747-7664","lreiber@cox.net","http://www.floralexpressions.com"
"Christiane","Eschberger","Casco Services Inc","96541 W Central Blvd","Phoenix","Maricopa","AZ",85034,"602-390-4944","602-330-6894","christiane.eschberger@yahoo.com","http://www.cascoservicesinc.com"
"Goldie","Schirpke","Reuter, Arthur C Jr","34 Saint George Ave #2","Bangor","Penobscot","ME","04401","207-295-7569","207-748-3722","goldie.schirpke@yahoo.com","http://www.reuterarthurcjr.com"
"Loreta","Timenez","Kaminski, Katherine Andritsaki","47857 Coney Island Ave","Clinton","Prince
  Georges","MD",20735,"301-696-6420","301-392-6698","loreta.timenez@hotmail.com","http://www.kaminskikatherineandritsaki.com"
"Fabiola","Hauenstein","Sidewinder Products Corp","8573 Lincoln Blvd","York","York","PA",17404,"717-809-3119","717-344-2804","fabiola.hauenstein@hauenstein.org","http://www.sidewinderproductscorp.com"
"Amie","Perigo","General Foam Corporation","596 Santa Maria Ave #7913","Mesquite","Dallas","TX",75150,"972-419-7946","972-898-1033","amie.perigo@yahoo.com","http://www.generalfoamcorporation.com"
"Raina","Brachle","Ikg Borden Divsn Harsco Corp","3829 Ventura Blvd","Butte","Silver Bow","MT",59701,"406-318-1515","406-374-7752","raina.brachle@brachle.org","http://www.ikgbordendivsnharscocorp.com"
"Erinn","Canlas","Anchor Computer Inc","13 S Hacienda Dr","Livingston","Essex","NJ","07039","973-767-3008","973-563-9502","erinn.canlas@canlas.com","http://www.anchorcomputerinc.com"
"Cherry","Lietz","Sebring & Co","40 9th Ave Sw #91","Waterford","Oakland","MI",48329,"248-9
 80-6904","248-697-7722","cherry@lietz.com","http://www.sebringco.com"
"Kattie","Vonasek","H A C Farm Lines Co Optv Assoc","2845 Boulder Crescent St","Cleveland","Cuyahoga","OH",44103,"216-923-3715","216-270-9653","kattie@vonasek.org","http://www.hacfarmlinescooptvassoc.com"
"Lilli","Scriven","Hunter, John J Esq","33 State St","Abilene","Taylor","TX",79601,"325-631-1560","325-667-7868","lilli@aol.com","http://www.hunterjohnjesq.com"
"Whitley","Tomasulo","Freehold Fence Co","2 S 15th St","Fort Worth","Tarrant","TX",76107,"817-526-4408","817-819-7799","whitley.tomasulo@aol.com","http://www.freeholdfenceco.com"
"Barbra","Adkin","Binswanger","4 Kohler Memorial Dr","Brooklyn","Kings","NY",11230,"718-201-3751","718-732-9475","badkin@hotmail.com","http://www.binswanger.com"
"Hermila","Thyberg","Chilton Malting Co","1 Rancho Del Mar Shopping C","Providence","Providence","RI","02903","401-893-4882","401-885-7681","hermila_thyberg@hotmail.com","http://www.chiltonmaltingco.com"
"Jesusita","Flis
 ter","Schoen, Edward J Jr","3943 N Highland Ave","Lancaster","Lancaster","PA",17601,"717-885-9118","717-686-7564","jesusita.flister@hotmail.com","http://www.schoenedwardjjr.com"
"Caitlin","Julia","Helderman, Seymour Cpa","5 Williams St","Johnston","Providence","RI","02919","401-948-4982","401-552-9059","caitlin.julia@julia.org","http://www.heldermanseymourcpa.com"
"Roosevelt","Hoffis","Denbrook, Myron","60 Old Dover Rd","Hialeah","Miami-Dade","FL",33014,"305-622-4739","305-302-1135","roosevelt.hoffis@aol.com","http://www.denbrookmyron.com"
"Helaine","Halter","Lippitt, Mike","8 Sheridan Rd","Jersey City","Hudson","NJ","07304","201-832-4168","201-412-3040","hhalter@yahoo.com","http://www.lippittmike.com"
"Lorean","Martabano","Hiram, Hogg P Esq","85092 Southern Blvd","San Antonio","Bexar","TX",78204,"210-856-4979","210-634-2447","lorean.martabano@hotmail.com","http://www.hiramhoggpesq.com"
"France","Buzick","In Travel Agency","64 Newman Springs Rd E","Brooklyn","Kings","NY",11219,"718-
 478-8504","718-853-3740","france.buzick@yahoo.com","http://www.intravelagency.com"
"Justine","Ferrario","Newhart Foods Inc","48 Stratford Ave","Pomona","Los Angeles","CA",91768,"909-993-3242","909-631-5703","jferrario@hotmail.com","http://www.newhartfoodsinc.com"
"Adelina","Nabours","Courtyard By Marriott","80 Pittsford Victor Rd #9","Cleveland","Cuyahoga","OH",44103,"216-230-4892","216-937-5320","adelina_nabours@gmail.com","http://www.courtyardbymarriott.com"
"Derick","Dhamer","Studer, Eugene A Esq","87163 N Main Ave","New York","New York","NY",10013,"212-304-4515","212-225-9676","ddhamer@cox.net","http://www.studereugeneaesq.com"
"Jerry","Dallen","Seashore Supply Co Waretown","393 Lafayette Ave","Richmond","Richmond City","VA",23219,"804-762-9576","804-808-9574","jerry.dallen@yahoo.com","http://www.seashoresupplycowaretown.com"
"Leota","Ragel","Mayar Silk Inc","99 5th Ave #33","Trion","Chattooga","GA",30753,"706-221-4243","706-616-5131","leota.ragel@gmail.com","http://www.mayarsil
 kinc.com"
"Jutta","Amyot","National Medical Excess Corp","49 N Mays St","Broussard","Lafayette","LA",70518,"337-515-1438","337-991-8070","jamyot@hotmail.com","http://www.nationalmedicalexcesscorp.com"
"Aja","Gehrett","Stero Company","993 Washington Ave","Nutley","Essex","NJ","07110","973-544-2677","973-986-4456","aja_gehrett@hotmail.com","http://www.sterocompany.com"
"Kirk","Herritt","Hasting, H Duane Esq","88 15th Ave Ne","Vestal","Broome","NY",13850,"607-407-3716","607-350-7690","kirk.herritt@aol.com","http://www.hastinghduaneesq.com"
"Leonora","Mauson","Insty Prints","3381 E 40th Ave","Passaic","Passaic","NJ","07055","973-412-2995","973-355-2120","leonora@yahoo.com","http://www.instyprints.com"
"Winfred","Brucato","Glenridge Manor Mobile Home Pk","201 Ridgewood Rd","Moscow","Latah","ID",83843,"208-252-4552","208-793-4108","winfred_brucato@hotmail.com","http://www.glenridgemanormobilehomepk.com"
"Tarra","Nachor","Circuit Solution Inc","39 Moccasin Dr","San Francisco","San Francisc
 o","CA",94104,"415-411-1775","415-284-2730","tarra.nachor@cox.net","http://www.circuitsolutioninc.com"
"Corinne","Loder","Local Office","4 Carroll St","North Attleboro","Bristol","MA","02760","508-942-4186","508-618-7826","corinne@loder.org","http://www.localoffice.com"
"Dulce","Labreche","Lee Kilkelly Paulson & Kabaker","9581 E Arapahoe Rd","Rochester","Oakland","MI",48307,"248-357-8718","248-811-5696","dulce_labreche@yahoo.com","http://www.leekilkellypaulsonkabaker.com"
"Kate","Keneipp","Davis, Maxon R Esq","33 N Michigan Ave","Green Bay","Brown","WI",54301,"920-353-6377","920-355-1610","kate_keneipp@yahoo.com","http://www.davismaxonresq.com"
"Kaitlyn","Ogg","Garrison, Paul E Esq","2 S Biscayne Blvd","Baltimore","Baltimore City","MD",21230,"410-665-4903","410-773-3862","kaitlyn.ogg@gmail.com","http://www.garrisonpauleesq.com"
"Sherita","Saras","Black History Resource Center","8 Us Highway 22","Colorado Springs","El Paso","CO",80937,"719-669-1664","719-547-9543","sherita.saras@cox.
 net","http://www.blackhistoryresourcecenter.com"
"Lashawnda","Stuer","Rodriguez, J Christopher Esq","7422 Martin Ave #8","Toledo","Lucas","OH",43607,"419-588-8719","419-399-1744","lstuer@cox.net","http://www.rodriguezjchristopheresq.com"
"Ernest","Syrop","Grant Family Health Center","94 Chase Rd","Hyattsville","Prince Georges","MD",20785,"301-998-9644","301-257-4883","ernest@cox.net","http://www.grantfamilyhealthcenter.com"
"Nobuko","Halsey","Goeman Wood Products Inc","8139 I Hwy 10 #92","New Bedford","Bristol","MA","02745","508-855-9887","508-897-7916","nobuko.halsey@yahoo.com","http://www.goemanwoodproductsinc.com"
"Lavonna","Wolny","Linhares, Kenneth A Esq","5 Cabot Rd","Mc Lean","Fairfax","VA",22102,"703-483-1970","703-892-2914","lavonna.wolny@hotmail.com","http://www.linhareskennethaesq.com"
"Lashaunda","Lizama","Earnhardt Printing","3387 Ryan Dr","Hanover","Anne Arundel","MD",21076,"410-678-2473","410-912-6032","llizama@cox.net","http://www.earnhardtprinting.com"
"Mariann","Bi
 lden","H P G Industrys Inc","3125 Packer Ave #9851","Austin","Travis","TX",78753,"512-223-4791","512-742-1149","mariann.bilden@aol.com","http://www.hpgindustrysinc.com"
"Helene","Rodenberger","Bailey Transportation Prod Inc","347 Chestnut St","Peoria","Maricopa","AZ",85381,"623-461-8551","623-426-4907","helene@aol.com","http://www.baileytransportationprodinc.com"
"Roselle","Estell","Mcglynn Bliss Pc","8116 Mount Vernon Ave","Bucyrus","Crawford","OH",44820,"419-571-5920","419-488-6648","roselle.estell@hotmail.com","http://www.mcglynnblisspc.com"
"Samira","Heintzman","Mutual Fish Co","8772 Old County Rd #5410","Kent","King","WA",98032,"206-311-4137","206-923-6042","sheintzman@hotmail.com","http://www.mutualfishco.com"
"Margart","Meisel","Yeates, Arthur L Aia","868 State St #38","Cincinnati","Hamilton","OH",45251,"513-617-2362","513-747-9603","margart_meisel@yahoo.com","http://www.yeatesarthurlaia.com"
"Kristofer","Bennick","Logan, Ronald J Esq","772 W River Dr","Bloomington","Monroe",
 "IN",47404,"812-368-1511","812-442-8544","kristofer.bennick@yahoo.com","http://www.loganronaldjesq.com"
"Weldon","Acuff","Advantage Martgage Company","73 W Barstow Ave","Arlington Heights","Cook","IL",60004,"847-353-2156","847-613-5866","wacuff@gmail.com","http://www.advantagemartgagecompany.com"
"Shalon","Shadrick","Germer And Gertz Llp","61047 Mayfield Ave","Brooklyn","Kings","NY",11223,"718-232-2337","718-394-4974","shalon@cox.net","http://www.germerandgertzllp.com"
"Denise","Patak","Spence Law Offices","2139 Santa Rosa Ave","Orlando","Orange","FL",32801,"407-446-4358","407-808-3254","denise@patak.org","http://www.spencelawoffices.com"
"Louvenia","Beech","John Ortiz Nts Therapy Center","598 43rd St","Beverly Hills","Los Angeles","CA",90210,"310-820-2117","310-652-2379","louvenia.beech@beech.com","http://www.johnortizntstherapycenter.com"
"Audry","Yaw","Mike Uchrin Htg & Air Cond Inc","70295 Pioneer Ct","Brandon","Hillsborough","FL",33511,"813-797-4816","813-744-7100","audry.yaw@y
 aw.org","http://www.mikeuchrinhtgaircondinc.com"
"Kristel","Ehmann","Mccoy, Joy Reynolds Esq","92899 Kalakaua Ave","El Paso","El Paso","TX",79925,"915-452-1290","915-300-6100","kristel.ehmann@aol.com","http://www.mccoyjoyreynoldsesq.com"
"Vincenza","Zepp","Kbor 1600 Am","395 S 6th St #2","El Cajon","San Diego","CA",92020,"619-603-5125","619-935-6661","vzepp@gmail.com","http://www.kboram.com"
"Elouise","Gwalthney","Quality Inn Northwest","9506 Edgemore Ave","Bladensburg","Prince Georges","MD",20710,"301-841-5012","301-591-3034","egwalthney@yahoo.com","http://www.qualityinnnorthwest.com"
"Venita","Maillard","Wallace Church Assoc Inc","72119 S Walker Ave #63","Anaheim","Orange","CA",92801,"714-523-6653","714-663-9740","venita_maillard@gmail.com","http://www.wallacechurchassocinc.com"
"Kasandra","Semidey","Can Tron","369 Latham St #500","Saint Louis","Saint Louis City","MO",63102,"314-732-9131","314-697-3652","kasandra_semidey@semidey.com","http://www.cantron.com"
"Xochitl","Discipio","
 Ravaal Enterprises Inc","3158 Runamuck Pl","Round Rock","Williamson","TX",78664,"512-233-1831","512-942-3411","xdiscipio@gmail.com","http://www.ravaalenterprisesinc.com"
"Maile","Linahan","Thompson Steel Company Inc","9 Plainsboro Rd #598","Greensboro","Guilford","NC",27409,"336-670-2640","336-364-6037","mlinahan@yahoo.com","http://www.thompsonsteelcompanyinc.com"
"Krissy","Rauser","Anderson, Mark A Esq","8728 S Broad St","Coram","Suffolk","NY",11727,"631-443-4710","631-288-2866","krauser@cox.net","http://www.andersonmarkaesq.com"
"Pete","Dubaldi","Womack & Galich","2215 Prosperity Dr","Lyndhurst","Bergen","NJ","07071","201-825-2514","201-749-8866","pdubaldi@hotmail.com","http://www.womackgalich.com"
"Linn","Paa","Valerie & Company","1 S Pine St","Memphis","Shelby","TN",38112,"901-412-4381","901-573-9024","linn_paa@paa.com","http://www.valeriecompany.com"
"Paris","Wide","Gehring Pumps Inc","187 Market St","Atlanta","Fulton","GA",30342,"404-505-4445","404-607-8435","paris@hotmail.com
 ","http://www.gehringpumpsinc.com"
"Wynell","Dorshorst","Haehnel, Craig W Esq","94290 S Buchanan St","Pacifica","San Mateo","CA",94044,"650-473-1262","650-749-9879","wynell_dorshorst@dorshorst.org","http://www.haehnelcraigwesq.com"
"Quentin","Birkner","Spoor Behrins Campbell & Young","7061 N 2nd St","Burnsville","Dakota","MN",55337,"952-702-7993","952-314-5871","qbirkner@aol.com","http://www.spoorbehrinscampbellyoung.com"
"Regenia","Kannady","Ken Jeter Store Equipment Inc","10759 Main St","Scottsdale","Maricopa","AZ",85260,"480-726-1280","480-205-5121","regenia.kannady@cox.net","http://www.kenjeterstoreequipmentinc.com"
"Sheron","Louissant","Potter, Brenda J Cpa","97 E 3rd St #9","Long Island City","Queens","NY",11101,"718-976-8610","718-613-9994","sheron@aol.com","http://www.potterbrendajcpa.com"
"Izetta","Funnell","Baird Kurtz & Dobson","82 Winsor St #54","Atlanta","Dekalb","GA",30340,"770-844-3447","770-584-4119","izetta.funnell@hotmail.com","http://www.bairdkurtzdobson.com"
"Rod
 olfo","Butzen","Minor, Cynthia A Esq","41 Steel Ct","Northfield","Rice","MN",55057,"507-210-3510","507-590-5237","rodolfo@hotmail.com","http://www.minorcynthiaaesq.com"
"Zona","Colla","Solove, Robert A Esq","49440 Dearborn St","Norwalk","Fairfield","CT","06854","203-461-1949","203-938-2557","zona@hotmail.com","http://www.soloverobertaesq.com"
"Serina","Zagen","Mark Ii Imports Inc","7 S Beverly Dr","Fort Wayne","Allen","IN",46802,"260-273-3725","260-382-4869","szagen@aol.com","http://www.markiiimportsinc.com"
"Paz","Sahagun","White Sign Div Ctrl Equip Co","919 Wall Blvd","Meridian","Lauderdale","MS",39307,"601-927-8287","601-249-4511","paz_sahagun@cox.net","http://www.whitesigndivctrlequipco.com"
"Markus","Lukasik","M & M Store Fixtures Co Inc","89 20th St E #779","Sterling Heights","Macomb","MI",48310,"586-970-7380","586-247-1614","markus@yahoo.com","http://www.mmstorefixturescoinc.com"
"Jaclyn","Bachman","Judah Caster & Wheel Co","721 Interstate 45 S","Colorado Springs","El Paso","
 CO",80919,"719-853-3600","719-223-2074","jaclyn@aol.com","http://www.judahcasterwheelco.com"
"Cyril","Daufeldt","Galaxy International Inc","3 Lawton St","New York","New York","NY",10013,"212-745-8484","212-422-5427","cyril_daufeldt@daufeldt.com","http://www.galaxyinternationalinc.com"
"Gayla","Schnitzler","Sigma Corp Of America","38 Pleasant Hill Rd","Hayward","Alameda","CA",94545,"510-686-3407","510-441-4055","gschnitzler@gmail.com","http://www.sigmacorpofamerica.com"
"Erick","Nievas","Soward, Anne Esq","45 E Acacia Ct","Chicago","Cook","IL",60624,"773-704-9903","773-359-6109","erick_nievas@aol.com","http://www.sowardanneesq.com"
"Jennie","Drymon","Osborne, Michelle M Esq","63728 Poway Rd #1","Scranton","Lackawanna","PA",18509,"570-218-4831","570-868-8688","jennie@cox.net","http://www.osbornemichellemesq.com"
"Mitsue","Scipione","Students In Free Entrprs Natl","77 222 Dr","Oroville","Butte","CA",95965,"530-986-9272","530-399-3254","mscipione@scipione.com","http://www.studentsinfree
 entrprsnatl.com"
"Ciara","Ventura","Johnson, Robert M Esq","53 W Carey St","Port Jervis","Orange","NY",12771,"845-823-8877","845-694-7919","cventura@yahoo.com","http://www.johnsonrobertmesq.com"
"Galen","Cantres","Del Charro Apartments","617 Nw 36th Ave","Brook Park","Cuyahoga","OH",44142,"216-600-6111","216-871-6876","galen@yahoo.com","http://www.delcharroapartments.com"
"Truman","Feichtner","Legal Search Inc","539 Coldwater Canyon Ave","Bloomfield","Essex","NJ","07003","973-852-2736","973-473-5108","tfeichtner@yahoo.com","http://www.legalsearchinc.com"
"Gail","Kitty","Service Supply Co Inc","735 Crawford Dr","Anchorage","Anchorage","AK",99501,"907-435-9166","907-770-3542","gail@kitty.com","http://www.servicesupplycoinc.com"
"Dalene","Schoeneck","Sameshima, Douglas J Esq","910 Rahway Ave","Philadelphia","Philadelphia","PA",19102,"215-268-1275","215-380-8820","dalene@schoeneck.org","http://www.sameshimadouglasjesq.com"
"Gertude","Witten","Thompson, John Randolph Jr","7 Tarrytown Rd"
 ,"Cincinnati","Hamilton","OH",45217,"513-977-7043","513-863-9471","gertude.witten@gmail.com","http://www.thompsonjohnrandolphjr.com"
"Lizbeth","Kohl","E T Balancing Co Inc","35433 Blake St #588","Gardena","Los Angeles","CA",90248,"310-699-1222","310-955-5788","lizbeth@yahoo.com","http://www.etbalancingcoinc.com"
"Glenn","Berray","Griswold, John E Esq","29 Cherry St #7073","Des Moines","Polk","IA",50315,"515-370-7348","515-372-1738","gberray@gmail.com","http://www.griswoldjohneesq.com"
"Lashandra","Klang","Acqua Group","810 N La Brea Ave","King of Prussia","Montgomery","PA",19406,"610-809-1818","610-378-7332","lashandra@yahoo.com","http://www.acquagroup.com"
"Lenna","Newville","Brooks, Morris J Jr","987 Main St","Raleigh","Wake","NC",27601,"919-623-2524","919-254-5987","lnewville@newville.com","http://www.brooksmorrisjjr.com"
"Laurel","Pagliuca","Printing Images Corp","36 Enterprise St Se","Richland","Benton","WA",99352,"509-695-5199","509-595-6485","laurel@yahoo.com","http://www.pri
 ntingimagescorp.com"
"Mireya","Frerking","Roberts Supply Co Inc","8429 Miller Rd","Pelham","Westchester","NY",10803,"914-868-5965","914-883-3061","mireya.frerking@hotmail.com","http://www.robertssupplycoinc.com"
"Annelle","Tagala","Vico Products Mfg Co","5 W 7th St","Parkville","Baltimore","MD",21234,"410-757-1035","410-234-2267","annelle@yahoo.com","http://www.vicoproductsmfgco.com"
"Dean","Ketelsen","J M Custom Design Millwork","2 Flynn Rd","Hicksville","Nassau","NY",11801,"516-847-4418","516-732-6649","dean_ketelsen@gmail.com","http://www.jmcustomdesignmillwork.com"
"Levi","Munis","Farrell & Johnson Office Equip","2094 Ne 36th Ave","Worcester","Worcester","MA","01603","508-456-4907","508-658-7802","levi.munis@gmail.com","http://www.farrelljohnsonofficeequip.com"
"Sylvie","Ryser","Millers Market & Deli","649 Tulane Ave","Tulsa","Tulsa","OK",74105,"918-644-9555","918-565-1706","sylvie@aol.com","http://www.millersmarketdeli.com"
"Sharee","Maile","Holiday Inn Naperville","2094 Montou
 r Blvd","Muskegon","Muskegon","MI",49442,"231-467-9978","231-265-6940","sharee_maile@aol.com","http://www.holidayinnnaperville.com"
"Cordelia","Storment","Burrows, Jon H Esq","393 Hammond Dr","Lafayette","Lafayette","LA",70506,"337-566-6001","337-255-3427","cordelia_storment@aol.com","http://www.burrowsjonhesq.com"
"Mollie","Mcdoniel","Dock Seal Specialty","8590 Lake Lizzie Dr","Bowling Green","Wood","OH",43402,"419-975-3182","419-417-4674","mollie_mcdoniel@yahoo.com","http://www.docksealspecialty.com"
"Brett","Mccullan","Five Star Limousines Of Tx Inc","87895 Concord Rd","La Mesa","San Diego","CA",91942,"619-461-9984","619-727-3892","brett.mccullan@mccullan.com","http://www.fivestarlimousinesoftxinc.com"
"Teddy","Pedrozo","Barkan, Neal J Esq","46314 Route 130","Bridgeport","Fairfield","CT","06610","203-892-3863","203-918-3939","teddy_pedrozo@aol.com","http://www.barkannealjesq.com"
"Tasia","Andreason","Campbell, Robert A","4 Cowesett Ave","Kearny","Hudson","NJ","07032","201-920-900
 2","201-969-7063","tasia_andreason@yahoo.com","http://www.campbellroberta.com"
"Hubert","Walthall","Dee, Deanna","95 Main Ave #2","Barberton","Summit","OH",44203,"330-903-1345","330-566-8898","hubert@walthall.org","http://www.deedeanna.com"
"Arthur","Farrow","Young, Timothy L Esq","28 S 7th St #2824","Englewood","Bergen","NJ","07631","201-238-5688","201-772-4377","arthur.farrow@yahoo.com","http://www.youngtimothylesq.com"
"Vilma","Berlanga","Wells, D Fred Esq","79 S Howell Ave","Grand Rapids","Kent","MI",49546,"616-737-3085","616-568-4113","vberlanga@berlanga.com","http://www.wellsdfredesq.com"
"Billye","Miro","Gray, Francine H Esq","36 Lancaster Dr Se","Pearl","Rankin","MS",39208,"601-567-5386","601-637-5479","billye_miro@cox.net","http://www.grayfrancinehesq.com"
"Glenna","Slayton","Toledo Iv Care","2759 Livingston Ave","Memphis","Shelby","TN",38118,"901-640-9178","901-869-4314","glenna_slayton@cox.net","http://www.toledoivcare.com"
"Mitzie","Hudnall","Cangro Transmission Co","17 
 Jersey Ave","Englewood","Arapahoe","CO",80110,"303-402-1940","303-997-7760","mitzie_hudnall@yahoo.com","http://www.cangrotransmissionco.com"
"Bernardine","Rodefer","Sat Poly Inc","2 W Grand Ave","Memphis","Shelby","TN",38112,"901-901-4726","901-739-5892","bernardine_rodefer@yahoo.com","http://www.satpolyinc.com"
"Staci","Schmaltz","Midwest Contracting & Mfg Inc","18 Coronado Ave #563","Pasadena","Los Angeles","CA",91106,"626-866-2339","626-293-7678","staci_schmaltz@aol.com","http://www.midwestcontractingmfginc.com"
"Nichelle","Meteer","Print Doctor","72 Beechwood Ter","Chicago","Cook","IL",60657,"773-225-9985","773-857-2231","nichelle_meteer@meteer.com","http://www.printdoctor.com"
"Janine","Rhoden","Nordic Group Inc","92 Broadway","Astoria","Queens","NY",11103,"718-228-5894","718-728-5051","jrhoden@yahoo.com","http://www.nordicgroupinc.com"
"Ettie","Hoopengardner","Jackson Millwork Co","39 Franklin Ave","Richland","Benton","WA",99352,"509-755-5393","509-847-3352","ettie.hoopengardn
 er@hotmail.com","http://www.jacksonmillworkco.com"
"Eden","Jayson","Harris Corporation","4 Iwaena St","Baltimore","Baltimore City","MD",21202,"410-890-7866","410-429-4888","eden_jayson@yahoo.com","http://www.harriscorporation.com"
"Lynelle","Auber","United Cerebral Palsy Of Ne Pa","32820 Corkwood Rd","Newark","Essex","NJ","07104","973-860-8610","973-605-6492","lynelle_auber@gmail.com","http://www.unitedcerebralpalsyofnepa.com"
"Merissa","Tomblin","One Day Surgery Center Inc","34 Raritan Center Pky","Bellflower","Los Angeles","CA",90706,"562-579-6900","562-719-7922","merissa.tomblin@gmail.com","http://www.onedaysurgerycenterinc.com"
"Golda","Kaniecki","Calaveras Prospect","6201 S Nevada Ave","Toms River","Ocean","NJ","08755","732-628-9909","732-617-5310","golda_kaniecki@yahoo.com","http://www.calaverasprospect.com"
"Catarina","Gleich","Terk, Robert E Esq","78 Maryland Dr #146","Denville","Morris","NJ","07834","973-210-3994","973-491-8723","catarina_gleich@hotmail.com","http://www.ter
 kroberteesq.com"
"Virgie","Kiel","Cullen, Terrence P Esq","76598 Rd  I 95 #1","Denver","Denver","CO",80216,"303-776-7548","303-845-5408","vkiel@hotmail.com","http://www.cullenterrencepesq.com"
"Jolene","Ostolaza","Central Die Casting Mfg Co Inc","1610 14th St Nw","Newport News","Newport News City","VA",23608,"757-682-7116","757-940-1741","jolene@yahoo.com","http://www.centraldiecastingmfgcoinc.com"
"Keneth","Borgman","Centerline Engineering","86350 Roszel Rd","Phoenix","Maricopa","AZ",85012,"602-919-4211","602-442-3092","keneth@yahoo.com","http://www.centerlineengineering.com"
"Rikki","Nayar","Targan & Kievit Pa","1644 Clove Rd","Miami","Miami-Dade","FL",33155,"305-968-9487","305-978-2069","rikki@nayar.com","http://www.targankievitpa.com"
"Elke","Sengbusch","Riley Riper Hollin & Colagreco","9 W Central Ave","Phoenix","Maricopa","AZ",85013,"602-896-2993","602-575-3457","elke_sengbusch@yahoo.com","http://www.rileyriperhollincolagreco.com"
"Hoa","Sarao","Kaplan, Joel S Esq","27846 Lafa
 yette Ave","Oak Hill","Volusia","FL",32759,"386-526-7800","386-599-7296","hoa@sarao.org","http://www.kaplanjoelsesq.com"
"Trinidad","Mcrae","Water Office","10276 Brooks St","San Francisco","San Francisco","CA",94105,"415-331-9634","415-419-1597","trinidad_mcrae@yahoo.com","http://www.wateroffice.com"
"Mari","Lueckenbach","Westbrooks, Nelson E Jr","1 Century Park E","San Diego","San Diego","CA",92110,"858-793-9684","858-228-5683","mari_lueckenbach@yahoo.com","http://www.westbrooksnelsonejr.com"
"Selma","Husser","Armon Communications","9 State Highway 57 #22","Jersey City","Hudson","NJ","07306","201-991-8369","201-772-7699","selma.husser@cox.net","http://www.armoncommunications.com"
"Antione","Onofrio","Jacobs & Gerber Inc","4 S Washington Ave","San Bernardino","San Bernardino","CA",92410,"909-430-7765","909-665-3223","aonofrio@onofrio.com","http://www.jacobsgerberinc.com"
"Luisa","Jurney","Forest Fire Laboratory","25 Se 176th Pl","Cambridge","Middlesex","MA","02138","617-365-2134","6
 17-544-2541","ljurney@hotmail.com","http://www.forestfirelaboratory.com"
"Clorinda","Heimann","Haughey, Charles Jr","105 Richmond Valley Rd","Escondido","San Diego","CA",92025,"760-291-5497","760-261-4786","clorinda.heimann@hotmail.com","http://www.haugheycharlesjr.com"
"Dick","Wenzinger","Wheaton Plastic Products","22 Spruce St #595","Gardena","Los Angeles","CA",90248,"310-510-9713","310-936-2258","dick@yahoo.com","http://www.wheatonplasticproducts.com"
"Ahmed","Angalich","Reese Plastics","2 W Beverly Blvd","Harrisburg","Dauphin","PA",17110,"717-528-8996","717-632-5831","ahmed.angalich@angalich.com","http://www.reeseplastics.com"
"Iluminada","Ohms","Nazette Marner Good Wendt","72 Southern Blvd","Mesa","Maricopa","AZ",85204,"480-293-2882","480-866-6544","iluminada.ohms@yahoo.com","http://www.nazettemarnergoodwendt.com"
"Joanna","Leinenbach","Levinson Axelrod Wheaton","1 Washington St","Lake Worth","Palm Beach","FL",33461,"561-470-4574","561-951-9734","joanna_leinenbach@hotmail.com",
 "http://www.levinsonaxelrodwheaton.com"
"Caprice","Suell","Egnor, W Dan Esq","90177 N 55th Ave","Nashville","Davidson","TN",37211,"615-246-1824","615-726-4537","caprice@aol.com","http://www.egnorwdanesq.com"
"Stephane","Myricks","Portland Central Thriftlodge","9 Tower Ave","Burlington","Boone","KY",41005,"859-717-7638","859-308-4286","stephane_myricks@cox.net","http://www.portlandcentralthriftlodge.com"
"Quentin","Swayze","Ulbrich Trucking","278 Bayview Ave","Milan","Monroe","MI",48160,"734-561-6170","734-851-8571","quentin_swayze@yahoo.com","http://www.ulbrichtrucking.com"
"Annmarie","Castros","Tipiak Inc","80312 W 32nd St","Conroe","Montgomery","TX",77301,"936-751-7961","936-937-2334","annmarie_castros@gmail.com","http://www.tipiakinc.com"
"Shonda","Greenbush","Saint George Well Drilling","82 Us Highway 46","Clifton","Passaic","NJ","07011","973-482-2430","973-644-2974","shonda_greenbush@cox.net","http://www.saintgeorgewelldrilling.com"
"Cecil","Lapage","Hawkes, Douglas D","4 Stova
 ll St #72","Union City","Hudson","NJ","07087","201-693-3967","201-856-2720","clapage@lapage.com","http://www.hawkesdouglasd.com"
"Jeanice","Claucherty","Accurel Systems Intrntl Corp","19 Amboy Ave","Miami","Miami-Dade","FL",33142,"305-988-4162","305-306-7834","jeanice.claucherty@yahoo.com","http://www.accurelsystemsintrntlcorp.com"
"Josphine","Villanueva","Santa Cruz Community Internet","63 Smith Ln #8343","Moss","Clay","TN",38575,"931-553-9774","931-486-6946","josphine_villanueva@villanueva.com","http://www.santacruzcommunityinternet.com"
"Daniel","Perruzza","Gersh & Danielson","11360 S Halsted St","Santa Ana","Orange","CA",92705,"714-771-3880","714-531-1391","dperruzza@perruzza.com","http://www.gershdanielson.com"
"Cassi","Wildfong","Cobb, James O Esq","26849 Jefferson Hwy","Rolling Meadows","Cook","IL",60008,"847-633-3216","847-755-9041","cassi.wildfong@aol.com","http://www.cobbjamesoesq.com"
"Britt","Galam","Wheatley Trucking Company","2500 Pringle Rd Se #508","Hatfield","Montgo
 mery","PA",19440,"215-888-3304","215-351-8523","britt@galam.org","http://www.wheatleytruckingcompany.com"
"Adell","Lipkin","Systems Graph Inc Ab Dick Dlr","65 Mountain View Dr","Whippany","Morris","NJ","07981","973-654-1561","973-662-8988","adell.lipkin@lipkin.com","http://www.systemsgraphincabdickdlr.com"
"Jacqueline","Rowling","John Hancock Mutl Life Ins Co","1 N San Saba","Erie","Erie","PA",16501,"814-865-8113","814-481-1700","jacqueline.rowling@yahoo.com","http://www.johnhancockmutllifeinsco.com"
"Lonny","Weglarz","History Division Of State","51120 State Route 18","Salt Lake City","Salt Lake","UT",84115,"801-293-9853","801-892-8781","lonny_weglarz@gmail.com","http://www.historydivisionofstate.com"
"Lonna","Diestel","Dimmock, Thomas J Esq","1482 College Ave","Fayetteville","Cumberland","NC",28301,"910-922-3672","910-200-7912","lonna_diestel@gmail.com","http://www.dimmockthomasjesq.com"
"Cristal","Samara","Intermed Inc","4119 Metropolitan Dr","Los Angeles","Los Angeles","CA",90021
 ,"213-975-8026","213-696-8004","cristal@cox.net","http://www.intermedinc.com"
"Kenneth","Grenet","Bank Of New York","2167 Sierra Rd","East Lansing","Ingham","MI",48823,"517-499-2322","517-867-8077","kenneth.grenet@grenet.org","http://www.bankofnewyork.com"
"Elli","Mclaird","Sportmaster Intrnatl","6 Sunrise Ave","Utica","Oneida","NY",13501,"315-818-2638","315-474-5570","emclaird@mclaird.com","http://www.sportmasterintrnatl.com"
"Alline","Jeanty","W W John Holden Inc","55713 Lake City Hwy","South Bend","St Joseph","IN",46601,"574-656-2800","574-405-1983","ajeanty@gmail.com","http://www.wwjohnholdeninc.com"
"Sharika","Eanes","Maccani & Delp","75698 N Fiesta Blvd","Orlando","Orange","FL",32806,"407-312-1691","407-472-1332","sharika.eanes@aol.com","http://www.maccanidelp.com"
"Nu","Mcnease","Amazonia Film Project","88 Sw 28th Ter","Harrison","Hudson","NJ","07029","973-751-9003","973-903-4175","nu@gmail.com","http://www.amazoniafilmproject.com"
"Daniela","Comnick","Water & Sewer Departmen
 t","7 Flowers Rd #403","Trenton","Mercer","NJ","08611","609-200-8577","609-398-2805","dcomnick@cox.net","http://www.watersewerdepartment.com"
"Cecilia","Colaizzo","Switchcraft Inc","4 Nw 12th St #3849","Madison","Dane","WI",53717,"608-382-4541","608-302-3387","cecilia_colaizzo@colaizzo.com","http://www.switchcraftinc.com"
"Leslie","Threets","C W D C Metal Fabricators","2 A Kelley Dr","Katonah","Westchester","NY",10536,"914-861-9748","914-396-2615","leslie@cox.net","http://www.cwdcmetalfabricators.com"
"Nan","Koppinger","Shimotani, Grace T","88827 Frankford Ave","Greensboro","Guilford","NC",27401,"336-370-5333","336-564-1492","nan@koppinger.com","http://www.shimotanigracet.com"
"Izetta","Dewar","Lisatoni, Jean Esq","2 W Scyene Rd #3","Baltimore","Baltimore City","MD",21217,"410-473-1708","410-522-7621","idewar@dewar.com","http://www.lisatonijeanesq.com"
"Tegan","Arceo","Ceramic Tile Sales Inc","62260 Park Stre","Monroe Township","Middlesex","NJ","08831","732-730-2692","732-705-6719",
 "tegan.arceo@arceo.org","http://www.ceramictilesalesinc.com"
"Ruthann","Keener","Maiden Craft Inc","3424 29th St Se","Kerrville","Kerr","TX",78028,"830-258-2769","830-919-5991","ruthann@hotmail.com","http://www.maidencraftinc.com"
"Joni","Breland","Carriage House Cllsn Rpr Inc","35 E Main St #43","Elk Grove Village","Cook","IL",60007,"847-519-5906","847-740-5304","joni_breland@cox.net","http://www.carriagehousecllsnrprinc.com"
"Vi","Rentfro","Video Workshop","7163 W Clark Rd","Freehold","Monmouth","NJ","07728","732-605-4781","732-724-7251","vrentfro@cox.net","http://www.videoworkshop.com"
"Colette","Kardas","Fresno Tile Center Inc","21575 S Apple Creek Rd","Omaha","Douglas","NE",68124,"402-896-5943","402-707-1602","colette.kardas@yahoo.com","http://www.fresnotilecenterinc.com"
"Malcolm","Tromblay","Versatile Sash & Woodwork","747 Leonis Blvd","Annandale","Fairfax","VA",22003,"703-221-5602","703-874-4248","malcolm_tromblay@cox.net","http://www.versatilesashwoodwork.com"
"Ryan","Harno
 s","Warner Electric Brk & Cltch Co","13 Gunnison St","Plano","Collin","TX",75075,"972-558-1665","972-961-4968","ryan@cox.net","http://www.warnerelectricbrkcltchco.com"
"Jess","Chaffins","New York Public Library","18 3rd Ave","New York","New York","NY",10016,"212-510-4633","212-428-9538","jess.chaffins@chaffins.org","http://www.newyorkpubliclibrary.com"
"Sharen","Bourbon","Mccaleb, John A Esq","62 W Austin St","Syosset","Nassau","NY",11791,"516-816-1541","516-749-3188","sbourbon@yahoo.com","http://www.mccalebjohnaesq.com"
"Nickolas","Juvera","United Oil Co Inc","177 S Rider Trl #52","Crystal River","Citrus","FL",34429,"352-598-8301","352-947-6152","nickolas_juvera@cox.net","http://www.unitedoilcoinc.com"
"Gary","Nunlee","Irving Foot Center","2 W Mount Royal Ave","Fortville","Hancock","IN",46040,"317-542-6023","317-887-8486","gary_nunlee@nunlee.org","http://www.irvingfootcenter.com"
"Diane","Devreese","Acme Supply Co","1953 Telegraph Rd","Saint Joseph","Buchanan","MO",64504,"816-557-9
 673","816-329-5565","diane@cox.net","http://www.acmesupplyco.com"
"Roslyn","Chavous","Mcrae, James L","63517 Dupont St","Jackson","Hinds","MS",39211,"601-234-9632","601-973-5754","roslyn.chavous@chavous.org","http://www.mcraejamesl.com"
"Glory","Schieler","Mcgraths Seafood","5 E Truman Rd","Abilene","Taylor","TX",79602,"325-869-2649","325-740-3778","glory@yahoo.com","http://www.mcgrathsseafood.com"
"Rasheeda","Sayaphon","Kummerer, J Michael Esq","251 Park Ave #979","Saratoga","Santa Clara","CA",95070,"408-805-4309","408-997-7490","rasheeda@aol.com","http://www.kummererjmichaelesq.com"
"Alpha","Palaia","Stoffer, James M Jr","43496 Commercial Dr #29","Cherry Hill","Camden","NJ","08003","856-312-2629","856-513-7024","alpha@yahoo.com","http://www.stofferjamesmjr.com"
"Refugia","Jacobos","North Central Fl Sfty Cncl","2184 Worth St","Hayward","Alameda","CA",94545,"510-974-8671","510-509-3496","refugia.jacobos@jacobos.com","http://www.northcentralflsftycncl.com"
"Shawnda","Yori","Fiorucci 
 Foods Usa Inc","50126 N Plankinton Ave",

<TRUNCATED>

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

Posted by ma...@apache.org.
NIFI-1280 added support for RecordSchema in SchemaRegistry

Signed-off-by: Mark Payne <ma...@hotmail.com>
Signed-off-by: Matt Burgess <ma...@apache.org>

NIFI-1280: Updated SimpleKeyValueSchemaRegistry to make use of new CHOICE RecordFieldType - Update Record Readers to use SchemaRegistry controller service. Moved SchemaRegistry api into its own maven module and added to standard-services-api so that we can properly add dependencies on it. Code cleanup and bug fixes

Signed-off-by: Matt Burgess <ma...@apache.org>

NIFI-1280: Fixed checkstyle violations and license exclusions for RAT plugin

Signed-off-by: Matt Burgess <ma...@apache.org>

NIFI-1280: Addressed feedback from PR Review

Signed-off-by: Matt Burgess <ma...@apache.org>

NIFI-1280: Additional changes/doc to support QueryFlowFile and Record Readers/Writers

This closes #1652


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

Branch: refs/heads/master
Commit: 68c592ea43d30754ec07c42cf10563fe9db185ae
Parents: a88d3bf
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Thu Mar 30 09:12:07 2017 -0400
Committer: Matt Burgess <ma...@apache.org>
Committed: Tue Apr 11 19:29:35 2017 -0400

----------------------------------------------------------------------
 .../apache/nifi/util/MockProcessSession.java    |   4 +-
 .../nifi/util/TestMockProcessSession.java       |  16 +-
 nifi-nar-bundles/nifi-hive-bundle/pom.xml       |  10 +
 .../nifi-registry-nar/pom.xml                   |   7 +-
 .../nifi-registry-processors/pom.xml            |  74 ---
 .../processors/AbstractCSVTransformer.java      |  57 --
 .../processors/AbstractContentTransformer.java  | 101 ---
 .../processors/AbstractTransformer.java         |  93 ---
 .../schemaregistry/processors/AvroUtils.java    |  67 --
 .../processors/BaseContentTransformer.java      |  51 --
 .../processors/BaseTransformer.java             | 189 ------
 .../schemaregistry/processors/CSVUtils.java     | 302 ---------
 .../processors/ExtractAvroFields.java           | 100 ---
 .../schemaregistry/processors/JsonUtils.java    |  74 ---
 .../processors/RegistryCommon.java              |  84 ---
 .../processors/TransformAvroToCSV.java          |  57 --
 .../processors/TransformAvroToJson.java         |  46 --
 .../processors/TransformCSVToAvro.java          |  80 ---
 .../processors/TransformCSVToJson.java          |  80 ---
 .../processors/TransformJsonToAvro.java         |  45 --
 .../processors/TransformJsonToCSV.java          |  45 --
 .../org.apache.nifi.processor.Processor         |  21 -
 .../processors/TransformersTest.java            | 220 -------
 .../expected_ouput_csv/decimal_logicalType.txt  |   1 -
 .../decimal_logicalType_invalid_scale.txt       |   1 -
 ...mal_logicalType_valid_scale_with_default.txt |   1 -
 .../decimal_logicalType_with_default.txt        |   1 -
 .../expected_ouput_csv/primitive_types.txt      |   1 -
 .../primitive_types_with_matching_default.txt   |   1 -
 .../union_null_last_field_with_default.txt      |   1 -
 .../union_null_middle_field_with_default.txt    |   1 -
 .../expected_ouput_csv/union_with_default.txt   |   1 -
 ...l_logicalType_invalid_scale_with_default.txt |  16 -
 ...mal_logicalType_valid_scale_with_default.txt |  16 -
 ..._logicalType_valid_scale_with_no_default.txt |  15 -
 .../input_avro/primitive_types_no_defaults.txt  |  11 -
 .../primitive_types_union_with_defaults.txt     |  11 -
 .../primitive_types_with_matching_default.txt   |  11 -
 .../primitive_types_with_mismatch_default.txt   |  11 -
 .../input_avro/union_and_matching_defaults.txt  |  18 -
 .../input_avro/union_and_mismatch_defaults.txt  |  18 -
 .../resources/input_csv/decimal_logicalType.txt |   1 -
 .../decimal_logicalType_missing_value.txt       |   1 -
 .../resources/input_csv/primitive_types.txt     |   1 -
 .../primitive_types_with_matching_default.txt   |   1 -
 .../union_null_last_field_with_default.txt      |   1 -
 .../union_null_middle_field_with_default.txt    |   1 -
 .../resources/input_csv/union_with_default.txt  |   1 -
 .../input_csv/union_with_missing_value.txt      |   1 -
 .../nifi-registry-service/pom.xml               |  12 +
 .../services/AvroSchemaRegistry.java            | 217 +++++++
 .../services/AvroSchemaValidator.java           |  57 ++
 .../schemaregistry/services/SchemaRegistry.java |  46 --
 .../services/SimpleKeyValueSchemaRegistry.java  |  96 ---
 ...org.apache.nifi.controller.ControllerService |   2 +-
 .../SimpleKeyValueSchemaRegistryTest.java       |  70 ---
 .../services/TestAvroSchemaRegistry.java        | 111 ++++
 nifi-nar-bundles/nifi-registry-bundle/pom.xml   |   5 -
 .../nifi-standard-processors/pom.xml            |   4 +-
 .../nifi/processors/standard/QueryFlowFile.java |  23 +-
 .../nifi/queryflowfile/FlowFileEnumerator.java  |   6 +-
 .../nifi/queryflowfile/FlowFileTable.java       |   4 +-
 .../additionalDetails.html                      |   3 +-
 .../processors/standard/TestQueryFlowFile.java  | 130 +---
 .../standard/util/record/MockRecordParser.java  | 107 ++++
 .../standard/util/record/MockRecordWriter.java  |  80 +++
 .../pom.xml                                     |   2 +-
 .../apache/nifi/serialization/RecordReader.java |   1 -
 .../apache/nifi/serialization/RecordWriter.java |   2 +-
 .../serialization/RowRecordReaderFactory.java   |   7 +-
 .../nifi/serialization/record/DataType.java     |  31 +-
 .../nifi/serialization/record/MapRecord.java    | 201 +-----
 .../nifi/serialization/record/Record.java       |   4 +-
 .../serialization/record/RecordFieldType.java   | 208 ++++++-
 .../record/ResultSetRecordSet.java              | 168 ++++-
 .../record/type/ArrayDataType.java              |  67 ++
 .../record/type/ChoiceDataType.java             |  68 +++
 .../record/type/RecordDataType.java             |  63 ++
 .../record/util/DataTypeUtils.java              | 608 +++++++++++++++++++
 .../util/IllegalTypeConversionException.java    |  29 +
 .../src/main/resources/META-INF/NOTICE          |  11 +-
 .../.gitignore                                  |   1 -
 .../nifi-record-serialization-services/pom.xml  |  19 +-
 .../java/org/apache/nifi/avro/AvroReader.java   |  13 +-
 .../org/apache/nifi/avro/AvroRecordReader.java  | 177 +++---
 .../apache/nifi/avro/AvroRecordSetWriter.java   |  15 +-
 .../java/org/apache/nifi/avro/AvroTypeUtil.java | 159 +++++
 .../org/apache/nifi/avro/WriteAvroResult.java   | 308 ++++------
 .../java/org/apache/nifi/csv/CSVReader.java     |  58 +-
 .../org/apache/nifi/csv/CSVRecordReader.java    | 197 ++----
 .../org/apache/nifi/csv/CSVRecordSetWriter.java |  40 +-
 .../main/java/org/apache/nifi/csv/CSVUtils.java | 204 +++++++
 .../nifi/csv/SingleCharacterValidator.java      |  62 ++
 .../org/apache/nifi/csv/WriteCSVResult.java     |  33 +-
 .../java/org/apache/nifi/grok/GrokReader.java   |  26 +-
 .../org/apache/nifi/grok/GrokRecordReader.java  |  54 +-
 .../nifi/json/AbstractJsonRowRecordReader.java  | 127 +---
 .../org/apache/nifi/json/JsonPathReader.java    |  47 +-
 .../nifi/json/JsonPathRowRecordReader.java      | 241 ++------
 .../org/apache/nifi/json/JsonPathValidator.java |  12 -
 .../apache/nifi/json/JsonRecordSetWriter.java   |   8 +-
 .../org/apache/nifi/json/JsonTreeReader.java    |  54 +-
 .../nifi/json/JsonTreeRowRecordReader.java      | 121 ++--
 .../org/apache/nifi/json/PropertyNameUtil.java  |  88 ---
 .../org/apache/nifi/json/WriteJsonResult.java   | 162 ++---
 .../serialization/AbstractRecordSetWriter.java  |  84 ---
 .../nifi/serialization/DataTypeUtils.java       | 165 -----
 .../DateTimeTextRecordSetWriter.java            |  57 ++
 .../nifi/serialization/DateTimeUtils.java       |  50 ++
 .../SchemaRegistryRecordReader.java             | 110 ++++
 .../UserTypeOverrideRowReader.java              |  78 ---
 .../nifi/text/FreeFormTextRecordSetWriter.java  |   6 +-
 .../additionalDetails.html                      | 150 +++--
 .../additionalDetails.html                      |  97 +--
 .../additionalDetails.html                      | 318 +++++-----
 .../additionalDetails.html                      | 281 ++++++---
 .../apache/nifi/avro/TestAvroRecordReader.java  | 121 +++-
 .../apache/nifi/avro/TestWriteAvroResult.java   | 202 ++++++
 .../apache/nifi/csv/TestCSVRecordReader.java    |  94 +--
 .../org/apache/nifi/csv/TestWriteCSVResult.java |  23 +-
 .../apache/nifi/grok/TestGrokRecordReader.java  |  11 +-
 .../nifi/json/TestJsonPathRowRecordReader.java  | 109 +++-
 .../nifi/json/TestJsonTreeRowRecordReader.java  | 104 +++-
 .../apache/nifi/json/TestWriteJsonResult.java   |   9 +-
 .../src/test/resources/avro/datatypes.avsc      |  47 ++
 .../src/test/resources/avro/logical-types.avsc  |  34 ++
 .../nifi-schema-registry-service-api/pom.xml    |  32 +
 .../schemaregistry/services/SchemaRegistry.java |  53 ++
 .../nifi-standard-services-api-nar/pom.xml      |   5 +
 nifi-nar-bundles/nifi-standard-services/pom.xml |   1 +
 pom.xml                                         |  12 +-
 131 files changed, 4423 insertions(+), 4466 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/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 7dd9714..faf6e42 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
@@ -218,8 +218,8 @@ 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);
+            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/68c592ea/nifi-mock/src/test/java/org/apache/nifi/util/TestMockProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/test/java/org/apache/nifi/util/TestMockProcessSession.java b/nifi-mock/src/test/java/org/apache/nifi/util/TestMockProcessSession.java
index e16afb3..d1c2bea 100644
--- a/nifi-mock/src/test/java/org/apache/nifi/util/TestMockProcessSession.java
+++ b/nifi-mock/src/test/java/org/apache/nifi/util/TestMockProcessSession.java
@@ -16,6 +16,14 @@
  */
 package org.apache.nifi.util;
 
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -28,14 +36,6 @@ import org.apache.nifi.stream.io.StreamUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Collections;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
-
 public class TestMockProcessSession {
 
     @Test(expected = AssertionError.class)

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-hive-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
index 5c3f2a5..342e7ed 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
@@ -38,6 +38,16 @@
         <module>nifi-hive-nar</module>
     </modules>
 
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>org.apache.avro</groupId>
+                <artifactId>avro</artifactId>
+                <version>1.7.7</version>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
+
     <build>
         <plugins>
             <plugin>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-nar/pom.xml b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-nar/pom.xml
index dfdf214..0780c85 100644
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-nar/pom.xml
@@ -25,7 +25,12 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-registry-processors</artifactId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-registry-service</artifactId>
         </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/pom.xml b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/pom.xml
deleted file mode 100644
index 0ea83ee..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/pom.xml
+++ /dev/null
@@ -1,74 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!-- Copyright 2016 Hortoworks, Inc. All rights reserved. Hortonworks, Inc. 
-	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. 
-	See the associated NOTICE file for additional information regarding copyright 
-	ownership. -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.nifi</groupId>
-		<artifactId>nifi-registry-bundle</artifactId>
-		<version>1.2.0-SNAPSHOT</version>
-	</parent>
-
-	<artifactId>nifi-registry-processors</artifactId>
-	<packaging>jar</packaging>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.rat</groupId>
-				<artifactId>apache-rat-plugin</artifactId>
-				<configuration>
-					<excludes combine.children="append">
-						<exclude>src/test/resources/expected_ouput_csv/*</exclude>
-						<exclude>src/test/resources/input_avro/*</exclude>
-						<exclude>src/test/resources/input_csv/*</exclude>					
-					</excludes>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.nifi</groupId>
-			<artifactId>nifi-api</artifactId>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.nifi</groupId>
-			<artifactId>nifi-utils</artifactId>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.avro</groupId>
-			<artifactId>avro</artifactId>
-		</dependency>
-		<dependency>
-			<groupId>org.codehaus.jackson</groupId>
-			<artifactId>jackson-mapper-asl</artifactId>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.nifi</groupId>
-			<artifactId>nifi-registry-service</artifactId>
-			<version>1.2.0-SNAPSHOT</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.nifi</groupId>
-			<artifactId>nifi-mock</artifactId>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>pl.pragmatists</groupId>
-			<artifactId>JUnitParams</artifactId>
-			<version>1.0.5</version>
-			<scope>test</scope>
-		</dependency>
-	</dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractCSVTransformer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractCSVTransformer.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractCSVTransformer.java
deleted file mode 100644
index 54497dc..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractCSVTransformer.java
+++ /dev/null
@@ -1,57 +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.schemaregistry.processors;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.schemaregistry.services.SchemaRegistry;
-
-/**
- * Base processor for implementing transform-like processors for CSV
- * transformations that integrate with Schema Registry (see
- * {@link SchemaRegistry})
- */
-abstract class AbstractCSVTransformer extends AbstractContentTransformer {
-
-    static final List<PropertyDescriptor> BASE_CSV_DESCRIPTORS;
-
-    static {
-        List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
-        descriptors.addAll(BASE_DESCRIPTORS);
-        descriptors.add(DELIMITER);
-        BASE_CSV_DESCRIPTORS = Collections.unmodifiableList(descriptors);
-    }
-
-    protected volatile char delimiter;
-
-    @Override
-    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return BASE_CSV_DESCRIPTORS;
-    }
-
-    @Override
-    @OnScheduled
-    public void onScheduled(ProcessContext context) {
-        super.onScheduled(context);
-        this.delimiter = context.getProperty(DELIMITER).getValue().charAt(0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractContentTransformer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractContentTransformer.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractContentTransformer.java
deleted file mode 100644
index 403b52a..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractContentTransformer.java
+++ /dev/null
@@ -1,101 +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.schemaregistry.processors;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.schemaregistry.services.SchemaRegistry;
-
-/**
- * Base processor for implementing transform-like processors that integrate with
- * Schema Registry (see {@link SchemaRegistry})
- */
-abstract class AbstractContentTransformer extends BaseContentTransformer implements RegistryCommon {
-
-    static final List<PropertyDescriptor> BASE_DESCRIPTORS;
-
-    static {
-        List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
-        descriptors.add(REGISTRY_SERVICE);
-        descriptors.add(SCHEMA_NAME);
-        descriptors.add(SCHEMA_TYPE);
-        BASE_DESCRIPTORS = Collections.unmodifiableList(descriptors);
-    }
-
-    volatile SchemaRegistry schemaRegistryDelegate;
-
-    /**
-     *
-     */
-    @Override
-    @OnScheduled
-    public void onScheduled(ProcessContext context) {
-        this.schemaRegistryDelegate = context.getProperty(REGISTRY_SERVICE).asControllerService(SchemaRegistry.class);
-    }
-
-    /**
-     *
-     */
-    @Override
-    protected Map<String, String> transform(InputStream in, OutputStream out, InvocationContextProperties contextProperties) {
-        Schema schema = RegistryCommon.retrieveSchema(this.schemaRegistryDelegate, contextProperties);
-        return this.transform(in, out, contextProperties, schema);
-    }
-
-    /**
-     * This operation is designed to allow sub-classes to provide
-     * implementations that read content of the provided {@link InputStream} and
-     * write content (same or different) into the provided {@link OutputStream}.
-     * Both {@link InputStream} and {@link OutputStream} represent the content
-     * of the in/out {@link FlowFile} and are both required to NOT be null;
-     * <p>
-     * The returned {@link Map} represents attributes that will be added to the
-     * outgoing FlowFile. It can be null, in which case no attributes will be
-     * added to the resulting {@link FlowFile}.
-     *
-     *
-     * @param in
-     *            {@link InputStream} representing data to be transformed
-     * @param out
-     *            {@link OutputStream} representing target stream to wrote
-     *            transformed data. Can be null if no output needs to be
-     *            written.
-     * @param contextProperties
-     *            instance of {@link InvocationContextProperties}
-     * @param schema
-     *            instance of {@link Schema}
-     */
-    protected abstract Map<String, String> transform(InputStream in, OutputStream out, InvocationContextProperties contextProperties, Schema schema);
-
-    /**
-     *
-     */
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return BASE_DESCRIPTORS;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractTransformer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractTransformer.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractTransformer.java
deleted file mode 100644
index 13dd4a5..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AbstractTransformer.java
+++ /dev/null
@@ -1,93 +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.schemaregistry.processors;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.schemaregistry.services.SchemaRegistry;
-
-/**
- * Base processor for implementing transform-like processors that integrate with
- * Schema Registry (see {@link SchemaRegistry})
- */
-abstract class AbstractTransformer extends BaseTransformer implements RegistryCommon {
-
-    static final List<PropertyDescriptor> BASE_DESCRIPTORS;
-
-    static {
-        List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
-        descriptors.add(REGISTRY_SERVICE);
-        descriptors.add(SCHEMA_NAME);
-        BASE_DESCRIPTORS = Collections.unmodifiableList(descriptors);
-    }
-
-    volatile SchemaRegistry schemaRegistryDelegate;
-
-    /**
-     *
-     */
-    @Override
-    @OnScheduled
-    public void onScheduled(ProcessContext context) {
-        this.schemaRegistryDelegate = context.getProperty(REGISTRY_SERVICE).asControllerService(SchemaRegistry.class);
-    }
-
-    /**
-     * This operation is designed to allow sub-classes to provide
-     * implementations that read content of the provided {@link InputStream}
-     * that represent the content of the incoming {@link FlowFile}.
-     * <p>
-     * The returned {@link Map} represents attributes that will be added to the
-     * outgoing FlowFile.
-     *
-     *
-     * @param in
-     *            {@link InputStream} representing data to be transformer
-     * @param contextProperties
-     *            instance of {@link InvocationContextProperties}
-     * @param schema
-     *            instance of avro {@link Schema}
-     */
-    protected abstract Map<String, String> transform(InputStream in, InvocationContextProperties contextProperties, Schema schema);
-
-    /**
-     *
-     */
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return BASE_DESCRIPTORS;
-    }
-
-    /**
-     *
-     */
-    @Override
-    protected Map<String, String> transform(InputStream in, OutputStream out, InvocationContextProperties contextProperties) {
-        Schema schema = RegistryCommon.retrieveSchema(this.schemaRegistryDelegate, contextProperties);
-        return this.transform(in, contextProperties, schema);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AvroUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AvroUtils.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AvroUtils.java
deleted file mode 100644
index b967af9..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/AvroUtils.java
+++ /dev/null
@@ -1,67 +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.schemaregistry.processors;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericDatumReader;
-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.DecoderFactory;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.nifi.flowfile.FlowFile;
-
-/**
- * Various Avro related utility operations relevant to transforming contents of
- * the {@link FlowFile} between Avro formats.
- */
-class AvroUtils {
-
-    /**
-     * Reads provided {@link InputStream} into Avro {@link GenericRecord}
-     * applying provided {@link Schema} returning the resulting GenericRecord.
-     */
-    public static GenericRecord read(InputStream in, Schema schema) {
-        GenericDatumReader<GenericRecord> datumReader = new GenericDatumReader<>(schema);
-        GenericRecord avroRecord = null;
-        try {
-            avroRecord = datumReader.read(null, DecoderFactory.get().binaryDecoder(in, null));
-            return avroRecord;
-        } catch (Exception e) {
-            throw new IllegalStateException("Failed to read AVRO record", e);
-        }
-    }
-
-    /**
-     * Writes provided {@link GenericRecord} into the provided
-     * {@link OutputStream}.
-     */
-    public static void write(GenericRecord record, OutputStream out) {
-        BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
-        DatumWriter<GenericRecord> writer = new GenericDatumWriter<>(record.getSchema());
-        try {
-            writer.write(record, encoder);
-            encoder.flush();
-        } catch (Exception e) {
-            throw new IllegalStateException("Failed to write AVRO record", e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/BaseContentTransformer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/BaseContentTransformer.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/BaseContentTransformer.java
deleted file mode 100644
index 12586ac..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/BaseContentTransformer.java
+++ /dev/null
@@ -1,51 +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.schemaregistry.processors;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.io.StreamCallback;
-
-/**
- * Base processor which contains common functionality for processors that
- * receive {@link FlowFile} and output {@link FlowFile} while also modifying the
- * content of the {@link FlowFile}
- */
-public abstract class BaseContentTransformer extends BaseTransformer {
-
-    @Override
-    protected FlowFile doTransform(ProcessContext context, ProcessSession session, FlowFile flowFile, InvocationContextProperties contextProperties) {
-        AtomicReference<Map<String, String>> attributeRef = new AtomicReference<Map<String, String>>();
-        flowFile = session.write(flowFile, new StreamCallback() {
-            @Override
-            public void process(InputStream in, OutputStream out) throws IOException {
-                attributeRef.set(transform(in, out, contextProperties));
-            }
-        });
-        if (attributeRef.get() != null) {
-            flowFile = session.putAllAttributes(flowFile, attributeRef.get());
-        }
-        return flowFile;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/BaseTransformer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/BaseTransformer.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/BaseTransformer.java
deleted file mode 100644
index e1cc98c..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/BaseTransformer.java
+++ /dev/null
@@ -1,189 +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.schemaregistry.processors;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.PropertyValue;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.AbstractProcessor;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.ProcessSessionFactory;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
-
-/**
- * Base processor which contains common functionality for processors that
- * receive {@link FlowFile} and output {@link FlowFile} and contain only two
- * {@link Relationship}s (i.e., success and failure). Every successful execution
- * of
- * {@link #doTransform(ProcessContext, ProcessSession, FlowFile, InvocationContextProperties)}
- * operation will result in transferring {@link FlowFile} to 'success'
- * relationship while any exception will result in such file going to 'failure'.
- */
-public abstract class BaseTransformer extends AbstractProcessor {
-
-    public static final Relationship REL_SUCCESS = new Relationship.Builder()
-            .name("success")
-            .description("Successfully retrieved schema from Schema Registry")
-            .build();
-
-    public static final Relationship REL_FAILURE = new Relationship.Builder()
-            .name("failure")
-            .description("FlowFiles that failed to find a schema are sent to this relationship")
-            .build();
-
-    private static final Set<Relationship> BASE_RELATIONSHIPS;
-
-    static {
-        Set<Relationship> relationships = new HashSet<>();
-        relationships.add(REL_SUCCESS);
-        relationships.add(REL_FAILURE);
-        BASE_RELATIONSHIPS = Collections.unmodifiableSet(relationships);
-    }
-
-    private final Map<PropertyDescriptor, String> propertyInstanceValues = new HashMap<>();
-
-    @Override
-    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
-        FlowFile flowFile = session.get();
-        if (flowFile != null) {
-            try {
-                InvocationContextProperties contextProperties = new InvocationContextProperties(context, flowFile);
-                flowFile = this.doTransform(context, session, flowFile, contextProperties);
-                session.transfer(flowFile, REL_SUCCESS);
-            } catch (Exception e) {
-                this.getLogger().error("Failed FlowFile processing, routing to failure. Issue: " + e.getMessage(), e);
-                session.transfer(flowFile, REL_FAILURE);
-            }
-        } else {
-            context.yield();
-        }
-    }
-
-    @OnScheduled
-    public void onScheduled(ProcessContext context) {
-        List<PropertyDescriptor> propertyDescriptors = this.getSupportedPropertyDescriptors();
-        for (PropertyDescriptor propertyDescriptor : propertyDescriptors) {
-            if (!propertyDescriptor.isExpressionLanguageSupported()){
-                this.propertyInstanceValues.put(propertyDescriptor, context.getProperty(propertyDescriptor).getValue());
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    protected FlowFile doTransform(ProcessContext context, ProcessSession session, FlowFile flowFile,  InvocationContextProperties contextProperties) {
-        AtomicReference<Map<String, String>> attributeRef = new AtomicReference<Map<String, String>>();
-        session.read(flowFile, new InputStreamCallback() {
-            @Override
-            public void process(InputStream in) throws IOException {
-                attributeRef.set(transform(in, null, contextProperties));
-            }
-        });
-        if (attributeRef.get() != null) {
-            flowFile = session.putAllAttributes(flowFile, attributeRef.get());
-        }
-        return flowFile;
-    }
-
-    @Override
-    public Set<Relationship> getRelationships() {
-        return BASE_RELATIONSHIPS;
-    }
-
-    /**
-     * This operation is designed to allow sub-classes to provide
-     * implementations that read content of the provided {@link InputStream} and
-     * write content (same or different) it into the provided
-     * {@link OutputStream}. Both {@link InputStream} and {@link OutputStream}
-     * represent the content of the in/out {@link FlowFile}. The
-     * {@link OutputStream} can be null if no output needs to be written.
-     * <p>
-     * The returned {@link Map} represents attributes that will be added to the
-     * outgoing FlowFile.
-     *
-     *
-     * @param in
-     *            {@link InputStream} representing data to be transformed
-     * @param out
-     *            {@link OutputStream} representing target stream to wrote
-     *            transformed data. Can be null if no output needs to be
-     *            written.
-     * @param contextProperties
-     *            instance of {@link InvocationContextProperties}
-     */
-    protected abstract Map<String, String> transform(InputStream in, OutputStream out, InvocationContextProperties contextProperties);
-
-    /**
-     * Properties object that gathers the value of the
-     * {@link PropertyDescriptor} within the context of
-     * {@link Processor#onTrigger(ProcessContext, ProcessSessionFactory)}
-     * invocation. It maintains the knowledge of instance properties vs.
-     * invocation properties that the values of which are set by evaluating
-     * expression against the incoming {@link FlowFile}.
-     */
-    public class InvocationContextProperties {
-        private final Map<PropertyDescriptor, String> propertyInvocationValues = new HashMap<>();
-
-        InvocationContextProperties(ProcessContext context, FlowFile flowFile) {
-            List<PropertyDescriptor> propertyDescriptors = BaseTransformer.this.getSupportedPropertyDescriptors();
-            for (PropertyDescriptor propertyDescriptor : propertyDescriptors) {
-                if (propertyDescriptor.isExpressionLanguageSupported()) {
-                    PropertyValue value = context.getProperty(propertyDescriptor)
-                            .evaluateAttributeExpressions(flowFile);
-                    this.propertyInvocationValues.put(propertyDescriptor, value.getValue());
-                }
-            }
-        }
-
-        /**
-         * Returns the value of the property within the context of
-         * {@link Processor#onTrigger(ProcessContext, ProcessSessionFactory)}
-         * invocation.
-         */
-        public String getPropertyValue(PropertyDescriptor propertyDescriptor, boolean notNull) {
-            String propertyValue = propertyInstanceValues.containsKey(propertyDescriptor)
-                    ? propertyInstanceValues.get(propertyDescriptor)
-                            : propertyInvocationValues.get(propertyDescriptor);
-            if (notNull && propertyValue == null) {
-                throw new IllegalArgumentException("Property '" + propertyDescriptor + "' evaluatd to null");
-            }
-            return propertyValue;
-        }
-
-        @Override
-        public String toString() {
-            return "Instance: " + propertyInstanceValues + "; Invocation: " + propertyInvocationValues;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/CSVUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/CSVUtils.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/CSVUtils.java
deleted file mode 100644
index 58d5d6b..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/CSVUtils.java
+++ /dev/null
@@ -1,302 +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.schemaregistry.processors;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.StringWriter;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.text.DecimalFormat;
-import java.text.NumberFormat;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.avro.Schema;
-import org.apache.avro.Schema.Field;
-import org.apache.avro.Schema.Type;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericData.Record;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang3.StringEscapeUtils;
-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.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.node.BooleanNode;
-import org.codehaus.jackson.node.DoubleNode;
-import org.codehaus.jackson.node.IntNode;
-import org.codehaus.jackson.node.LongNode;
-import org.codehaus.jackson.node.TextNode;
-
-/**
- * Various CSV related utility operations relevant to transforming contents of
- * the {@link FlowFile} between CSV and AVRO formats.
- */
-class CSVUtils {
-    /**
-     * Provides a {@link Validator} to ensure that provided value is a valid
-     * character.
-     */
-    public static final Validator CHAR_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(String subject, String input, ValidationContext context) {
-            // Allows special, escaped characters as input, which is then un-escaped and converted to a single character.
-            // Examples for special characters: \t (or \u0009), \f.
-            if (input.length() > 1) {
-                input = StringEscapeUtils.unescapeJava(input);
-            }
-            return new ValidationResult.Builder().subject(subject).input(input)
-                    .explanation("Only non-null single characters are supported")
-                    .valid(input.length() == 1 && input.charAt(0) != 0).build();
-        }
-    };
-
-    public static GenericRecord read(InputStream record, char delimiter, Schema schema, char quoteChar) {
-        Record avroRecord = new GenericData.Record(schema);
-        String[] parsedRecord = parseFields(convertInputStreamToString(record), delimiter, quoteChar);
-        List<Field> fields = schema.getFields();
-        if (parsedRecord.length != fields.size()) {
-            throw new IllegalStateException("Incompatible schema. Parsed fields count does not match the count of fields from schema. "
-                    + "Schema: " + schema.toString(true) + "\n Record: " + record);
-        }
-
-        for (int i = 0; i < fields.size(); i++) {
-            Field field = fields.get(i);
-            Type type = field.schema().getType();
-            updateRecord(field, type, parsedRecord[i], avroRecord);
-        }
-        return avroRecord;
-    }
-
-    /**
-     * Parses provided record into fields using provided delimiter. The
-     * 'quoteChar' is used to ensure that if a delimiter char is in quotes it
-     * will not be parsed into a separate filed.
-     */
-    public static String[] parseFields(String record, char delimiter, char quoteChar) {
-        List<String> result = new ArrayList<String>();
-        int start = 0;
-        boolean inQuotes = false;
-        for (int i = 0; i < record.length(); i++) {
-            if (record.charAt(i) == quoteChar) {
-                inQuotes = !inQuotes;
-            }
-            boolean atLastChar = (i == record.length() - 1);
-            if (atLastChar) {
-                if (record.charAt(i) == delimiter) {
-                    //missing last column value, add NULL
-                    result.add(record.substring(start,i));
-                    result.add(null);
-                } else {
-                    result.add(record.substring(start));
-                }
-            } else if (record.charAt(i) == delimiter && !inQuotes) {
-                if (start == i) {
-                    //There is no value, so add NULL to indicated the absence of a value for this field.
-                    result.add(null);
-                } else {
-                    result.add(record.substring(start, i));
-                }
-                start = i + 1;
-            }
-        }
-        return result.toArray(new String[]{});
-    }
-
-    /**
-     * Writes {@link GenericRecord} as CSV (delimited) record to the
-     * {@link OutputStream} using provided delimiter.
-     */
-    public static void write(GenericRecord record, char delimiter, OutputStream out) {
-        List<Field> fields = record.getSchema().getFields();
-
-        String delimiterToUse = "";
-        try {
-            for (Field field : fields) {
-                out.write(delimiterToUse.getBytes(StandardCharsets.UTF_8));
-                Object fieldValue = record.get(field.name());
-                if (null == fieldValue) {
-                    out.write(new byte[0]);
-                } else {
-                    if (Type.BYTES == field.schema().getType()) {
-                        // need to create it from the ByteBuffer it is serialized as.
-                        // need to ensure the type is one of the logical ones we support and if so convert it.
-                        if(!"decimal".contentEquals(field.getProp("logicalType"))){
-                            throw new IllegalArgumentException("The field '" + field.name() + "' has a logical type of '" +
-                                    field.getProp("logicalType") + "' that is currently not supported.");
-                        }
-
-                        JsonNode rawPrecision = field.getJsonProp("precision");
-                        if(null == rawPrecision){
-                            throw new IllegalArgumentException("The field '" + field.name() + "' is missing the required precision property");
-                        }
-                        int precision = rawPrecision.asInt();
-                        JsonNode rawScale = field.getJsonProp("scale");
-                        int scale = null == rawScale ? 0 : rawScale.asInt();
-
-                        // write out the decimal with the precision and scale.
-                        NumberFormat numberFormat = DecimalFormat.getInstance();
-                        numberFormat.setGroupingUsed(false);
-                        normalizeNumberFormat(numberFormat, scale, precision);
-                        String rawValue = new String(((ByteBuffer)fieldValue).array());
-                        // raw value needs to be parsed to ensure that BigDecimal will not throw an exception for specific locale
-                        rawValue = numberFormat.parse(rawValue).toString();
-                        out.write(numberFormat.format(new BigDecimal(rawValue)).getBytes(StandardCharsets.UTF_8));
-                    } else {
-                        out.write(fieldValue.toString().getBytes(StandardCharsets.UTF_8));
-                    }
-                }
-                if (delimiterToUse.length() == 0) {
-                    delimiterToUse = String.valueOf(delimiter);
-                }
-            }
-        } catch (IOException | ParseException e) {
-            throw new IllegalStateException("Failed to parse AVRO Record", e);
-        }
-    }
-
-    /**
-     * According to the 1.7.7 spec If a logical type is invalid, for example a
-     * decimal with scale greater than its precision,then implementations should
-     * ignore the logical type and use the underlying Avro type.
-     */
-    private static void normalizeNumberFormat(NumberFormat numberFormat, int scale, int precision) {
-        if (scale < precision) {
-            // write out with the specified precision and scale.
-            numberFormat.setMaximumIntegerDigits(precision);
-            numberFormat.setMaximumFractionDigits(scale);
-            numberFormat.setMinimumFractionDigits(scale);
-        }
-    }
-
-    /**
-     *
-     */
-    private static String convertInputStreamToString(InputStream record) {
-        StringWriter writer = new StringWriter();
-        try {
-            IOUtils.copy(record, writer, StandardCharsets.UTF_8);
-        } catch (Exception e) {
-            throw new IllegalStateException("Failed to read InputStream into String", e);
-        }
-        return writer.toString();
-    }
-
-    /**
-     *
-     */
-    private static ByteBuffer encodeLogicalType(final Field field, final String fieldValue) {
-        String logicalType = field.getProp("logicalType");
-        if (!"decimal".contentEquals(logicalType)) {
-            throw new IllegalArgumentException("The field '" + field.name() + "' has a logical type of '" + logicalType
-                    + "' that is currently not supported.");
-        }
-
-        JsonNode rawPrecision = field.getJsonProp("precision");
-        if (null == rawPrecision) {
-            throw new IllegalArgumentException("The field '" + field.name() + "' is missing the required precision property");
-        }
-        int precision = rawPrecision.asInt();
-        JsonNode rawScale = field.getJsonProp("scale");
-        int scale = null == rawScale ? 0 : rawScale.asInt();
-
-        NumberFormat numberFormat = DecimalFormat.getInstance();
-        numberFormat.setGroupingUsed(false);
-        normalizeNumberFormat(numberFormat, scale, precision);
-        BigDecimal decimal = null == fieldValue ? new BigDecimal(retrieveDefaultFieldValue(field).asText()) : new BigDecimal(fieldValue);
-        return ByteBuffer.wrap(numberFormat.format(decimal).getBytes(StandardCharsets.UTF_8));
-    }
-
-    /**
-     *
-     */
-    private static JsonNode retrieveDefaultFieldValue(Field field) {
-        JsonNode jsonNode = field.defaultValue();
-        if (null == jsonNode) {
-            throw new IllegalArgumentException("The field '" + field.name() + "' is NULL and there is no default value supplied in the Avro Schema");
-        }
-        return jsonNode;
-    }
-
-    /**
-     *
-     */
-    private static void updateRecord(Field field, Type type, String providedValue, Record avroRecord) {
-        if (Type.NULL != type) {
-            Object value;
-            if (Type.INT == type) {
-                value = null == providedValue ? possiblyGetDefaultValue(field, IntNode.class).getIntValue()
-                        : Integer.parseInt(providedValue);
-                avroRecord.put(field.name(), value);
-            } else if (Type.BOOLEAN == type) {
-                value = null == providedValue
-                        ? possiblyGetDefaultValue(field, BooleanNode.class).getBooleanValue()
-                        : Boolean.parseBoolean(providedValue);
-                avroRecord.put(field.name(), value);
-            } else if (Type.DOUBLE == type) {
-                value = null == providedValue ? possiblyGetDefaultValue(field, DoubleNode.class).getDoubleValue()
-                        : Double.parseDouble(providedValue);
-                avroRecord.put(field.name(), value);
-            } else if (Type.FLOAT == type) {
-                value = null == providedValue ? possiblyGetDefaultValue(field, DoubleNode.class).getDoubleValue()
-                        : Float.parseFloat(providedValue);
-                avroRecord.put(field.name(), value);
-            } else if (Type.LONG == type) {
-                value = null == providedValue ? possiblyGetDefaultValue(field, LongNode.class).getLongValue()
-                        : Long.parseLong(providedValue);
-                avroRecord.put(field.name(), value);
-            } else if (Type.STRING == type) {
-                value = null == providedValue ? possiblyGetDefaultValue(field, TextNode.class).getTextValue()
-                        : providedValue;
-                avroRecord.put(field.name(), value);
-            } else if (Type.BYTES == type) {
-                value = encodeLogicalType(field, providedValue);
-                avroRecord.put(field.name(), value);
-            } else if (Type.UNION == type) {
-                field.schema().getTypes()
-                        .forEach(schema -> updateRecord(field, schema.getType(), providedValue, avroRecord));
-            } else if (Type.ARRAY == type || Type.ENUM == type || Type.FIXED == type || Type.MAP == type
-                    || Type.NULL == type || Type.RECORD == type) {
-                throw new IllegalArgumentException("The field type '" + type + "' is not supported at the moment");
-            } else {
-                avroRecord.put(field.name(), providedValue);
-            }
-        }
-    }
-
-    /**
-     * Check to see if there is a default value to use, if not will throw
-     * {@link IllegalArgumentException}
-     */
-    private static <T extends JsonNode> JsonNode possiblyGetDefaultValue(Field field, Class<T> expectedDefaultType) {
-        JsonNode jsonNode = retrieveDefaultFieldValue(field);
-        if (field.schema().getType() != Type.UNION && !expectedDefaultType.isAssignableFrom(jsonNode.getClass())) {
-            // since we do not support schema evolution here we need to throw an
-            // exception here as the data is in error.
-            throw new IllegalArgumentException("The field '" + field.name() + "' has a default value that "
-                    + "does not match the field type. Field Type is: '" + expectedDefaultType.getName() + "' and the "
-                    + "default value type is: '" + field.defaultValue().toString());
-        }
-        return jsonNode;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/ExtractAvroFields.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/ExtractAvroFields.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/ExtractAvroFields.java
deleted file mode 100644
index 2ab83c5..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/ExtractAvroFields.java
+++ /dev/null
@@ -1,100 +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.schemaregistry.processors;
-
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.nifi.annotation.behavior.DynamicProperty;
-import org.apache.nifi.annotation.behavior.InputRequirement;
-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.components.PropertyDescriptor;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.util.StandardValidators;
-
-@Tags({ "registry", "schema", "avro", "extract", "evaluate" })
-@CapabilityDescription("Extracts Avro field and assigns it to the FlowFile attribute")
-@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
-@DynamicProperty(name = "Avro field name", value = "FlowFile attribute name to set the extracted field",
-                 description = "The value of the Avro field specified by 'Avro field name' will be extracted and set as "
-                         + "FlowFile attribute under name specified by the value of this property.")
-public final class ExtractAvroFields extends AbstractTransformer {
-
-    private static final List<PropertyDescriptor> DESCRIPTORS;
-
-    static {
-        List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
-        descriptors.addAll(BASE_DESCRIPTORS);
-        descriptors.add(SCHEMA_TYPE);
-        DESCRIPTORS = Collections.unmodifiableList(descriptors);
-    }
-
-    private volatile Map<String, String> dynamicProperties;
-
-    /**
-     *
-     */
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return DESCRIPTORS;
-    }
-
-    /**
-     *
-     */
-    @Override
-    @OnScheduled
-    public void onScheduled(ProcessContext context) {
-        super.onScheduled(context);
-        this.dynamicProperties = context.getProperties().entrySet().stream()
-                .filter(p -> p.getKey().isDynamic())
-                .collect(Collectors.toMap(p -> p.getKey().getName(), p -> p.getValue()));
-    }
-
-    /**
-     *
-     */
-    @Override
-    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder()
-                .name(propertyDescriptorName)
-                .expressionLanguageSupported(false)
-                .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-                .required(false)
-                .dynamic(true)
-                .build();
-    }
-
-    /**
-     *
-     */
-    @Override
-    protected Map<String, String> transform(InputStream in, InvocationContextProperties contextProperties, Schema schema) {
-        GenericRecord avroRecord = AvroUtils.read(in, schema);
-        Map<String, String> attributes = this.dynamicProperties.entrySet().stream().collect(
-                Collectors.toMap(dProp -> dProp.getValue(), dProp -> String.valueOf(avroRecord.get(dProp.getKey()))));
-        return Collections.unmodifiableMap(attributes);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/JsonUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/JsonUtils.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/JsonUtils.java
deleted file mode 100644
index 81c98b3..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/JsonUtils.java
+++ /dev/null
@@ -1,74 +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.schemaregistry.processors;
-
-import java.io.DataInputStream;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.Decoder;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.avro.io.JsonEncoder;
-import org.apache.nifi.flowfile.FlowFile;
-
-/**
- * Various Json related utility operations relevant to transforming contents of
- * the {@link FlowFile} between JSON and AVRO formats.
- */
-class JsonUtils {
-
-    /**
-     * Writes provided {@link GenericRecord} into the provided
-     * {@link OutputStream} as JSON.
-     */
-    public static void write(GenericRecord record, OutputStream out) {
-        try {
-            DatumWriter<GenericRecord> writer = new GenericDatumWriter<GenericRecord>(record.getSchema());
-            JsonEncoder encoder = EncoderFactory.get().jsonEncoder(record.getSchema(), out);
-            writer.write(record, encoder);
-            encoder.flush();
-        } catch (Exception e) {
-            throw new IllegalStateException("Failed to read GenericRecord", e);
-        }
-    }
-
-    /**
-     * Reads provided {@link InputStream} as ISON into Avro
-     * {@link GenericRecord} applying provided {@link Schema} returning the
-     * resulting GenericRecord.
-     */
-    public static GenericRecord read(InputStream jsonIs, Schema schema) {
-        DataInputStream din = new DataInputStream(jsonIs);
-        try {
-            Decoder decoder = DecoderFactory.get().jsonDecoder(schema, din);
-            DatumReader<GenericData.Record> reader = new GenericDatumReader<>(schema);
-            return reader.read(null, decoder);
-        } catch (Exception e) {
-            throw new IllegalStateException("Failed to parse incoming Json input stream into Avro GenericRecord. "
-                    + "Possible reason: the value may not be a valid JSON or incompatible schema is provided. Schema was '"
-                    + schema.toString(true) + "'.", e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/RegistryCommon.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/RegistryCommon.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/RegistryCommon.java
deleted file mode 100644
index 3fc1530..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/RegistryCommon.java
+++ /dev/null
@@ -1,84 +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.schemaregistry.processors;
-
-import org.apache.avro.Schema;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.schemaregistry.processors.BaseTransformer.InvocationContextProperties;
-import org.apache.nifi.schemaregistry.services.SchemaRegistry;
-
-/**
- * Strategy that encapsulates common properties and functionalities used by all
- * processors that integrate with Schema Registry.
- */
-interface RegistryCommon {
-
-    static final String SCHEMA_ATTRIBUTE_NAME = "schema.text";
-
-    static final PropertyDescriptor REGISTRY_SERVICE = new PropertyDescriptor.Builder()
-            .name("schema-registry-service")
-            .displayName("Schema Registry Service")
-            .description("The Schema Registry Service for serializing/deserializing messages as well as schema retrieval.")
-            .required(true)
-            .identifiesControllerService(SchemaRegistry.class)
-            .build();
-
-    static final PropertyDescriptor SCHEMA_NAME = new PropertyDescriptor.Builder()
-            .name("schema-name")
-            .displayName("Schema Name")
-            .description("The name of schema.")
-            .required(true)
-            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
-
-    static final PropertyDescriptor SCHEMA_TYPE = new PropertyDescriptor.Builder()
-            .name("schema-type")
-            .displayName("Schema Type")
-            .description("The type of schema (avro is the the only current supported schema).")
-            .required(true)
-            .allowableValues("avro")
-            .defaultValue("avro")
-            .expressionLanguageSupported(true)
-            .build();
-
-    static final PropertyDescriptor DELIMITER = new PropertyDescriptor.Builder()
-            .name("csv-delimiter")
-            .displayName("CSV delimiter")
-            .description("Delimiter character for CSV records")
-            .addValidator(CSVUtils.CHAR_VALIDATOR)
-            .defaultValue(",")
-            .build();
-
-    static final PropertyDescriptor QUOTE = new PropertyDescriptor.Builder()
-            .name("csv-quote-character")
-            .displayName("CSV quote character")
-            .description("Quote character for CSV values")
-            .addValidator(CSVUtils.CHAR_VALIDATOR)
-            .defaultValue("\"")
-            .build();
-    /**
-     * Utility operation to retrieve and parse {@link Schema} from Schema
-     * Registry using provided {@link SchemaRegistry};
-     */
-    static Schema retrieveSchema(SchemaRegistry schemaRegistry, InvocationContextProperties contextProperties) {
-        String schemaName = contextProperties.getPropertyValue(SCHEMA_NAME, true);
-        String schemaText = schemaRegistry.retrieveSchemaText(schemaName);
-        return new Schema.Parser().parse(schemaText);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToCSV.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToCSV.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToCSV.java
deleted file mode 100644
index aa0d418..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToCSV.java
+++ /dev/null
@@ -1,57 +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.schemaregistry.processors;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collections;
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.commons.io.IOUtils;
-import org.apache.nifi.annotation.behavior.InputRequirement;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-
-@Tags({ "registry", "schema", "avro", "csv", "transform" })
-@CapabilityDescription("Transforms AVRO content of the Flow File to CSV using the schema provided by the Schema Registry Service.")
-@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
-public final class TransformAvroToCSV extends AbstractCSVTransformer {
-
-    /**
-     *
-     */
-    @Override
-    protected Map<String, String> transform(InputStream in, OutputStream out, InvocationContextProperties contextProperties, Schema schema) {
-        byte[] buff = null;
-        try {
-            ByteArrayOutputStream bos = new ByteArrayOutputStream();
-            IOUtils.copy(in, bos);
-            buff = bos.toByteArray();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-        ByteArrayInputStream is = new ByteArrayInputStream(buff);
-        GenericRecord avroRecord = AvroUtils.read(is, schema);
-        CSVUtils.write(avroRecord, this.delimiter, out);
-        return Collections.singletonMap(CoreAttributes.MIME_TYPE.key(), "text/csv");
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToJson.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToJson.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToJson.java
deleted file mode 100644
index ba45563..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToJson.java
+++ /dev/null
@@ -1,46 +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.schemaregistry.processors;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collections;
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.nifi.annotation.behavior.InputRequirement;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-
-
-@Tags({ "registry", "schema", "avro", "json", "transform" })
-@CapabilityDescription("Transforms AVRO content of the Flow File to JSON using the schema provided by the Schema Registry Service.")
-@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
-public final class TransformAvroToJson extends AbstractContentTransformer {
-
-    /**
-     *
-     */
-    @Override
-    protected Map<String, String> transform(InputStream in, OutputStream out, InvocationContextProperties contextProperties, Schema schema) {
-        GenericRecord avroRecord = AvroUtils.read(in, schema);
-        JsonUtils.write(avroRecord, out);
-        return Collections.singletonMap(CoreAttributes.MIME_TYPE.key(), "application/json");
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToAvro.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToAvro.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToAvro.java
deleted file mode 100644
index f44e440..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToAvro.java
+++ /dev/null
@@ -1,80 +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.schemaregistry.processors;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.nifi.annotation.behavior.InputRequirement;
-import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
-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.components.PropertyDescriptor;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.processor.ProcessContext;
-
-@Tags({ "csv", "avro", "transform", "registry", "schema" })
-@InputRequirement(Requirement.INPUT_REQUIRED)
-@CapabilityDescription("Transforms CSV content of the Flow File to Avro using the schema provided by the Schema Registry Service.")
-public final class TransformCSVToAvro extends AbstractCSVTransformer {
-
-    private static final List<PropertyDescriptor> DESCRIPTORS;
-
-    static {
-        List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
-        descriptors.addAll(BASE_CSV_DESCRIPTORS);
-        descriptors.add(QUOTE);
-        DESCRIPTORS = Collections.unmodifiableList(descriptors);
-    }
-
-    private volatile char quoteChar;
-
-    /**
-     *
-     */
-    @Override
-    @OnScheduled
-    public void onScheduled(ProcessContext context) {
-        super.onScheduled(context);
-        this.quoteChar = context.getProperty(QUOTE).getValue().charAt(0);
-    }
-
-    /**
-     *
-     */
-    @Override
-    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return DESCRIPTORS;
-    }
-
-    /**
-     *
-     */
-    @Override
-    protected Map<String, String> transform(InputStream in, OutputStream out, InvocationContextProperties contextProperties, Schema schema) {
-        GenericRecord avroRecord = CSVUtils.read(in, this.delimiter, schema, this.quoteChar);
-        AvroUtils.write(avroRecord, out);
-        return Collections.singletonMap(CoreAttributes.MIME_TYPE.key(), "binary/avro");
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/68c592ea/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToJson.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToJson.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToJson.java
deleted file mode 100644
index 2ce9fbe..0000000
--- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToJson.java
+++ /dev/null
@@ -1,80 +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.schemaregistry.processors;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.nifi.annotation.behavior.InputRequirement;
-import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
-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.components.PropertyDescriptor;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.processor.ProcessContext;
-
-@Tags({ "csv", "json", "transform", "registry", "schema" })
-@InputRequirement(Requirement.INPUT_REQUIRED)
-@CapabilityDescription("Transforms CSV content of the Flow File to JSON using the schema provided by the Schema Registry Service.")
-public final class TransformCSVToJson extends AbstractCSVTransformer {
-
-    private static final List<PropertyDescriptor> DESCRIPTORS;
-
-    static {
-        List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
-        descriptors.addAll(BASE_CSV_DESCRIPTORS);
-        descriptors.add(QUOTE);
-        DESCRIPTORS = Collections.unmodifiableList(descriptors);
-    }
-
-    private volatile char quoteChar;
-
-    /**
-     *
-     */
-    @Override
-    @OnScheduled
-    public void onScheduled(ProcessContext context) {
-        super.onScheduled(context);
-        this.quoteChar = context.getProperty(QUOTE).getValue().charAt(0);
-    }
-
-    /**
-     *
-     */
-    @Override
-    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return DESCRIPTORS;
-    }
-
-    /**
-     *
-     */
-    @Override
-    protected Map<String, String> transform(InputStream in, OutputStream out, InvocationContextProperties contextProperties, Schema schema) {
-        GenericRecord avroRecord = CSVUtils.read(in, this.delimiter, schema, this.quoteChar);
-        JsonUtils.write(avroRecord, out);
-        return Collections.singletonMap(CoreAttributes.MIME_TYPE.key(), "application/json");
-    }
-}