You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2018/06/13 18:33:21 UTC

[1/6] nifi git commit: NIFI-4963: Added Hive3 bundle - Incorporated review comments - Added more defensive code for PutHive3Streaming error handling

Repository: nifi
Updated Branches:
  refs/heads/master 8feac9ae5 -> da99f873a


http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java
new file mode 100644
index 0000000..4682d76
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java
@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.util.orc;
+
+
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.hadoop.hive.ql.io.orc.NiFiOrcUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.UnionObject;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Unit tests for the NiFiOrcUtils helper class
+ */
+public class TestNiFiOrcUtils {
+
+    @Test
+    public void test_getOrcField_primitive() {
+        // Expected ORC types
+        TypeInfo[] expectedTypes = {
+                TypeInfoFactory.getPrimitiveTypeInfo("int"),
+                TypeInfoFactory.getPrimitiveTypeInfo("bigint"),
+                TypeInfoFactory.getPrimitiveTypeInfo("boolean"),
+                TypeInfoFactory.getPrimitiveTypeInfo("float"),
+                TypeInfoFactory.getPrimitiveTypeInfo("double"),
+                TypeInfoFactory.getPrimitiveTypeInfo("binary"),
+                TypeInfoFactory.getPrimitiveTypeInfo("string")
+        };
+
+        // Build a fake Avro record with all types
+        Schema testSchema = buildPrimitiveAvroSchema();
+        List<Schema.Field> fields = testSchema.getFields();
+        for (int i = 0; i < fields.size(); i++) {
+            assertEquals(expectedTypes[i], NiFiOrcUtils.getOrcField(fields.get(i).schema(), false));
+        }
+
+    }
+
+    @Test
+    public void test_getOrcField_union_optional_type() {
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
+        builder.name("union").type().unionOf().nullBuilder().endNull().and().booleanType().endUnion().noDefault();
+        Schema testSchema = builder.endRecord();
+        TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("union").schema(), false);
+        assertEquals(TypeInfoCreator.createBoolean(), orcType);
+    }
+
+    @Test
+    public void test_getOrcField_union() {
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
+        builder.name("union").type().unionOf().intType().and().booleanType().endUnion().noDefault();
+        Schema testSchema = builder.endRecord();
+        TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("union").schema(), false);
+        assertEquals(
+                TypeInfoFactory.getUnionTypeInfo(Arrays.asList(
+                        TypeInfoCreator.createInt(),
+                        TypeInfoCreator.createBoolean())),
+                orcType);
+    }
+
+    @Test
+    public void test_getOrcField_map() {
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
+        builder.name("map").type().map().values().doubleType().noDefault();
+        Schema testSchema = builder.endRecord();
+        TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("map").schema(), true);
+        assertEquals(
+                TypeInfoFactory.getMapTypeInfo(
+                        TypeInfoCreator.createString(),
+                        TypeInfoCreator.createDouble()),
+                orcType);
+    }
+
+    @Test
+    public void test_getOrcField_nested_map() {
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
+        builder.name("map").type().map().values().map().values().doubleType().noDefault();
+        Schema testSchema = builder.endRecord();
+        TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("map").schema(), false);
+        assertEquals(
+                TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(),
+                        TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(), TypeInfoCreator.createDouble())),
+                orcType);
+    }
+
+    @Test
+    public void test_getOrcField_array() {
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
+        builder.name("array").type().array().items().longType().noDefault();
+        Schema testSchema = builder.endRecord();
+        TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("array").schema(), false);
+        assertEquals(
+                TypeInfoFactory.getListTypeInfo(TypeInfoCreator.createLong()),
+                orcType);
+    }
+
+    @Test
+    public void test_getOrcField_complex_array() {
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
+        builder.name("Array").type().array().items().map().values().floatType().noDefault();
+        Schema testSchema = builder.endRecord();
+        TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("Array").schema(), true);
+        assertEquals(
+                TypeInfoFactory.getListTypeInfo(TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(), TypeInfoCreator.createFloat())),
+                orcType);
+    }
+
+    @Test
+    public void test_getOrcField_record() {
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
+        builder.name("Int").type().intType().noDefault();
+        builder.name("Long").type().longType().longDefault(1L);
+        builder.name("Array").type().array().items().stringType().noDefault();
+        Schema testSchema = builder.endRecord();
+        // Normalize field names for Hive, assert that their names are now lowercase
+        TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema, true);
+        assertEquals(
+                TypeInfoFactory.getStructTypeInfo(
+                        Arrays.asList("int", "long", "array"),
+                        Arrays.asList(
+                                TypeInfoCreator.createInt(),
+                                TypeInfoCreator.createLong(),
+                                TypeInfoFactory.getListTypeInfo(TypeInfoCreator.createString()))),
+                orcType);
+    }
+
+    @Test
+    public void test_getOrcField_enum() {
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
+        builder.name("enumField").type().enumeration("enum").symbols("a", "b", "c").enumDefault("a");
+        Schema testSchema = builder.endRecord();
+        TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("enumField").schema(), true);
+        assertEquals(TypeInfoCreator.createString(), orcType);
+    }
+
+    @Test
+    public void test_getPrimitiveOrcTypeFromPrimitiveAvroType() {
+        // Expected ORC types
+        TypeInfo[] expectedTypes = {
+                TypeInfoCreator.createInt(),
+                TypeInfoCreator.createLong(),
+                TypeInfoCreator.createBoolean(),
+                TypeInfoCreator.createFloat(),
+                TypeInfoCreator.createDouble(),
+                TypeInfoCreator.createBinary(),
+                TypeInfoCreator.createString(),
+        };
+
+        Schema testSchema = buildPrimitiveAvroSchema();
+        List<Schema.Field> fields = testSchema.getFields();
+        for (int i = 0; i < fields.size(); i++) {
+            assertEquals(expectedTypes[i], NiFiOrcUtils.getPrimitiveOrcTypeFromPrimitiveAvroType(fields.get(i).schema().getType()));
+        }
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void test_getPrimitiveOrcTypeFromPrimitiveAvroType_badType() {
+        Schema.Type nonPrimitiveType = Schema.Type.ARRAY;
+        NiFiOrcUtils.getPrimitiveOrcTypeFromPrimitiveAvroType(nonPrimitiveType);
+    }
+
+    @Test
+    public void test_getWritable() throws Exception {
+        assertTrue(NiFiOrcUtils.convertToORCObject(null, 1, true) instanceof IntWritable);
+        assertTrue(NiFiOrcUtils.convertToORCObject(null, 1L, true) instanceof LongWritable);
+        assertTrue(NiFiOrcUtils.convertToORCObject(null, 1.0f, true) instanceof FloatWritable);
+        assertTrue(NiFiOrcUtils.convertToORCObject(null, 1.0, true) instanceof DoubleWritable);
+        assertTrue(NiFiOrcUtils.convertToORCObject(null, new int[]{1, 2, 3}, true) instanceof List);
+        assertTrue(NiFiOrcUtils.convertToORCObject(null, Arrays.asList(1, 2, 3), true) instanceof List);
+        Map<String, Float> map = new HashMap<>();
+        map.put("Hello", 1.0f);
+        map.put("World", 2.0f);
+
+        Object convMap = NiFiOrcUtils.convertToORCObject(TypeInfoUtils.getTypeInfoFromTypeString("map<string,float>"), map, true);
+        assertTrue(convMap instanceof Map);
+        ((Map) convMap).forEach((key, value) -> {
+            assertTrue(key instanceof Text);
+            assertTrue(value instanceof FloatWritable);
+        });
+    }
+
+    @Test
+    public void test_getHiveTypeFromAvroType_primitive() {
+        // Expected ORC types
+        String[] expectedTypes = {
+                "INT",
+                "BIGINT",
+                "BOOLEAN",
+                "FLOAT",
+                "DOUBLE",
+                "BINARY",
+                "STRING",
+        };
+
+        Schema testSchema = buildPrimitiveAvroSchema();
+        List<Schema.Field> fields = testSchema.getFields();
+        for (int i = 0; i < fields.size(); i++) {
+            assertEquals(expectedTypes[i], NiFiOrcUtils.getHiveTypeFromAvroType(fields.get(i).schema(), false));
+        }
+    }
+
+    @Test
+    public void test_getHiveTypeFromAvroType_complex() {
+        // Expected ORC types
+        String[] expectedTypes = {
+                "INT",
+                "MAP<STRING, DOUBLE>",
+                "STRING",
+                "UNIONTYPE<BIGINT, FLOAT>",
+                "ARRAY<INT>"
+        };
+
+        Schema testSchema = buildComplexAvroSchema();
+        List<Schema.Field> fields = testSchema.getFields();
+        for (int i = 0; i < fields.size(); i++) {
+            assertEquals(expectedTypes[i], NiFiOrcUtils.getHiveTypeFromAvroType(fields.get(i).schema(), false));
+        }
+
+        assertEquals("STRUCT<myInt:INT, myMap:MAP<STRING, DOUBLE>, myEnum:STRING, myLongOrFloat:UNIONTYPE<BIGINT, FLOAT>, myIntList:ARRAY<INT>>",
+                NiFiOrcUtils.getHiveTypeFromAvroType(testSchema, false));
+    }
+
+    @Test
+    public void test_generateHiveDDL_primitive() {
+        Schema avroSchema = buildPrimitiveAvroSchema();
+        String ddl = NiFiOrcUtils.generateHiveDDL(avroSchema, "myHiveTable", false);
+        assertEquals("CREATE EXTERNAL TABLE IF NOT EXISTS myHiveTable (int INT, long BIGINT, boolean BOOLEAN, float FLOAT, double DOUBLE, bytes BINARY, string STRING)"
+                + " STORED AS ORC", ddl);
+    }
+
+    @Test
+    public void test_generateHiveDDL_complex() {
+        Schema avroSchema = buildComplexAvroSchema();
+        String ddl = NiFiOrcUtils.generateHiveDDL(avroSchema, "myHiveTable", false);
+        assertEquals("CREATE EXTERNAL TABLE IF NOT EXISTS myHiveTable "
+                + "(myInt INT, myMap MAP<STRING, DOUBLE>, myEnum STRING, myLongOrFloat UNIONTYPE<BIGINT, FLOAT>, myIntList ARRAY<INT>)"
+                + " STORED AS ORC", ddl);
+    }
+
+    @Test
+    public void test_generateHiveDDL_complex_normalize() {
+        Schema avroSchema = buildComplexAvroSchema();
+        String ddl = NiFiOrcUtils.generateHiveDDL(avroSchema, "myHiveTable", true);
+        assertEquals("CREATE EXTERNAL TABLE IF NOT EXISTS myHiveTable "
+                + "(myint INT, mymap MAP<STRING, DOUBLE>, myenum STRING, mylongorfloat UNIONTYPE<BIGINT, FLOAT>, myintlist ARRAY<INT>)"
+                + " STORED AS ORC", ddl);
+    }
+
+    @Test
+    public void test_convertToORCObject() {
+        Schema schema = SchemaBuilder.enumeration("myEnum").symbols("x", "y", "z");
+        List<Object> objects = Arrays.asList(new Utf8("Hello"), new GenericData.EnumSymbol(schema, "x"));
+        objects.forEach((avroObject) -> {
+            Object o = NiFiOrcUtils.convertToORCObject(TypeInfoUtils.getTypeInfoFromTypeString("uniontype<bigint,string>"), avroObject, true);
+            assertTrue(o instanceof UnionObject);
+            UnionObject uo = (UnionObject) o;
+            assertTrue(uo.getObject() instanceof Text);
+        });
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void test_convertToORCObjectBadUnion() {
+        NiFiOrcUtils.convertToORCObject(TypeInfoUtils.getTypeInfoFromTypeString("uniontype<bigint,long>"), "Hello", true);
+    }
+
+    @Test
+    public void test_getHiveTypeFromAvroType_complex_normalize() {
+        // Expected ORC types
+        String[] expectedTypes = {
+                "INT",
+                "MAP<STRING, DOUBLE>",
+                "STRING",
+                "UNIONTYPE<BIGINT, FLOAT>",
+                "ARRAY<INT>"
+        };
+
+        Schema testSchema = buildComplexAvroSchema();
+        List<Schema.Field> fields = testSchema.getFields();
+        for (int i = 0; i < fields.size(); i++) {
+            assertEquals(expectedTypes[i], NiFiOrcUtils.getHiveTypeFromAvroType(fields.get(i).schema(), true));
+        }
+
+        assertEquals("STRUCT<myint:INT, mymap:MAP<STRING, DOUBLE>, myenum:STRING, mylongorfloat:UNIONTYPE<BIGINT, FLOAT>, myintlist:ARRAY<INT>>",
+                NiFiOrcUtils.getHiveTypeFromAvroType(testSchema, true));
+    }
+
+    //////////////////
+    // Helper methods
+    //////////////////
+
+    public static Schema buildPrimitiveAvroSchema() {
+        // Build a fake Avro record with all primitive types
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("test.record").namespace("any.data").fields();
+        builder.name("int").type().intType().noDefault();
+        builder.name("long").type().longType().longDefault(1L);
+        builder.name("boolean").type().booleanType().booleanDefault(true);
+        builder.name("float").type().floatType().floatDefault(0.0f);
+        builder.name("double").type().doubleType().doubleDefault(0.0);
+        builder.name("bytes").type().bytesType().noDefault();
+        builder.name("string").type().stringType().stringDefault("default");
+        return builder.endRecord();
+    }
+
+    public static GenericData.Record buildPrimitiveAvroRecord(int i, long l, boolean b, float f, double d, ByteBuffer bytes, String string) {
+        Schema schema = buildPrimitiveAvroSchema();
+        GenericData.Record row = new GenericData.Record(schema);
+        row.put("int", i);
+        row.put("long", l);
+        row.put("boolean", b);
+        row.put("float", f);
+        row.put("double", d);
+        row.put("bytes", bytes);
+        row.put("string", string);
+        return row;
+    }
+
+    public static TypeInfo buildPrimitiveOrcSchema() {
+        return TypeInfoFactory.getStructTypeInfo(Arrays.asList("int", "long", "boolean", "float", "double", "bytes", "string"),
+                Arrays.asList(
+                        TypeInfoCreator.createInt(),
+                        TypeInfoCreator.createLong(),
+                        TypeInfoCreator.createBoolean(),
+                        TypeInfoCreator.createFloat(),
+                        TypeInfoCreator.createDouble(),
+                        TypeInfoCreator.createBinary(),
+                        TypeInfoCreator.createString()));
+    }
+
+    public static Schema buildComplexAvroSchema() {
+        // Build a fake Avro record with nested  types
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("complex.record").namespace("any.data").fields();
+        builder.name("myInt").type().unionOf().nullType().and().intType().endUnion().nullDefault();
+        builder.name("myMap").type().map().values().doubleType().noDefault();
+        builder.name("myEnum").type().enumeration("myEnum").symbols("ABC", "DEF", "XYZ").enumDefault("ABC");
+        builder.name("myLongOrFloat").type().unionOf().longType().and().floatType().endUnion().noDefault();
+        builder.name("myIntList").type().array().items().intType().noDefault();
+        return builder.endRecord();
+    }
+
+    public static GenericData.Record buildComplexAvroRecord(Integer i, Map<String, Double> m, String e, Object unionVal, List<Integer> intArray) {
+        Schema schema = buildComplexAvroSchema();
+        GenericData.Record row = new GenericData.Record(schema);
+        row.put("myInt", i);
+        row.put("myMap", m);
+        row.put("myEnum", e);
+        row.put("myLongOrFloat", unionVal);
+        row.put("myIntList", intArray);
+        return row;
+    }
+
+    public static TypeInfo buildComplexOrcSchema() {
+        return TypeInfoUtils.getTypeInfoFromTypeString("struct<myInt:int,myMap:map<string,double>,myEnum:string,myLongOrFloat:uniontype<int>,myIntList:array<int>>");
+    }
+
+    public static Schema buildNestedComplexAvroSchema() {
+        // Build a fake Avro record with nested complex types
+        final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("nested.complex.record").namespace("any.data").fields();
+        builder.name("myMapOfArray").type().map().values().array().items().doubleType().noDefault();
+        builder.name("myArrayOfMap").type().array().items().map().values().stringType().noDefault();
+        return builder.endRecord();
+    }
+
+    public static GenericData.Record buildNestedComplexAvroRecord(Map<String, List<Double>> m, List<Map<String, String>> a) {
+        Schema schema = buildNestedComplexAvroSchema();
+        GenericData.Record row = new GenericData.Record(schema);
+        row.put("myMapOfArray", m);
+        row.put("myArrayOfMap", a);
+        return row;
+    }
+
+    public static TypeInfo buildNestedComplexOrcSchema() {
+        return TypeInfoUtils.getTypeInfoFromTypeString("struct<myMapOfArray:map<string,array<double>>,myArrayOfMap:array<map<string,string>>>");
+    }
+
+    private static class TypeInfoCreator {
+        static TypeInfo createInt() {
+            return TypeInfoFactory.getPrimitiveTypeInfo("int");
+        }
+
+        static TypeInfo createLong() {
+            return TypeInfoFactory.getPrimitiveTypeInfo("bigint");
+        }
+
+        static TypeInfo createBoolean() {
+            return TypeInfoFactory.getPrimitiveTypeInfo("boolean");
+        }
+
+        static TypeInfo createFloat() {
+            return TypeInfoFactory.getPrimitiveTypeInfo("float");
+        }
+
+        static TypeInfo createDouble() {
+            return TypeInfoFactory.getPrimitiveTypeInfo("double");
+        }
+
+        static TypeInfo createBinary() {
+            return TypeInfoFactory.getPrimitiveTypeInfo("binary");
+        }
+
+        static TypeInfo createString() {
+            return TypeInfoFactory.getPrimitiveTypeInfo("string");
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/array_of_records.avsc
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/array_of_records.avsc b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/array_of_records.avsc
new file mode 100644
index 0000000..1e5154c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/array_of_records.avsc
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ {
+  "namespace" : "org.apache.nifi",
+  "name" : "outer_record",
+  "type" : "record",
+  "fields" : [ {
+    "name" : "records",
+    "type" : {
+      "type" : "array",
+      "items" : {
+        "type" : "record",
+        "name" : "inner_record",
+        "fields" : [ {
+          "name" : "name",
+          "type" : "string"
+        }, {
+          "name" : "age",
+          "type" : "string"
+        } ]
+      }
+    }
+  } ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/core-site-security.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/core-site-security.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/core-site-security.xml
new file mode 100644
index 0000000..eefc74e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/core-site-security.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<configuration>
+    <property>
+        <name>fs.default.name</name>
+        <value>hdfs://hive</value>
+    </property>
+    <property>
+        <name>hadoop.security.authentication</name>
+        <value>kerberos</value>
+    </property>
+    <property>
+        <name>hadoop.security.authorization</name>
+        <value>true</value>
+    </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/core-site.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/core-site.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/core-site.xml
new file mode 100644
index 0000000..8a7d178
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/core-site.xml
@@ -0,0 +1,22 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<configuration>
+    <property>
+        <name>fs.default.name</name>
+        <value>file:///</value>
+    </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/fake.keytab
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/fake.keytab b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/fake.keytab
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/hive-site-security.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/hive-site-security.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/hive-site-security.xml
new file mode 100644
index 0000000..07fd74c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/hive-site-security.xml
@@ -0,0 +1,26 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<configuration>
+  <property>
+    <name>fs.default.name</name>
+    <value>hdfs://hive</value>
+  </property>
+  <property>
+    <name>hive.server2.authentication</name>
+    <value>KERBEROS</value>
+  </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/hive-site.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/hive-site.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/hive-site.xml
new file mode 100644
index 0000000..7e7f86c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/hive-site.xml
@@ -0,0 +1,22 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<configuration>
+  <property>
+    <name>fs.default.name</name>
+    <value>file:///</value>
+  </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/krb5.conf
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/krb5.conf b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/krb5.conf
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/user.avsc
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/user.avsc b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/user.avsc
new file mode 100644
index 0000000..95ef6e4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/user.avsc
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+{"namespace": "example.avro",
+ "type": "record",
+ "name": "User",
+ "fields": [
+     {"name": "name", "type": "string"},
+     {"name": "favorite_number",  "type": ["int", "null"]},
+     {"name": "favorite_color", "type": ["string", "null"]},
+     {"name": "scale", "type": ["double", "null"]}
+ ]
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/user_logical_types.avsc
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/user_logical_types.avsc b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/user_logical_types.avsc
new file mode 100644
index 0000000..054a2f2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/resources/user_logical_types.avsc
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+{"namespace": "example.avro",
+ "type": "record",
+ "name": "User",
+ "fields": [
+     {"name": "id",  "type": ["null","int"]},
+     {"name": "timeMillis","type": {"type": "int","logicalType": "time-millis"}},
+     {"name": "timestampMillis","type": {"type": "long","logicalType": "timestamp-millis"}},
+     {"name": "dt","type": {"type": "int","logicalType": "date"}},
+     {"name": "dec","type": {"type": "bytes","logicalType": "decimal", "precision": 4, "scale": 2}}
+ ]
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/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 fc7a751..51c9640 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
@@ -26,63 +26,22 @@
     <version>1.7.0-SNAPSHOT</version>
     <packaging>pom</packaging>
 
-    <properties>
-        <!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
-        <hive.hadoop.version>2.6.2</hive.hadoop.version>
-        <hadoop.version>${hive.hadoop.version}</hadoop.version>
-    </properties>
-
     <modules>
         <module>nifi-hive-services-api</module>
         <module>nifi-hive-services-api-nar</module>
         <module>nifi-hive-processors</module>
         <module>nifi-hive-nar</module>
+        <module>nifi-hive3-processors</module>
+        <module>nifi-hive3-nar</module>
     </modules>
 
-    <dependencyManagement>
-        <dependencies>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-common</artifactId>
-                <version>${hadoop.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-hdfs</artifactId>
-                <version>${hadoop.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-yarn-api</artifactId>
-                <version>${hadoop.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-mapreduce-client-core</artifactId>
-                <version>${hadoop.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-annotations</artifactId>
-                <version>${hadoop.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-client</artifactId>
-                <version>${hadoop.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-auth</artifactId>
-                <version>${hadoop.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.avro</groupId>
-                <artifactId>avro</artifactId>
-                <version>1.7.7</version>
-            </dependency>
-        </dependencies>
-    </dependencyManagement>
+    <properties>
+        <hive.version>1.2.1</hive.version>
+        <hive.hadoop.version>2.6.2</hive.hadoop.version>
+        <hive3.version>3.0.0</hive3.version>
+        <hive3.hadoop.version>3.0.0</hive3.hadoop.version>
+        <orc.version>1.4.3</orc.version>
+    </properties>
 
     <build>
         <plugins>


[5/6] nifi git commit: NIFI-4963: Added Hive3 bundle - Incorporated review comments - Added more defensive code for PutHive3Streaming error handling

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/dbcp/hive/Hive3ConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/dbcp/hive/Hive3ConnectionPool.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/dbcp/hive/Hive3ConnectionPool.java
new file mode 100644
index 0000000..b0662b8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/dbcp/hive/Hive3ConnectionPool.java
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.dbcp.hive;
+
+import java.io.File;
+
+import org.apache.commons.dbcp.BasicDataSource;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.jdbc.HiveDriver;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+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.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.hadoop.KerberosProperties;
+import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.kerberos.KerberosCredentialsService;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.hive.AuthenticationFailedException;
+import org.apache.nifi.util.hive.HiveConfigurator;
+import org.apache.nifi.util.hive.HiveUtils;
+import org.apache.nifi.util.hive.ValidationResources;
+
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.nifi.controller.ControllerServiceInitializationContext;
+
+/**
+ * Implementation for Database Connection Pooling Service used for Apache Hive
+ * connections. Apache DBCP is used for connection pooling functionality.
+ */
+@RequiresInstanceClassLoading
+@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive 3.x. Connections can be asked from pool and returned after usage.")
+public class Hive3ConnectionPool extends AbstractControllerService implements Hive3DBCPService {
+    private static final String ALLOW_EXPLICIT_KEYTAB = "NIFI_ALLOW_EXPLICIT_KEYTAB";
+
+    static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
+            .name("hive-db-connect-url")
+            .displayName("Database Connection URL")
+            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
+                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
+                    + "as a connection parameter when connecting to a secure Hive server.")
+            .defaultValue(null)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
+            .name("hive-config-resources")
+            .displayName("Hive Configuration Resources")
+            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
+                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration. Note that to enable authentication "
+                    + "with Kerberos e.g., the appropriate properties must be set in the configuration files. Please see the Hive documentation for more details.")
+            .required(false)
+            .addValidator(HiveUtils.createMultipleFilesExistValidator())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
+            .name("hive-db-user")
+            .displayName("Database User")
+            .description("Database user name")
+            .defaultValue(null)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
+            .name("hive-db-password")
+            .displayName("Password")
+            .description("The password for the database user")
+            .defaultValue(null)
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+            .name("hive-max-wait-time")
+            .displayName("Max Wait Time")
+            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
+                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
+            .defaultValue("500 millis")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+            .name("hive-max-total-connections")
+            .displayName("Max Total Connections")
+            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
+                    + "or negative for no limit.")
+            .defaultValue("8")
+            .required(true)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+            .name("Validation-query")
+            .displayName("Validation query")
+            .description("Validation query used to validate connections before returning them. "
+                    + "When a borrowed connection is invalid, it gets dropped and a new valid connection will be returned. "
+                    + "NOTE: Using validation may have a performance penalty.")
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    private static final PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("kerberos-credentials-service")
+            .displayName("Kerberos Credentials Service")
+            .description("Specifies the Kerberos Credentials Controller Service that should be used for authenticating with Kerberos")
+            .identifiesControllerService(KerberosCredentialsService.class)
+            .required(false)
+            .build();
+
+
+    private List<PropertyDescriptor> properties;
+
+    private String connectionUrl = "unknown";
+
+    // Holder of cached Configuration information so validation does not reload the same config over and over
+    private final AtomicReference<ValidationResources> validationResourceHolder = new AtomicReference<>();
+
+    private volatile BasicDataSource dataSource;
+
+    private volatile HiveConfigurator hiveConfigurator = new HiveConfigurator();
+    private volatile UserGroupInformation ugi;
+    private volatile File kerberosConfigFile = null;
+    private volatile KerberosProperties kerberosProperties;
+
+    @Override
+    protected void init(final ControllerServiceInitializationContext context) {
+        List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(DATABASE_URL);
+        props.add(HIVE_CONFIGURATION_RESOURCES);
+        props.add(DB_USER);
+        props.add(DB_PASSWORD);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(VALIDATION_QUERY);
+        props.add(KERBEROS_CREDENTIALS_SERVICE);
+
+        kerberosConfigFile = context.getKerberosConfigurationFile();
+        kerberosProperties = new KerberosProperties(kerberosConfigFile);
+        props.add(kerberosProperties.getKerberosPrincipal());
+        props.add(kerberosProperties.getKerberosKeytab());
+        properties = props;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        boolean confFileProvided = validationContext.getProperty(HIVE_CONFIGURATION_RESOURCES).isSet();
+
+        final List<ValidationResult> problems = new ArrayList<>();
+
+        if (confFileProvided) {
+            final String explicitPrincipal = validationContext.getProperty(kerberosProperties.getKerberosPrincipal()).evaluateAttributeExpressions().getValue();
+            final String explicitKeytab = validationContext.getProperty(kerberosProperties.getKerberosKeytab()).evaluateAttributeExpressions().getValue();
+            final KerberosCredentialsService credentialsService = validationContext.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
+
+            final String resolvedPrincipal;
+            final String resolvedKeytab;
+            if (credentialsService == null) {
+                resolvedPrincipal = explicitPrincipal;
+                resolvedKeytab = explicitKeytab;
+            } else {
+                resolvedPrincipal = credentialsService.getPrincipal();
+                resolvedKeytab = credentialsService.getKeytab();
+            }
+
+
+            final String configFiles = validationContext.getProperty(HIVE_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue();
+            problems.addAll(hiveConfigurator.validate(configFiles, resolvedPrincipal, resolvedKeytab, validationResourceHolder, getLogger()));
+
+            if (credentialsService != null && (explicitPrincipal != null || explicitKeytab != null)) {
+                problems.add(new ValidationResult.Builder()
+                        .subject("Kerberos Credentials")
+                        .valid(false)
+                        .explanation("Cannot specify both a Kerberos Credentials Service and a principal/keytab")
+                        .build());
+            }
+
+            final String allowExplicitKeytabVariable = System.getenv(ALLOW_EXPLICIT_KEYTAB);
+            if ("false".equalsIgnoreCase(allowExplicitKeytabVariable) && (explicitPrincipal != null || explicitKeytab != null)) {
+                problems.add(new ValidationResult.Builder()
+                        .subject("Kerberos Credentials")
+                        .valid(false)
+                        .explanation("The '" + ALLOW_EXPLICIT_KEYTAB + "' system environment variable is configured to forbid explicitly configuring principal/keytab in processors. "
+                                + "The Kerberos Credentials Service should be used instead of setting the Kerberos Keytab or Kerberos Principal property.")
+                        .build());
+            }
+        }
+
+        return problems;
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     * <p/>
+     * As of Apache NiFi 1.5.0, due to changes made to
+     * {@link SecurityUtil#loginKerberos(Configuration, String, String)}, which is used by this class invoking
+     * {@link HiveConfigurator#authenticate(Configuration, String, String)}
+     * to authenticate a principal with Kerberos, Hive controller services no longer
+     * attempt relogins explicitly.  For more information, please read the documentation for
+     * {@link SecurityUtil#loginKerberos(Configuration, String, String)}.
+     * <p/>
+     * In previous versions of NiFi, a {@link org.apache.nifi.hadoop.KerberosTicketRenewer} was started by
+     * {@link HiveConfigurator#authenticate(Configuration, String, String, long)} when the Hive
+     * controller service was enabled.  The use of a separate thread to explicitly relogin could cause race conditions
+     * with the implicit relogin attempts made by hadoop/Hive code on a thread that references the same
+     * {@link UserGroupInformation} instance.  One of these threads could leave the
+     * {@link javax.security.auth.Subject} in {@link UserGroupInformation} to be cleared or in an unexpected state
+     * while the other thread is attempting to use the {@link javax.security.auth.Subject}, resulting in failed
+     * authentication attempts that would leave the Hive controller service in an unrecoverable state.
+     *
+     * @see SecurityUtil#loginKerberos(Configuration, String, String)
+     * @see HiveConfigurator#authenticate(Configuration, String, String)
+     * @see HiveConfigurator#authenticate(Configuration, String, String, long)
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+
+        ComponentLog log = getLogger();
+
+        final String configFiles = context.getProperty(HIVE_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue();
+        final Configuration hiveConfig = hiveConfigurator.getConfigurationFromFiles(configFiles);
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+
+        // add any dynamic properties to the Hive configuration
+        for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
+            final PropertyDescriptor descriptor = entry.getKey();
+            if (descriptor.isDynamic()) {
+                hiveConfig.set(descriptor.getName(), context.getProperty(descriptor).evaluateAttributeExpressions().getValue());
+            }
+        }
+
+        final String drv = HiveDriver.class.getName();
+        if (SecurityUtil.isSecurityEnabled(hiveConfig)) {
+            final String explicitPrincipal = context.getProperty(kerberosProperties.getKerberosPrincipal()).evaluateAttributeExpressions().getValue();
+            final String explicitKeytab = context.getProperty(kerberosProperties.getKerberosKeytab()).evaluateAttributeExpressions().getValue();
+            final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
+
+            final String resolvedPrincipal;
+            final String resolvedKeytab;
+            if (credentialsService == null) {
+                resolvedPrincipal = explicitPrincipal;
+                resolvedKeytab = explicitKeytab;
+            } else {
+                resolvedPrincipal = credentialsService.getPrincipal();
+                resolvedKeytab = credentialsService.getKeytab();
+            }
+
+            log.info("Hive Security Enabled, logging in as principal {} with keytab {}", new Object[] {resolvedPrincipal, resolvedKeytab});
+            try {
+                ugi = hiveConfigurator.authenticate(hiveConfig, resolvedPrincipal, resolvedKeytab);
+            } catch (AuthenticationFailedException ae) {
+                log.error(ae.getMessage(), ae);
+            }
+
+            getLogger().info("Successfully logged in as principal {} with keytab {}", new Object[] {resolvedPrincipal, resolvedKeytab});
+        }
+
+        final String user = context.getProperty(DB_USER).evaluateAttributeExpressions().getValue();
+        final String passw = context.getProperty(DB_PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS);
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+
+        dataSource = new BasicDataSource();
+        dataSource.setDriverClassName(drv);
+
+        connectionUrl = context.getProperty(DATABASE_URL).evaluateAttributeExpressions().getValue();
+
+        dataSource.setMaxWait(maxWaitMillis);
+        dataSource.setMaxActive(maxTotal);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        dataSource.setUrl(connectionUrl);
+        dataSource.setUsername(user);
+        dataSource.setPassword(passw);
+    }
+
+    /**
+     * Shutdown pool, close all open connections.
+     */
+    @OnDisabled
+    public void shutdown() {
+        try {
+            dataSource.close();
+        } catch (final SQLException e) {
+            throw new ProcessException(e);
+        }
+    }
+
+    @Override
+    public Connection getConnection() throws ProcessException {
+        try {
+            if (ugi != null) {
+                try {
+                    return ugi.doAs((PrivilegedExceptionAction<Connection>) () -> dataSource.getConnection());
+                } catch (UndeclaredThrowableException e) {
+                    Throwable cause = e.getCause();
+                    if (cause instanceof SQLException) {
+                        throw (SQLException) cause;
+                    } else {
+                        throw e;
+                    }
+                }
+            } else {
+                getLogger().info("Simple Authentication");
+                return dataSource.getConnection();
+            }
+        } catch (SQLException | IOException | InterruptedException e) {
+            getLogger().error("Error getting Hive connection", e);
+            throw new ProcessException(e);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "Hive3ConnectionPool[id=" + getIdentifier() + "]";
+    }
+
+    @Override
+    public String getConnectionURL() {
+        return connectionUrl;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/AbstractHive3QLProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/AbstractHive3QLProcessor.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/AbstractHive3QLProcessor.java
new file mode 100644
index 0000000..4fcce19
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/AbstractHive3QLProcessor.java
@@ -0,0 +1,348 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.hive;
+
+import org.antlr.runtime.tree.CommonTree;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseException;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.dbcp.hive.Hive3DBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.stream.io.StreamUtils;
+
+import java.math.BigDecimal;
+import java.nio.charset.Charset;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.SQLDataException;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * An abstract base class for HiveQL processors to share common data, methods, etc.
+ */
+public abstract class AbstractHive3QLProcessor extends AbstractSessionFactoryProcessor {
+
+    protected static final Pattern HIVEQL_TYPE_ATTRIBUTE_PATTERN = Pattern.compile("hiveql\\.args\\.(\\d+)\\.type");
+    protected static final Pattern NUMBER_PATTERN = Pattern.compile("-?\\d+");
+    static String ATTR_INPUT_TABLES = "query.input.tables";
+    static String ATTR_OUTPUT_TABLES = "query.output.tables";
+
+
+    public static final PropertyDescriptor HIVE_DBCP_SERVICE = new PropertyDescriptor.Builder()
+            .name("hive3-dbcp-service")
+            .displayName("Hive Database Connection Pooling Service")
+            .description("The Hive Controller Service that is used to obtain connection(s) to the Hive database")
+            .required(true)
+            .identifiesControllerService(Hive3DBCPService.class)
+            .build();
+
+    public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
+            .name("hive3-charset")
+            .displayName("Character Set")
+            .description("Specifies the character set of the record data.")
+            .required(true)
+            .defaultValue("UTF-8")
+            .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor QUERY_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("hive3-query-timeout")
+            .displayName("Query timeout")
+            .description("Sets the number of seconds the driver will wait for a query to execute. "
+                    + "A value of 0 means no timeout. NOTE: Non-zero values may not be supported by the driver.")
+            .defaultValue("0")
+            .required(true)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    /**
+     * Determines the HiveQL statement that should be executed for the given FlowFile
+     *
+     * @param session  the session that can be used to access the given FlowFile
+     * @param flowFile the FlowFile whose HiveQL statement should be executed
+     * @return the HiveQL that is associated with the given FlowFile
+     */
+    protected String getHiveQL(final ProcessSession session, final FlowFile flowFile, final Charset charset) {
+        // Read the HiveQL from the FlowFile's content
+        final byte[] buffer = new byte[(int) flowFile.getSize()];
+        session.read(flowFile, in -> StreamUtils.fillBuffer(in, buffer));
+
+        // Create the PreparedStatement to use for this FlowFile.
+        return new String(buffer, charset);
+    }
+
+    private class ParameterHolder {
+        String attributeName;
+        int jdbcType;
+        String value;
+    }
+
+    /**
+     * Sets all of the appropriate parameters on the given PreparedStatement, based on the given FlowFile attributes.
+     *
+     * @param stmt       the statement to set the parameters on
+     * @param attributes the attributes from which to derive parameter indices, values, and types
+     * @throws SQLException if the PreparedStatement throws a SQLException when the appropriate setter is called
+     */
+    protected int setParameters(int base, final PreparedStatement stmt, int paramCount, final Map<String, String> attributes) throws SQLException {
+
+        Map<Integer, ParameterHolder> parmMap = new TreeMap<Integer, ParameterHolder>();
+
+        for (final Map.Entry<String, String> entry : attributes.entrySet()) {
+                final String key = entry.getKey();
+                final Matcher matcher = HIVEQL_TYPE_ATTRIBUTE_PATTERN.matcher(key);
+                if (matcher.matches()) {
+                    final int parameterIndex = Integer.parseInt(matcher.group(1));
+                    if (parameterIndex >= base && parameterIndex < base + paramCount) {
+                        final boolean isNumeric = NUMBER_PATTERN.matcher(entry.getValue()).matches();
+                        if (!isNumeric) {
+                            throw new SQLDataException("Value of the " + key + " attribute is '" + entry.getValue() + "', which is not a valid JDBC numeral jdbcType");
+                        }
+
+                        final String valueAttrName = "hiveql.args." + parameterIndex + ".value";
+
+                        ParameterHolder ph = new ParameterHolder();
+                        int realIndexLoc = parameterIndex - base +1;
+
+                        ph.jdbcType = Integer.parseInt(entry.getValue());
+                        ph.value = attributes.get(valueAttrName);
+                        ph.attributeName = valueAttrName;
+
+                        parmMap.put(realIndexLoc, ph);
+
+                    }
+                }
+        }
+
+
+        // Now that's we've retrieved the correct number of parameters and it's sorted, let's set them.
+        for (final Map.Entry<Integer, ParameterHolder> entry : parmMap.entrySet()) {
+            final Integer index = entry.getKey();
+            final ParameterHolder ph = entry.getValue();
+
+            try {
+                setParameter(stmt, ph.attributeName, index, ph.value, ph.jdbcType);
+            } catch (final NumberFormatException nfe) {
+                throw new SQLDataException("The value of the " + ph.attributeName + " is '" + ph.value + "', which cannot be converted into the necessary data jdbcType", nfe);
+            }
+        }
+        return base + paramCount;
+    }
+
+    /**
+     * Determines how to map the given value to the appropriate JDBC data jdbcType and sets the parameter on the
+     * provided PreparedStatement
+     *
+     * @param stmt           the PreparedStatement to set the parameter on
+     * @param attrName       the name of the attribute that the parameter is coming from - for logging purposes
+     * @param parameterIndex the index of the HiveQL parameter to set
+     * @param parameterValue the value of the HiveQL parameter to set
+     * @param jdbcType       the JDBC Type of the HiveQL parameter to set
+     * @throws SQLException if the PreparedStatement throws a SQLException when calling the appropriate setter
+     */
+    protected void setParameter(final PreparedStatement stmt, final String attrName, final int parameterIndex, final String parameterValue, final int jdbcType) throws SQLException {
+        if (parameterValue == null) {
+            stmt.setNull(parameterIndex, jdbcType);
+        } else {
+            try {
+                switch (jdbcType) {
+                    case Types.BIT:
+                    case Types.BOOLEAN:
+                        stmt.setBoolean(parameterIndex, Boolean.parseBoolean(parameterValue));
+                        break;
+                    case Types.TINYINT:
+                        stmt.setByte(parameterIndex, Byte.parseByte(parameterValue));
+                        break;
+                    case Types.SMALLINT:
+                        stmt.setShort(parameterIndex, Short.parseShort(parameterValue));
+                        break;
+                    case Types.INTEGER:
+                        stmt.setInt(parameterIndex, Integer.parseInt(parameterValue));
+                        break;
+                    case Types.BIGINT:
+                        stmt.setLong(parameterIndex, Long.parseLong(parameterValue));
+                        break;
+                    case Types.REAL:
+                        stmt.setFloat(parameterIndex, Float.parseFloat(parameterValue));
+                        break;
+                    case Types.FLOAT:
+                    case Types.DOUBLE:
+                        stmt.setDouble(parameterIndex, Double.parseDouble(parameterValue));
+                        break;
+                    case Types.DECIMAL:
+                    case Types.NUMERIC:
+                        stmt.setBigDecimal(parameterIndex, new BigDecimal(parameterValue));
+                        break;
+                    case Types.DATE:
+                        stmt.setDate(parameterIndex, new Date(Long.parseLong(parameterValue)));
+                        break;
+                    case Types.TIME:
+                        stmt.setTime(parameterIndex, new Time(Long.parseLong(parameterValue)));
+                        break;
+                    case Types.TIMESTAMP:
+                        stmt.setTimestamp(parameterIndex, new Timestamp(Long.parseLong(parameterValue)));
+                        break;
+                    case Types.CHAR:
+                    case Types.VARCHAR:
+                    case Types.LONGNVARCHAR:
+                    case Types.LONGVARCHAR:
+                        stmt.setString(parameterIndex, parameterValue);
+                        break;
+                    default:
+                        stmt.setObject(parameterIndex, parameterValue, jdbcType);
+                        break;
+                }
+            } catch (SQLException e) {
+                // Log which attribute/parameter had an error, then rethrow to be handled at the top level
+                getLogger().error("Error setting parameter {} to value from {} ({})", new Object[]{parameterIndex, attrName, parameterValue}, e);
+                throw e;
+            }
+        }
+    }
+
+    protected static class TableName {
+        private final String database;
+        private final String table;
+        private final boolean input;
+
+        TableName(String database, String table, boolean input) {
+            this.database = database;
+            this.table = table;
+            this.input = input;
+        }
+
+        public String getDatabase() {
+            return database;
+        }
+
+        public String getTable() {
+            return table;
+        }
+
+        public boolean isInput() {
+            return input;
+        }
+
+        @Override
+        public String toString() {
+            return database == null || database.isEmpty() ? table : database + '.' + table;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            TableName tableName = (TableName) o;
+
+            if (input != tableName.input) return false;
+            if (database != null ? !database.equals(tableName.database) : tableName.database != null) return false;
+            return table.equals(tableName.table);
+        }
+
+        @Override
+        public int hashCode() {
+            int result = database != null ? database.hashCode() : 0;
+            result = 31 * result + table.hashCode();
+            result = 31 * result + (input ? 1 : 0);
+            return result;
+        }
+    }
+
+    protected Set<TableName> findTableNames(final String query) {
+        final ASTNode node;
+        try {
+            node = new ParseDriver().parse(normalize(query));
+        } catch (ParseException e) {
+            // If failed to parse the query, just log a message, but continue.
+            getLogger().debug("Failed to parse query: {} due to {}", new Object[]{query, e}, e);
+            return Collections.emptySet();
+        }
+        final HashSet<TableName> tableNames = new HashSet<>();
+        findTableNames(node, tableNames);
+        return tableNames;
+    }
+
+    /**
+     * Normalize query.
+     * Hive resolves prepared statement parameters before executing a query,
+     * see {@link org.apache.hive.jdbc.HivePreparedStatement#updateSql(String, HashMap)} for detail.
+     * HiveParser does not expect '?' to be in a query string, and throws an Exception if there is one.
+     * In this normalize method, '?' is replaced to 'x' to avoid that.
+     */
+    private String normalize(String query) {
+        return query.replace('?', 'x');
+    }
+
+    private void findTableNames(final Object obj, final Set<TableName> tableNames) {
+        if (!(obj instanceof CommonTree)) {
+            return;
+        }
+        final CommonTree tree = (CommonTree) obj;
+        final int childCount = tree.getChildCount();
+        if ("TOK_TABNAME".equals(tree.getText())) {
+            final TableName tableName;
+            final boolean isInput = "TOK_TABREF".equals(tree.getParent().getText());
+            switch (childCount) {
+                case 1 :
+                    tableName = new TableName(null, tree.getChild(0).getText(), isInput);
+                    break;
+                case 2:
+                    tableName = new TableName(tree.getChild(0).getText(), tree.getChild(1).getText(), isInput);
+                    break;
+                default:
+                    throw new IllegalStateException("TOK_TABNAME does not have expected children, childCount=" + childCount);
+            }
+            // If parent is TOK_TABREF, then it is an input table.
+            tableNames.add(tableName);
+            return;
+        }
+        for (int i = 0; i < childCount; i++) {
+            findTableNames(tree.getChild(i), tableNames);
+        }
+    }
+
+    protected Map<String, String> toQueryTableAttributes(Set<TableName> tableNames) {
+        final Map<String, String> attributes = new HashMap<>();
+        for (TableName tableName : tableNames) {
+            final String attributeName = tableName.isInput() ? ATTR_INPUT_TABLES : ATTR_OUTPUT_TABLES;
+            if (attributes.containsKey(attributeName)) {
+                attributes.put(attributeName, attributes.get(attributeName) + "," + tableName);
+            } else {
+                attributes.put(attributeName, tableName.toString());
+            }
+        }
+        return attributes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3QL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3QL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3QL.java
new file mode 100644
index 0000000..989d085
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3QL.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.processors.hive;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.ReadsAttributes;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.dbcp.hive.Hive3DBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.pattern.ErrorTypes;
+import org.apache.nifi.processor.util.pattern.ExceptionHandler;
+import org.apache.nifi.processor.util.pattern.ExceptionHandler.OnError;
+import org.apache.nifi.processor.util.pattern.PartialFunctions.FetchFlowFiles;
+import org.apache.nifi.processor.util.pattern.PartialFunctions.InitConnection;
+import org.apache.nifi.processor.util.pattern.Put;
+import org.apache.nifi.processor.util.pattern.RollbackOnFailure;
+import org.apache.nifi.processor.util.pattern.RoutingResult;
+
+import java.nio.charset.Charset;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.SQLNonTransientException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+@SeeAlso(SelectHive3QL.class)
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"sql", "hive", "put", "database", "update", "insert"})
+@CapabilityDescription("Executes a HiveQL DDL/DML command (UPDATE, INSERT, e.g.). The content of an incoming FlowFile is expected to be the HiveQL command "
+        + "to execute. The HiveQL command may use the ? to escape parameters. In this case, the parameters to use must exist as FlowFile attributes "
+        + "with the naming convention hiveql.args.N.type and hiveql.args.N.value, where N is a positive integer. The hiveql.args.N.type is expected to be "
+        + "a number indicating the JDBC Type. The content of the FlowFile is expected to be in UTF-8 format.")
+@ReadsAttributes({
+        @ReadsAttribute(attribute = "hiveql.args.N.type", description = "Incoming FlowFiles are expected to be parametrized HiveQL statements. The type of each Parameter is specified as an integer "
+                + "that represents the JDBC Type of the parameter."),
+        @ReadsAttribute(attribute = "hiveql.args.N.value", description = "Incoming FlowFiles are expected to be parametrized HiveQL statements. The value of the Parameters are specified as "
+                + "hiveql.args.1.value, hiveql.args.2.value, hiveql.args.3.value, and so on. The type of the hiveql.args.1.value Parameter is specified by the hiveql.args.1.type attribute.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "query.input.tables", description = "This attribute is written on the flow files routed to the 'success' relationships, "
+                + "and contains input table names (if any) in comma delimited 'databaseName.tableName' format."),
+        @WritesAttribute(attribute = "query.output.tables", description = "This attribute is written on the flow files routed to the 'success' relationships, "
+                + "and contains the target table names in 'databaseName.tableName' format.")
+})
+public class PutHive3QL extends AbstractHive3QLProcessor {
+
+    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("hive-batch-size")
+            .displayName("Batch Size")
+            .description("The preferred number of FlowFiles to put to the database in a single transaction")
+            .required(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .defaultValue("100")
+            .build();
+
+    public static final PropertyDescriptor STATEMENT_DELIMITER = new PropertyDescriptor.Builder()
+            .name("statement-delimiter")
+            .displayName("Statement Delimiter")
+            .description("Statement Delimiter used to separate SQL statements in a multiple statement script")
+            .required(true)
+            .defaultValue(";")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("A FlowFile is routed to this relationship after the database is successfully updated")
+            .build();
+    public static final Relationship REL_RETRY = new Relationship.Builder()
+            .name("retry")
+            .description("A FlowFile is routed to this relationship if the database cannot be updated but attempting the operation again may succeed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("A FlowFile is routed to this relationship if the database cannot be updated and retrying the operation will also fail, "
+                    + "such as an invalid query or an integrity constraint violation")
+            .build();
+
+
+    private final static List<PropertyDescriptor> propertyDescriptors;
+    private final static Set<Relationship> relationships;
+
+    /*
+     * Will ensure that the list of property descriptors is built only once.
+     * Will also create a Set of relationships
+     */
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(HIVE_DBCP_SERVICE);
+        _propertyDescriptors.add(BATCH_SIZE);
+        _propertyDescriptors.add(QUERY_TIMEOUT);
+        _propertyDescriptors.add(CHARSET);
+        _propertyDescriptors.add(STATEMENT_DELIMITER);
+        _propertyDescriptors.add(RollbackOnFailure.ROLLBACK_ON_FAILURE);
+        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
+
+        Set<Relationship> _relationships = new HashSet<>();
+        _relationships.add(REL_SUCCESS);
+        _relationships.add(REL_FAILURE);
+        _relationships.add(REL_RETRY);
+        relationships = Collections.unmodifiableSet(_relationships);
+    }
+
+    private Put<FunctionContext, Connection> process;
+    private ExceptionHandler<FunctionContext> exceptionHandler;
+
+    @OnScheduled
+    public void constructProcess() {
+        exceptionHandler = new ExceptionHandler<>();
+        exceptionHandler.mapException(e -> {
+            if (e instanceof SQLNonTransientException) {
+                return ErrorTypes.InvalidInput;
+            } else if (e instanceof SQLException) {
+                return ErrorTypes.TemporalFailure;
+            } else {
+                return ErrorTypes.UnknownFailure;
+            }
+        });
+        exceptionHandler.adjustError(RollbackOnFailure.createAdjustError(getLogger()));
+
+        process = new Put<>();
+        process.setLogger(getLogger());
+        process.initConnection(initConnection);
+        process.fetchFlowFiles(fetchFlowFiles);
+        process.putFlowFile(putFlowFile);
+        process.adjustRoute(RollbackOnFailure.createAdjustRoute(REL_FAILURE, REL_RETRY));
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return propertyDescriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private class FunctionContext extends RollbackOnFailure {
+        final Charset charset;
+        final String statementDelimiter;
+        final long startNanos = System.nanoTime();
+
+        String connectionUrl;
+
+
+        private FunctionContext(boolean rollbackOnFailure, Charset charset, String statementDelimiter) {
+            super(rollbackOnFailure, false);
+            this.charset = charset;
+            this.statementDelimiter = statementDelimiter;
+        }
+    }
+
+    private InitConnection<FunctionContext, Connection> initConnection = (context, session, fc, ff) -> {
+        final Hive3DBCPService dbcpService = context.getProperty(HIVE_DBCP_SERVICE).asControllerService(Hive3DBCPService.class);
+        final Connection connection = dbcpService.getConnection();
+        fc.connectionUrl = dbcpService.getConnectionURL();
+        return connection;
+    };
+
+    private FetchFlowFiles<FunctionContext> fetchFlowFiles = (context, session, functionContext, result) -> {
+        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
+        return session.get(batchSize);
+    };
+
+    private Put.PutFlowFile<FunctionContext, Connection> putFlowFile = (context, session, fc, conn, flowFile, result) -> {
+        final String script = getHiveQL(session, flowFile, fc.charset);
+        String regex = "(?<!\\\\)" + Pattern.quote(fc.statementDelimiter);
+
+        String[] hiveQLs = script.split(regex);
+
+        final Set<TableName> tableNames = new HashSet<>();
+        exceptionHandler.execute(fc, flowFile, input -> {
+            int loc = 1;
+            for (String hiveQLStr: hiveQLs) {
+                getLogger().debug("HiveQL: {}", new Object[]{hiveQLStr});
+
+                final String hiveQL = hiveQLStr.trim();
+                if (!StringUtils.isEmpty(hiveQL)) {
+                    final PreparedStatement stmt = conn.prepareStatement(hiveQL);
+
+                    // Get ParameterMetadata
+                    // Hive JDBC Doesn't support this yet:
+                    // ParameterMetaData pmd = stmt.getParameterMetaData();
+                    // int paramCount = pmd.getParameterCount();
+                    int paramCount = StringUtils.countMatches(hiveQL, "?");
+
+                    if (paramCount > 0) {
+                        loc = setParameters(loc, stmt, paramCount, flowFile.getAttributes());
+                    }
+
+                    // Parse hiveQL and extract input/output tables
+                    try {
+                        tableNames.addAll(findTableNames(hiveQL));
+                    } catch (Exception e) {
+                        // If failed to parse the query, just log a warning message, but continue.
+                        getLogger().warn("Failed to parse hiveQL: {} due to {}", new Object[]{hiveQL, e}, e);
+                    }
+
+                    stmt.setQueryTimeout(context.getProperty(QUERY_TIMEOUT).evaluateAttributeExpressions(flowFile).asInteger());
+
+                    // Execute the statement
+                    stmt.execute();
+                    fc.proceed();
+                }
+            }
+
+            // Emit a Provenance SEND event
+            final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - fc.startNanos);
+
+            final FlowFile updatedFlowFile = session.putAllAttributes(flowFile, toQueryTableAttributes(tableNames));
+            session.getProvenanceReporter().send(updatedFlowFile, fc.connectionUrl, transmissionMillis, true);
+            result.routeTo(flowFile, REL_SUCCESS);
+
+        }, onFlowFileError(context, session, result));
+
+    };
+
+    private OnError<FunctionContext, FlowFile> onFlowFileError(final ProcessContext context, final ProcessSession session, final RoutingResult result) {
+        OnError<FunctionContext, FlowFile> onFlowFileError = ExceptionHandler.createOnError(context, session, result, REL_FAILURE, REL_RETRY);
+        onFlowFileError = onFlowFileError.andThen((c, i, r, e) -> {
+            switch (r.destination()) {
+                case Failure:
+                    getLogger().error("Failed to update Hive for {} due to {}; routing to failure", new Object[] {i, e}, e);
+                    break;
+                case Retry:
+                    getLogger().error("Failed to update Hive for {} due to {}; it is possible that retrying the operation will succeed, so routing to retry",
+                            new Object[] {i, e}, e);
+                    break;
+            }
+        });
+        return RollbackOnFailure.createOnError(onFlowFileError);
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
+        final Boolean rollbackOnFailure = context.getProperty(RollbackOnFailure.ROLLBACK_ON_FAILURE).asBoolean();
+        final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
+        final String statementDelimiter = context.getProperty(STATEMENT_DELIMITER).getValue();
+        final FunctionContext functionContext = new FunctionContext(rollbackOnFailure, charset, statementDelimiter);
+        RollbackOnFailure.onTrigger(context, sessionFactory, functionContext, getLogger(), session -> process.onTrigger(context, session, functionContext));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java
new file mode 100644
index 0000000..664915c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java
@@ -0,0 +1,560 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.hive;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.common.util.ShutdownHookManager;
+import org.apache.hive.streaming.ConnectionError;
+import org.apache.hive.streaming.HiveStreamingConnection;
+import org.apache.hive.streaming.InvalidTable;
+import org.apache.hive.streaming.SerializationError;
+import org.apache.hive.streaming.StreamingConnection;
+import org.apache.hive.streaming.StreamingException;
+import org.apache.hive.streaming.StreamingIOFailure;
+import org.apache.hive.streaming.TransactionError;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.kerberos.KerberosCredentialsService;
+import org.apache.nifi.logging.ComponentLog;
+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.util.StandardValidators;
+import org.apache.nifi.processor.util.pattern.DiscontinuedException;
+import org.apache.nifi.processor.util.pattern.RollbackOnFailure;
+import org.apache.nifi.processors.hadoop.exception.RecordReaderFactoryException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.util.StringUtils;
+import org.apache.nifi.util.hive.AuthenticationFailedException;
+import org.apache.nifi.util.hive.HiveConfigurator;
+import org.apache.nifi.util.hive.HiveOptions;
+import org.apache.hive.streaming.HiveRecordWriter;
+import org.apache.nifi.util.hive.HiveUtils;
+import org.apache.nifi.util.hive.ValidationResources;
+
+import java.io.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+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.Objects;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processors.hive.AbstractHive3QLProcessor.ATTR_OUTPUT_TABLES;
+
+@Tags({"hive", "streaming", "put", "database", "store"})
+@CapabilityDescription("This processor uses Hive Streaming to send flow file records to an Apache Hive 3.0+ table. "
+        + "The partition values are expected to be the 'last' fields of each record, so if the table is partitioned on column A for example, then the last field in "
+        + "each record should be field A.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "hivestreaming.record.count", description = "This attribute is written on the flow files routed to the 'success' "
+                + "and 'failure' relationships, and contains the number of records from the incoming flow file. All records in a flow file are committed as a single transaction."),
+        @WritesAttribute(attribute = "query.output.tables", description = "This attribute is written on the flow files routed to the 'success' "
+                + "and 'failure' relationships, and contains the target table name in 'databaseName.tableName' format.")
+})
+@RequiresInstanceClassLoading
+public class PutHive3Streaming extends AbstractProcessor {
+    // Attributes
+    public static final String HIVE_STREAMING_RECORD_COUNT_ATTR = "hivestreaming.record.count";
+
+    private static final String CLIENT_CACHE_DISABLED_PROPERTY = "hcatalog.hive.client.cache.disabled";
+
+    // Properties
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("The service for reading records from incoming flow files.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor METASTORE_URI = new PropertyDescriptor.Builder()
+            .name("hive3-stream-metastore-uri")
+            .displayName("Hive Metastore URI")
+            .description("The URI location for the Hive Metastore. Note that this is not the location of the Hive Server. The default port for the "
+                    + "Hive metastore is 9043.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.URI_VALIDATOR)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("(^[^/]+.*[^/]+$|^[^/]+$|^$)"))) // no start with / or end with /
+            .build();
+
+    static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
+            .name("hive3-config-resources")
+            .displayName("Hive Configuration Resources")
+            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
+                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration. Note that to enable authentication "
+                    + "with Kerberos e.g., the appropriate properties must be set in the configuration files. Also note that if Max Concurrent Tasks is set "
+                    + "to a number greater than one, the 'hcatalog.hive.client.cache.disabled' property will be forced to 'true' to avoid concurrency issues. "
+                    + "Please see the Hive documentation for more details.")
+            .required(false)
+            .addValidator(HiveUtils.createMultipleFilesExistValidator())
+            .build();
+
+    static final PropertyDescriptor DB_NAME = new PropertyDescriptor.Builder()
+            .name("hive3-stream-database-name")
+            .displayName("Database Name")
+            .description("The name of the database in which to put the data.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
+            .name("hive3-stream-table-name")
+            .displayName("Table Name")
+            .description("The name of the database table in which to put the data.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PARTITION_VALUES = new PropertyDescriptor.Builder()
+            .name("hive3-stream-part-vals")
+            .displayName("Partition Values")
+            .description("Specifies a comma-separated list of the values for the partition columns of the target table. If the incoming records all have the same values "
+                    + "for the partition columns, those values can be entered here, resulting in a performance gain. If specified, this property will often contain "
+                    + "Expression Language, for example if PartitionRecord is upstream and two partitions 'name' and 'age' are used, then this property can be set to "
+                    + "${name},${age}.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor AUTOCREATE_PARTITIONS = new PropertyDescriptor.Builder()
+            .name("hive3-stream-autocreate-partition")
+            .displayName("Auto-Create Partitions")
+            .description("Flag indicating whether partitions should be automatically created")
+            .required(true)
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    static final PropertyDescriptor CALL_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("hive3-stream-call-timeout")
+            .displayName("Call Timeout")
+            .description("The number of seconds allowed for a Hive Streaming operation to complete. A value of 0 indicates the processor should wait indefinitely on operations. "
+                    + "Note that although this property supports Expression Language, it will not be evaluated against incoming FlowFile attributes.")
+            .defaultValue("0")
+            .required(true)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor DISABLE_STREAMING_OPTIMIZATIONS = new PropertyDescriptor.Builder()
+            .name("hive3-stream-disable-optimizations")
+            .displayName("Disable Streaming Optimizations")
+            .description("Whether to disable streaming optimizations. Disabling streaming optimizations will have significant impact to performance and memory consumption.")
+            .required(true)
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+
+    static final PropertyDescriptor ROLLBACK_ON_FAILURE = RollbackOnFailure.createRollbackOnFailureProperty(
+            "NOTE: When an error occurred after a Hive streaming transaction which is derived from the same input FlowFile is already committed," +
+                    " (i.e. a FlowFile contains more records than 'Records per Transaction' and a failure occurred at the 2nd transaction or later)" +
+                    " then the succeeded records will be transferred to 'success' relationship while the original input FlowFile stays in incoming queue." +
+                    " Duplicated records can be created for the succeeded ones when the same FlowFile is processed again.");
+
+    static final PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("kerberos-credentials-service")
+            .displayName("Kerberos Credentials Service")
+            .description("Specifies the Kerberos Credentials Controller Service that should be used for authenticating with Kerberos")
+            .identifiesControllerService(KerberosCredentialsService.class)
+            .required(false)
+            .build();
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("A FlowFile containing Avro records routed to this relationship after the record has been successfully transmitted to Hive.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("A FlowFile containing Avro records routed to this relationship if the record could not be transmitted to Hive.")
+            .build();
+
+    public static final Relationship REL_RETRY = new Relationship.Builder()
+            .name("retry")
+            .description("The incoming FlowFile is routed to this relationship if its records cannot be transmitted to Hive. Note that "
+                    + "some records may have been processed successfully, they will be routed (as Avro flow files) to the success relationship. "
+                    + "The combination of the retry, success, and failure relationships indicate how many records succeeded and/or failed. This "
+                    + "can be used to provide a retry capability since full rollback is not possible.")
+            .build();
+
+    private List<PropertyDescriptor> propertyDescriptors;
+    private Set<Relationship> relationships;
+
+    protected volatile HiveConfigurator hiveConfigurator = new HiveConfigurator();
+    protected volatile UserGroupInformation ugi;
+    protected volatile HiveConf hiveConfig;
+
+    protected volatile int callTimeout;
+    protected ExecutorService callTimeoutPool;
+    protected volatile boolean rollbackOnFailure;
+
+    // Holder of cached Configuration information so validation does not reload the same config over and over
+    private final AtomicReference<ValidationResources> validationResourceHolder = new AtomicReference<>();
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(RECORD_READER);
+        props.add(METASTORE_URI);
+        props.add(HIVE_CONFIGURATION_RESOURCES);
+        props.add(DB_NAME);
+        props.add(TABLE_NAME);
+        props.add(PARTITION_VALUES);
+        props.add(AUTOCREATE_PARTITIONS);
+        props.add(CALL_TIMEOUT);
+        props.add(DISABLE_STREAMING_OPTIMIZATIONS);
+        props.add(ROLLBACK_ON_FAILURE);
+        props.add(KERBEROS_CREDENTIALS_SERVICE);
+
+        propertyDescriptors = Collections.unmodifiableList(props);
+
+        Set<Relationship> _relationships = new HashSet<>();
+        _relationships.add(REL_SUCCESS);
+        _relationships.add(REL_FAILURE);
+        _relationships.add(REL_RETRY);
+        relationships = Collections.unmodifiableSet(_relationships);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return propertyDescriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        boolean confFileProvided = validationContext.getProperty(HIVE_CONFIGURATION_RESOURCES).isSet();
+
+        final List<ValidationResult> problems = new ArrayList<>();
+
+        final KerberosCredentialsService credentialsService = validationContext.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
+
+        final String resolvedPrincipal = credentialsService != null ? credentialsService.getPrincipal() : null;
+        final String resolvedKeytab = credentialsService != null ? credentialsService.getKeytab() : null;
+        if (confFileProvided) {
+            final String configFiles = validationContext.getProperty(HIVE_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue();
+            problems.addAll(hiveConfigurator.validate(configFiles, resolvedPrincipal, resolvedKeytab, validationResourceHolder, getLogger()));
+        }
+
+        return problems;
+    }
+
+    @OnScheduled
+    public void setup(final ProcessContext context) {
+        ComponentLog log = getLogger();
+        rollbackOnFailure = context.getProperty(ROLLBACK_ON_FAILURE).asBoolean();
+
+        final String configFiles = context.getProperty(HIVE_CONFIGURATION_RESOURCES).getValue();
+        hiveConfig = hiveConfigurator.getConfigurationFromFiles(configFiles);
+
+        // If more than one concurrent task, force 'hcatalog.hive.client.cache.disabled' to true
+        if (context.getMaxConcurrentTasks() > 1) {
+            hiveConfig.setBoolean(CLIENT_CACHE_DISABLED_PROPERTY, true);
+        }
+
+        // add any dynamic properties to the Hive configuration
+        for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
+            final PropertyDescriptor descriptor = entry.getKey();
+            if (descriptor.isDynamic()) {
+                hiveConfig.set(descriptor.getName(), entry.getValue());
+            }
+        }
+
+        hiveConfigurator.preload(hiveConfig);
+
+        if (SecurityUtil.isSecurityEnabled(hiveConfig)) {
+            final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
+
+            final String resolvedPrincipal = credentialsService.getPrincipal();
+            final String resolvedKeytab = credentialsService.getKeytab();
+
+            log.info("Hive Security Enabled, logging in as principal {} with keytab {}", new Object[]{resolvedPrincipal, resolvedKeytab});
+            try {
+                ugi = hiveConfigurator.authenticate(hiveConfig, resolvedPrincipal, resolvedKeytab);
+            } catch (AuthenticationFailedException ae) {
+                throw new ProcessException("Kerberos authentication failed for Hive Streaming", ae);
+            }
+
+            log.info("Successfully logged in as principal {} with keytab {}", new Object[]{resolvedPrincipal, resolvedKeytab});
+        } else {
+            ugi = null;
+        }
+
+        callTimeout = context.getProperty(CALL_TIMEOUT).evaluateAttributeExpressions().asInteger() * 1000; // milliseconds
+        String timeoutName = "put-hive3-streaming-%d";
+        this.callTimeoutPool = Executors.newFixedThreadPool(1,
+                new ThreadFactoryBuilder().setNameFormat(timeoutName).build());
+    }
+
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final RecordReaderFactory recordReaderFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final String dbName = context.getProperty(DB_NAME).evaluateAttributeExpressions(flowFile).getValue();
+        final String tableName = context.getProperty(TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        final ComponentLog log = getLogger();
+        final String metastoreUri = context.getProperty(METASTORE_URI).evaluateAttributeExpressions(flowFile).getValue();
+
+        final String partitionValuesString = context.getProperty(PARTITION_VALUES).evaluateAttributeExpressions(flowFile).getValue();
+        final boolean autoCreatePartitions = context.getProperty(AUTOCREATE_PARTITIONS).asBoolean();
+        final boolean disableStreamingOptimizations = context.getProperty(DISABLE_STREAMING_OPTIMIZATIONS).asBoolean();
+
+        HiveOptions o = new HiveOptions(metastoreUri, dbName, tableName)
+                .withHiveConf(hiveConfig)
+                .withAutoCreatePartitions(autoCreatePartitions)
+                .withCallTimeout(callTimeout)
+                .withStreamingOptimizations(!disableStreamingOptimizations);
+
+        if (!StringUtils.isEmpty(partitionValuesString)) {
+            List<String> staticPartitionValues = Arrays.stream(partitionValuesString.split(",")).filter(Objects::nonNull).map(String::trim).collect(Collectors.toList());
+            o = o.withStaticPartitionValues(staticPartitionValues);
+        }
+
+        if (SecurityUtil.isSecurityEnabled(hiveConfig)) {
+            final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
+            o = o.withKerberosPrincipal(credentialsService.getPrincipal()).withKerberosKeytab(credentialsService.getKeytab());
+        }
+
+        final HiveOptions options = o;
+
+        // Store the original class loader, then explicitly set it to this class's classloader (for use by the Hive Metastore)
+        ClassLoader originalClassloader = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+
+        StreamingConnection hiveStreamingConnection = null;
+
+        try (final InputStream rawIn = session.read(flowFile)) {
+            final RecordReader reader;
+
+            try (final BufferedInputStream in = new BufferedInputStream(rawIn)) {
+
+                // if we fail to create the RecordReader then we want to route to failure, so we need to
+                // handle this separately from the other IOExceptions which normally route to retry
+                try {
+                    reader = recordReaderFactory.createRecordReader(flowFile, in, getLogger());
+                } catch (Exception e) {
+                    throw new RecordReaderFactoryException("Unable to create RecordReader", e);
+                }
+
+                hiveStreamingConnection = makeStreamingConnection(options, reader);
+                // Add shutdown handler with higher priority than FileSystem shutdown hook so that streaming connection gets closed first before
+                // filesystem close (to avoid ClosedChannelException)
+                ShutdownHookManager.addShutdownHook(hiveStreamingConnection::close, FileSystem.SHUTDOWN_HOOK_PRIORITY + 1);
+
+                // Write records to Hive streaming, then commit and close
+                hiveStreamingConnection.beginTransaction();
+                hiveStreamingConnection.write(in);
+                hiveStreamingConnection.commitTransaction();
+                rawIn.close();
+
+                Map<String, String> updateAttributes = new HashMap<>();
+                updateAttributes.put(HIVE_STREAMING_RECORD_COUNT_ATTR, Long.toString(hiveStreamingConnection.getConnectionStats().getRecordsWritten()));
+                updateAttributes.put(ATTR_OUTPUT_TABLES, options.getQualifiedTableName());
+                flowFile = session.putAllAttributes(flowFile, updateAttributes);
+                session.getProvenanceReporter().send(flowFile, hiveStreamingConnection.getMetastoreUri());
+                session.transfer(flowFile, REL_SUCCESS);
+            } catch (TransactionError te) {
+                if (rollbackOnFailure) {
+                    throw new ProcessException(te.getLocalizedMessage(), te);
+                } else {
+                    throw new ShouldRetryException(te.getLocalizedMessage(), te);
+                }
+            } catch (RecordReaderFactoryException rrfe) {
+                throw new ProcessException(rrfe);
+            }
+        } catch (InvalidTable | SerializationError | StreamingIOFailure | IOException e) {
+            if (rollbackOnFailure) {
+                if (hiveStreamingConnection != null) {
+                    abortConnection(hiveStreamingConnection);
+                }
+                throw new ProcessException(e.getLocalizedMessage(), e);
+            } else {
+                Map<String, String> updateAttributes = new HashMap<>();
+                updateAttributes.put(HIVE_STREAMING_RECORD_COUNT_ATTR, Long.toString(hiveStreamingConnection.getConnectionStats().getRecordsWritten()));
+                updateAttributes.put(ATTR_OUTPUT_TABLES, options.getQualifiedTableName());
+                flowFile = session.putAllAttributes(flowFile, updateAttributes);
+                session.transfer(flowFile, REL_FAILURE);
+            }
+        } catch (DiscontinuedException e) {
+            // The input FlowFile processing is discontinued. Keep it in the input queue.
+            getLogger().warn("Discontinued processing for {} due to {}", new Object[]{flowFile, e}, e);
+            session.transfer(flowFile, Relationship.SELF);
+        } catch (ConnectionError ce) {
+            // If we can't connect to the metastore, yield the processor
+            context.yield();
+            throw new ProcessException("A connection to metastore cannot be established", ce);
+        } catch (ShouldRetryException e) {
+            // This exception is already a result of adjusting an error, so simply transfer the FlowFile to retry. Still need to abort the txn
+            getLogger().error(e.getLocalizedMessage(), e);
+            if (hiveStreamingConnection != null) {
+                abortConnection(hiveStreamingConnection);
+            }
+            flowFile = session.penalize(flowFile);
+            session.transfer(flowFile, REL_RETRY);
+        } catch (StreamingException se) {
+            // Handle all other exceptions. These are often record-based exceptions (since Hive will throw a subclass of the exception caught above)
+            Throwable cause = se.getCause();
+            if (cause == null) cause = se;
+            // This is a failure on the incoming data, rollback on failure if specified; otherwise route to failure after penalizing (and abort txn in any case)
+            if (rollbackOnFailure) {
+                if (hiveStreamingConnection != null) {
+                    abortConnection(hiveStreamingConnection);
+                }
+                throw new ProcessException(cause.getLocalizedMessage(), cause);
+            } else {
+                flowFile = session.penalize(flowFile);
+                Map<String, String> updateAttributes = new HashMap<>();
+                updateAttributes.put(HIVE_STREAMING_RECORD_COUNT_ATTR, Long.toString(hiveStreamingConnection.getConnectionStats().getRecordsWritten()));
+                updateAttributes.put(ATTR_OUTPUT_TABLES, options.getQualifiedTableName());
+                flowFile = session.putAllAttributes(flowFile, updateAttributes);
+                session.transfer(flowFile, REL_FAILURE);
+            }
+
+        } catch (Throwable t) {
+            if (hiveStreamingConnection != null) {
+                abortConnection(hiveStreamingConnection);
+            }
+            throw (t instanceof ProcessException) ? (ProcessException) t : new ProcessException(t);
+        } finally {
+            closeConnection(hiveStreamingConnection);
+            // Restore original class loader, might not be necessary but is good practice since the processor task changed it
+            Thread.currentThread().setContextClassLoader(originalClassloader);
+        }
+    }
+
+    StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader) throws StreamingException {
+        return HiveStreamingConnection.newBuilder()
+                .withDatabase(options.getDatabaseName())
+                .withTable(options.getTableName())
+                .withStaticPartitionValues(options.getStaticPartitionValues())
+                .withHiveConf(options.getHiveConf())
+                .withRecordWriter(new HiveRecordWriter(reader, getLogger()))
+                .withAgentInfo("NiFi " + this.getClass().getSimpleName() + " [" + this.getIdentifier()
+                        + "] thread " + Thread.currentThread().getId() + "[" + Thread.currentThread().getName() + "]")
+                .connect();
+    }
+
+    @OnStopped
+    public void cleanup() {
+        validationResourceHolder.set(null); // trigger re-validation of resources
+
+        ComponentLog log = getLogger();
+
+        if (callTimeoutPool != null) {
+            callTimeoutPool.shutdown();
+            try {
+                while (!callTimeoutPool.isTerminated()) {
+                    callTimeoutPool.awaitTermination(callTimeout, TimeUnit.MILLISECONDS);
+                }
+            } catch (Throwable t) {
+                log.warn("shutdown interrupted on " + callTimeoutPool, t);
+            }
+            callTimeoutPool = null;
+        }
+
+        ugi = null;
+    }
+
+    private void abortAndCloseConnection(StreamingConnection connection) {
+        try {
+            abortConnection(connection);
+            closeConnection(connection);
+        } catch (Exception ie) {
+            getLogger().warn("unable to close hive connections. ", ie);
+        }
+    }
+
+    /**
+     * Abort current Txn on the connection
+     */
+    private void abortConnection(StreamingConnection connection) {
+        if (connection != null) {
+            try {
+                connection.abortTransaction();
+            } catch (Exception e) {
+                getLogger().error("Failed to abort Hive Streaming transaction " + connection + " due to exception ", e);
+            }
+        }
+    }
+
+    /**
+     * Close the streaming connection
+     */
+    private void closeConnection(StreamingConnection connection) {
+        if (connection != null) {
+            try {
+                connection.close();
+            } catch (Exception e) {
+                getLogger().error("Failed to close Hive Streaming connection " + connection + " due to exception ", e);
+            }
+        }
+    }
+
+    private static class ShouldRetryException extends RuntimeException {
+        private ShouldRetryException(String message, Throwable cause) {
+            super(message, cause);
+        }
+    }
+}
+


[4/6] nifi git commit: NIFI-4963: Added Hive3 bundle - Incorporated review comments - Added more defensive code for PutHive3Streaming error handling

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/SelectHive3QL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/SelectHive3QL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/SelectHive3QL.java
new file mode 100644
index 0000000..cb0b000
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/SelectHive3QL.java
@@ -0,0 +1,477 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.hive;
+
+import java.nio.charset.Charset;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.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.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.dbcp.hive.Hive3DBCPService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.pattern.PartialFunctions;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.hive.CsvOutputOptions;
+import org.apache.nifi.util.hive.HiveJdbcCommon;
+
+import static org.apache.nifi.util.hive.HiveJdbcCommon.AVRO;
+import static org.apache.nifi.util.hive.HiveJdbcCommon.CSV;
+import static org.apache.nifi.util.hive.HiveJdbcCommon.CSV_MIME_TYPE;
+import static org.apache.nifi.util.hive.HiveJdbcCommon.MIME_TYPE_AVRO_BINARY;
+import static org.apache.nifi.util.hive.HiveJdbcCommon.NORMALIZE_NAMES_FOR_AVRO;
+
+@EventDriven
+@InputRequirement(Requirement.INPUT_ALLOWED)
+@Tags({"hive", "sql", "select", "jdbc", "query", "database"})
+@CapabilityDescription("Execute provided HiveQL SELECT query against a Hive database connection. Query result will be converted to Avro or CSV format."
+        + " Streaming is used so arbitrarily large result sets are supported. This processor can be scheduled to run on "
+        + "a timer, or cron expression, using the standard scheduling methods, or it can be triggered by an incoming FlowFile. "
+        + "If it is triggered by an incoming FlowFile, then attributes of that FlowFile will be available when evaluating the "
+        + "select query. FlowFile attribute 'selecthiveql.row.count' indicates how many rows were selected.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the MIME type for the outgoing flowfile to application/avro-binary for Avro or text/csv for CSV."),
+        @WritesAttribute(attribute = "filename", description = "Adds .avro or .csv to the filename attribute depending on which output format is selected."),
+        @WritesAttribute(attribute = "selecthiveql.row.count", description = "Indicates how many rows were selected/returned by the query."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Rows Per Flow File' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Rows Per Flow File' is set then this is the total number of  "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Rows Per Flow File' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order  "
+                + "FlowFiles were produced"),
+        @WritesAttribute(attribute = "query.input.tables", description = "Contains input table names in comma delimited 'databaseName.tableName' format.")
+})
+public class SelectHive3QL extends AbstractHive3QLProcessor {
+
+    static final String RESULT_ROW_COUNT = "selecthiveql.row.count";
+
+    // Relationships
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Successfully created FlowFile from HiveQL query result set.")
+            .build();
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("HiveQL query execution failed. Incoming FlowFile will be penalized and routed to this relationship")
+            .build();
+
+
+    static final PropertyDescriptor HIVEQL_SELECT_QUERY = new PropertyDescriptor.Builder()
+            .name("hive-query")
+            .displayName("HiveQL Select Query")
+            .description("HiveQL SELECT query to execute. If this is not set, the query is assumed to be in the content of an incoming FlowFile.")
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    static final PropertyDescriptor FETCH_SIZE = new PropertyDescriptor.Builder()
+            .name("hive-fetch-size")
+            .displayName("Fetch Size")
+            .description("The number of result rows to be fetched from the result set at a time. This is a hint to the driver and may not be "
+                    + "honored and/or exact. If the value specified is zero, then the hint is ignored.")
+            .defaultValue("0")
+            .required(true)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    static final PropertyDescriptor MAX_ROWS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("hive-max-rows")
+            .displayName("Max Rows Per Flow File")
+            .description("The maximum number of result rows that will be included in a single FlowFile. " +
+                    "This will allow you to break up very large result sets into multiple FlowFiles. If the value specified is zero, then all rows are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    static final PropertyDescriptor MAX_FRAGMENTS = new PropertyDescriptor.Builder()
+            .name("hive-max-frags")
+            .displayName("Maximum Number of Fragments")
+            .description("The maximum number of fragments. If the value specified is zero, then all fragments are returned. " +
+                    "This prevents OutOfMemoryError when this processor ingests huge table.")
+            .defaultValue("0")
+            .required(true)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    static final PropertyDescriptor HIVEQL_CSV_HEADER = new PropertyDescriptor.Builder()
+            .name("csv-header")
+            .displayName("CSV Header")
+            .description("Include Header in Output")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor HIVEQL_CSV_ALT_HEADER = new PropertyDescriptor.Builder()
+            .name("csv-alt-header")
+            .displayName("Alternate CSV Header")
+            .description("Comma separated list of header fields")
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    static final PropertyDescriptor HIVEQL_CSV_DELIMITER = new PropertyDescriptor.Builder()
+            .name("csv-delimiter")
+            .displayName("CSV Delimiter")
+            .description("CSV Delimiter used to separate fields")
+            .required(true)
+            .defaultValue(",")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    static final PropertyDescriptor HIVEQL_CSV_QUOTE = new PropertyDescriptor.Builder()
+            .name("csv-quote")
+            .displayName("CSV Quote")
+            .description("Whether to force quoting of CSV fields. Note that this might conflict with the setting for CSV Escape.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+    static final PropertyDescriptor HIVEQL_CSV_ESCAPE = new PropertyDescriptor.Builder()
+            .name("csv-escape")
+            .displayName("CSV Escape")
+            .description("Whether to escape CSV strings in output. Note that this might conflict with the setting for CSV Quote.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor HIVEQL_OUTPUT_FORMAT = new PropertyDescriptor.Builder()
+            .name("hive-output-format")
+            .displayName("Output Format")
+            .description("How to represent the records coming from Hive (Avro, CSV, e.g.)")
+            .required(true)
+            .allowableValues(AVRO, CSV)
+            .defaultValue(AVRO)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .build();
+
+    private final static List<PropertyDescriptor> propertyDescriptors;
+    private final static Set<Relationship> relationships;
+
+    /*
+     * Will ensure that the list of property descriptors is built only once.
+     * Will also create a Set of relationships
+     */
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(HIVE_DBCP_SERVICE);
+        _propertyDescriptors.add(HIVEQL_SELECT_QUERY);
+        _propertyDescriptors.add(FETCH_SIZE);
+        _propertyDescriptors.add(QUERY_TIMEOUT);
+        _propertyDescriptors.add(MAX_ROWS_PER_FLOW_FILE);
+        _propertyDescriptors.add(MAX_FRAGMENTS);
+        _propertyDescriptors.add(HIVEQL_OUTPUT_FORMAT);
+        _propertyDescriptors.add(NORMALIZE_NAMES_FOR_AVRO);
+        _propertyDescriptors.add(HIVEQL_CSV_HEADER);
+        _propertyDescriptors.add(HIVEQL_CSV_ALT_HEADER);
+        _propertyDescriptors.add(HIVEQL_CSV_DELIMITER);
+        _propertyDescriptors.add(HIVEQL_CSV_QUOTE);
+        _propertyDescriptors.add(HIVEQL_CSV_ESCAPE);
+        _propertyDescriptors.add(CHARSET);
+        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
+
+        Set<Relationship> _relationships = new HashSet<>();
+        _relationships.add(REL_SUCCESS);
+        _relationships.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(_relationships);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return propertyDescriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @OnScheduled
+    public void setup(ProcessContext context) {
+        // If the query is not set, then an incoming flow file is needed. Otherwise fail the initialization
+        if (!context.getProperty(HIVEQL_SELECT_QUERY).isSet() && !context.hasIncomingConnection()) {
+            final String errorString = "Either the Select Query must be specified or there must be an incoming connection "
+                    + "providing flowfile(s) containing a SQL select query";
+            getLogger().error(errorString);
+            throw new ProcessException(errorString);
+        }
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
+        PartialFunctions.onTrigger(context, sessionFactory, getLogger(), session -> onTrigger(context, session));
+    }
+
+    private void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile fileToProcess = (context.hasIncomingConnection() ? session.get() : null);
+        FlowFile flowfile = null;
+
+        // If we have no FlowFile, and all incoming connections are self-loops then we can continue on.
+        // However, if we have no FlowFile and we have connections coming from other Processors, then
+        // we know that we should run only if we have a FlowFile.
+        if (context.hasIncomingConnection()) {
+            if (fileToProcess == null && context.hasNonLoopConnection()) {
+                return;
+            }
+        }
+
+        final ComponentLog logger = getLogger();
+        final Hive3DBCPService dbcpService = context.getProperty(HIVE_DBCP_SERVICE).asControllerService(Hive3DBCPService.class);
+        final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
+
+        final boolean flowbased = !(context.getProperty(HIVEQL_SELECT_QUERY).isSet());
+
+        // Source the SQL
+        final String selectQuery;
+
+        if (context.getProperty(HIVEQL_SELECT_QUERY).isSet()) {
+            selectQuery = context.getProperty(HIVEQL_SELECT_QUERY).evaluateAttributeExpressions(fileToProcess).getValue();
+        } else {
+            // If the query is not set, then an incoming flow file is required, and expected to contain a valid SQL select query.
+            // If there is no incoming connection, onTrigger will not be called as the processor will fail when scheduled.
+            final StringBuilder queryContents = new StringBuilder();
+            session.read(fileToProcess, in -> queryContents.append(IOUtils.toString(in, charset)));
+            selectQuery = queryContents.toString();
+        }
+
+
+        final Integer fetchSize = context.getProperty(FETCH_SIZE).evaluateAttributeExpressions(fileToProcess).asInteger();
+        final Integer maxRowsPerFlowFile = context.getProperty(MAX_ROWS_PER_FLOW_FILE).evaluateAttributeExpressions(fileToProcess).asInteger();
+        final Integer maxFragments = context.getProperty(MAX_FRAGMENTS).isSet()
+                ? context.getProperty(MAX_FRAGMENTS).evaluateAttributeExpressions(fileToProcess).asInteger()
+                : 0;
+        final String outputFormat = context.getProperty(HIVEQL_OUTPUT_FORMAT).getValue();
+        final boolean convertNamesForAvro = context.getProperty(NORMALIZE_NAMES_FOR_AVRO).asBoolean();
+        final StopWatch stopWatch = new StopWatch(true);
+        final boolean header = context.getProperty(HIVEQL_CSV_HEADER).asBoolean();
+        final String altHeader = context.getProperty(HIVEQL_CSV_ALT_HEADER).evaluateAttributeExpressions(fileToProcess).getValue();
+        final String delimiter = context.getProperty(HIVEQL_CSV_DELIMITER).evaluateAttributeExpressions(fileToProcess).getValue();
+        final boolean quote = context.getProperty(HIVEQL_CSV_QUOTE).asBoolean();
+        final boolean escape = context.getProperty(HIVEQL_CSV_HEADER).asBoolean();
+        final String fragmentIdentifier = UUID.randomUUID().toString();
+
+        try (final Connection con = dbcpService.getConnection();
+             final Statement st = (flowbased ? con.prepareStatement(selectQuery) : con.createStatement())
+        ) {
+
+            st.setQueryTimeout(context.getProperty(QUERY_TIMEOUT).evaluateAttributeExpressions(fileToProcess).asInteger());
+
+            if (fetchSize != null && fetchSize > 0) {
+                try {
+                    st.setFetchSize(fetchSize);
+                } catch (SQLException se) {
+                    // Not all drivers support this, just log the error (at debug level) and move on
+                    logger.debug("Cannot set fetch size to {} due to {}", new Object[]{fetchSize, se.getLocalizedMessage()}, se);
+                }
+            }
+
+            final List<FlowFile> resultSetFlowFiles = new ArrayList<>();
+            try {
+                logger.debug("Executing query {}", new Object[]{selectQuery});
+                if (flowbased) {
+                    // Hive JDBC Doesn't Support this yet:
+                    // ParameterMetaData pmd = ((PreparedStatement)st).getParameterMetaData();
+                    // int paramCount = pmd.getParameterCount();
+
+                    // Alternate way to determine number of params in SQL.
+                    int paramCount = StringUtils.countMatches(selectQuery, "?");
+
+                    if (paramCount > 0) {
+                        setParameters(1, (PreparedStatement) st, paramCount, fileToProcess.getAttributes());
+                    }
+                }
+
+                final ResultSet resultSet;
+
+                try {
+                    resultSet = (flowbased ? ((PreparedStatement) st).executeQuery() : st.executeQuery(selectQuery));
+                } catch (SQLException se) {
+                    // If an error occurs during the query, a flowfile is expected to be routed to failure, so ensure one here
+                    flowfile = (fileToProcess == null) ? session.create() : fileToProcess;
+                    fileToProcess = null;
+                    throw se;
+                }
+
+                int fragmentIndex = 0;
+                String baseFilename = (fileToProcess != null) ? fileToProcess.getAttribute(CoreAttributes.FILENAME.key()) : null;
+                while (true) {
+                    final AtomicLong nrOfRows = new AtomicLong(0L);
+                    flowfile = (flowfile == null) ? session.create() : session.create(flowfile);
+                    if (baseFilename == null) {
+                        baseFilename = flowfile.getAttribute(CoreAttributes.FILENAME.key());
+                    }
+                    try {
+                        flowfile = session.write(flowfile, out -> {
+                            try {
+                                if (AVRO.equals(outputFormat)) {
+                                    nrOfRows.set(HiveJdbcCommon.convertToAvroStream(resultSet, out, maxRowsPerFlowFile, convertNamesForAvro));
+                                } else if (CSV.equals(outputFormat)) {
+                                    CsvOutputOptions options = new CsvOutputOptions(header, altHeader, delimiter, quote, escape, maxRowsPerFlowFile);
+                                    nrOfRows.set(HiveJdbcCommon.convertToCsvStream(resultSet, out, options));
+                                } else {
+                                    nrOfRows.set(0L);
+                                    throw new ProcessException("Unsupported output format: " + outputFormat);
+                                }
+                            } catch (final SQLException | RuntimeException e) {
+                                throw new ProcessException("Error during database query or conversion of records.", e);
+                            }
+                        });
+                    } catch (ProcessException e) {
+                        // Add flowfile to results before rethrowing so it will be removed from session in outer catch
+                        resultSetFlowFiles.add(flowfile);
+                        throw e;
+                    }
+
+                    if (nrOfRows.get() > 0 || resultSetFlowFiles.isEmpty()) {
+                        final Map<String, String> attributes = new HashMap<>();
+                        // Set attribute for how many rows were selected
+                        attributes.put(RESULT_ROW_COUNT, String.valueOf(nrOfRows.get()));
+
+                        try {
+                            // Set input/output table names by parsing the query
+                            attributes.putAll(toQueryTableAttributes(findTableNames(selectQuery)));
+                        } catch (Exception e) {
+                            // If failed to parse the query, just log a warning message, but continue.
+                            getLogger().warn("Failed to parse query: {} due to {}", new Object[]{selectQuery, e}, e);
+                        }
+
+                        // Set MIME type on output document and add extension to filename
+                        if (AVRO.equals(outputFormat)) {
+                            attributes.put(CoreAttributes.MIME_TYPE.key(), MIME_TYPE_AVRO_BINARY);
+                            attributes.put(CoreAttributes.FILENAME.key(), baseFilename + "." + fragmentIndex + ".avro");
+                        } else if (CSV.equals(outputFormat)) {
+                            attributes.put(CoreAttributes.MIME_TYPE.key(), CSV_MIME_TYPE);
+                            attributes.put(CoreAttributes.FILENAME.key(), baseFilename + "." + fragmentIndex + ".csv");
+                        }
+
+                        if (maxRowsPerFlowFile > 0) {
+                            attributes.put("fragment.identifier", fragmentIdentifier);
+                            attributes.put("fragment.index", String.valueOf(fragmentIndex));
+                        }
+
+                        flowfile = session.putAllAttributes(flowfile, attributes);
+
+                        logger.info("{} contains {} Avro records; transferring to 'success'",
+                                new Object[]{flowfile, nrOfRows.get()});
+
+                        if (context.hasIncomingConnection()) {
+                            // If the flow file came from an incoming connection, issue a Modify Content provenance event
+
+                            session.getProvenanceReporter().modifyContent(flowfile, "Retrieved " + nrOfRows.get() + " rows",
+                                    stopWatch.getElapsed(TimeUnit.MILLISECONDS));
+                        } else {
+                            // If we created a flow file from rows received from Hive, issue a Receive provenance event
+                            session.getProvenanceReporter().receive(flowfile, dbcpService.getConnectionURL(), stopWatch.getElapsed(TimeUnit.MILLISECONDS));
+                        }
+                        resultSetFlowFiles.add(flowfile);
+                    } else {
+                        // If there were no rows returned (and the first flow file has been sent, we're done processing, so remove the flowfile and carry on
+                        session.remove(flowfile);
+                        break;
+                    }
+
+                    fragmentIndex++;
+                    if (maxFragments > 0 && fragmentIndex >= maxFragments) {
+                        break;
+                    }
+                }
+
+                for (int i = 0; i < resultSetFlowFiles.size(); i++) {
+                    // Set count on all FlowFiles
+                    if (maxRowsPerFlowFile > 0) {
+                        resultSetFlowFiles.set(i,
+                                session.putAttribute(resultSetFlowFiles.get(i), "fragment.count", Integer.toString(fragmentIndex)));
+                    }
+                }
+
+            } catch (final SQLException e) {
+                throw e;
+            }
+
+            session.transfer(resultSetFlowFiles, REL_SUCCESS);
+
+        } catch (final ProcessException | SQLException e) {
+            logger.error("Issue processing SQL {} due to {}.", new Object[]{selectQuery, e});
+            if (flowfile == null) {
+                // This can happen if any exceptions occur while setting up the connection, statement, etc.
+                logger.error("Unable to execute HiveQL select query {} due to {}. No FlowFile to route to failure",
+                        new Object[]{selectQuery, e});
+                context.yield();
+            } else {
+                if (context.hasIncomingConnection()) {
+                    logger.error("Unable to execute HiveQL select query {} for {} due to {}; routing to failure",
+                            new Object[]{selectQuery, flowfile, e});
+                    flowfile = session.penalize(flowfile);
+                } else {
+                    logger.error("Unable to execute HiveQL select query {} due to {}; routing to failure",
+                            new Object[]{selectQuery, e});
+                    context.yield();
+                }
+                session.transfer(flowfile, REL_FAILURE);
+            }
+        } finally {
+            if (fileToProcess != null) {
+                session.remove(fileToProcess);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/PutORC.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/PutORC.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/PutORC.java
new file mode 100644
index 0000000..a0a5d13
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/PutORC.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.orc;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.orc.CompressionKind;
+import org.apache.hadoop.hive.ql.io.orc.NiFiOrcUtils;
+import org.apache.hadoop.hive.ql.io.orc.Writer;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.avro.AvroTypeUtil;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.hadoop.AbstractPutHDFSRecord;
+import org.apache.nifi.processors.hadoop.record.HDFSRecordWriter;
+import org.apache.nifi.processors.orc.record.ORCHDFSRecordWriter;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.util.hive.HiveUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"put", "ORC", "hadoop", "HDFS", "filesystem", "restricted", "record"})
+@CapabilityDescription("Reads records from an incoming FlowFile using the provided Record Reader, and writes those records " +
+        "to a ORC file in the location/filesystem specified in the configuration.")
+@ReadsAttribute(attribute = "filename", description = "The name of the file to write comes from the value of this attribute.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "filename", description = "The name of the file is stored in this attribute."),
+        @WritesAttribute(attribute = "absolute.hdfs.path", description = "The absolute path to the file is stored in this attribute."),
+        @WritesAttribute(attribute = "record.count", description = "The number of records written to the ORC file"),
+        @WritesAttribute(attribute = "hive.ddl", description = "Creates a partial Hive DDL statement for creating an external table in Hive from the destination folder. "
+                + "This can be used in ReplaceText for setting the content to the DDL. To make it valid DDL, add \"LOCATION '<path_to_orc_file_in_hdfs>'\", where "
+                + "the path is the directory that contains this ORC file on HDFS. For example, this processor can send flow files downstream to ReplaceText to set the content "
+                + "to this DDL (plus the LOCATION clause as described), then to PutHiveQL processor to create the table if it doesn't exist.")
+})
+@Restricted("Provides operator the ability to write to any file that NiFi has access to in HDFS or the local filesystem.")
+public class PutORC extends AbstractPutHDFSRecord {
+
+    public static final String HIVE_DDL_ATTRIBUTE = "hive.ddl";
+
+    public static final PropertyDescriptor ORC_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
+            .name("putorc-config-resources")
+            .displayName("ORC Configuration Resources")
+            .description("A file or comma separated list of files which contains the ORC configuration (hive-site.xml, e.g.). Without this, Hadoop "
+                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration. Please see the ORC documentation for more details.")
+            .required(false).addValidator(HiveUtils.createMultipleFilesExistValidator()).build();
+
+    public static final PropertyDescriptor STRIPE_SIZE = new PropertyDescriptor.Builder()
+            .name("putorc-stripe-size")
+            .displayName("Stripe Size")
+            .description("The size of the memory buffer (in bytes) for writing stripes to an ORC file")
+            .required(true)
+            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+            .defaultValue("64 MB")
+            .build();
+
+    public static final PropertyDescriptor BUFFER_SIZE = new PropertyDescriptor.Builder()
+            .name("putorc-buffer-size")
+            .displayName("Buffer Size")
+            .description("The maximum size of the memory buffers (in bytes) used for compressing and storing a stripe in memory. This is a hint to the ORC writer, "
+                    + "which may choose to use a smaller buffer size based on stripe size and number of columns for efficient stripe writing and memory utilization.")
+            .required(true)
+            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+            .defaultValue("10 KB")
+            .build();
+
+    static final PropertyDescriptor HIVE_TABLE_NAME = new PropertyDescriptor.Builder()
+            .name("putorc-hive-table-name")
+            .displayName("Hive Table Name")
+            .description("An optional table name to insert into the hive.ddl attribute. The generated DDL can be used by "
+                    + "a PutHive3QL processor (presumably after a PutHDFS processor) to create a table backed by the converted ORC file. "
+                    + "If this property is not provided, the full name (including namespace) of the incoming Avro record will be normalized "
+                    + "and used as the table name.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor HIVE_FIELD_NAMES = new PropertyDescriptor.Builder()
+            .name("putorc-hive-field-names")
+            .displayName("Normalize Field Names for Hive")
+            .description("Whether to normalize field names for Hive (force lowercase, e.g.). If the ORC file is going to "
+                    + "be part of a Hive table, this property should be set to true. To preserve the original field names from the "
+                    + "schema, this property should be set to false.")
+            .required(true)
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+
+    public static final List<AllowableValue> COMPRESSION_TYPES;
+
+    static {
+        final List<AllowableValue> compressionTypes = new ArrayList<>();
+        compressionTypes.add(new AllowableValue("NONE", "NONE", "No compression"));
+        compressionTypes.add(new AllowableValue("ZLIB", "ZLIB", "ZLIB compression"));
+        compressionTypes.add(new AllowableValue("SNAPPY", "SNAPPY", "Snappy compression"));
+        compressionTypes.add(new AllowableValue("LZO", "LZO", "LZO compression"));
+        COMPRESSION_TYPES = Collections.unmodifiableList(compressionTypes);
+    }
+
+    @Override
+    public List<AllowableValue> getCompressionTypes(final ProcessorInitializationContext context) {
+        return COMPRESSION_TYPES;
+    }
+
+    @Override
+    public String getDefaultCompressionType(final ProcessorInitializationContext context) {
+        return "NONE";
+    }
+
+    @Override
+    public List<PropertyDescriptor> getAdditionalProperties() {
+        final List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(ORC_CONFIGURATION_RESOURCES);
+        _propertyDescriptors.add(STRIPE_SIZE);
+        _propertyDescriptors.add(BUFFER_SIZE);
+        _propertyDescriptors.add(HIVE_TABLE_NAME);
+        _propertyDescriptors.add(HIVE_FIELD_NAMES);
+        return Collections.unmodifiableList(_propertyDescriptors);
+    }
+
+    @Override
+    public HDFSRecordWriter createHDFSRecordWriter(final ProcessContext context, final FlowFile flowFile, final Configuration conf, final Path path, final RecordSchema schema)
+            throws IOException, SchemaNotFoundException {
+
+        final Schema avroSchema = AvroTypeUtil.extractAvroSchema(schema);
+
+        final long stripeSize = context.getProperty(STRIPE_SIZE).asDataSize(DataUnit.B).longValue();
+        final int bufferSize = context.getProperty(BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
+        final CompressionKind compressionType = CompressionKind.valueOf(context.getProperty(COMPRESSION_TYPE).getValue());
+        final boolean normalizeForHive = context.getProperty(HIVE_FIELD_NAMES).asBoolean();
+        TypeInfo orcSchema = NiFiOrcUtils.getOrcField(avroSchema, normalizeForHive);
+        final Writer orcWriter = NiFiOrcUtils.createWriter(path, conf, orcSchema, stripeSize, compressionType, bufferSize);
+        final String hiveTableName = context.getProperty(HIVE_TABLE_NAME).isSet()
+                ? context.getProperty(HIVE_TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue()
+                : NiFiOrcUtils.normalizeHiveTableName(avroSchema.getFullName());
+        final boolean hiveFieldNames = context.getProperty(HIVE_FIELD_NAMES).asBoolean();
+
+        return new ORCHDFSRecordWriter(orcWriter, avroSchema, hiveTableName, hiveFieldNames);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/record/ORCHDFSRecordWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/record/ORCHDFSRecordWriter.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/record/ORCHDFSRecordWriter.java
new file mode 100644
index 0000000..bd386a0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/record/ORCHDFSRecordWriter.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.processors.orc.record;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.hive.ql.io.orc.NiFiOrcUtils;
+import org.apache.hadoop.hive.ql.io.orc.Writer;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.nifi.processors.hadoop.record.HDFSRecordWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.nifi.processors.orc.PutORC.HIVE_DDL_ATTRIBUTE;
+
+/**
+ * HDFSRecordWriter that writes ORC files using Avro as the schema representation.
+ */
+
+public class ORCHDFSRecordWriter implements HDFSRecordWriter {
+
+    private final Schema avroSchema;
+    private final TypeInfo orcSchema;
+    private final Writer orcWriter;
+    private final String hiveTableName;
+    private final boolean hiveFieldNames;
+    private final List<Schema.Field> recordFields;
+    private final int numRecordFields;
+    private Object[] workingRow;
+
+    public ORCHDFSRecordWriter(final Writer orcWriter, final Schema avroSchema, final String hiveTableName, final boolean hiveFieldNames) {
+        this.avroSchema = avroSchema;
+        this.orcWriter = orcWriter;
+        this.hiveFieldNames = hiveFieldNames;
+        this.orcSchema = NiFiOrcUtils.getOrcField(avroSchema, this.hiveFieldNames);
+        this.hiveTableName = hiveTableName;
+        this.recordFields = avroSchema != null ? avroSchema.getFields() : null;
+        this.numRecordFields = recordFields != null ? recordFields.size() : -1;
+        // Reuse row object
+        this.workingRow = numRecordFields > -1 ? new Object[numRecordFields] : null;
+    }
+
+    @Override
+    public void write(final Record record) throws IOException {
+        if (recordFields != null) {
+            for (int i = 0; i < numRecordFields; i++) {
+                final Schema.Field field = recordFields.get(i);
+                final Schema fieldSchema = field.schema();
+                final String fieldName = field.name();
+                Object o = record.getValue(fieldName);
+                try {
+                    workingRow[i] = NiFiOrcUtils.convertToORCObject(NiFiOrcUtils.getOrcField(fieldSchema, hiveFieldNames), o, hiveFieldNames);
+                } catch (ArrayIndexOutOfBoundsException aioobe) {
+                    final String errorMsg = "Index out of bounds for column " + i + ", type " + fieldName + ", and object " + o.toString();
+                    throw new IOException(errorMsg, aioobe);
+                }
+            }
+            orcWriter.addRow(NiFiOrcUtils.createOrcStruct(orcSchema, workingRow));
+        }
+    }
+
+    /**
+     * @param recordSet the RecordSet to write
+     * @return the result of writing the record set
+     * @throws IOException if an I/O error happens reading from the RecordSet, or writing a Record
+     */
+    public WriteResult write(final RecordSet recordSet) throws IOException {
+        int recordCount = 0;
+
+        Record record;
+        while ((record = recordSet.next()) != null) {
+            write(record);
+            recordCount++;
+        }
+
+        // Add Hive DDL Attribute
+        String hiveDDL = NiFiOrcUtils.generateHiveDDL(avroSchema, hiveTableName, hiveFieldNames);
+        Map<String, String> attributes = new HashMap<String, String>() {{
+            put(HIVE_DDL_ATTRIBUTE, hiveDDL);
+        }};
+
+        return WriteResult.of(recordCount, attributes);
+    }
+
+    @Override
+    public void close() throws IOException {
+        orcWriter.close();
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/AuthenticationFailedException.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/AuthenticationFailedException.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/AuthenticationFailedException.java
new file mode 100644
index 0000000..70cc6c1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/AuthenticationFailedException.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.util.hive;
+
+public class AuthenticationFailedException extends Exception {
+    public AuthenticationFailedException(String reason, Exception cause) {
+        super(reason, cause);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/CsvOutputOptions.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/CsvOutputOptions.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/CsvOutputOptions.java
new file mode 100644
index 0000000..3688912
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/CsvOutputOptions.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.util.hive;
+
+public class CsvOutputOptions {
+
+    private boolean header = true;
+    private String altHeader = null;
+    private String delimiter = ",";
+    private boolean quote = false;
+    private boolean escape = true;
+
+    private int maxRowsPerFlowFile = 0;
+
+    public boolean isHeader() {
+        return header;
+    }
+
+    public String getAltHeader() {
+        return altHeader;
+    }
+
+
+    public String getDelimiter() {
+        return delimiter;
+    }
+
+
+    public boolean isQuote() {
+        return quote;
+    }
+
+    public boolean isEscape() {
+        return escape;
+    }
+
+    public int getMaxRowsPerFlowFile() {
+        return maxRowsPerFlowFile;
+    }
+
+    public CsvOutputOptions(boolean header, String altHeader, String delimiter, boolean quote, boolean escape, int maxRowsPerFlowFile) {
+        this.header = header;
+        this.altHeader = altHeader;
+        this.delimiter = delimiter;
+        this.quote = quote;
+        this.escape = escape;
+        this.maxRowsPerFlowFile = maxRowsPerFlowFile;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveConfigurator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveConfigurator.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveConfigurator.java
new file mode 100644
index 0000000..6d53683
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveConfigurator.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.util.hive;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.hadoop.KerberosProperties;
+import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Created by mburgess on 5/4/16.
+ */
+public class HiveConfigurator {
+
+    public Collection<ValidationResult> validate(String configFiles, String principal, String keyTab, AtomicReference<ValidationResources> validationResourceHolder, ComponentLog log) {
+
+        final List<ValidationResult> problems = new ArrayList<>();
+        ValidationResources resources = validationResourceHolder.get();
+
+        // if no resources in the holder, or if the holder has different resources loaded,
+        // then load the Configuration and set the new resources in the holder
+        if (resources == null || !configFiles.equals(resources.getConfigResources())) {
+            log.debug("Reloading validation resources");
+            resources = new ValidationResources(configFiles, getConfigurationFromFiles(configFiles));
+            validationResourceHolder.set(resources);
+        }
+
+        final Configuration hiveConfig = resources.getConfiguration();
+
+        problems.addAll(KerberosProperties.validatePrincipalAndKeytab(this.getClass().getSimpleName(), hiveConfig, principal, keyTab, log));
+
+        return problems;
+    }
+
+    public HiveConf getConfigurationFromFiles(final String configFiles) {
+        final HiveConf hiveConfig = new HiveConf();
+        if (StringUtils.isNotBlank(configFiles)) {
+            for (final String configFile : configFiles.split(",")) {
+                hiveConfig.addResource(new Path(configFile.trim()));
+            }
+        }
+        return hiveConfig;
+    }
+
+    public void preload(Configuration configuration) {
+        try {
+            FileSystem.get(configuration).close();
+            UserGroupInformation.setConfiguration(configuration);
+        } catch (IOException ioe) {
+            // Suppress exception as future uses of this configuration will fail
+        }
+    }
+
+    /**
+     * As of Apache NiFi 1.5.0, due to changes made to
+     * {@link SecurityUtil#loginKerberos(Configuration, String, String)}, which is used by this
+     * class to authenticate a principal with Kerberos, Hive controller services no longer
+     * attempt relogins explicitly.  For more information, please read the documentation for
+     * {@link SecurityUtil#loginKerberos(Configuration, String, String)}.
+     * <p/>
+     * In previous versions of NiFi, a {@link org.apache.nifi.hadoop.KerberosTicketRenewer} was started by
+     * {@link HiveConfigurator#authenticate(Configuration, String, String, long)} when the Hive
+     * controller service was enabled.  The use of a separate thread to explicitly relogin could cause race conditions
+     * with the implicit relogin attempts made by hadoop/Hive code on a thread that references the same
+     * {@link UserGroupInformation} instance.  One of these threads could leave the
+     * {@link javax.security.auth.Subject} in {@link UserGroupInformation} to be cleared or in an unexpected state
+     * while the other thread is attempting to use the {@link javax.security.auth.Subject}, resulting in failed
+     * authentication attempts that would leave the Hive controller service in an unrecoverable state.
+     *
+     * @see SecurityUtil#loginKerberos(Configuration, String, String)
+     */
+    public UserGroupInformation authenticate(final Configuration hiveConfig, String principal, String keyTab) throws AuthenticationFailedException {
+        UserGroupInformation ugi;
+        try {
+            ugi = SecurityUtil.loginKerberos(hiveConfig, principal, keyTab);
+        } catch (IOException ioe) {
+            throw new AuthenticationFailedException("Kerberos Authentication for Hive failed", ioe);
+        }
+        return ugi;
+    }
+
+    /**
+     * As of Apache NiFi 1.5.0, this method has been deprecated and is now a wrapper
+     * method which invokes {@link HiveConfigurator#authenticate(Configuration, String, String)}. It will no longer start a
+     * {@link org.apache.nifi.hadoop.KerberosTicketRenewer} to perform explicit relogins.
+     *
+     * @see HiveConfigurator#authenticate(Configuration, String, String)
+     */
+    @Deprecated
+    public UserGroupInformation authenticate(final Configuration hiveConfig, String principal, String keyTab, long ticketRenewalPeriod) throws AuthenticationFailedException {
+        return authenticate(hiveConfig, principal, keyTab);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java
new file mode 100644
index 0000000..ff06495
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java
@@ -0,0 +1,450 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.util.hive;
+
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.apache.avro.SchemaBuilder.FieldAssembler;
+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.commons.lang3.StringEscapeUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.nifi.components.PropertyDescriptor;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static java.sql.Types.ARRAY;
+import static java.sql.Types.BIGINT;
+import static java.sql.Types.BINARY;
+import static java.sql.Types.BIT;
+import static java.sql.Types.BLOB;
+import static java.sql.Types.BOOLEAN;
+import static java.sql.Types.CHAR;
+import static java.sql.Types.CLOB;
+import static java.sql.Types.DATE;
+import static java.sql.Types.DECIMAL;
+import static java.sql.Types.DOUBLE;
+import static java.sql.Types.FLOAT;
+import static java.sql.Types.INTEGER;
+import static java.sql.Types.JAVA_OBJECT;
+import static java.sql.Types.LONGNVARCHAR;
+import static java.sql.Types.LONGVARBINARY;
+import static java.sql.Types.LONGVARCHAR;
+import static java.sql.Types.NCHAR;
+import static java.sql.Types.NUMERIC;
+import static java.sql.Types.NVARCHAR;
+import static java.sql.Types.REAL;
+import static java.sql.Types.ROWID;
+import static java.sql.Types.SMALLINT;
+import static java.sql.Types.STRUCT;
+import static java.sql.Types.TIME;
+import static java.sql.Types.TIMESTAMP;
+import static java.sql.Types.TINYINT;
+import static java.sql.Types.VARBINARY;
+import static java.sql.Types.VARCHAR;
+
+/**
+ * JDBC / HiveQL common functions.
+ */
+public class HiveJdbcCommon {
+
+    public static final String AVRO = "Avro";
+    public static final String CSV = "CSV";
+
+    public static final String MIME_TYPE_AVRO_BINARY = "application/avro-binary";
+    public static final String CSV_MIME_TYPE = "text/csv";
+
+
+    public static final PropertyDescriptor NORMALIZE_NAMES_FOR_AVRO = new PropertyDescriptor.Builder()
+            .name("hive-normalize-avro")
+            .displayName("Normalize Table/Column Names")
+            .description("Whether to change non-Avro-compatible characters in column names to Avro-compatible characters. For example, colons and periods "
+                    + "will be changed to underscores in order to build a valid Avro record.")
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true)
+            .build();
+
+    public static long convertToAvroStream(final ResultSet rs, final OutputStream outStream, final int maxRows, boolean convertNames) throws SQLException, IOException {
+        return convertToAvroStream(rs, outStream, null, maxRows, convertNames, null);
+    }
+
+
+    public static long convertToAvroStream(final ResultSet rs, final OutputStream outStream, String recordName, final int maxRows, boolean convertNames, ResultSetRowCallback callback)
+            throws SQLException, IOException {
+        final Schema schema = createSchema(rs, recordName, convertNames);
+        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, outStream);
+
+            final ResultSetMetaData meta = rs.getMetaData();
+            final int nrOfColumns = meta.getColumnCount();
+            long nrOfRows = 0;
+            while (rs.next()) {
+                if (callback != null) {
+                    callback.processRow(rs);
+                }
+                for (int i = 1; i <= nrOfColumns; i++) {
+                    final int javaSqlType = meta.getColumnType(i);
+                    Object value = rs.getObject(i);
+
+                    if (value == null) {
+                        rec.put(i - 1, null);
+
+                    } else if (javaSqlType == BINARY || javaSqlType == VARBINARY || javaSqlType == LONGVARBINARY || javaSqlType == BLOB || javaSqlType == CLOB) {
+                        // bytes requires little bit different handling
+                        ByteBuffer bb = null;
+                        if (value instanceof byte[]) {
+                            bb = ByteBuffer.wrap((byte[]) value);
+                        } else if (value instanceof ByteBuffer) {
+                            bb = (ByteBuffer) value;
+                        }
+                        if (bb != null) {
+                            rec.put(i - 1, bb);
+                        } else {
+                            throw new IOException("Could not process binary object of type " + value.getClass().getName());
+                        }
+
+                    } 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
+                        rec.put(i - 1, ((Byte) value).intValue());
+
+                    } else if (value instanceof BigDecimal || value instanceof BigInteger) {
+                        // Avro can't handle BigDecimal and BigInteger as numbers - it will throw an AvroRuntimeException such as: "Unknown datum type: java.math.BigDecimal: 38"
+                        rec.put(i - 1, value.toString());
+
+                    } else if (value instanceof Number) {
+                        // Need to call the right getXYZ() method (instead of the getObject() method above), since Doubles are sometimes returned
+                        // when the JDBC type is 6 (Float) for example.
+                        if (javaSqlType == FLOAT) {
+                            value = rs.getFloat(i);
+                        } else if (javaSqlType == DOUBLE) {
+                            value = rs.getDouble(i);
+                        } else if (javaSqlType == INTEGER || javaSqlType == TINYINT || javaSqlType == SMALLINT) {
+                            value = rs.getInt(i);
+                        }
+
+                        rec.put(i - 1, value);
+
+                    } else if (value instanceof Boolean) {
+                        rec.put(i - 1, value);
+                    } else {
+                        // 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.
+                        rec.put(i - 1, value.toString());
+                    }
+                }
+                dataFileWriter.append(rec);
+                nrOfRows += 1;
+
+                if (maxRows > 0 && nrOfRows == maxRows)
+                    break;
+            }
+
+            return nrOfRows;
+        }
+    }
+
+    public static Schema createSchema(final ResultSet rs, boolean convertNames) throws SQLException {
+        return createSchema(rs, null, false);
+    }
+
+    /**
+     * Creates an Avro schema from a result set. If the table/record name is known a priori and provided, use that as a
+     * fallback for the record name if it cannot be retrieved from the result set, and finally fall back to a default value.
+     *
+     * @param rs         The result set to convert to Avro
+     * @param recordName The a priori record name to use if it cannot be determined from the result set.
+     * @param convertNames  Whether to convert column/table names to be legal Avro names
+     * @return A Schema object representing the result set converted to an Avro record
+     * @throws SQLException if any error occurs during conversion
+     */
+    public static Schema createSchema(final ResultSet rs, String recordName, boolean convertNames) throws SQLException {
+        final ResultSetMetaData meta = rs.getMetaData();
+        final int nrOfColumns = meta.getColumnCount();
+        String tableName = StringUtils.isEmpty(recordName) ? "NiFi_SelectHiveQL_Record" : recordName;
+        try {
+            if (nrOfColumns > 0) {
+                // Hive JDBC doesn't support getTableName, instead it returns table.column for column name. Grab the table name from the first column
+                String firstColumnNameFromMeta = meta.getColumnName(1);
+                int tableNameDelimiter = firstColumnNameFromMeta.lastIndexOf(".");
+                if (tableNameDelimiter > -1) {
+                    String tableNameFromMeta = firstColumnNameFromMeta.substring(0, tableNameDelimiter);
+                    if (!StringUtils.isBlank(tableNameFromMeta)) {
+                        tableName = tableNameFromMeta;
+                    }
+                }
+            }
+        } catch (SQLException se) {
+            // Not all drivers support getTableName, so just use the previously-set default
+        }
+
+        if (convertNames) {
+            tableName = normalizeNameForAvro(tableName);
+        }
+        final FieldAssembler<Schema> builder = SchemaBuilder.record(tableName).namespace("any.data").fields();
+
+        /**
+         * Some missing Avro types - Decimal, Date types. May need some additional work.
+         */
+        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(".");
+            String columnName = columnNameFromMeta.substring(columnNameDelimiter + 1);
+            switch (meta.getColumnType(i)) {
+                case CHAR:
+                case LONGNVARCHAR:
+                case LONGVARCHAR:
+                case NCHAR:
+                case NVARCHAR:
+                case VARCHAR:
+                case ARRAY:
+                case STRUCT:
+                case JAVA_OBJECT:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault();
+                    break;
+
+                case BIT:
+                case BOOLEAN:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().booleanType().endUnion().noDefault();
+                    break;
+
+                case INTEGER:
+                    // Default to signed type unless otherwise noted. Some JDBC drivers don't implement isSigned()
+                    boolean signedType = true;
+                    try {
+                        signedType = meta.isSigned(i);
+                    } catch (SQLException se) {
+                        // Use signed types as default
+                    }
+                    if (signedType) {
+                        builder.name(columnName).type().unionOf().nullBuilder().endNull().and().intType().endUnion().noDefault();
+                    } else {
+                        builder.name(columnName).type().unionOf().nullBuilder().endNull().and().longType().endUnion().noDefault();
+                    }
+                    break;
+
+                case SMALLINT:
+                case TINYINT:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().intType().endUnion().noDefault();
+                    break;
+
+                case BIGINT:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().longType().endUnion().noDefault();
+                    break;
+
+                // java.sql.RowId is interface, is seems to be database
+                // implementation specific, let's convert to String
+                case ROWID:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault();
+                    break;
+
+                case FLOAT:
+                case REAL:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().floatType().endUnion().noDefault();
+                    break;
+
+                case DOUBLE:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().doubleType().endUnion().noDefault();
+                    break;
+
+                // Did not find direct suitable type, need to be clarified!!!!
+                case DECIMAL:
+                case NUMERIC:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault();
+                    break;
+
+                // Did not find direct suitable type, need to be clarified!!!!
+                case DATE:
+                case TIME:
+                case TIMESTAMP:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault();
+                    break;
+
+                case BINARY:
+                case VARBINARY:
+                case LONGVARBINARY:
+                case BLOB:
+                case CLOB:
+                    builder.name(columnName).type().unionOf().nullBuilder().endNull().and().bytesType().endUnion().noDefault();
+                    break;
+
+
+                default:
+                    throw new IllegalArgumentException("createSchema: Unknown SQL type " + meta.getColumnType(i) + " cannot be converted to Avro type");
+            }
+        }
+
+        return builder.endRecord();
+    }
+
+    public static long convertToCsvStream(final ResultSet rs, final OutputStream outStream, CsvOutputOptions outputOptions) throws SQLException, IOException {
+        return convertToCsvStream(rs, outStream, null, null, outputOptions);
+    }
+
+    public static long convertToCsvStream(final ResultSet rs, final OutputStream outStream, String recordName, ResultSetRowCallback callback, CsvOutputOptions outputOptions)
+            throws SQLException, IOException {
+
+        final ResultSetMetaData meta = rs.getMetaData();
+        final int nrOfColumns = meta.getColumnCount();
+        List<String> columnNames = new ArrayList<>(nrOfColumns);
+
+        if (outputOptions.isHeader()) {
+            if (outputOptions.getAltHeader() == null) {
+                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));
+                }
+            } else {
+                String[] altHeaderNames = outputOptions.getAltHeader().split(",");
+                columnNames = Arrays.asList(altHeaderNames);
+            }
+        }
+
+        // Write column names as header row
+        outStream.write(StringUtils.join(columnNames, outputOptions.getDelimiter()).getBytes(StandardCharsets.UTF_8));
+        if (outputOptions.isHeader()) {
+            outStream.write("\n".getBytes(StandardCharsets.UTF_8));
+        }
+
+        // Iterate over the rows
+        int maxRows = outputOptions.getMaxRowsPerFlowFile();
+        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:
+                        String valueString = rs.getString(i);
+                        if (valueString != null) {
+                            // Removed extra quotes as those are a part of the escapeCsv when required.
+                            StringBuilder sb = new StringBuilder();
+                            if (outputOptions.isQuote()) {
+                                sb.append("\"");
+                                if (outputOptions.isEscape()) {
+                                    sb.append(StringEscapeUtils.escapeCsv(valueString));
+                                } else {
+                                    sb.append(valueString);
+                                }
+                                sb.append("\"");
+                                rowValues.add(sb.toString());
+                            } else {
+                                if (outputOptions.isEscape()) {
+                                    rowValues.add(StringEscapeUtils.escapeCsv(valueString));
+                                } else {
+                                    rowValues.add(valueString);
+                                }
+                            }
+                        } else {
+                            rowValues.add("");
+                        }
+                        break;
+                    case ARRAY:
+                    case STRUCT:
+                    case JAVA_OBJECT:
+                        String complexValueString = rs.getString(i);
+                        if (complexValueString != null) {
+                            rowValues.add(StringEscapeUtils.escapeCsv(complexValueString));
+                        } else {
+                            rowValues.add("");
+                        }
+                        break;
+                    default:
+                        if (value != null) {
+                            rowValues.add(value.toString());
+                        } else {
+                            rowValues.add("");
+                        }
+                }
+            }
+            // Write row values
+            outStream.write(StringUtils.join(rowValues, outputOptions.getDelimiter()).getBytes(StandardCharsets.UTF_8));
+            outStream.write("\n".getBytes(StandardCharsets.UTF_8));
+            nrOfRows++;
+
+            if (maxRows > 0 && nrOfRows == maxRows)
+                break;
+        }
+        return nrOfRows;
+    }
+
+    public static String normalizeNameForAvro(String inputName) {
+        String normalizedName = inputName.replaceAll("[^A-Za-z0-9_]", "_");
+        if (Character.isDigit(normalizedName.charAt(0))) {
+            normalizedName = "_" + normalizedName;
+        }
+        return normalizedName;
+    }
+
+    /**
+     * 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;
+    }
+
+    public static Configuration getConfigurationFromFiles(final String configFiles) {
+        final Configuration hiveConfig = new HiveConf();
+        if (StringUtils.isNotBlank(configFiles)) {
+            for (final String configFile : configFiles.split(",")) {
+                hiveConfig.addResource(new Path(configFile.trim()));
+            }
+        }
+        return hiveConfig;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.java
new file mode 100644
index 0000000..ca6e6eb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveOptions.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.util.hive;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+import java.io.Serializable;
+import java.util.List;
+
+
+public class HiveOptions implements Serializable {
+
+    protected String databaseName;
+    protected String tableName;
+    protected String metaStoreURI;
+    protected Integer idleTimeout = 60000;
+    protected Integer callTimeout = 0;
+    protected List<String> staticPartitionValues = null;
+    protected Boolean autoCreatePartitions = true;
+    protected String kerberosPrincipal;
+    protected String kerberosKeytab;
+    protected HiveConf hiveConf;
+    protected boolean streamingOptimizations = true;
+
+    public HiveOptions(String metaStoreURI, String databaseName, String tableName) {
+        this.metaStoreURI = metaStoreURI;
+        this.databaseName = databaseName;
+        this.tableName = tableName;
+    }
+
+    public HiveOptions withCallTimeout(Integer callTimeout) {
+        this.callTimeout = callTimeout;
+        return this;
+    }
+
+    public HiveOptions withStaticPartitionValues(List<String> staticPartitionValues) {
+        this.staticPartitionValues = staticPartitionValues;
+        return this;
+    }
+
+    public HiveOptions withAutoCreatePartitions(Boolean autoCreatePartitions) {
+        this.autoCreatePartitions = autoCreatePartitions;
+        return this;
+    }
+
+    public HiveOptions withKerberosKeytab(String kerberosKeytab) {
+        this.kerberosKeytab = kerberosKeytab;
+        return this;
+    }
+
+    public HiveOptions withKerberosPrincipal(String kerberosPrincipal) {
+        this.kerberosPrincipal = kerberosPrincipal;
+        return this;
+    }
+
+    public HiveOptions withHiveConf(HiveConf hiveConf) {
+        this.hiveConf = hiveConf;
+        return this;
+    }
+
+    public HiveOptions withStreamingOptimizations(boolean streamingOptimizations) {
+        this.streamingOptimizations = streamingOptimizations;
+        return this;
+    }
+
+    public String getMetaStoreURI() {
+        return metaStoreURI;
+    }
+
+    public String getDatabaseName() {
+        return databaseName;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getQualifiedTableName() {
+        return databaseName + "." + tableName;
+    }
+
+    public List<String> getStaticPartitionValues() {
+        return staticPartitionValues;
+    }
+
+    public Integer getCallTimeOut() {
+        return callTimeout;
+    }
+
+    public Integer getIdleTimeout() {
+        return idleTimeout;
+    }
+
+    public HiveConf getHiveConf() {
+        return hiveConf;
+    }
+
+    public boolean getStreamingOptimizations() {
+        return streamingOptimizations;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveUtils.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveUtils.java
new file mode 100644
index 0000000..cb4107f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveUtils.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.util.hive;
+
+
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+
+public class HiveUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(HiveUtils.class);
+
+    public static final Validator GREATER_THAN_ONE_VALIDATOR = (subject, value, context) -> {
+        if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value)) {
+            return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build();
+        }
+
+        String reason = null;
+        try {
+            final int intVal = Integer.parseInt(value);
+
+            if (intVal < 2) {
+                reason = "value is less than 2";
+            }
+        } catch (final NumberFormatException e) {
+            reason = "value is not a valid integer";
+        }
+
+        return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
+    };
+
+    /**
+     * Validates that one or more files exist, as specified in a single property.
+     */
+    public static Validator createMultipleFilesExistValidator() {
+        return (subject, input, context) -> {
+            if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input)) {
+                return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build();
+            }
+            final String[] files = input.split("\\s*,\\s*");
+            for (String filename : files) {
+                try {
+                    final File file = new File(filename.trim());
+                    final boolean valid = file.exists() && file.isFile();
+                    if (!valid) {
+                        final String message = "File " + file + " does not exist or is not a file";
+                        return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation(message).build();
+                    }
+                } catch (SecurityException e) {
+                    final String message = "Unable to access " + filename + " due to " + e.getMessage();
+                    return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation(message).build();
+                }
+            }
+            return new ValidationResult.Builder().subject(subject).input(input).valid(true).build();
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/ValidationResources.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/ValidationResources.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/ValidationResources.java
new file mode 100644
index 0000000..1014efb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/ValidationResources.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.util.hive;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A helper class for maintaining loaded configurations (to avoid reloading on use unless necessary)
+ */
+public class ValidationResources {
+
+    private final String configResources;
+    private final Configuration configuration;
+
+    public ValidationResources(String configResources, Configuration configuration) {
+        this.configResources = configResources;
+        this.configuration = configuration;
+    }
+
+    public String getConfigResources() {
+        return configResources;
+    }
+
+    public Configuration getConfiguration() {
+        return configuration;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
new file mode 100644
index 0000000..0bf68f3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -0,0 +1,15 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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.dbcp.hive.Hive3ConnectionPool
\ No newline at end of file


[3/6] nifi git commit: NIFI-4963: Added Hive3 bundle - Incorporated review comments - Added more defensive code for PutHive3Streaming error handling

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
new file mode 100644
index 0000000..8766887
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -0,0 +1,18 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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.processors.hive.SelectHive3QL
+org.apache.nifi.processors.hive.PutHive3QL
+org.apache.nifi.processors.hive.PutHive3Streaming
+org.apache.nifi.processors.orc.PutORC

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubConnectionError.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubConnectionError.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubConnectionError.java
new file mode 100644
index 0000000..2854cae
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubConnectionError.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.hive.streaming;
+
+public class StubConnectionError extends ConnectionError {
+    public StubConnectionError(String msg) {
+        super(msg);
+    }
+
+    public StubConnectionError(String msg, Exception innerEx) {
+        super(msg, innerEx);
+    }
+
+    public StubConnectionError(StreamingConnection endPoint, Exception innerEx) {
+        super(endPoint, innerEx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubSerializationError.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubSerializationError.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubSerializationError.java
new file mode 100644
index 0000000..58e65a8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubSerializationError.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.streaming;
+
+public class StubSerializationError extends SerializationError {
+    public StubSerializationError(String msg, Exception e) {
+        super(msg, e);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubStreamingIOFailure.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubStreamingIOFailure.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubStreamingIOFailure.java
new file mode 100644
index 0000000..ec9931c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubStreamingIOFailure.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.hive.streaming;
+
+public class StubStreamingIOFailure extends StreamingIOFailure {
+
+    public StubStreamingIOFailure(String msg, Exception cause) {
+        super(msg, cause);
+    }
+
+    public StubStreamingIOFailure(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubTransactionError.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubTransactionError.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubTransactionError.java
new file mode 100644
index 0000000..0fddcb8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/hive/streaming/StubTransactionError.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.streaming;
+
+public class StubTransactionError extends TransactionError {
+    public StubTransactionError(String msg, Exception e) {
+        super(msg, e);
+    }
+
+    public StubTransactionError(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/dbcp/hive/Hive3ConnectionPoolTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/dbcp/hive/Hive3ConnectionPoolTest.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/dbcp/hive/Hive3ConnectionPoolTest.java
new file mode 100644
index 0000000..5d9f87c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/dbcp/hive/Hive3ConnectionPoolTest.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.dbcp.hive;
+
+import org.apache.commons.dbcp.BasicDataSource;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.util.MockConfigurationContext;
+import org.apache.nifi.util.MockVariableRegistry;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class Hive3ConnectionPoolTest {
+    private UserGroupInformation userGroupInformation;
+    private Hive3ConnectionPool hive3ConnectionPool;
+    private BasicDataSource basicDataSource;
+    private ComponentLog componentLog;
+
+    @Before
+    public void setup() throws Exception {
+        userGroupInformation = mock(UserGroupInformation.class);
+        basicDataSource = mock(BasicDataSource.class);
+        componentLog = mock(ComponentLog.class);
+
+        when(userGroupInformation.doAs(isA(PrivilegedExceptionAction.class))).thenAnswer(invocation -> {
+            try {
+                return ((PrivilegedExceptionAction) invocation.getArguments()[0]).run();
+            } catch (IOException | Error | RuntimeException | InterruptedException e) {
+                throw e;
+            } catch (Throwable e) {
+                throw new UndeclaredThrowableException(e);
+            }
+        });
+        initPool();
+    }
+
+    private void initPool() throws Exception {
+        hive3ConnectionPool = new Hive3ConnectionPool();
+
+        Field ugiField = Hive3ConnectionPool.class.getDeclaredField("ugi");
+        ugiField.setAccessible(true);
+        ugiField.set(hive3ConnectionPool, userGroupInformation);
+
+        Field dataSourceField = Hive3ConnectionPool.class.getDeclaredField("dataSource");
+        dataSourceField.setAccessible(true);
+        dataSourceField.set(hive3ConnectionPool, basicDataSource);
+
+        Field componentLogField = AbstractControllerService.class.getDeclaredField("logger");
+        componentLogField.setAccessible(true);
+        componentLogField.set(hive3ConnectionPool, componentLog);
+    }
+
+    @Test(expected = ProcessException.class)
+    public void testGetConnectionSqlException() throws SQLException {
+        SQLException sqlException = new SQLException("bad sql");
+        when(basicDataSource.getConnection()).thenThrow(sqlException);
+        try {
+            hive3ConnectionPool.getConnection();
+        } catch (ProcessException e) {
+            assertEquals(sqlException, e.getCause());
+            throw e;
+        }
+    }
+
+    @Test
+    public void testExpressionLanguageSupport() throws Exception {
+        final String URL = "jdbc:hive2://localhost:10000/default";
+        final String USER = "user";
+        final String PASS = "pass";
+        final int MAX_CONN = 7;
+        final String MAX_WAIT = "10 sec"; // 10000 milliseconds
+        final String CONF = "/path/to/hive-site.xml";
+        hive3ConnectionPool = new Hive3ConnectionPool();
+
+        Map<PropertyDescriptor, String> props = new HashMap<PropertyDescriptor, String>() {{
+            put(Hive3ConnectionPool.DATABASE_URL, "${url}");
+            put(Hive3ConnectionPool.DB_USER, "${username}");
+            put(Hive3ConnectionPool.DB_PASSWORD, "${password}");
+            put(Hive3ConnectionPool.MAX_TOTAL_CONNECTIONS, "${maxconn}");
+            put(Hive3ConnectionPool.MAX_WAIT_TIME, "${maxwait}");
+            put(Hive3ConnectionPool.HIVE_CONFIGURATION_RESOURCES, "${hiveconf}");
+        }};
+
+        MockVariableRegistry registry = new MockVariableRegistry();
+        registry.setVariable(new VariableDescriptor("url"), URL);
+        registry.setVariable(new VariableDescriptor("username"), USER);
+        registry.setVariable(new VariableDescriptor("password"), PASS);
+        registry.setVariable(new VariableDescriptor("maxconn"), Integer.toString(MAX_CONN));
+        registry.setVariable(new VariableDescriptor("maxwait"), MAX_WAIT);
+        registry.setVariable(new VariableDescriptor("hiveconf"), CONF);
+
+
+        MockConfigurationContext context = new MockConfigurationContext(props, null, registry);
+        hive3ConnectionPool.onConfigured(context);
+
+        Field dataSourceField = Hive3ConnectionPool.class.getDeclaredField("dataSource");
+        dataSourceField.setAccessible(true);
+        basicDataSource = (BasicDataSource) dataSourceField.get(hive3ConnectionPool);
+        assertEquals(URL, basicDataSource.getUrl());
+        assertEquals(USER, basicDataSource.getUsername());
+        assertEquals(PASS, basicDataSource.getPassword());
+        assertEquals(MAX_CONN, basicDataSource.getMaxActive());
+        assertEquals(10000L, basicDataSource.getMaxWait());
+        assertEquals(URL, hive3ConnectionPool.getConnectionURL());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestHive3Parser.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestHive3Parser.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestHive3Parser.java
new file mode 100644
index 0000000..63c893d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestHive3Parser.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.processors.hive;
+
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.MockProcessContext;
+import org.apache.nifi.util.MockProcessorInitializationContext;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestHive3Parser extends AbstractHive3QLProcessor {
+
+    @Before
+    public void initialize() {
+        final MockProcessContext processContext = new MockProcessContext(this);
+        final ProcessorInitializationContext initializationContext = new MockProcessorInitializationContext(this, processContext);
+        initialize(initializationContext);
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
+
+    }
+
+    @Test
+    public void parseSelect() {
+        String query = "select a.empid, to_something(b.saraly) from " +
+                "company.emp a inner join default.salary b where a.empid = b.empid";
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(2, tableNames.size());
+        assertTrue(tableNames.contains(new TableName("company", "emp", true)));
+        assertTrue(tableNames.contains(new TableName("default", "salary", true)));
+    }
+
+    @Test
+    public void parseSelectPrepared() {
+        String query = "select empid from company.emp a where a.firstName = ?";
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(1, tableNames.size());
+        assertTrue(tableNames.contains(new TableName("company", "emp", true)));
+    }
+
+
+    @Test
+    public void parseLongSelect() {
+        String query = "select\n" +
+                "\n" +
+                "    i_item_id,\n" +
+                "\n" +
+                "    i_item_desc,\n" +
+                "\n" +
+                "    s_state,\n" +
+                "\n" +
+                "    count(ss_quantity) as store_sales_quantitycount,\n" +
+                "\n" +
+                "    avg(ss_quantity) as store_sales_quantityave,\n" +
+                "\n" +
+                "    stddev_samp(ss_quantity) as store_sales_quantitystdev,\n" +
+                "\n" +
+                "    stddev_samp(ss_quantity) / avg(ss_quantity) as store_sales_quantitycov,\n" +
+                "\n" +
+                "    count(sr_return_quantity) as store_returns_quantitycount,\n" +
+                "\n" +
+                "    avg(sr_return_quantity) as store_returns_quantityave,\n" +
+                "\n" +
+                "    stddev_samp(sr_return_quantity) as store_returns_quantitystdev,\n" +
+                "\n" +
+                "    stddev_samp(sr_return_quantity) / avg(sr_return_quantity) as store_returns_quantitycov,\n" +
+                "\n" +
+                "    count(cs_quantity) as catalog_sales_quantitycount,\n" +
+                "\n" +
+                "    avg(cs_quantity) as catalog_sales_quantityave,\n" +
+                "\n" +
+                "    stddev_samp(cs_quantity) / avg(cs_quantity) as catalog_sales_quantitystdev,\n" +
+                "\n" +
+                "    stddev_samp(cs_quantity) / avg(cs_quantity) as catalog_sales_quantitycov\n" +
+                "\n" +
+                "from\n" +
+                "\n" +
+                "    store_sales,\n" +
+                "\n" +
+                "    store_returns,\n" +
+                "\n" +
+                "    catalog_sales,\n" +
+                "\n" +
+                "    date_dim d1,\n" +
+                "\n" +
+                "    date_dim d2,\n" +
+                "\n" +
+                "    date_dim d3,\n" +
+                "\n" +
+                "    store,\n" +
+                "\n" +
+                "    item\n" +
+                "\n" +
+                "where\n" +
+                "\n" +
+                "    d1.d_quarter_name = '2000Q1'\n" +
+                "\n" +
+                "        and d1.d_date_sk = ss_sold_date_sk\n" +
+                "\n" +
+                "        and i_item_sk = ss_item_sk\n" +
+                "\n" +
+                "        and s_store_sk = ss_store_sk\n" +
+                "\n" +
+                "        and ss_customer_sk = sr_customer_sk\n" +
+                "\n" +
+                "        and ss_item_sk = sr_item_sk\n" +
+                "\n" +
+                "        and ss_ticket_number = sr_ticket_number\n" +
+                "\n" +
+                "        and sr_returned_date_sk = d2.d_date_sk\n" +
+                "\n" +
+                "        and d2.d_quarter_name in ('2000Q1' , '2000Q2', '2000Q3')\n" +
+                "\n" +
+                "        and sr_customer_sk = cs_bill_customer_sk\n" +
+                "\n" +
+                "        and sr_item_sk = cs_item_sk\n" +
+                "\n" +
+                "        and cs_sold_date_sk = d3.d_date_sk\n" +
+                "\n" +
+                "        and d3.d_quarter_name in ('2000Q1' , '2000Q2', '2000Q3')\n" +
+                "\n" +
+                "group by i_item_id , i_item_desc , s_state\n" +
+                "\n" +
+                "order by i_item_id , i_item_desc , s_state\n" +
+                "\n" +
+                "limit 100";
+
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(6, tableNames.size());
+        AtomicInteger cnt = new AtomicInteger(0);
+        for (TableName tableName : tableNames) {
+            if (tableName.equals(new TableName(null, "store_sales", true))) {
+                cnt.incrementAndGet();
+            } else if (tableName.equals(new TableName(null, "store_returns", true))) {
+                cnt.incrementAndGet();
+            } else if (tableName.equals(new TableName(null, "catalog_sales", true))) {
+                cnt.incrementAndGet();
+            } else if (tableName.equals(new TableName(null, "date_dim", true))) {
+                cnt.incrementAndGet();
+            } else if (tableName.equals(new TableName(null, "store", true))) {
+                cnt.incrementAndGet();
+            } else if (tableName.equals(new TableName(null, "item", true))) {
+                cnt.incrementAndGet();
+            }
+        }
+        assertEquals(6, cnt.get());
+    }
+
+    @Test
+    public void parseSelectInsert() {
+        String query = "insert into databaseA.tableA select key, max(value) from databaseA.tableA where category = 'x'";
+
+        // The same database.tableName can appear two times for input and output.
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(2, tableNames.size());
+        AtomicInteger cnt = new AtomicInteger(0);
+        tableNames.forEach(tableName -> {
+            if (tableName.equals(new TableName("databaseA", "tableA", false))) {
+                cnt.incrementAndGet();
+            } else if (tableName.equals(new TableName("databaseA", "tableA", true))) {
+                cnt.incrementAndGet();
+            }
+        });
+        assertEquals(2, cnt.get());
+    }
+
+    @Test
+    public void parseInsert() {
+        String query = "insert into databaseB.tableB1 select something from tableA1 a1 inner join tableA2 a2 where a1.id = a2.id";
+
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(3, tableNames.size());
+        AtomicInteger cnt = new AtomicInteger(0);
+        tableNames.forEach(tableName -> {
+            if (tableName.equals(new TableName("databaseB", "tableB1", false))) {
+                cnt.incrementAndGet();
+            } else if (tableName.equals(new TableName(null, "tableA1", true))) {
+                cnt.incrementAndGet();
+            } else if (tableName.equals(new TableName(null, "tableA2", true))) {
+                cnt.incrementAndGet();
+            }
+        });
+        assertEquals(3, cnt.get());
+    }
+
+    @Test
+    public void parseUpdate() {
+        String query = "update table_a set y = 'updated' where x > 100";
+
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(1, tableNames.size());
+        assertTrue(tableNames.contains(new TableName(null, "table_a", false)));
+    }
+
+    @Test
+    public void parseDelete() {
+        String query = "delete from table_a where x > 100";
+
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(1, tableNames.size());
+        assertTrue(tableNames.contains(new TableName(null, "table_a", false)));
+    }
+
+    @Test
+    public void parseDDL() {
+        String query = "CREATE TABLE IF NOT EXISTS EMPLOYEES(\n" +
+                "EmployeeID INT,FirstName STRING, Title STRING,\n" +
+                "State STRING, Laptop STRING)\n" +
+                "COMMENT 'Employee Names'\n" +
+                "STORED AS ORC";
+
+
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(1, tableNames.size());
+        assertTrue(tableNames.contains(new TableName(null, "EMPLOYEES", false)));
+    }
+
+    @Test
+    public void parseSetProperty() {
+        String query = " set 'hive.exec.dynamic.partition.mode'=nonstrict";
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(0, tableNames.size());
+    }
+
+    @Test
+    public void parseSetRole() {
+        String query = "set role all";
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(0, tableNames.size());
+    }
+
+    @Test
+    public void parseShowRoles() {
+        String query = "show roles";
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(0, tableNames.size());
+    }
+
+    @Test
+    public void parseMsck() {
+        String query = "msck repair table table_a";
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(1, tableNames.size());
+        assertTrue(tableNames.contains(new TableName(null, "table_a", false)));
+    }
+
+    @Test
+    public void parseAddJar() {
+        String query = "ADD JAR hdfs:///tmp/my_jar.jar";
+        final Set<TableName> tableNames = findTableNames(query);
+        System.out.printf("tableNames=%s\n", tableNames);
+        assertEquals(0, tableNames.size());
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3QL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3QL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3QL.java
new file mode 100644
index 0000000..99b0b7d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3QL.java
@@ -0,0 +1,792 @@
+package org.apache.nifi.processors.hive;/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.dbcp.hive.Hive3DBCPService;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.pattern.RollbackOnFailure;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Types;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestPutHive3QL {
+    private static final String createPersons = "CREATE TABLE PERSONS (id integer primary key, name varchar(100), code integer)";
+    private static final String createPersonsAutoId = "CREATE TABLE PERSONS (id INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1), name VARCHAR(100), code INTEGER check(code <= 100))";
+
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+
+    @BeforeClass
+    public static void setup() {
+        System.setProperty("derby.stream.error.file", "target/derby.log");
+    }
+
+    @Test
+    public void testDirectStatements() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersons);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+        runner.enqueue("INSERT INTO PERSONS (ID, NAME, CODE) VALUES (1, 'Mark', 84)".getBytes());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+                assertEquals(1, rs.getInt(1));
+                assertEquals("Mark", rs.getString(2));
+                assertEquals(84, rs.getInt(3));
+                assertFalse(rs.next());
+            }
+        }
+
+        runner.enqueue("UPDATE PERSONS SET NAME='George' WHERE ID=1".getBytes());
+        runner.run();
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+                assertEquals(1, rs.getInt(1));
+                assertEquals("George", rs.getString(2));
+                assertEquals(84, rs.getInt(3));
+                assertFalse(rs.next());
+            }
+        }
+    }
+
+    @Test
+    public void testFailInMiddleWithBadStatement() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersonsAutoId);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+        runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Mark', 84)".getBytes());
+        runner.enqueue("INSERT INTO PERSONS".getBytes()); // intentionally wrong syntax
+        runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Tom', 3)".getBytes());
+        runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Harry', 44)".getBytes());
+        runner.run();
+
+        runner.assertTransferCount(PutHive3QL.REL_FAILURE, 1);
+        runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 3);
+        runner.getFlowFilesForRelationship(PutHive3QL.REL_SUCCESS)
+                .forEach(f -> f.assertAttributeEquals(PutHive3QL.ATTR_OUTPUT_TABLES, "PERSONS"));
+    }
+
+    @Test
+    public void testFailInMiddleWithBadStatementRollbackOnFailure() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+        runner.setProperty(RollbackOnFailure.ROLLBACK_ON_FAILURE, "true");
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersonsAutoId);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+        runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Mark', 84)".getBytes());
+        runner.enqueue("INSERT INTO PERSONS".getBytes()); // intentionally wrong syntax
+        runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Tom', 3)".getBytes());
+        runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Harry', 44)".getBytes());
+        runner.run();
+
+        // The 1st one should be routed to success, others should stay in queue.
+        assertEquals(3, runner.getQueueSize().getObjectCount());
+        runner.assertTransferCount(PutHive3QL.REL_FAILURE, 0);
+        runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 1);
+    }
+
+    @Test
+    public void testFailAtBeginning() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersonsAutoId);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+        runner.enqueue("INSERT INTO PERSONS".getBytes()); // intentionally wrong syntax
+        runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Tom', 3)".getBytes());
+        runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Harry', 44)".getBytes());
+        runner.run();
+
+        runner.assertTransferCount(PutHive3QL.REL_FAILURE, 1);
+        runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 2);
+    }
+
+    @Test
+    public void testFailAtBeginningRollbackOnFailure() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+        runner.setProperty(RollbackOnFailure.ROLLBACK_ON_FAILURE, "true");
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersonsAutoId);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+        runner.enqueue("INSERT INTO PERSONS".getBytes()); // intentionally wrong syntax
+        runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Tom', 3)".getBytes());
+        runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Harry', 44)".getBytes());
+        try {
+            runner.run();
+            fail("ProcessException should be thrown");
+        } catch (AssertionError e) {
+            assertTrue(e.getCause() instanceof ProcessException);
+        }
+
+        assertEquals(3, runner.getQueueSize().getObjectCount());
+        runner.assertTransferCount(PutHive3QL.REL_FAILURE, 0);
+        runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 0);
+    }
+
+    @Test
+    public void testFailInMiddleWithBadParameterType() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersonsAutoId);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+
+        final Map<String, String> goodAttributes = new HashMap<>();
+        goodAttributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        goodAttributes.put("hiveql.args.1.value", "84");
+
+        final Map<String, String> badAttributes = new HashMap<>();
+        badAttributes.put("hiveql.args.1.type", String.valueOf(Types.VARCHAR));
+        badAttributes.put("hiveql.args.1.value", "hello");
+
+        final byte[] data = "INSERT INTO PERSONS (NAME, CODE) VALUES ('Mark', ?)".getBytes();
+        runner.enqueue(data, goodAttributes);
+        runner.enqueue(data, badAttributes);
+        runner.enqueue(data, goodAttributes);
+        runner.enqueue(data, goodAttributes);
+        runner.run();
+
+        runner.assertTransferCount(PutHive3QL.REL_FAILURE, 1);
+        runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 3);
+    }
+
+
+    @Test
+    public void testFailInMiddleWithBadParameterValue() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersonsAutoId);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+
+        final Map<String, String> goodAttributes = new HashMap<>();
+        goodAttributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        goodAttributes.put("hiveql.args.1.value", "84");
+
+        final Map<String, String> badAttributes = new HashMap<>();
+        badAttributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        badAttributes.put("hiveql.args.1.value", "101"); // Constraint violation, up to 100
+
+        final byte[] data = "INSERT INTO PERSONS (NAME, CODE) VALUES ('Mark', ?)".getBytes();
+        runner.enqueue(data, goodAttributes);
+        runner.enqueue(data, badAttributes);
+        runner.enqueue(data, goodAttributes);
+        runner.enqueue(data, goodAttributes);
+        runner.run();
+
+        runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 3);
+        runner.assertTransferCount(PutHive3QL.REL_FAILURE, 1);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+                assertEquals(1, rs.getInt(1));
+                assertEquals("Mark", rs.getString(2));
+                assertEquals(84, rs.getInt(3));
+                assertTrue(rs.next());
+                assertTrue(rs.next());
+                assertFalse(rs.next());
+            }
+        }
+    }
+
+    @Test
+    public void testFailInMiddleWithBadNumberFormat() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersonsAutoId);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+
+        final Map<String, String> goodAttributes = new HashMap<>();
+        goodAttributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        goodAttributes.put("hiveql.args.1.value", "84");
+
+        final Map<String, String> badAttributes = new HashMap<>();
+        badAttributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        badAttributes.put("hiveql.args.1.value", "NOT_NUMBER");
+
+        final byte[] data = "INSERT INTO PERSONS (NAME, CODE) VALUES ('Mark', ?)".getBytes();
+        runner.enqueue(data, goodAttributes);
+        runner.enqueue(data, badAttributes);
+        runner.enqueue(data, goodAttributes);
+        runner.enqueue(data, goodAttributes);
+        runner.run();
+
+        runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 3);
+        runner.assertTransferCount(PutHive3QL.REL_FAILURE, 1);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+                assertEquals(1, rs.getInt(1));
+                assertEquals("Mark", rs.getString(2));
+                assertEquals(84, rs.getInt(3));
+                assertTrue(rs.next());
+                assertTrue(rs.next());
+                assertFalse(rs.next());
+            }
+        }
+    }
+
+
+    @Test
+    public void testUsingSqlDataTypesWithNegativeValues() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate("CREATE TABLE PERSONS (id integer primary key, name varchar(100), code bigint)");
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hiveql.args.1.type", "-5");
+        attributes.put("hiveql.args.1.value", "84");
+        runner.enqueue("INSERT INTO PERSONS (ID, NAME, CODE) VALUES (1, 'Mark', ?)".getBytes(), attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
+        runner.getFlowFilesForRelationship(PutHive3QL.REL_SUCCESS).get(0).assertAttributeEquals(PutHive3QL.ATTR_OUTPUT_TABLES, "PERSONS");
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+                assertEquals(1, rs.getInt(1));
+                assertEquals("Mark", rs.getString(2));
+                assertEquals(84, rs.getInt(3));
+                assertFalse(rs.next());
+            }
+        }
+    }
+
+    @Test
+    public void testStatementsWithPreparedParameters() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersons);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.1.value", "1");
+
+        attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
+        attributes.put("hiveql.args.2.value", "Mark");
+
+        attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.3.value", "84");
+
+        runner.enqueue("INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?)".getBytes(), attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+                assertEquals(1, rs.getInt(1));
+                assertEquals("Mark", rs.getString(2));
+                assertEquals(84, rs.getInt(3));
+                assertFalse(rs.next());
+            }
+        }
+
+        runner.clearTransferState();
+
+        attributes.clear();
+        attributes.put("hiveql.args.1.type", String.valueOf(Types.VARCHAR));
+        attributes.put("hiveql.args.1.value", "George");
+
+        attributes.put("hiveql.args.2.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.2.value", "1");
+
+        runner.enqueue("UPDATE PERSONS SET NAME=? WHERE ID=?".getBytes(), attributes);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+                assertEquals(1, rs.getInt(1));
+                assertEquals("George", rs.getString(2));
+                assertEquals(84, rs.getInt(3));
+                assertFalse(rs.next());
+            }
+        }
+    }
+
+
+    @Test
+    public void testMultipleStatementsWithinFlowFile() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersons);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+
+        final String sql = "INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?); " +
+            "UPDATE PERSONS SET NAME='George' WHERE ID=?; ";
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.1.value", "1");
+
+        attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
+        attributes.put("hiveql.args.2.value", "Mark");
+
+        attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.3.value", "84");
+
+        attributes.put("hiveql.args.4.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.4.value", "1");
+
+        runner.enqueue(sql.getBytes(), attributes);
+        runner.run();
+
+        // should fail because of the semicolon
+        runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
+        runner.getFlowFilesForRelationship(PutHive3QL.REL_SUCCESS)
+                .forEach(f -> f.assertAttributeEquals(PutHive3QL.ATTR_OUTPUT_TABLES, "PERSONS"));
+
+        // Now we can check that the values were inserted by the multi-statement script.
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+                assertEquals("Record ID mismatch", 1, rs.getInt(1));
+                assertEquals("Record NAME mismatch", "George", rs.getString(2));
+            }
+        }
+    }
+
+    @Test
+    public void testMultipleStatementsWithinFlowFilePlusEmbeddedDelimiter() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersons);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+
+        final String sql = "INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?); " +
+                "UPDATE PERSONS SET NAME='George\\;' WHERE ID=?; ";
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.1.value", "1");
+
+        attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
+        attributes.put("hiveql.args.2.value", "Mark");
+
+        attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.3.value", "84");
+
+        attributes.put("hiveql.args.4.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.4.value", "1");
+
+        runner.enqueue(sql.getBytes(), attributes);
+        runner.run();
+
+        // should fail because of the semicolon
+        runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
+
+        // Now we can check that the values were inserted by the multi-statement script.
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+                assertEquals("Record ID mismatch", 1, rs.getInt(1));
+                assertEquals("Record NAME mismatch", "George\\;", rs.getString(2));
+            }
+        }
+    }
+
+
+    @Test
+    public void testWithNullParameter() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersons);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.1.value", "1");
+
+        attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
+        attributes.put("hiveql.args.2.value", "Mark");
+
+        attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
+
+        runner.enqueue("INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?)".getBytes(), attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+                assertEquals(1, rs.getInt(1));
+                assertEquals("Mark", rs.getString(2));
+                assertEquals(0, rs.getInt(3));
+                assertFalse(rs.next());
+            }
+        }
+    }
+
+    @Test
+    public void testInvalidStatement() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final File tempDir = folder.getRoot();
+        final File dbDir = new File(tempDir, "db");
+        final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate(createPersons);
+            }
+        }
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+
+        final String sql = "INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?); " +
+            "UPDATE SOME_RANDOM_TABLE NAME='George' WHERE ID=?; ";
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.1.value", "1");
+
+        attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
+        attributes.put("hiveql.args.2.value", "Mark");
+
+        attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.3.value", "84");
+
+        attributes.put("hiveql.args.4.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.4.value", "1");
+
+        runner.enqueue(sql.getBytes(), attributes);
+        runner.run();
+
+        // should fail because of the table is invalid
+        runner.assertAllFlowFilesTransferred(PutHive3QL.REL_FAILURE, 1);
+
+        try (final Connection conn = service.getConnection()) {
+            try (final Statement stmt = conn.createStatement()) {
+                final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
+                assertTrue(rs.next());
+            }
+        }
+    }
+
+
+    @Test
+    public void testRetryableFailure() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final DBCPService service = new SQLExceptionService(null);
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+
+        final String sql = "INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?); " +
+            "UPDATE PERSONS SET NAME='George' WHERE ID=?; ";
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.1.value", "1");
+
+        attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
+        attributes.put("hiveql.args.2.value", "Mark");
+
+        attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.3.value", "84");
+
+        attributes.put("hiveql.args.4.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.4.value", "1");
+
+        runner.enqueue(sql.getBytes(), attributes);
+        runner.run();
+
+        // should fail because there isn't a valid connection and tables don't exist.
+        runner.assertAllFlowFilesTransferred(PutHive3QL.REL_RETRY, 1);
+    }
+
+    @Test
+    public void testRetryableFailureRollbackOnFailure() throws InitializationException, ProcessException, SQLException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
+        final DBCPService service = new SQLExceptionService(null);
+        runner.addControllerService("dbcp", service);
+        runner.enableControllerService(service);
+
+        runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+        runner.setProperty(RollbackOnFailure.ROLLBACK_ON_FAILURE, "true");
+
+        final String sql = "INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?); " +
+                "UPDATE PERSONS SET NAME='George' WHERE ID=?; ";
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.1.value", "1");
+
+        attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
+        attributes.put("hiveql.args.2.value", "Mark");
+
+        attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.3.value", "84");
+
+        attributes.put("hiveql.args.4.type", String.valueOf(Types.INTEGER));
+        attributes.put("hiveql.args.4.value", "1");
+
+        runner.enqueue(sql.getBytes(), attributes);
+        try {
+            runner.run();
+            fail("Should throw ProcessException");
+        } catch (AssertionError e) {
+            assertTrue(e.getCause() instanceof ProcessException);
+        }
+
+        assertEquals(1, runner.getQueueSize().getObjectCount());
+        runner.assertAllFlowFilesTransferred(PutHive3QL.REL_RETRY, 0);
+    }
+
+    /**
+     * Simple implementation only for testing purposes
+     */
+    private static class MockDBCPService extends AbstractControllerService implements Hive3DBCPService {
+        private final String dbLocation;
+
+        MockDBCPService(final String dbLocation) {
+            this.dbLocation = dbLocation;
+        }
+
+        @Override
+        public String getIdentifier() {
+            return "dbcp";
+        }
+
+        @Override
+        public Connection getConnection() throws ProcessException {
+            try {
+                Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
+                return DriverManager.getConnection("jdbc:derby:" + dbLocation + ";create=true");
+            } catch (final Exception e) {
+                e.printStackTrace();
+                throw new ProcessException("getConnection failed: " + e);
+            }
+        }
+
+        @Override
+        public String getConnectionURL() {
+            return "jdbc:derby:" + dbLocation + ";create=true";
+        }
+    }
+
+    /**
+     * Simple implementation only for testing purposes
+     */
+    private static class SQLExceptionService extends AbstractControllerService implements Hive3DBCPService {
+        private final Hive3DBCPService service;
+        private int allowedBeforeFailure = 0;
+        private int successful = 0;
+
+        SQLExceptionService(final Hive3DBCPService service) {
+            this.service = service;
+        }
+
+        @Override
+        public String getIdentifier() {
+            return "dbcp";
+        }
+
+        @Override
+        public Connection getConnection() throws ProcessException {
+            try {
+                if (++successful > allowedBeforeFailure) {
+                    final Connection conn = Mockito.mock(Connection.class);
+                    Mockito.when(conn.prepareStatement(Mockito.any(String.class))).thenThrow(new SQLException("Unit Test Generated SQLException"));
+                    return conn;
+                } else {
+                    return service.getConnection();
+                }
+            } catch (final Exception e) {
+                e.printStackTrace();
+                throw new ProcessException("getConnection failed: " + e);
+            }
+        }
+
+        @Override
+        public String getConnectionURL() {
+            return service != null ? service.getConnectionURL() : null;
+        }
+    }
+}


[6/6] nifi git commit: NIFI-4963: Added Hive3 bundle - Incorporated review comments - Added more defensive code for PutHive3Streaming error handling

Posted by bb...@apache.org.
NIFI-4963: Added Hive3 bundle
- Incorporated review comments
- Added more defensive code for PutHive3Streaming error handling

This closes #2755.

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/master
Commit: da99f873a7d2f636465efd86178e578da75674a0
Parents: 8feac9a
Author: Matthew Burgess <ma...@apache.org>
Authored: Mon Jun 4 11:37:48 2018 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Wed Jun 13 14:32:58 2018 -0400

----------------------------------------------------------------------
 .travis.yml                                     |   2 +-
 nifi-assembly/pom.xml                           |  23 +
 .../nifi-hive-bundle/nifi-hive-nar/pom.xml      |   2 +
 .../nifi-hive-processors/pom.xml                |   7 +-
 .../nifi-hive-services-api-nar/pom.xml          |   2 +
 .../nifi-hive-services-api/pom.xml              |   6 +
 .../apache/nifi/dbcp/hive/Hive3DBCPService.java |  30 +
 .../nifi-hive-bundle/nifi-hive3-nar/pom.xml     |  49 ++
 .../src/main/resources/META-INF/NOTICE          | 349 ++++++++
 .../nifi-hive3-processors/pom.xml               | 140 +++
 .../hadoop/hive/ql/io/orc/NiFiOrcUtils.java     | 533 +++++++++++
 .../apache/hive/streaming/HiveRecordWriter.java | 106 +++
 .../apache/hive/streaming/NiFiRecordSerDe.java  | 282 ++++++
 .../nifi/dbcp/hive/Hive3ConnectionPool.java     | 385 ++++++++
 .../hive/AbstractHive3QLProcessor.java          | 348 ++++++++
 .../apache/nifi/processors/hive/PutHive3QL.java | 280 ++++++
 .../nifi/processors/hive/PutHive3Streaming.java | 560 ++++++++++++
 .../nifi/processors/hive/SelectHive3QL.java     | 477 ++++++++++
 .../org/apache/nifi/processors/orc/PutORC.java  | 175 ++++
 .../orc/record/ORCHDFSRecordWriter.java         | 110 +++
 .../hive/AuthenticationFailedException.java     |  23 +
 .../apache/nifi/util/hive/CsvOutputOptions.java |  63 ++
 .../apache/nifi/util/hive/HiveConfigurator.java | 119 +++
 .../apache/nifi/util/hive/HiveJdbcCommon.java   | 450 ++++++++++
 .../org/apache/nifi/util/hive/HiveOptions.java  | 117 +++
 .../org/apache/nifi/util/hive/HiveUtils.java    |  76 ++
 .../nifi/util/hive/ValidationResources.java     |  41 +
 ...org.apache.nifi.controller.ControllerService |  15 +
 .../org.apache.nifi.processor.Processor         |  18 +
 .../hive/streaming/StubConnectionError.java     |  31 +
 .../hive/streaming/StubSerializationError.java  |  23 +
 .../hive/streaming/StubStreamingIOFailure.java  |  28 +
 .../hive/streaming/StubTransactionError.java    |  27 +
 .../nifi/dbcp/hive/Hive3ConnectionPoolTest.java | 138 +++
 .../nifi/processors/hive/TestHive3Parser.java   | 292 ++++++
 .../nifi/processors/hive/TestPutHive3QL.java    | 792 +++++++++++++++++
 .../processors/hive/TestPutHive3Streaming.java  | 878 +++++++++++++++++++
 .../nifi/processors/hive/TestSelectHive3QL.java | 539 ++++++++++++
 .../apache/nifi/processors/orc/PutORCTest.java  | 416 +++++++++
 .../apache/nifi/util/orc/TestNiFiOrcUtils.java  | 437 +++++++++
 .../src/test/resources/array_of_records.avsc    |  38 +
 .../src/test/resources/core-site-security.xml   |  30 +
 .../src/test/resources/core-site.xml            |  22 +
 .../src/test/resources/fake.keytab              |   0
 .../src/test/resources/hive-site-security.xml   |  26 +
 .../src/test/resources/hive-site.xml            |  22 +
 .../src/test/resources/krb5.conf                |   0
 .../src/test/resources/user.avsc                |  26 +
 .../src/test/resources/user_logical_types.avsc  |  27 +
 nifi-nar-bundles/nifi-hive-bundle/pom.xml       |  59 +-
 50 files changed, 8587 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index d6c9b39..05351b5 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -55,4 +55,4 @@ script:
   # Note: The reason the sed is done as part of script is to ensure the pom hack 
   # won't affect the 'clean install' above
   - bash .travis.sh
-  - mvn -T 2 clean install -Pcontrib-check,include-grpc,include-atlas -Ddir-only | grep -v -F -f .travis-output-filters && exit ${PIPESTATUS[0]}
+  - mvn -T 2 clean install -Pcontrib-check,include-grpc,include-atlas,include-hive3 -Ddir-only | grep -v -F -f .travis-output-filters && exit ${PIPESTATUS[0]}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index e610aa0..3f473c8 100755
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -549,6 +549,12 @@ language governing permissions and limitations under the License. -->
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-hive3-nar</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-site-to-site-reporting-nar</artifactId>
             <version>1.7.0-SNAPSHOT</version>
             <type>nar</type>
@@ -746,6 +752,23 @@ language governing permissions and limitations under the License. -->
             </dependencies>
         </profile>
         <profile>
+            <id>include-hive3</id>
+            <!-- This profile handles the inclusion of Hive 3 artifacts. The NAR
+            is quite large and makes the resultant binary distribution significantly
+            larger (275+ MB). -->
+            <activation>
+                <activeByDefault>false</activeByDefault>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.nifi</groupId>
+                    <artifactId>nifi-hive3-nar</artifactId>
+                    <version>1.7.0-SNAPSHOT</version>
+                    <type>nar</type>
+                </dependency>
+            </dependencies>
+        </profile>
+        <profile>
             <id>rpm</id>
             <activation>
                 <activeByDefault>false</activeByDefault>

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-nar/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-nar/pom.xml
index 41e0159..cb2d60d 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-nar/pom.xml
@@ -28,6 +28,8 @@
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
         <source.skip>true</source.skip>
+        <!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
+        <hadoop.version>${hive.hadoop.version}</hadoop.version>
     </properties>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
index f7b7b0b..4a6be6d 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
@@ -24,13 +24,17 @@
 
     <artifactId>nifi-hive-processors</artifactId>
     <packaging>jar</packaging>
+
     <properties>
-        <hive.version>1.2.1</hive.version>
+        <!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
+        <hadoop.version>${hive.hadoop.version}</hadoop.version>
     </properties>
+
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -84,6 +88,7 @@
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client</artifactId>
+            <version>${hadoop.version}</version>
             <exclusions>
                 <exclusion>
                     <groupId>com.google.code.findbugs</groupId>

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api-nar/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api-nar/pom.xml
index 1060225..b0b9a4c 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api-nar/pom.xml
@@ -28,6 +28,8 @@
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
         <source.skip>true</source.skip>
+        <!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
+        <hadoop.version>${hive.hadoop.version}</hadoop.version>
     </properties>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/pom.xml
index 6d85c38..2db9b34 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/pom.xml
@@ -25,10 +25,16 @@
     <artifactId>nifi-hive-services-api</artifactId>
     <packaging>jar</packaging>
 
+    <properties>
+        <!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
+        <hadoop.version>${hive.hadoop.version}</hadoop.version>
+    </properties>
+
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/src/main/java/org/apache/nifi/dbcp/hive/Hive3DBCPService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/src/main/java/org/apache/nifi/dbcp/hive/Hive3DBCPService.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/src/main/java/org/apache/nifi/dbcp/hive/Hive3DBCPService.java
new file mode 100644
index 0000000..e3af3aa
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-services-api/src/main/java/org/apache/nifi/dbcp/hive/Hive3DBCPService.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.dbcp.hive;
+
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+
+/**
+ * Definition for Database Connection Pooling Service.
+ *
+ */
+@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
+public interface Hive3DBCPService extends HiveDBCPService {
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml
new file mode 100644
index 0000000..41286d5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml
@@ -0,0 +1,49 @@
+<?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/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-hive-bundle</artifactId>
+        <version>1.7.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-hive3-nar</artifactId>
+    <version>1.7.0-SNAPSHOT</version>
+    <packaging>nar</packaging>
+    <properties>
+        <maven.javadoc.skip>true</maven.javadoc.skip>
+        <source.skip>true</source.skip>
+        <!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
+        <hadoop.version>${hive3.hadoop.version}</hadoop.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-hive-services-api-nar</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-hive3-processors</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..9da3e38
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,349 @@
+nifi-hive-nar
+Copyright 2014-2017 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+This includes derived works from the Apache Storm (ASLv2 licensed) project (https://github.com/apache/storm):
+  Copyright 2015 The Apache Software Foundation
+  The derived work is adapted from
+    org/apache/storm/hive/common/HiveWriter.java
+    org/apache/storm/hive/common/HiveOptions.java
+  and can be found in the org.apache.nifi.util.hive package
+
+This includes derived works from the Apache Hive (ASLv2 licensed) project (https://github.com/apache/hive):
+  Copyright 2008-2016 The Apache Software Foundation
+  The derived work is adapted from
+    release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
+  and can be found in the org.apache.hadoop.hive.ql.io.orc package
+  The derived work is adapted from
+      branch-3.0/streaming/src/java/org/apache/hive/streaming/StrictJsonWriter.java
+    and can be found in the org.apache.hive.streaming.HiveRecordWriter class
+  The derived work is adapted from
+      branch-3.0/serde/src/java/org/apache/hadoop/hive/serde2/JsonSerDe.java
+    and can be found in the org.apache.hive.streaming.NiFiRecordSerDe class
+
+===========================================
+Apache Software License v2
+===========================================
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) Apache Ant
+    The following NOTICE information applies:
+      Apache Ant
+      Copyright 1999-2016 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Codec
+      The following NOTICE information applies:
+        Apache Commons Codec
+        Copyright 2002-2014 The Apache Software Foundation
+
+        src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java
+        contains test data from http://aspell.net/test/orig/batch0.tab.
+        Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
+
+        ===============================================================================
+
+        The content of package org.apache.commons.codec.language.bm has been translated
+        from the original php source code available at http://stevemorse.org/phoneticinfo.htm
+        with permission from the original authors.
+        Original source copyright:
+        Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
+
+  (ASLv2) Apache Commons DBCP
+    The following NOTICE information applies:
+      Apache Commons DBCP
+      Copyright 2001-2015 The Apache Software Foundation.
+
+  (ASLv2) Apache Commons EL
+    The following NOTICE information applies:
+      Apache Commons EL
+      Copyright 1999-2016 The Apache Software Foundation
+
+      EL-8 patch - Copyright 2004-2007 Jamie Taylor
+      http://issues.apache.org/jira/browse/EL-8
+
+  (ASLv2) Apache HttpComponents
+      The following NOTICE information applies:
+        Apache HttpComponents Client
+        Copyright 1999-2016 The Apache Software Foundation
+        Apache HttpComponents Core - HttpCore
+        Copyright 2006-2009 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Logging
+    The following NOTICE information applies:
+      Apache Commons Logging
+      Copyright 2003-2014 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Pool
+    The following NOTICE information applies:
+      Apache Commons Pool
+      Copyright 1999-2009 The Apache Software Foundation.
+
+  (ASLv2) Apache Commons IO
+    The following NOTICE information applies:
+      Apache Commons IO
+      Copyright 2002-2016 The Apache Software Foundation
+
+  (ASLv2) Apache Hive
+    The following NOTICE information applies:
+      Apache Hive
+      Copyright 2008-2015 The Apache Software Foundation
+
+      This product includes software developed by The Apache Software
+      Foundation (http://www.apache.org/).
+
+      This product includes Jersey (https://jersey.java.net/)
+      Copyright (c) 2010-2014 Oracle and/or its affiliates.
+
+      This project includes software copyrighted by Microsoft Corporation and
+      licensed under the Apache License, Version 2.0.
+
+      This project includes software copyrighted by Dell SecureWorks and
+      licensed under the Apache License, Version 2.0.
+
+  (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) BoneCP
+    The following NOTICE information applies:
+       BoneCP
+       Copyright 2010 Wallace Wadge
+
+  (ASLv2) Apache Hadoop
+    The following NOTICE information applies:
+      The binary distribution of this product bundles binaries of
+      org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the
+      following notices:
+      * Copyright 2011 Dain Sundstrom <da...@iq80.com>
+      * Copyright 2011 FuseSource Corp. http://fusesource.com
+
+      The binary distribution of this product bundles binaries of
+      org.fusesource.hawtjni:hawtjni-runtime (https://github.com/fusesource/hawtjni),
+      which has the following notices:
+      * This product includes software developed by FuseSource Corp.
+        http://fusesource.com
+      * This product includes software developed at
+        Progress Software Corporation and/or its  subsidiaries or affiliates.
+      * This product includes software developed by IBM Corporation and others.
+
+  (ASLv2) Apache HBase
+    The following NOTICE information applies:
+      Apache HBase
+      Copyright 2007-2015 The Apache Software Foundation
+
+      --
+      This product incorporates portions of the 'Hadoop' project
+
+      Copyright 2007-2009 The Apache Software Foundation
+
+      Licensed under the Apache License v2.0
+      --
+      Our Orca logo we got here: http://www.vectorfree.com/jumping-orca
+      It is licensed Creative Commons Attribution 3.0.
+      See https://creativecommons.org/licenses/by/3.0/us/
+      We changed the logo by stripping the colored background, inverting
+      it and then rotating it some.
+
+      Later we found that vectorfree.com image is not properly licensed.
+      The original is owned by vectorportal.com. The original was
+      relicensed so we could use it as Creative Commons Attribution 3.0.
+      The license is bundled with the download available here:
+      http://www.vectorportal.com/subcategory/205/KILLER-WHALE-FREE-VECTOR.eps/ifile/9136/detailtest.asp
+      --
+      This product includes portions of the Bootstrap project v3.0.0
+
+      Copyright 2013 Twitter, Inc.
+
+      Licensed under the Apache License v2.0
+
+      This product uses the Glyphicons Halflings icon set.
+
+      http://glyphicons.com/
+
+      Copyright Jan Kovařík
+
+      Licensed under the Apache License v2.0 as a part of the Bootstrap project.
+
+      --
+      This product includes portions of the Guava project v14, specifically
+      'hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java'
+
+      Copyright (C) 2007 The Guava Authors
+
+      Licensed under the Apache License, Version 2.0
+
+  (ASLv2) Apache Commons Lang
+    The following NOTICE information applies:
+      Apache Commons Lang
+      Copyright 2001-2015 The Apache Software Foundation
+
+  (ASLv2) Apache Curator
+    The following NOTICE information applies:
+      Apache Curator
+      Copyright 2013-2014 The Apache Software Foundation
+
+  (ASLv2) Apache Derby
+    The following NOTICE information applies:
+      Apache Derby
+      Copyright 2004-2014 Apache, Apache DB, Apache Derby, Apache Torque, Apache JDO, Apache DDLUtils,
+      the Derby hat logo, the Apache JDO logo, and the Apache feather logo are trademarks of The Apache Software Foundation.
+
+  (ASLv2) Apache DS
+    The following NOTICE information applies:
+      ApacheDS
+      Copyright 2003-2015 The Apache Software Foundation
+
+  (ASLv2) Apache Geronimo
+    The following NOTICE information applies:
+      Apache Geronimo
+      Copyright 2003-2008 The Apache Software Foundation
+
+  (ASLv2) HTrace Core
+    The following NOTICE information applies:
+      In addition, this product includes software dependencies. See
+      the accompanying LICENSE.txt for a listing of dependencies
+      that are NOT Apache licensed (with pointers to their licensing)
+
+      Apache HTrace includes an Apache Thrift connector to Zipkin. Zipkin
+      is a distributed tracing system that is Apache 2.0 Licensed.
+      Copyright 2012 Twitter, Inc.
+
+  (ASLv2) Jettison
+    The following NOTICE information applies:
+       Copyright 2006 Envoi Solutions LLC
+
+  (ASLv2) Jetty
+    The following NOTICE information applies:
+       Jetty Web Container
+       Copyright 1995-2017 Mort Bay Consulting Pty Ltd.
+
+  (ASLv2) Apache log4j
+    The following NOTICE information applies:
+      Apache log4j
+      Copyright 2007 The Apache Software Foundation
+
+  (ASLv2) Parquet MR
+    The following NOTICE information applies:
+      Parquet MR
+      Copyright 2012 Twitter, Inc.
+
+      This project includes code from https://github.com/lemire/JavaFastPFOR
+      parquet-column/src/main/java/parquet/column/values/bitpacking/LemireBitPacking.java
+      Apache License Version 2.0 http://www.apache.org/licenses/.
+      (c) Daniel Lemire, http://lemire.me/en/
+
+  (ASLv2) Apache Thrift
+    The following NOTICE information applies:
+      Apache Thrift
+      Copyright 2006-2010 The Apache Software Foundation.
+
+  (ASLv2) Apache Twill
+    The following NOTICE information applies:
+      Apache Twill
+      Copyright 2013-2016 The Apache Software Foundation
+
+  (ASLv2) Dropwizard Metrics
+    The following NOTICE information applies:
+      Metrics
+      Copyright 2010-2013 Coda Hale and Yammer, Inc.
+
+      This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+      LongAdder), which was released with the following comments:
+
+          Written by Doug Lea with assistance from members of JCP JSR-166
+          Expert Group and released to the public domain, as explained at
+          http://creativecommons.org/publicdomain/zero/1.0/
+
+  (ASLv2) Joda Time
+      The following NOTICE information applies:
+        This product includes software developed by
+        Joda.org (http://www.joda.org/).
+
+  (ASLv2) The Netty Project
+      The following NOTICE information applies:
+        The Netty Project
+        Copyright 2011 The Netty Project
+
+  (ASLv2) Apache Tomcat
+      The following NOTICE information applies:
+        Apache Tomcat
+        Copyright 2007 The Apache Software Foundation
+
+          Java Management Extensions (JMX) support is provided by
+          the MX4J package, which is open source software.  The
+          original software and related information is available
+          at http://mx4j.sourceforge.net.
+
+          Java compilation software for JSP pages is provided by Eclipse,
+          which is open source software.  The orginal software and
+          related infomation is available at
+          http://www.eclipse.org.
+
+  (ASLv2) Apache ZooKeeper
+     The following NOTICE information applies:
+       Apache ZooKeeper
+       Copyright 2009-2012 The Apache Software Foundation
+
+  (ASLv2) Google GSON
+     The following NOTICE information applies:
+       Copyright 2008 Google Inc.
+
+  (ASLv2) JPam
+    The following NOTICE information applies:
+      Copyright 2003-2006 Greg Luck
+
+  ************************
+  Common Development and Distribution License 1.1
+  ************************
+
+  The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details.
+
+      (CDDL 1.1) (GPL2 w/ CPE) jersey-client (com.sun.jersey:jersey-client:jar:1.9 - https://jersey.java.net)
+      (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:jar:1.9 - https://jersey.java.net/)
+      (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:jar:1.9 - https://jersey.java.net/)
+      (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:jar:1.9 - https://jersey.java.net/)
+      (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:jar:1.9 - https://jersey.java.net/)
+      (CDDL 1.1) (GPL2 w/ CPE) Java Architecture For XML Binding (javax.xml.bind:jaxb-api:jar:2.2.2 - https://jaxb.dev.java.net/)
+      (CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
+
+
+  ************************
+  Common Development and Distribution License 1.0
+  ************************
+
+    The following binary components are provided under the Common Development and Distribution License 1.0.  See project link for details.
+
+      (CDDL 1.0) JavaServlet(TM) Specification (javax.servlet:servlet-api:jar:2.5 - no url available)
+      (CDDL 1.0) (GPL3) Streaming API For XML (javax.xml.stream:stax-api:jar:1.0-2 - no url provided)
+      (CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp)
+      (CDDL 1.0) JavaServer Pages(TM) API (javax.servlet.jsp:jsp-api:jar:2.1 - http://jsp.java.net)
+
+  *****************
+  Public Domain
+  *****************
+
+  The following binary components are provided to the 'Public Domain'.  See project link for details.
+
+      (Public Domain) AOP Alliance 1.0 (http://aopalliance.sourceforge.net/)

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml
new file mode 100644
index 0000000..c62268b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml
@@ -0,0 +1,140 @@
+<?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/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-hive-bundle</artifactId>
+        <version>1.7.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-hive3-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <properties>
+        <!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
+        <hadoop.version>${hive3.hadoop.version}</hadoop.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-dbcp-service-api</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-hive-services-api</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-hadoop-record-utils</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kerberos-credentials-service-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-jdbc</artifactId>
+            <version>${hive3.version}</version>
+            <exclusions>
+                <exclusion>
+                        <groupId>org.json</groupId>
+                        <artifactId>json</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-streaming</artifactId>
+            <version>${hive3.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hive.hcatalog</groupId>
+            <artifactId>hive-hcatalog-core</artifactId>
+            <version>${hive3.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+            <version>${hadoop.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.code.findbugs</groupId>
+                    <artifactId>jsr305</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-hadoop-utils</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>com.github.stephenc.findbugs</groupId>
+            <artifactId>findbugs-annotations</artifactId>
+            <version>1.3.9-1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock-record-utils</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java
new file mode 100644
index 0000000..7231421
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.hadoop.hive.ql.io.orc;
+
+import org.apache.avro.LogicalType;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.orc.MemoryManager;
+import org.apache.orc.OrcConf;
+import org.apache.orc.impl.MemoryManagerImpl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+
+/**
+ * Utility methods for ORC support (conversion from Avro, conversion to Hive types, e.g.
+ */
+public class NiFiOrcUtils {
+
+    public static Object convertToORCObject(TypeInfo typeInfo, Object o, final boolean hiveFieldNames) {
+        if (o != null) {
+            if (typeInfo instanceof UnionTypeInfo) {
+                OrcUnion union = new OrcUnion();
+                // Avro uses Utf8 and GenericData.EnumSymbol objects instead of Strings. This is handled in other places in the method, but here
+                // we need to determine the union types from the objects, so choose String.class if the object is one of those Avro classes
+                Class clazzToCompareTo = o.getClass();
+                if (o instanceof org.apache.avro.util.Utf8 || o instanceof GenericData.EnumSymbol) {
+                    clazzToCompareTo = String.class;
+                }
+                // Need to find which of the union types correspond to the primitive object
+                TypeInfo objectTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(
+                        ObjectInspectorFactory.getReflectionObjectInspector(clazzToCompareTo, ObjectInspectorFactory.ObjectInspectorOptions.JAVA));
+                List<TypeInfo> unionTypeInfos = ((UnionTypeInfo) typeInfo).getAllUnionObjectTypeInfos();
+
+                int index = 0;
+                while (index < unionTypeInfos.size() && !unionTypeInfos.get(index).equals(objectTypeInfo)) {
+                    index++;
+                }
+                if (index < unionTypeInfos.size()) {
+                    union.set((byte) index, convertToORCObject(objectTypeInfo, o, hiveFieldNames));
+                } else {
+                    throw new IllegalArgumentException("Object Type for class " + o.getClass().getName() + " not in Union declaration");
+                }
+                return union;
+            }
+            if (o instanceof Integer) {
+                return new IntWritable((int) o);
+            }
+            if (o instanceof Boolean) {
+                return new BooleanWritable((boolean) o);
+            }
+            if (o instanceof Long) {
+                return new LongWritable((long) o);
+            }
+            if (o instanceof Float) {
+                return new FloatWritable((float) o);
+            }
+            if (o instanceof Double) {
+                return new DoubleWritable((double) o);
+            }
+            if (o instanceof String || o instanceof Utf8 || o instanceof GenericData.EnumSymbol) {
+                return new Text(o.toString());
+            }
+            if (o instanceof ByteBuffer) {
+                return new BytesWritable(((ByteBuffer) o).array());
+            }
+            if (o instanceof Timestamp) {
+                return new TimestampWritable((Timestamp) o);
+            }
+            if (o instanceof Date) {
+                return new DateWritable((Date) o);
+            }
+            if (o instanceof Object[]) {
+                Object[] objArray = (Object[]) o;
+                TypeInfo listTypeInfo = ((ListTypeInfo) typeInfo).getListElementTypeInfo();
+                return Arrays.stream(objArray)
+                        .map(o1 -> convertToORCObject(listTypeInfo, o1, hiveFieldNames))
+                        .collect(Collectors.toList());
+            }
+            if (o instanceof int[]) {
+                int[] intArray = (int[]) o;
+                return Arrays.stream(intArray)
+                        .mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("int"), element, hiveFieldNames))
+                        .collect(Collectors.toList());
+            }
+            if (o instanceof long[]) {
+                long[] longArray = (long[]) o;
+                return Arrays.stream(longArray)
+                        .mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("bigint"), element, hiveFieldNames))
+                        .collect(Collectors.toList());
+            }
+            if (o instanceof float[]) {
+                float[] floatArray = (float[]) o;
+                return IntStream.range(0, floatArray.length)
+                        .mapToDouble(i -> floatArray[i])
+                        .mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("float"), (float) element, hiveFieldNames))
+                        .collect(Collectors.toList());
+            }
+            if (o instanceof double[]) {
+                double[] doubleArray = (double[]) o;
+                return Arrays.stream(doubleArray)
+                        .mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("double"), element, hiveFieldNames))
+                        .collect(Collectors.toList());
+            }
+            if (o instanceof boolean[]) {
+                boolean[] booleanArray = (boolean[]) o;
+                return IntStream.range(0, booleanArray.length)
+                        .map(i -> booleanArray[i] ? 1 : 0)
+                        .mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("boolean"), element == 1, hiveFieldNames))
+                        .collect(Collectors.toList());
+            }
+            if (o instanceof GenericData.Array) {
+                GenericData.Array array = ((GenericData.Array) o);
+                // The type information in this case is interpreted as a List
+                TypeInfo listTypeInfo = ((ListTypeInfo) typeInfo).getListElementTypeInfo();
+                return array.stream().map((element) -> convertToORCObject(listTypeInfo, element, hiveFieldNames)).collect(Collectors.toList());
+            }
+            if (o instanceof List) {
+                return o;
+            }
+            if (o instanceof Map) {
+                Map map = new HashMap();
+                TypeInfo keyInfo = ((MapTypeInfo) typeInfo).getMapKeyTypeInfo();
+                TypeInfo valueInfo = ((MapTypeInfo) typeInfo).getMapValueTypeInfo();
+                // Unions are not allowed as key/value types, so if we convert the key and value objects,
+                // they should return Writable objects
+                ((Map) o).forEach((key, value) -> {
+                    Object keyObject = convertToORCObject(keyInfo, key, hiveFieldNames);
+                    Object valueObject = convertToORCObject(valueInfo, value, hiveFieldNames);
+                    if (keyObject == null) {
+                        throw new IllegalArgumentException("Maps' key cannot be null");
+                    }
+                    map.put(keyObject, valueObject);
+                });
+                return map;
+            }
+            if (o instanceof GenericData.Record) {
+                GenericData.Record record = (GenericData.Record) o;
+                TypeInfo recordSchema = NiFiOrcUtils.getOrcField(record.getSchema(), hiveFieldNames);
+                List<Schema.Field> recordFields = record.getSchema().getFields();
+                if (recordFields != null) {
+                    Object[] fieldObjects = new Object[recordFields.size()];
+                    for (int i = 0; i < recordFields.size(); i++) {
+                        Schema.Field field = recordFields.get(i);
+                        Schema fieldSchema = field.schema();
+                        Object fieldObject = record.get(field.name());
+                        fieldObjects[i] = NiFiOrcUtils.convertToORCObject(NiFiOrcUtils.getOrcField(fieldSchema, hiveFieldNames), fieldObject, hiveFieldNames);
+                    }
+                    return NiFiOrcUtils.createOrcStruct(recordSchema, fieldObjects);
+                }
+            }
+            throw new IllegalArgumentException("Error converting object of type " + o.getClass().getName() + " to ORC type " + typeInfo.getTypeName());
+        } else {
+            return null;
+        }
+    }
+
+
+    /**
+     * Create an object of OrcStruct given a TypeInfo and a list of objects
+     *
+     * @param typeInfo The TypeInfo object representing the ORC record schema
+     * @param objs     ORC objects/Writables
+     * @return an OrcStruct containing the specified objects for the specified schema
+     */
+    @SuppressWarnings("unchecked")
+    public static OrcStruct createOrcStruct(TypeInfo typeInfo, Object... objs) {
+        SettableStructObjectInspector oi = (SettableStructObjectInspector) OrcStruct
+                .createObjectInspector(typeInfo);
+        List<StructField> fields = (List<StructField>) oi.getAllStructFieldRefs();
+        OrcStruct result = (OrcStruct) oi.create();
+        result.setNumFields(fields.size());
+        for (int i = 0; i < fields.size(); i++) {
+            oi.setStructFieldData(result, fields.get(i), objs[i]);
+        }
+        return result;
+    }
+
+    public static String normalizeHiveTableName(String name) {
+        return name.replaceAll("[\\. ]", "_");
+    }
+
+    public static String generateHiveDDL(Schema avroSchema, String tableName, boolean hiveFieldNames) {
+        Schema.Type schemaType = avroSchema.getType();
+        StringBuilder sb = new StringBuilder("CREATE EXTERNAL TABLE IF NOT EXISTS ");
+        sb.append(tableName);
+        sb.append(" (");
+        if (Schema.Type.RECORD.equals(schemaType)) {
+            List<String> hiveColumns = new ArrayList<>();
+            List<Schema.Field> fields = avroSchema.getFields();
+            if (fields != null) {
+                hiveColumns.addAll(
+                        fields.stream().map(field -> (hiveFieldNames ? field.name().toLowerCase() : field.name()) + " "
+                                + getHiveTypeFromAvroType(field.schema(), hiveFieldNames)).collect(Collectors.toList()));
+            }
+            sb.append(StringUtils.join(hiveColumns, ", "));
+            sb.append(") STORED AS ORC");
+            return sb.toString();
+        } else {
+            throw new IllegalArgumentException("Avro schema is of type " + schemaType.getName() + ", not RECORD");
+        }
+    }
+
+
+    public static TypeInfo getOrcField(Schema fieldSchema, boolean hiveFieldNames) throws IllegalArgumentException {
+        Schema.Type fieldType = fieldSchema.getType();
+        LogicalType logicalType = fieldSchema.getLogicalType();
+
+        switch (fieldType) {
+            case INT:
+            case LONG:
+                // Handle logical types
+                if (logicalType != null) {
+                    if (LogicalTypes.date().equals(logicalType)) {
+                        return TypeInfoFactory.dateTypeInfo;
+                    } else if (LogicalTypes.timeMicros().equals(logicalType)) {
+                        // Time micros isn't supported by our Record Field types (see AvroTypeUtil)
+                        throw new IllegalArgumentException("time-micros is not a supported field type");
+                    } else if (LogicalTypes.timeMillis().equals(logicalType)) {
+                        return TypeInfoFactory.intTypeInfo;
+                    } else if (LogicalTypes.timestampMicros().equals(logicalType)) {
+                        // Timestamp micros isn't supported by our Record Field types (see AvroTypeUtil)
+                        throw new IllegalArgumentException("timestamp-micros is not a supported field type");
+                    } else if (LogicalTypes.timestampMillis().equals(logicalType)) {
+                        return TypeInfoFactory.timestampTypeInfo;
+                    }
+                }
+                return getPrimitiveOrcTypeFromPrimitiveAvroType(fieldType);
+            case BYTES:
+                // Handle logical types
+                if (logicalType != null) {
+                    if (logicalType instanceof LogicalTypes.Decimal) {
+                        return TypeInfoFactory.doubleTypeInfo;
+                    }
+                }
+                return getPrimitiveOrcTypeFromPrimitiveAvroType(fieldType);
+
+            case BOOLEAN:
+            case DOUBLE:
+            case FLOAT:
+            case STRING:
+                return getPrimitiveOrcTypeFromPrimitiveAvroType(fieldType);
+
+            case UNION:
+                List<Schema> unionFieldSchemas = fieldSchema.getTypes();
+
+                if (unionFieldSchemas != null) {
+                    // Ignore null types in union
+                    List<TypeInfo> orcFields = unionFieldSchemas.stream().filter(
+                            unionFieldSchema -> !Schema.Type.NULL.equals(unionFieldSchema.getType()))
+                            .map((it) -> NiFiOrcUtils.getOrcField(it, hiveFieldNames))
+                            .collect(Collectors.toList());
+
+                    // Flatten the field if the union only has one non-null element
+                    if (orcFields.size() == 1) {
+                        return orcFields.get(0);
+                    } else {
+                        return TypeInfoFactory.getUnionTypeInfo(orcFields);
+                    }
+                }
+                return null;
+
+            case ARRAY:
+                return TypeInfoFactory.getListTypeInfo(getOrcField(fieldSchema.getElementType(), hiveFieldNames));
+
+            case MAP:
+                return TypeInfoFactory.getMapTypeInfo(
+                        getPrimitiveOrcTypeFromPrimitiveAvroType(Schema.Type.STRING),
+                        getOrcField(fieldSchema.getValueType(), hiveFieldNames));
+
+            case RECORD:
+                List<Schema.Field> avroFields = fieldSchema.getFields();
+                if (avroFields != null) {
+                    List<String> orcFieldNames = new ArrayList<>(avroFields.size());
+                    List<TypeInfo> orcFields = new ArrayList<>(avroFields.size());
+                    avroFields.forEach(avroField -> {
+                        String fieldName = hiveFieldNames ? avroField.name().toLowerCase() : avroField.name();
+                        orcFieldNames.add(fieldName);
+                        orcFields.add(getOrcField(avroField.schema(), hiveFieldNames));
+                    });
+                    return TypeInfoFactory.getStructTypeInfo(orcFieldNames, orcFields);
+                }
+                return null;
+
+            case ENUM:
+                // An enum value is just a String for ORC/Hive
+                return getPrimitiveOrcTypeFromPrimitiveAvroType(Schema.Type.STRING);
+
+            default:
+                throw new IllegalArgumentException("Did not recognize Avro type " + fieldType.getName());
+        }
+
+    }
+
+    public static Schema.Type getAvroSchemaTypeOfObject(Object o) {
+        if (o == null) {
+            return Schema.Type.NULL;
+        } else if (o instanceof Integer) {
+            return Schema.Type.INT;
+        } else if (o instanceof Long) {
+            return Schema.Type.LONG;
+        } else if (o instanceof Boolean) {
+            return Schema.Type.BOOLEAN;
+        } else if (o instanceof byte[]) {
+            return Schema.Type.BYTES;
+        } else if (o instanceof Float) {
+            return Schema.Type.FLOAT;
+        } else if (o instanceof Double) {
+            return Schema.Type.DOUBLE;
+        } else if (o instanceof Enum) {
+            return Schema.Type.ENUM;
+        } else if (o instanceof Object[]) {
+            return Schema.Type.ARRAY;
+        } else if (o instanceof List) {
+            return Schema.Type.ARRAY;
+        } else if (o instanceof Map) {
+            return Schema.Type.MAP;
+        } else {
+            throw new IllegalArgumentException("Object of class " + o.getClass() + " is not a supported Avro Type");
+        }
+    }
+
+    public static TypeInfo getPrimitiveOrcTypeFromPrimitiveAvroType(Schema.Type avroType) throws IllegalArgumentException {
+        if (avroType == null) {
+            throw new IllegalArgumentException("Avro type is null");
+        }
+        switch (avroType) {
+            case INT:
+                return TypeInfoFactory.getPrimitiveTypeInfo("int");
+            case LONG:
+                return TypeInfoFactory.getPrimitiveTypeInfo("bigint");
+            case BOOLEAN:
+                return TypeInfoFactory.getPrimitiveTypeInfo("boolean");
+            case BYTES:
+                return TypeInfoFactory.getPrimitiveTypeInfo("binary");
+            case DOUBLE:
+                return TypeInfoFactory.getPrimitiveTypeInfo("double");
+            case FLOAT:
+                return TypeInfoFactory.getPrimitiveTypeInfo("float");
+            case STRING:
+                return TypeInfoFactory.getPrimitiveTypeInfo("string");
+            default:
+                throw new IllegalArgumentException("Avro type " + avroType.getName() + " is not a primitive type");
+        }
+    }
+
+    public static String getHiveTypeFromAvroType(Schema avroSchema, boolean hiveFieldNames) {
+        if (avroSchema == null) {
+            throw new IllegalArgumentException("Avro schema is null");
+        }
+
+        Schema.Type avroType = avroSchema.getType();
+        LogicalType logicalType = avroSchema.getLogicalType();
+
+        switch (avroType) {
+            case INT:
+                if (logicalType != null) {
+                    if (LogicalTypes.date().equals(logicalType)) {
+                        return "DATE";
+                    }
+                    // Time-millis has no current corresponding Hive type, perhaps an INTERVAL type when that is fully supported.
+                }
+                return "INT";
+            case LONG:
+                if (logicalType != null) {
+                    if (LogicalTypes.timestampMillis().equals(logicalType)) {
+                        return "TIMESTAMP";
+                    }
+                    // Timestamp-micros and time-micros are not supported by our Record Field type system
+                }
+                return "BIGINT";
+            case BOOLEAN:
+                return "BOOLEAN";
+            case BYTES:
+                if (logicalType != null) {
+                    if (logicalType instanceof LogicalTypes.Decimal) {
+                        return "DOUBLE";
+                    }
+                }
+                return "BINARY";
+            case DOUBLE:
+                return "DOUBLE";
+            case FLOAT:
+                return "FLOAT";
+            case STRING:
+            case ENUM:
+                return "STRING";
+            case UNION:
+                List<Schema> unionFieldSchemas = avroSchema.getTypes();
+                if (unionFieldSchemas != null) {
+                    List<String> hiveFields = new ArrayList<>();
+                    for (Schema unionFieldSchema : unionFieldSchemas) {
+                        Schema.Type unionFieldSchemaType = unionFieldSchema.getType();
+                        // Ignore null types in union
+                        if (!Schema.Type.NULL.equals(unionFieldSchemaType)) {
+                            hiveFields.add(getHiveTypeFromAvroType(unionFieldSchema, hiveFieldNames));
+                        }
+                    }
+                    // Flatten the field if the union only has one non-null element
+                    return (hiveFields.size() == 1)
+                            ? hiveFields.get(0)
+                            : "UNIONTYPE<" + StringUtils.join(hiveFields, ", ") + ">";
+
+                }
+                break;
+            case MAP:
+                return "MAP<STRING, " + getHiveTypeFromAvroType(avroSchema.getValueType(), hiveFieldNames) + ">";
+            case ARRAY:
+                return "ARRAY<" + getHiveTypeFromAvroType(avroSchema.getElementType(), hiveFieldNames) + ">";
+            case RECORD:
+                List<Schema.Field> recordFields = avroSchema.getFields();
+                if (recordFields != null) {
+                    List<String> hiveFields = recordFields.stream().map(
+                            recordField -> (hiveFieldNames ? recordField.name().toLowerCase() : recordField.name()) + ":"
+                                    + getHiveTypeFromAvroType(recordField.schema(), hiveFieldNames)).collect(Collectors.toList());
+                    return "STRUCT<" + StringUtils.join(hiveFields, ", ") + ">";
+                }
+                break;
+            default:
+                break;
+        }
+
+        throw new IllegalArgumentException("Error converting Avro type " + avroType.getName() + " to Hive type");
+    }
+
+
+    public static Writer createWriter(
+            Path path,
+            Configuration conf,
+            TypeInfo orcSchema,
+            long stripeSize,
+            CompressionKind compress,
+            int bufferSize) throws IOException {
+
+        int rowIndexStride = (int) OrcConf.ROW_INDEX_STRIDE.getLong(conf);
+
+        boolean addBlockPadding = OrcConf.BLOCK_PADDING.getBoolean(conf);
+
+        String versionName = OrcConf.WRITE_FORMAT.getString(conf);
+        OrcFile.Version versionValue = (versionName == null)
+                ? OrcFile.Version.CURRENT
+                : OrcFile.Version.byName(versionName);
+
+        OrcFile.EncodingStrategy encodingStrategy;
+        String enString = OrcConf.ENCODING_STRATEGY.getString(conf);
+        if (enString == null) {
+            encodingStrategy = OrcFile.EncodingStrategy.SPEED;
+        } else {
+            encodingStrategy = OrcFile.EncodingStrategy.valueOf(enString);
+        }
+
+        final double paddingTolerance = OrcConf.BLOCK_PADDING_TOLERANCE.getDouble(conf);
+
+        long blockSizeValue = OrcConf.BLOCK_SIZE.getLong(conf);
+
+        double bloomFilterFpp = OrcConf.BLOOM_FILTER_FPP.getDouble(conf);
+
+        ObjectInspector inspector = OrcStruct.createObjectInspector(orcSchema);
+
+        OrcFile.WriterOptions writerOptions = OrcFile.writerOptions(conf)
+                .rowIndexStride(rowIndexStride)
+                .blockPadding(addBlockPadding)
+                .version(versionValue)
+                .encodingStrategy(encodingStrategy)
+                .paddingTolerance(paddingTolerance)
+                .blockSize(blockSizeValue)
+                .bloomFilterFpp(bloomFilterFpp)
+                .memory(getMemoryManager(conf))
+                .inspector(inspector)
+                .stripeSize(stripeSize)
+                .bufferSize(bufferSize)
+                .compress(compress);
+
+        return OrcFile.createWriter(path, writerOptions);
+    }
+
+    private static MemoryManager memoryManager = null;
+
+    private static synchronized MemoryManager getMemoryManager(Configuration conf) {
+        if (memoryManager == null) {
+            memoryManager = new MemoryManagerImpl(conf);
+        }
+        return memoryManager;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/HiveRecordWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/HiveRecordWriter.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/HiveRecordWriter.java
new file mode 100644
index 0000000..6edb374
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/HiveRecordWriter.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.hive.streaming;
+
+import com.google.common.base.Joiner;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.io.ObjectWritable;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+import java.util.Properties;
+
+public class HiveRecordWriter extends AbstractRecordWriter {
+
+    private RecordReader recordReader;
+    private NiFiRecordSerDe serde;
+    private ComponentLog log;
+
+    public HiveRecordWriter(RecordReader recordReader, ComponentLog log) {
+        super(null);
+        this.recordReader = recordReader;
+        this.log = log;
+    }
+
+    @Override
+    public AbstractSerDe createSerde() throws SerializationError {
+        try {
+            Properties tableProps = table.getMetadata();
+            tableProps.setProperty(serdeConstants.LIST_COLUMNS, Joiner.on(",").join(inputColumns));
+            tableProps.setProperty(serdeConstants.LIST_COLUMN_TYPES, Joiner.on(":").join(inputTypes));
+            NiFiRecordSerDe serde = new NiFiRecordSerDe(recordReader, log);
+            SerDeUtils.initializeSerDe(serde, conf, tableProps, null);
+            this.serde = serde;
+            return serde;
+        } catch (SerDeException e) {
+            throw new SerializationError("Error initializing serde " + NiFiRecordSerDe.class.getName(), e);
+        }
+    }
+
+    @Override
+    public Object encode(byte[] bytes) {
+        throw new UnsupportedOperationException(this.getClass().getName() + " does not support encoding of records via bytes, only via an InputStream");
+    }
+
+    @Override
+    public void write(long writeId, byte[] record) {
+        throw new UnsupportedOperationException(this.getClass().getName() + " does not support writing of records via bytes, only via an InputStream");
+    }
+
+    @Override
+    public void write(long writeId, InputStream inputStream) throws StreamingException {
+        // The inputStream is already available to the recordReader, so just iterate through the records
+        try {
+            Record record;
+            while ((record = recordReader.nextRecord()) != null) {
+                write(writeId, record);
+            }
+        } catch (MalformedRecordException | IOException e) {
+            throw new StreamingException(e.getLocalizedMessage(), e);
+        }
+    }
+
+    public Object encode(Record record) throws SerializationError {
+        try {
+            ObjectWritable blob = new ObjectWritable(record);
+            return serde.deserialize(blob);
+        } catch (SerDeException e) {
+            throw new SerializationError("Unable to convert Record into Object", e);
+        }
+    }
+
+    private void write(long writeId, Record record) throws StreamingException {
+        checkAutoFlush();
+        try {
+            Object encodedRow = encode(record);
+            int bucket = getBucket(encodedRow);
+            List<String> partitionValues = getPartitionValues(encodedRow);
+            getRecordUpdater(partitionValues, bucket).insert(writeId, encodedRow);
+            conn.getConnectionStats().incrementRecordsWritten();
+        } catch (IOException e) {
+            throw new StreamingIOFailure("Error writing record in transaction write id (" + writeId + ")", e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java
new file mode 100644
index 0000000..d4b444a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.hive.streaming;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeStats;
+import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.ObjectWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hive.common.util.HiveStringUtils;
+import org.apache.hive.common.util.TimestampParser;
+import org.apache.nifi.avro.AvroTypeUtil;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class NiFiRecordSerDe extends AbstractSerDe {
+
+    protected RecordReader recordReader;
+    protected ComponentLog log;
+    protected List<String> columnNames;
+    protected StructTypeInfo schema;
+    protected SerDeStats stats;
+
+    protected StandardStructObjectInspector cachedObjectInspector;
+    protected TimestampParser tsParser;
+
+    private final static Pattern INTERNAL_PATTERN = Pattern.compile("_col([0-9]+)");
+
+    private Map<String, Integer> fieldPositionMap;
+
+    public NiFiRecordSerDe(RecordReader recordReader, ComponentLog log) {
+        this.recordReader = recordReader;
+        this.log = log;
+    }
+
+    @Override
+    public void initialize(Configuration conf, Properties tbl) throws SerDeException {
+        List<TypeInfo> columnTypes;
+        StructTypeInfo rowTypeInfo;
+
+        log.debug("Initializing NiFiRecordSerDe: {}", tbl.entrySet().toArray());
+
+        // Get column names and types
+        String columnNameProperty = tbl.getProperty(serdeConstants.LIST_COLUMNS);
+        String columnTypeProperty = tbl.getProperty(serdeConstants.LIST_COLUMN_TYPES);
+        final String columnNameDelimiter = tbl.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ? tbl
+                .getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : String.valueOf(SerDeUtils.COMMA);
+        // all table column names
+        if (columnNameProperty.isEmpty()) {
+            columnNames = new ArrayList<>(0);
+        } else {
+            columnNames = new ArrayList<>(Arrays.asList(columnNameProperty.split(columnNameDelimiter)));
+        }
+
+        // all column types
+        if (columnTypeProperty.isEmpty()) {
+            columnTypes = new ArrayList<>(0);
+        } else {
+            columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty);
+        }
+
+        log.debug("columns: {}, {}", new Object[]{columnNameProperty, columnNames});
+        log.debug("types: {}, {} ", new Object[]{columnTypeProperty, columnTypes});
+
+        assert (columnNames.size() == columnTypes.size());
+
+        rowTypeInfo = (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes);
+        schema = rowTypeInfo;
+        log.debug("schema : {}", new Object[]{schema});
+        cachedObjectInspector = (StandardStructObjectInspector) TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(rowTypeInfo);
+        tsParser = new TimestampParser(HiveStringUtils.splitAndUnEscape(tbl.getProperty(serdeConstants.TIMESTAMP_FORMATS)));
+        // Populate mapping of field names to column positions
+        try {
+            populateFieldPositionMap();
+        } catch (MalformedRecordException | IOException e) {
+            throw new SerDeException(e);
+        }
+        stats = new SerDeStats();
+    }
+
+    @Override
+    public Class<? extends Writable> getSerializedClass() {
+        return ObjectWritable.class;
+    }
+
+    @Override
+    public Writable serialize(Object o, ObjectInspector objectInspector) throws SerDeException {
+        throw new UnsupportedOperationException("This SerDe only supports deserialization");
+    }
+
+    @Override
+    public SerDeStats getSerDeStats() {
+        return stats;
+    }
+
+    @Override
+    public Object deserialize(Writable writable) throws SerDeException {
+        ObjectWritable t = (ObjectWritable) writable;
+        Record record = (Record) t.get();
+        List<Object> r = new ArrayList<>(Collections.nCopies(columnNames.size(), null));
+        try {
+            RecordSchema recordSchema = record.getSchema();
+            for (RecordField field : recordSchema.getFields()) {
+                String fieldName = field.getFieldName();
+                String normalizedFieldName = fieldName.toLowerCase();
+
+                // Get column position of field name, and set field value there
+                Integer fpos = fieldPositionMap.get(normalizedFieldName);
+                if(fpos == null || fpos == -1) {
+                    // This is either a partition column or not a column in the target table, ignore either way
+                    continue;
+                }
+                Object currField = extractCurrentField(record, field, schema.getStructFieldTypeInfo(normalizedFieldName));
+                r.set(fpos, currField);
+            }
+            stats.setRowCount(stats.getRowCount() + 1);
+
+        } catch (Exception e) {
+            log.warn("Error [{}] parsing Record [{}].", new Object[]{e.getLocalizedMessage(), t}, e);
+            throw new SerDeException(e);
+        }
+
+        return r;
+    }
+
+    /**
+     * Utility method to extract current expected field from given JsonParser
+     * isTokenCurrent is a boolean variable also passed in, which determines
+     * if the JsonParser is already at the token we expect to read next, or
+     * needs advancing to the next before we read.
+     */
+    private Object extractCurrentField(Record record, RecordField field, TypeInfo fieldTypeInfo) {
+        Object val;
+        String fieldName = (field != null) ? field.getFieldName() : null;
+
+        switch (fieldTypeInfo.getCategory()) {
+            case PRIMITIVE:
+                PrimitiveObjectInspector.PrimitiveCategory primitiveCategory = PrimitiveObjectInspector.PrimitiveCategory.UNKNOWN;
+                if (fieldTypeInfo instanceof PrimitiveTypeInfo) {
+                    primitiveCategory = ((PrimitiveTypeInfo) fieldTypeInfo).getPrimitiveCategory();
+                }
+                switch (primitiveCategory) {
+                    case INT:
+                    case BYTE:
+                    case SHORT:
+                        val = record.getAsInt(fieldName);
+                        break;
+                    case LONG:
+                        val = record.getAsLong(fieldName);
+                        break;
+                    case BOOLEAN:
+                        val = record.getAsBoolean(fieldName);
+                        break;
+                    case FLOAT:
+                        val = record.getAsFloat(fieldName);
+                        break;
+                    case DOUBLE:
+                        val = record.getAsDouble(fieldName);
+                        break;
+                    case STRING:
+                    case VARCHAR:
+                    case CHAR:
+                        val = record.getAsString(fieldName);
+                        break;
+                    case BINARY:
+                        val = AvroTypeUtil.convertByteArray(record.getAsArray(fieldName)).array();
+                        break;
+                    case DATE:
+                        val = record.getAsDate(fieldName, field.getDataType().getFormat());
+                        break;
+                    case TIMESTAMP:
+                        val = DataTypeUtils.toTimestamp(record.getValue(fieldName), () -> DataTypeUtils.getDateFormat(field.getDataType().getFormat()), fieldName);
+                        break;
+                    case DECIMAL:
+                        val = record.getAsDouble(fieldName);
+                        break;
+                    default:
+                        throw new IllegalArgumentException("Field " + fieldName + " cannot be converted to unknown type: " + primitiveCategory.name());
+                }
+                break;
+            case LIST:
+                val = Arrays.asList(record.getAsArray(fieldName));
+                break;
+            case MAP:
+                val = DataTypeUtils.convertRecordFieldtoObject(record.getValue(fieldName), field.getDataType());
+                break;
+            case STRUCT:
+                val = DataTypeUtils.convertRecordFieldtoObject(record.getValue(fieldName), field.getDataType());
+                break;
+            default:
+                log.error("Unknown type found: " + fieldTypeInfo + "for field of type: " + field.getDataType().toString());
+                return null;
+        }
+        return val;
+    }
+
+    @Override
+    public ObjectInspector getObjectInspector() {
+        return cachedObjectInspector;
+    }
+
+    private void populateFieldPositionMap() throws MalformedRecordException, IOException {
+        // Populate the mapping of field names to column positions only once
+        fieldPositionMap = new HashMap<>(columnNames.size());
+
+        RecordSchema recordSchema = recordReader.getSchema();
+        for (RecordField field : recordSchema.getFields()) {
+            String fieldName = field.getFieldName();
+            String normalizedFieldName = fieldName.toLowerCase();
+
+            int fpos = schema.getAllStructFieldNames().indexOf(fieldName.toLowerCase());
+            if (fpos == -1) {
+                Matcher m = INTERNAL_PATTERN.matcher(fieldName);
+                fpos = m.matches() ? Integer.parseInt(m.group(1)) : -1;
+
+                log.debug("NPE finding position for field [{}] in schema [{}],"
+                        + " attempting to check if it is an internal column name like _col0", new Object[]{fieldName, schema});
+                if (fpos == -1) {
+                    // unknown field, we return. We'll continue from the next field onwards. Log at debug level because partition columns will be "unknown fields"
+                    log.debug("Field {} is not found in the target table, ignoring...", new Object[]{field.getFieldName()});
+                    continue;
+                }
+                // If we get past this, then the column name did match the hive pattern for an internal
+                // column name, such as _col0, etc, so it *MUST* match the schema for the appropriate column.
+                // This means people can't use arbitrary column names such as _col0, and expect us to ignore it
+                // if we find it.
+                if (!fieldName.equalsIgnoreCase(HiveConf.getColumnInternalName(fpos))) {
+                    log.error("Hive internal column name {} and position "
+                            + "encoding {} for the column name are at odds", new Object[]{fieldName, fpos});
+                    throw new IOException("Hive internal column name (" + fieldName
+                            + ") and position encoding (" + fpos
+                            + ") for the column name are at odds");
+                }
+                // If we reached here, then we were successful at finding an alternate internal
+                // column mapping, and we're about to proceed.
+            }
+            fieldPositionMap.put(normalizedFieldName, fpos);
+        }
+    }
+}


[2/6] nifi git commit: NIFI-4963: Added Hive3 bundle - Incorporated review comments - Added more defensive code for PutHive3Streaming error handling

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
new file mode 100644
index 0000000..6a65783
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
@@ -0,0 +1,878 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.hive;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.file.DataFileWriter;
+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.DatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.streaming.ConnectionStats;
+import org.apache.hive.streaming.HiveRecordWriter;
+import org.apache.hive.streaming.PartitionInfo;
+import org.apache.hive.streaming.RecordWriter;
+import org.apache.hive.streaming.StreamingConnection;
+import org.apache.hive.streaming.StreamingException;
+import org.apache.hive.streaming.StubConnectionError;
+import org.apache.hive.streaming.StubSerializationError;
+import org.apache.hive.streaming.StubStreamingIOFailure;
+import org.apache.hive.streaming.StubTransactionError;
+import org.apache.nifi.avro.AvroTypeUtil;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.kerberos.KerberosCredentialsService;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.util.hive.HiveConfigurator;
+import org.apache.nifi.util.hive.HiveOptions;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiFunction;
+
+import static org.apache.nifi.processors.hive.AbstractHive3QLProcessor.ATTR_OUTPUT_TABLES;
+import static org.apache.nifi.processors.hive.PutHive3Streaming.HIVE_STREAMING_RECORD_COUNT_ATTR;
+import static org.apache.nifi.processors.hive.PutHive3Streaming.KERBEROS_CREDENTIALS_SERVICE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Unit tests for PutHive3Streaming processor.
+ */
+public class TestPutHive3Streaming {
+
+    private static final String TEST_CONF_PATH = "src/test/resources/core-site.xml";
+    private static final String TARGET_HIVE = "target/hive";
+
+    private TestRunner runner;
+    private MockPutHive3Streaming processor;
+
+    private HiveConfigurator hiveConfigurator;
+    private HiveConf hiveConf;
+    private UserGroupInformation ugi;
+    private Schema schema;
+
+    @Before
+    public void setUp() throws Exception {
+
+        final String avroSchema = IOUtils.toString(new FileInputStream("src/test/resources/user.avsc"), StandardCharsets.UTF_8);
+        schema = new Schema.Parser().parse(avroSchema);
+
+        Configuration testConf = new Configuration();
+        testConf.addResource(new Path(TEST_CONF_PATH));
+
+        // needed for calls to UserGroupInformation.setConfiguration() to work when passing in
+        // config with Kerberos authentication enabled
+        System.setProperty("java.security.krb5.realm", "nifi.com");
+        System.setProperty("java.security.krb5.kdc", "nifi.kdc");
+
+        ugi = null;
+        processor = new MockPutHive3Streaming();
+        hiveConfigurator = mock(HiveConfigurator.class);
+        hiveConf = new HiveConf();
+        when(hiveConfigurator.getConfigurationFromFiles(anyString())).thenReturn(hiveConf);
+        processor.hiveConfigurator = hiveConfigurator;
+
+        // Delete any temp files from previous tests
+        try {
+            FileUtils.deleteDirectory(new File(TARGET_HIVE));
+        } catch (IOException ioe) {
+            // Do nothing, directory may not have existed
+        }
+    }
+
+    private void configure(final PutHive3Streaming processor, final int numUsers) throws InitializationException {
+        configure(processor, numUsers, -1);
+    }
+
+    private void configure(final PutHive3Streaming processor, final int numUsers, int failAfter) throws InitializationException {
+        configure(processor, numUsers, failAfter, null);
+    }
+
+    private void configure(final PutHive3Streaming processor, final int numUsers, final int failAfter,
+                           final BiFunction<Integer, MockRecordParser, Void> recordGenerator) throws InitializationException {
+        runner = TestRunners.newTestRunner(processor);
+        runner.setProperty(PutHive3Streaming.HIVE_CONFIGURATION_RESOURCES, TEST_CONF_PATH);
+        MockRecordParser readerFactory = new MockRecordParser();
+        final RecordSchema recordSchema = AvroTypeUtil.createSchema(schema);
+        for (final RecordField recordField : recordSchema.getFields()) {
+            readerFactory.addSchemaField(recordField.getFieldName(), recordField.getDataType().getFieldType(), recordField.isNullable());
+        }
+
+        if (recordGenerator == null) {
+            for (int i = 0; i < numUsers; i++) {
+                readerFactory.addRecord("name" + i, i, "blue" + i, i * 10.0);
+            }
+        } else {
+            recordGenerator.apply(numUsers, readerFactory);
+        }
+
+        readerFactory.failAfter(failAfter);
+
+        runner.addControllerService("mock-reader-factory", readerFactory);
+        runner.enableControllerService(readerFactory);
+
+        runner.setProperty(PutHive3Streaming.RECORD_READER, "mock-reader-factory");
+    }
+
+    private void configureComplex(final MockPutHive3Streaming processor, final int numUsers, final int failAfter,
+                                  final BiFunction<Integer, MockRecordParser, Void> recordGenerator) throws IOException, InitializationException {
+        final String avroSchema = IOUtils.toString(new FileInputStream("src/test/resources/array_of_records.avsc"), StandardCharsets.UTF_8);
+        schema = new Schema.Parser().parse(avroSchema);
+        processor.setFields(Arrays.asList(new FieldSchema("records",
+                serdeConstants.LIST_TYPE_NAME + "<"
+                        + serdeConstants.MAP_TYPE_NAME + "<"
+                        + serdeConstants.STRING_TYPE_NAME + ","
+                        +  serdeConstants.STRING_TYPE_NAME + ">>", "")));
+        runner = TestRunners.newTestRunner(processor);
+        runner.setProperty(PutHive3Streaming.HIVE_CONFIGURATION_RESOURCES, TEST_CONF_PATH);
+        MockRecordParser readerFactory = new MockRecordParser();
+        final RecordSchema recordSchema = AvroTypeUtil.createSchema(schema);
+        for (final RecordField recordField : recordSchema.getFields()) {
+            readerFactory.addSchemaField(recordField.getFieldName(), recordField.getDataType().getFieldType(), recordField.isNullable());
+        }
+
+        if (recordGenerator == null) {
+            Object[] mapArray = new Object[numUsers];
+            for (int i = 0; i < numUsers; i++) {
+                final int x = i;
+                Map<String, Object> map = new HashMap<String, Object>() {{
+                    put("name", "name" + x);
+                    put("age", x * 5);
+                }};
+                mapArray[i] = map;
+            }
+            readerFactory.addRecord((Object)mapArray);
+        } else {
+            recordGenerator.apply(numUsers, readerFactory);
+        }
+
+        readerFactory.failAfter(failAfter);
+
+        runner.addControllerService("mock-reader-factory", readerFactory);
+        runner.enableControllerService(readerFactory);
+
+        runner.setProperty(PutHive3Streaming.RECORD_READER, "mock-reader-factory");
+    }
+
+    @Test
+    public void testSetup() throws Exception {
+        configure(processor, 0);
+        runner.assertNotValid();
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.assertNotValid();
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.assertValid();
+        runner.run();
+    }
+
+    @Test
+    public void testUgiGetsCleared() throws Exception {
+        configure(processor, 0);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        processor.ugi = mock(UserGroupInformation.class);
+        runner.run();
+        assertNull(processor.ugi);
+    }
+
+    @Test
+    public void testUgiGetsSetIfSecure() throws Exception {
+        configure(processor, 1);
+        hiveConf.set(SecurityUtil.HADOOP_SECURITY_AUTHENTICATION, SecurityUtil.KERBEROS);
+        KerberosCredentialsService kcs = new MockKerberosCredentialsService();
+        runner.addControllerService("kcs", kcs);
+        runner.setProperty(KERBEROS_CREDENTIALS_SERVICE, "kcs");
+        runner.enableControllerService(kcs);
+        ugi = mock(UserGroupInformation.class);
+        when(hiveConfigurator.authenticate(eq(hiveConf), anyString(), anyString())).thenReturn(ugi);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.enqueue(new byte[0]);
+        runner.run();
+    }
+
+    @Test(expected = AssertionError.class)
+    public void testSetupWithKerberosAuthFailed() throws Exception {
+        configure(processor, 0);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.setProperty(PutHive3Streaming.HIVE_CONFIGURATION_RESOURCES, "src/test/resources/core-site-security.xml, src/test/resources/hive-site-security.xml");
+
+        hiveConf.set(SecurityUtil.HADOOP_SECURITY_AUTHENTICATION, SecurityUtil.KERBEROS);
+        KerberosCredentialsService kcs = new MockKerberosCredentialsService(null, null);
+        runner.addControllerService("kcs", kcs);
+        runner.setProperty(KERBEROS_CREDENTIALS_SERVICE, "kcs");
+        runner.enableControllerService(kcs);
+        runner.assertNotValid();
+        runner.run();
+    }
+
+    @Test
+    public void onTrigger() throws Exception {
+        configure(processor, 1);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.enqueue(new byte[0]);
+        runner.run();
+
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 1);
+        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutHive3Streaming.REL_SUCCESS).get(0);
+        assertEquals("1", flowFile.getAttribute(HIVE_STREAMING_RECORD_COUNT_ATTR));
+        assertEquals("default.users", flowFile.getAttribute(ATTR_OUTPUT_TABLES));
+    }
+
+    @Test
+    public void onTriggerComplex() throws Exception {
+        configureComplex(processor, 10, -1, null);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.enqueue(new byte[0]);
+        runner.run();
+
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 1);
+        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutHive3Streaming.REL_SUCCESS).get(0);
+        // Schema is an array of size 10, so only one record is output
+        assertEquals("1", flowFile.getAttribute(HIVE_STREAMING_RECORD_COUNT_ATTR));
+        assertEquals("default.users", flowFile.getAttribute(ATTR_OUTPUT_TABLES));
+    }
+
+    @Test
+    public void onTriggerBadInput() throws Exception {
+        configure(processor, 1, 0);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.enqueue("I am not an Avro record".getBytes());
+        runner.run();
+
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
+    }
+
+    @Test
+    public void onTriggerBadInputRollbackOnFailure() throws Exception {
+        configure(processor, 1, 0);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+
+        runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
+        runner.enqueue("I am not an Avro record".getBytes());
+        try {
+            runner.run();
+            fail("ProcessException should be thrown");
+        } catch (AssertionError e) {
+            assertTrue(e.getCause() instanceof ProcessException);
+        }
+
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
+        // Assert incoming FlowFile stays in input queue.
+        assertEquals(1, runner.getQueueSize().getObjectCount());
+    }
+
+
+    @Test
+    public void onTriggerMultipleRecordsSingleTransaction() throws Exception {
+        configure(processor, 3);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        Map<String, Object> user1 = new HashMap<String, Object>() {
+            {
+                put("name", "Joe");
+                put("favorite_number", 146);
+            }
+        };
+        Map<String, Object> user2 = new HashMap<String, Object>() {
+            {
+                put("name", "Mary");
+                put("favorite_number", 42);
+            }
+        };
+        Map<String, Object> user3 = new HashMap<String, Object>() {
+            {
+                put("name", "Matt");
+                put("favorite_number", 3);
+            }
+        };
+        final List<Map<String, Object>> users = Arrays.asList(user1, user2, user3);
+        runner.enqueue(createAvroRecord(users));
+        runner.run();
+
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 1);
+        MockFlowFile resultFlowFile = runner.getFlowFilesForRelationship(PutHive3Streaming.REL_SUCCESS).get(0);
+        assertOutputAvroRecords(users, resultFlowFile);
+    }
+
+    @Test
+    public void onTriggerMultipleRecordsFailInMiddle() throws Exception {
+        configure(processor, 4);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        processor.setGenerateWriteFailure(true);
+        runner.enqueue(new byte[0]);
+        runner.run();
+
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
+        runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
+    }
+
+    @Test
+    public void onTriggerMultipleRecordsFailInMiddleRollbackOnFailure() throws Exception {
+        configure(processor, 3);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
+        processor.setGenerateWriteFailure(true);
+        runner.enqueue(new byte[0]);
+        try {
+            runner.run();
+            fail("ProcessException should be thrown, because any Hive Transaction is committed yet.");
+        } catch (AssertionError e) {
+            assertTrue(e.getCause() instanceof ProcessException);
+        }
+
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
+        // Assert incoming FlowFile stays in input queue.
+        assertEquals(1, runner.getQueueSize().getObjectCount());
+    }
+
+    private void assertOutputAvroRecords(List<Map<String, Object>> expectedRecords, MockFlowFile resultFlowFile) throws IOException {
+        assertEquals(String.valueOf(expectedRecords.size()), resultFlowFile.getAttribute(PutHive3Streaming.HIVE_STREAMING_RECORD_COUNT_ATTR));
+
+        final DataFileStream<GenericRecord> reader = new DataFileStream<>(
+                new ByteArrayInputStream(resultFlowFile.toByteArray()),
+                new GenericDatumReader<>());
+
+        Schema schema = reader.getSchema();
+
+        // Verify that the schema is preserved
+        assertEquals(schema, new Schema.Parser().parse(new File("src/test/resources/user.avsc")));
+
+        GenericRecord record = null;
+        for (Map<String, Object> expectedRecord : expectedRecords) {
+            assertTrue(reader.hasNext());
+            record = reader.next(record);
+            final String name = record.get("name").toString();
+            final Integer favorite_number = (Integer) record.get("favorite_number");
+            assertNotNull(name);
+            assertNotNull(favorite_number);
+            assertNull(record.get("favorite_color"));
+            assertNull(record.get("scale"));
+
+            assertEquals(expectedRecord.get("name"), name);
+            assertEquals(expectedRecord.get("favorite_number"), favorite_number);
+        }
+        assertFalse(reader.hasNext());
+    }
+
+    @Test
+    public void onTriggerWithConnectFailure() throws Exception {
+        configure(processor, 1);
+        processor.setGenerateConnectFailure(true);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.enqueue(new byte[0]);
+        try {
+            runner.run();
+            fail("ProcessException should be thrown");
+        } catch (AssertionError e) {
+            assertTrue(e.getCause() instanceof ProcessException);
+        }
+        runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
+        // Assert incoming FlowFile stays in input queue.
+        assertEquals(1, runner.getQueueSize().getObjectCount());
+    }
+
+    @Test
+    public void onTriggerWithConnectFailureRollbackOnFailure() throws Exception {
+        configure(processor, 1);
+        processor.setGenerateConnectFailure(true);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
+        runner.enqueue(new byte[0]);
+        try {
+            runner.run();
+            fail("ProcessException should be thrown");
+        } catch (AssertionError e) {
+            assertTrue(e.getCause() instanceof ProcessException);
+        }
+
+        runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
+        // Assert incoming FlowFile stays in input queue.
+        assertEquals(1, runner.getQueueSize().getObjectCount());
+    }
+
+    @Test
+    public void onTriggerWithWriteFailure() throws Exception {
+        configure(processor, 2);
+        processor.setGenerateWriteFailure(true);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.enqueue(new byte[0]);
+        runner.run();
+
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
+        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutHive3Streaming.REL_FAILURE).get(0);
+        assertEquals("0", flowFile.getAttribute(HIVE_STREAMING_RECORD_COUNT_ATTR));
+        assertEquals("default.users", flowFile.getAttribute(ATTR_OUTPUT_TABLES));
+    }
+
+    @Test
+    public void onTriggerWithWriteFailureRollbackOnFailure() throws Exception {
+        configure(processor, 2);
+        processor.setGenerateWriteFailure(true);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
+        Map<String, Object> user1 = new HashMap<String, Object>() {
+            {
+                put("name", "Joe");
+                put("favorite_number", 146);
+            }
+        };
+        Map<String, Object> user2 = new HashMap<String, Object>() {
+            {
+                put("name", "Mary");
+                put("favorite_number", 42);
+            }
+        };
+        runner.enqueue(createAvroRecord(Arrays.asList(user1, user2)));
+        try {
+            runner.run();
+            fail("ProcessException should be thrown");
+        } catch (AssertionError e) {
+            assertTrue(e.getCause() instanceof ProcessException);
+        }
+
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
+        // Assert incoming FlowFile stays in input queue.
+        assertEquals(1, runner.getQueueSize().getObjectCount());
+    }
+
+    @Test
+    public void onTriggerWithSerializationError() throws Exception {
+        configure(processor, 1);
+        processor.setGenerateSerializationError(true);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        Map<String, Object> user1 = new HashMap<String, Object>() {
+            {
+                put("name", "Joe");
+                put("favorite_number", 146);
+            }
+        };
+        runner.enqueue(createAvroRecord(Collections.singletonList(user1)));
+        runner.run();
+
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
+    }
+
+    @Test
+    public void onTriggerWithSerializationErrorRollbackOnFailure() throws Exception {
+        configure(processor, 1);
+        processor.setGenerateSerializationError(true);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
+        Map<String, Object> user1 = new HashMap<String, Object>() {
+            {
+                put("name", "Joe");
+                put("favorite_number", 146);
+            }
+        };
+        runner.enqueue(createAvroRecord(Collections.singletonList(user1)));
+        try {
+            runner.run();
+            fail("ProcessException should be thrown");
+        } catch (AssertionError e) {
+            assertTrue(e.getCause() instanceof ProcessException);
+        }
+
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
+        // Assert incoming FlowFile stays in input queue.
+        assertEquals(1, runner.getQueueSize().getObjectCount());
+    }
+
+    @Test
+    public void onTriggerWithCommitFailure() throws Exception {
+        configure(processor, 1);
+        processor.setGenerateCommitFailure(true);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "false");
+        runner.enqueue(new byte[0]);
+        runner.run();
+
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 1);
+    }
+
+    @Test
+    public void onTriggerWithCommitFailureRollbackOnFailure() throws Exception {
+        configure(processor, 1);
+        processor.setGenerateCommitFailure(true);
+        runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
+        runner.setProperty(PutHive3Streaming.DB_NAME, "default");
+        runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
+        runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
+        runner.enqueue(new byte[0]);
+        try {
+            runner.run();
+            fail("ProcessException should be thrown");
+        } catch (AssertionError e) {
+            assertTrue(e.getCause() instanceof ProcessException);
+        }
+
+        runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
+        runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
+        // Assert incoming FlowFile stays in input queue.
+        assertEquals(1, runner.getQueueSize().getObjectCount());
+    }
+
+    @Test
+    public void cleanup() {
+        processor.cleanup();
+    }
+
+    private byte[] createAvroRecord(List<Map<String, Object>> records) throws IOException {
+        final Schema schema = new Schema.Parser().parse(new File("src/test/resources/user.avsc"));
+
+        List<GenericRecord> users = new LinkedList<>();
+        for (Map<String, Object> record : records) {
+            final GenericRecord user = new GenericData.Record(schema);
+            user.put("name", record.get("name"));
+            user.put("favorite_number", record.get("favorite_number"));
+            user.put("favorite_color", record.get("favorite_color"));
+            users.add(user);
+        }
+        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+        ByteArrayOutputStream out = new ByteArrayOutputStream();
+        try (DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
+            dataFileWriter.create(schema, out);
+            for (final GenericRecord user : users) {
+                dataFileWriter.append(user);
+            }
+        }
+        return out.toByteArray();
+
+    }
+
+    private class MockPutHive3Streaming extends PutHive3Streaming {
+
+        private boolean generateConnectFailure = false;
+        private boolean generateWriteFailure = false;
+        private boolean generateSerializationError = false;
+        private boolean generateCommitFailure = false;
+        private List<FieldSchema> schema = Arrays.asList(
+                new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""),
+                new FieldSchema("favorite_number", serdeConstants.INT_TYPE_NAME, ""),
+                new FieldSchema("favorite_color", serdeConstants.STRING_TYPE_NAME, ""),
+                new FieldSchema("scale", serdeConstants.DOUBLE_TYPE_NAME, "")
+        );
+
+        @Override
+        StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader) throws StreamingException {
+
+            if (generateConnectFailure) {
+                throw new StubConnectionError("Unit Test - Connection Error");
+            }
+
+            HiveRecordWriter hiveRecordWriter = new HiveRecordWriter(reader, getLogger());
+            MockHiveStreamingConnection hiveConnection = new MockHiveStreamingConnection(options, reader, hiveRecordWriter, schema);
+            hiveConnection.setGenerateWriteFailure(generateWriteFailure);
+            hiveConnection.setGenerateSerializationError(generateSerializationError);
+            hiveConnection.setGenerateCommitFailure(generateCommitFailure);
+            return hiveConnection;
+        }
+
+        void setGenerateConnectFailure(boolean generateConnectFailure) {
+            this.generateConnectFailure = generateConnectFailure;
+        }
+
+        void setGenerateWriteFailure(boolean generateWriteFailure) {
+            this.generateWriteFailure = generateWriteFailure;
+        }
+
+        void setGenerateSerializationError(boolean generateSerializationError) {
+            this.generateSerializationError = generateSerializationError;
+        }
+
+        void setGenerateCommitFailure(boolean generateCommitFailure) {
+            this.generateCommitFailure = generateCommitFailure;
+        }
+
+        void setFields(List<FieldSchema> schema) {
+            this.schema = schema;
+        }
+    }
+
+    private class MockHiveStreamingConnection implements StreamingConnection {
+
+        private boolean generateWriteFailure = false;
+        private boolean generateSerializationError = false;
+        private boolean generateCommitFailure = false;
+        private int writeAttemptCount = 0;
+        private ConnectionStats connectionStats;
+        private HiveOptions options;
+        private RecordWriter writer;
+        private HiveConf hiveConf;
+        private Table table;
+        private String metastoreURI;
+
+        MockHiveStreamingConnection(HiveOptions options, RecordReader reader, RecordWriter recordWriter, List<FieldSchema> schema) {
+            this.options = options;
+            metastoreURI = options.getMetaStoreURI();
+            this.writer = recordWriter;
+            this.hiveConf = this.options.getHiveConf();
+            connectionStats = new ConnectionStats();
+            this.table = new Table(Table.getEmptyTable(options.getDatabaseName(), options.getTableName()));
+            this.table.setFields(schema);
+            StorageDescriptor sd = this.table.getSd();
+            sd.setOutputFormat(OrcOutputFormat.class.getName());
+            sd.setLocation(TARGET_HIVE);
+        }
+
+        @Override
+        public HiveConf getHiveConf() {
+            return hiveConf;
+        }
+
+        @Override
+        public void beginTransaction() throws StreamingException {
+            writer.init(this, 0, 100);
+        }
+
+        @Override
+        public synchronized void write(byte[] record) throws StreamingException {
+            throw new UnsupportedOperationException(this.getClass().getName() + " does not support writing of records via bytes, only via an InputStream");
+        }
+
+        @Override
+        public void write(InputStream inputStream) throws StreamingException {
+            try {
+                if (generateWriteFailure) {
+                    throw new StubStreamingIOFailure("Unit Test - Streaming IO Failure");
+                }
+                if (generateSerializationError) {
+                    throw new StubSerializationError("Unit Test - Serialization error", new Exception());
+                }
+                this.writer.write(writeAttemptCount, inputStream);
+            } finally {
+                writeAttemptCount++;
+            }
+        }
+
+        @Override
+        public void commitTransaction() throws StreamingException {
+            if (generateCommitFailure) {
+                throw new StubTransactionError("Unit Test - Commit Failure");
+            }
+            connectionStats.incrementCommittedTransactions();
+        }
+
+        @Override
+        public void abortTransaction() throws StreamingException {
+            connectionStats.incrementAbortedTransactions();
+        }
+
+        @Override
+        public void close() {
+            // closing the connection shouldn't throw an exception
+        }
+
+        @Override
+        public ConnectionStats getConnectionStats() {
+            return connectionStats;
+        }
+
+        public void setGenerateWriteFailure(boolean generateWriteFailure) {
+            this.generateWriteFailure = generateWriteFailure;
+        }
+
+        public void setGenerateSerializationError(boolean generateSerializationError) {
+            this.generateSerializationError = generateSerializationError;
+        }
+
+        public void setGenerateCommitFailure(boolean generateCommitFailure) {
+            this.generateCommitFailure = generateCommitFailure;
+        }
+
+        @Override
+        public String getMetastoreUri() {
+            return metastoreURI;
+        }
+
+        @Override
+        public Table getTable() {
+            return table;
+        }
+
+        @Override
+        public List<String> getStaticPartitionValues() {
+            return null;
+        }
+
+        @Override
+        public boolean isPartitionedTable() {
+            return false;
+        }
+
+        @Override
+        public boolean isDynamicPartitioning() {
+            return false;
+        }
+
+        @Override
+        public String getAgentInfo() {
+            return null;
+        }
+
+        @Override
+        public PartitionInfo createPartitionIfNotExists(List<String> list) throws StreamingException {
+            return null;
+        }
+    }
+
+    private static class MockKerberosCredentialsService implements KerberosCredentialsService, ControllerService {
+
+        private String keytab = "src/test/resources/fake.keytab";
+        private String principal = "test@REALM.COM";
+
+        public MockKerberosCredentialsService() {
+        }
+
+        public MockKerberosCredentialsService(String keytab, String principal) {
+            this.keytab = keytab;
+            this.principal = principal;
+        }
+
+        @Override
+        public String getKeytab() {
+            return keytab;
+        }
+
+        @Override
+        public String getPrincipal() {
+            return principal;
+        }
+
+        @Override
+        public void initialize(ControllerServiceInitializationContext context) throws InitializationException {
+
+        }
+
+        @Override
+        public Collection<ValidationResult> validate(ValidationContext context) {
+            return Collections.EMPTY_LIST;
+        }
+
+        @Override
+        public PropertyDescriptor getPropertyDescriptor(String name) {
+            return null;
+        }
+
+        @Override
+        public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+
+        }
+
+        @Override
+        public List<PropertyDescriptor> getPropertyDescriptors() {
+            return null;
+        }
+
+        @Override
+        public String getIdentifier() {
+            return "kcs";
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestSelectHive3QL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestSelectHive3QL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestSelectHive3QL.java
new file mode 100644
index 0000000..50e83ac
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestSelectHive3QL.java
@@ -0,0 +1,539 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.hive;
+
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.dbcp.hive.Hive3DBCPService;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.exception.ProcessException;
+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.apache.nifi.util.hive.HiveJdbcCommon;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Types;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import static org.apache.nifi.processors.hive.SelectHive3QL.HIVEQL_OUTPUT_FORMAT;
+import static org.apache.nifi.util.hive.HiveJdbcCommon.AVRO;
+import static org.apache.nifi.util.hive.HiveJdbcCommon.CSV;
+import static org.apache.nifi.util.hive.HiveJdbcCommon.CSV_MIME_TYPE;
+import static org.apache.nifi.util.hive.HiveJdbcCommon.MIME_TYPE_AVRO_BINARY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestSelectHive3QL {
+
+    private static final Logger LOGGER;
+    private final static String MAX_ROWS_KEY = "maxRows";
+
+    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.hive.SelectHive3QL", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.hive.TestSelectHive3QL", "debug");
+        LOGGER = LoggerFactory.getLogger(TestSelectHive3QL.class);
+    }
+
+    private final static String DB_LOCATION = "target/db";
+
+    private final static String QUERY_WITH_EL = "select "
+            + "  PER.ID as PersonId, PER.NAME as PersonName, PER.CODE as PersonCode"
+            + " from persons PER"
+            + " where PER.ID > ${person.id}";
+
+    private final static String QUERY_WITHOUT_EL = "select "
+            + "  PER.ID as PersonId, PER.NAME as PersonName, PER.CODE as PersonCode"
+            + " from persons PER"
+            + " where PER.ID > 10";
+
+
+    @BeforeClass
+    public static void setupClass() {
+        System.setProperty("derby.stream.error.file", "target/derby.log");
+    }
+
+    private TestRunner runner;
+
+    @Before
+    public void setup() throws InitializationException {
+        final DBCPService dbcp = new DBCPServiceSimpleImpl();
+        final Map<String, String> dbcpProperties = new HashMap<>();
+
+        runner = TestRunners.newTestRunner(SelectHive3QL.class);
+        runner.addControllerService("dbcp", dbcp, dbcpProperties);
+        runner.enableControllerService(dbcp);
+        runner.setProperty(SelectHive3QL.HIVE_DBCP_SERVICE, "dbcp");
+    }
+
+    @Test
+    public void testIncomingConnectionWithNoFlowFile() throws InitializationException {
+        runner.setIncomingConnection(true);
+        runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT * FROM persons");
+        runner.run();
+        runner.assertTransferCount(SelectHive3QL.REL_SUCCESS, 0);
+        runner.assertTransferCount(SelectHive3QL.REL_FAILURE, 0);
+    }
+
+    @Test
+    public void testNoIncomingConnection() throws ClassNotFoundException, SQLException, InitializationException, IOException {
+        runner.setIncomingConnection(false);
+        invokeOnTrigger(QUERY_WITHOUT_EL, false, "Avro");
+    }
+
+    @Test
+    public void testNoTimeLimit() throws InitializationException, ClassNotFoundException, SQLException, IOException {
+        invokeOnTrigger(QUERY_WITH_EL, true, "Avro");
+    }
+
+
+    @Test
+    public void testWithNullIntColumn() throws SQLException {
+        // remove previous test database, if any
+        final File dbLocation = new File(DB_LOCATION);
+        dbLocation.delete();
+
+        // load test data to database
+        final Connection con = ((Hive3DBCPService) runner.getControllerService("dbcp")).getConnection();
+        Statement stmt = con.createStatement();
+
+        try {
+            stmt.execute("drop table TEST_NULL_INT");
+        } catch (final SQLException sqle) {
+            // Nothing to do, probably means the table didn't exist
+        }
+
+        stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
+
+        stmt.execute("insert into TEST_NULL_INT (id, val1, val2) VALUES (0, NULL, 1)");
+        stmt.execute("insert into TEST_NULL_INT (id, val1, val2) VALUES (1, 1, 1)");
+
+        runner.setIncomingConnection(false);
+        runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT * FROM TEST_NULL_INT");
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 1);
+        runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(0).assertAttributeEquals(SelectHive3QL.RESULT_ROW_COUNT, "2");
+    }
+
+    @Test
+    public void testWithSqlException() throws SQLException {
+        // remove previous test database, if any
+        final File dbLocation = new File(DB_LOCATION);
+        dbLocation.delete();
+
+        // load test data to database
+        final Connection con = ((Hive3DBCPService) runner.getControllerService("dbcp")).getConnection();
+        Statement stmt = con.createStatement();
+
+        try {
+            stmt.execute("drop table TEST_NO_ROWS");
+        } catch (final SQLException sqle) {
+            // Nothing to do, probably means the table didn't exist
+        }
+
+        stmt.execute("create table TEST_NO_ROWS (id integer)");
+
+        runner.setIncomingConnection(false);
+        // Try a valid SQL statement that will generate an error (val1 does not exist, e.g.)
+        runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT val1 FROM TEST_NO_ROWS");
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_FAILURE, 1);
+    }
+
+    @Test
+    public void testWithBadSQL() throws SQLException {
+        final String BAD_SQL = "create table TEST_NO_ROWS (id integer)";
+
+        // Test with incoming flow file (it should be routed to failure intact, i.e. same content and no parent)
+        runner.setIncomingConnection(true);
+        // Try a valid SQL statement that will generate an error (val1 does not exist, e.g.)
+        runner.enqueue(BAD_SQL);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_FAILURE, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(SelectHive3QL.REL_FAILURE).get(0);
+        flowFile.assertContentEquals(BAD_SQL);
+        flowFile.assertAttributeEquals("parentIds", null);
+        runner.clearTransferState();
+
+        // Test with no incoming flow file (an empty flow file is transferred)
+        runner.setIncomingConnection(false);
+        // Try a valid SQL statement that will generate an error (val1 does not exist, e.g.)
+        runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, BAD_SQL);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_FAILURE, 1);
+        flowFile = runner.getFlowFilesForRelationship(SelectHive3QL.REL_FAILURE).get(0);
+        flowFile.assertContentEquals("");
+    }
+
+    @Test
+    public void invokeOnTriggerWithCsv()
+            throws InitializationException, ClassNotFoundException, SQLException, IOException {
+        invokeOnTrigger(QUERY_WITHOUT_EL, false, CSV);
+    }
+
+    @Test
+    public void invokeOnTriggerWithAvro()
+            throws InitializationException, ClassNotFoundException, SQLException, IOException {
+        invokeOnTrigger(QUERY_WITHOUT_EL, false, AVRO);
+    }
+
+    public void invokeOnTrigger(final String query, final boolean incomingFlowFile, String outputFormat)
+            throws InitializationException, ClassNotFoundException, SQLException, IOException {
+
+        // remove previous test database, if any
+        final File dbLocation = new File(DB_LOCATION);
+        dbLocation.delete();
+
+        // load test data to database
+        final Connection con = ((Hive3DBCPService) runner.getControllerService("dbcp")).getConnection();
+        final Statement stmt = con.createStatement();
+        try {
+            stmt.execute("drop table persons");
+        } catch (final SQLException sqle) {
+            // Nothing to do here, the table didn't exist
+        }
+
+        stmt.execute("create table persons (id integer, name varchar(100), code integer)");
+        Random rng = new Random(53496);
+        final int nrOfRows = 100;
+        stmt.executeUpdate("insert into persons values (1, 'Joe Smith', " + rng.nextInt(469947) + ")");
+        for (int i = 2; i < nrOfRows; i++) {
+            stmt.executeUpdate("insert into persons values (" + i + ", 'Someone Else', " + rng.nextInt(469947) + ")");
+        }
+        stmt.executeUpdate("insert into persons values (" + nrOfRows + ", 'Last Person', NULL)");
+
+        LOGGER.info("test data loaded");
+
+        runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, query);
+        runner.setProperty(HIVEQL_OUTPUT_FORMAT, outputFormat);
+
+        if (incomingFlowFile) {
+            // incoming FlowFile content is not used, but attributes are used
+            final Map<String, String> attributes = new HashMap<>();
+            attributes.put("person.id", "10");
+            runner.enqueue("Hello".getBytes(), attributes);
+        }
+
+        runner.setIncomingConnection(incomingFlowFile);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 1);
+
+        final List<MockFlowFile> flowfiles = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS);
+        MockFlowFile flowFile = flowfiles.get(0);
+        final InputStream in = new ByteArrayInputStream(flowFile.toByteArray());
+        long recordsFromStream = 0;
+        if (AVRO.equals(outputFormat)) {
+            assertEquals(MIME_TYPE_AVRO_BINARY, flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
+            final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
+            try (DataFileStream<GenericRecord> dataFileReader = new DataFileStream<>(in, datumReader)) {
+                GenericRecord record = null;
+                while (dataFileReader.hasNext()) {
+                    // Reuse record object by passing it to next(). This saves us from
+                    // allocating and garbage collecting many objects for files with
+                    // many items.
+                    record = dataFileReader.next(record);
+                    recordsFromStream++;
+                }
+            }
+        } else {
+            assertEquals(CSV_MIME_TYPE, flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
+            BufferedReader br = new BufferedReader(new InputStreamReader(in));
+
+            String headerRow = br.readLine();
+            // Derby capitalizes column names
+            assertEquals("PERSONID,PERSONNAME,PERSONCODE", headerRow);
+
+            // Validate rows
+            String line;
+            while ((line = br.readLine()) != null) {
+                recordsFromStream++;
+                String[] values = line.split(",");
+                if (recordsFromStream < (nrOfRows - 10)) {
+                    assertEquals(3, values.length);
+                    assertTrue(values[1].startsWith("\""));
+                    assertTrue(values[1].endsWith("\""));
+                } else {
+                    assertEquals(2, values.length); // Middle value is null
+                }
+            }
+        }
+        assertEquals(nrOfRows - 10, recordsFromStream);
+        assertEquals(recordsFromStream, Integer.parseInt(flowFile.getAttribute(SelectHive3QL.RESULT_ROW_COUNT)));
+        flowFile.assertAttributeEquals(AbstractHive3QLProcessor.ATTR_INPUT_TABLES, "persons");
+    }
+
+    @Test
+    public void testMaxRowsPerFlowFileAvro() throws ClassNotFoundException, SQLException, InitializationException, IOException {
+
+        // load test data to database
+        final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
+        Statement stmt = con.createStatement();
+        InputStream in;
+        MockFlowFile mff;
+
+        try {
+            stmt.execute("drop table TEST_QUERY_DB_TABLE");
+        } catch (final SQLException sqle) {
+            // Ignore this error, probably a "table does not exist" since Derby doesn't yet support DROP IF EXISTS [DERBY-4842]
+        }
+
+        stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
+        int rowCount = 0;
+        //create larger row set
+        for (int batch = 0; batch < 100; batch++) {
+            stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '1962-09-23 03:23:34.234')");
+            rowCount++;
+        }
+
+        runner.setIncomingConnection(false);
+        runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT * FROM TEST_QUERY_DB_TABLE");
+        runner.setProperty(SelectHive3QL.MAX_ROWS_PER_FLOW_FILE, "${" + MAX_ROWS_KEY + "}");
+        runner.setProperty(SelectHive3QL.HIVEQL_OUTPUT_FORMAT, HiveJdbcCommon.AVRO);
+        runner.setVariable(MAX_ROWS_KEY, "9");
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 12);
+
+        //ensure all but the last file have 9 records each
+        for (int ff = 0; ff < 11; ff++) {
+            mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(ff);
+            in = new ByteArrayInputStream(mff.toByteArray());
+            assertEquals(9, getNumberOfRecordsFromStream(in));
+
+            mff.assertAttributeExists("fragment.identifier");
+            assertEquals(Integer.toString(ff), mff.getAttribute("fragment.index"));
+            assertEquals("12", mff.getAttribute("fragment.count"));
+        }
+
+        //last file should have 1 record
+        mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(11);
+        in = new ByteArrayInputStream(mff.toByteArray());
+        assertEquals(1, getNumberOfRecordsFromStream(in));
+        mff.assertAttributeExists("fragment.identifier");
+        assertEquals(Integer.toString(11), mff.getAttribute("fragment.index"));
+        assertEquals("12", mff.getAttribute("fragment.count"));
+        runner.clearTransferState();
+    }
+
+    @Test
+    public void testParametrizedQuery() throws ClassNotFoundException, SQLException, InitializationException, IOException {
+        // load test data to database
+        final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
+        Statement stmt = con.createStatement();
+
+        try {
+            stmt.execute("drop table TEST_QUERY_DB_TABLE");
+        } catch (final SQLException sqle) {
+            // Ignore this error, probably a "table does not exist" since Derby doesn't yet support DROP IF EXISTS [DERBY-4842]
+        }
+
+        stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
+        int rowCount = 0;
+        //create larger row set
+        for (int batch = 0; batch < 100; batch++) {
+            stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '1962-09-23 03:23:34.234')");
+            rowCount++;
+        }
+
+        runner.setIncomingConnection(true);
+        runner.setProperty(SelectHive3QL.MAX_ROWS_PER_FLOW_FILE, "${" + MAX_ROWS_KEY + "}");
+        runner.setProperty(SelectHive3QL.HIVEQL_OUTPUT_FORMAT, HiveJdbcCommon.AVRO);
+        runner.setVariable(MAX_ROWS_KEY, "9");
+
+        Map<String, String> attributes = new HashMap<String, String>();
+        attributes.put("hiveql.args.1.value", "1");
+        attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
+        runner.enqueue("SELECT * FROM TEST_QUERY_DB_TABLE WHERE id = ?", attributes );
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 1);
+        runner.clearTransferState();
+    }
+
+    @Test
+    public void testMaxRowsPerFlowFileCSV() throws ClassNotFoundException, SQLException, InitializationException, IOException {
+
+        // load test data to database
+        final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
+        Statement stmt = con.createStatement();
+        InputStream in;
+        MockFlowFile mff;
+
+        try {
+            stmt.execute("drop table TEST_QUERY_DB_TABLE");
+        } catch (final SQLException sqle) {
+            // Ignore this error, probably a "table does not exist" since Derby doesn't yet support DROP IF EXISTS [DERBY-4842]
+        }
+
+        stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
+        int rowCount = 0;
+        //create larger row set
+        for (int batch = 0; batch < 100; batch++) {
+            stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '1962-09-23 03:23:34.234')");
+            rowCount++;
+        }
+
+        runner.setIncomingConnection(true);
+        runner.setProperty(SelectHive3QL.MAX_ROWS_PER_FLOW_FILE, "${" + MAX_ROWS_KEY + "}");
+        runner.setProperty(SelectHive3QL.HIVEQL_OUTPUT_FORMAT, HiveJdbcCommon.CSV);
+
+        runner.enqueue("SELECT * FROM TEST_QUERY_DB_TABLE", new HashMap<String, String>() {{
+            put(MAX_ROWS_KEY, "9");
+        }});
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 12);
+
+        //ensure all but the last file have 9 records (10 lines = 9 records + header) each
+        for (int ff = 0; ff < 11; ff++) {
+            mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(ff);
+            in = new ByteArrayInputStream(mff.toByteArray());
+            BufferedReader br = new BufferedReader(new InputStreamReader(in));
+            assertEquals(10, br.lines().count());
+
+            mff.assertAttributeExists("fragment.identifier");
+            assertEquals(Integer.toString(ff), mff.getAttribute("fragment.index"));
+            assertEquals("12", mff.getAttribute("fragment.count"));
+        }
+
+        //last file should have 1 record (2 lines = 1 record + header)
+        mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(11);
+        in = new ByteArrayInputStream(mff.toByteArray());
+        BufferedReader br = new BufferedReader(new InputStreamReader(in));
+        assertEquals(2, br.lines().count());
+        mff.assertAttributeExists("fragment.identifier");
+        assertEquals(Integer.toString(11), mff.getAttribute("fragment.index"));
+        assertEquals("12", mff.getAttribute("fragment.count"));
+        runner.clearTransferState();
+    }
+
+    @Test
+    public void testMaxRowsPerFlowFileWithMaxFragments() throws ClassNotFoundException, SQLException, InitializationException, IOException {
+
+        // load test data to database
+        final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
+        Statement stmt = con.createStatement();
+        InputStream in;
+        MockFlowFile mff;
+
+        try {
+            stmt.execute("drop table TEST_QUERY_DB_TABLE");
+        } catch (final SQLException sqle) {
+            // Ignore this error, probably a "table does not exist" since Derby doesn't yet support DROP IF EXISTS [DERBY-4842]
+        }
+
+        stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
+        int rowCount = 0;
+        //create larger row set
+        for (int batch = 0; batch < 100; batch++) {
+            stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '1962-09-23 03:23:34.234')");
+            rowCount++;
+        }
+
+        runner.setIncomingConnection(false);
+        runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT * FROM TEST_QUERY_DB_TABLE");
+        runner.setProperty(SelectHive3QL.MAX_ROWS_PER_FLOW_FILE, "9");
+        Integer maxFragments = 3;
+        runner.setProperty(SelectHive3QL.MAX_FRAGMENTS, maxFragments.toString());
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, maxFragments);
+
+        for (int i = 0; i < maxFragments; i++) {
+            mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(i);
+            in = new ByteArrayInputStream(mff.toByteArray());
+            assertEquals(9, getNumberOfRecordsFromStream(in));
+
+            mff.assertAttributeExists("fragment.identifier");
+            assertEquals(Integer.toString(i), mff.getAttribute("fragment.index"));
+            assertEquals(maxFragments.toString(), mff.getAttribute("fragment.count"));
+        }
+
+        runner.clearTransferState();
+    }
+
+    private long getNumberOfRecordsFromStream(InputStream in) throws IOException {
+        final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
+        try (DataFileStream<GenericRecord> dataFileReader = new DataFileStream<>(in, datumReader)) {
+            GenericRecord record = null;
+            long recordsFromStream = 0;
+            while (dataFileReader.hasNext()) {
+                // Reuse record object by passing it to next(). This saves us from
+                // allocating and garbage collecting many objects for files with
+                // many items.
+                record = dataFileReader.next(record);
+                recordsFromStream += 1;
+            }
+
+            return recordsFromStream;
+        }
+    }
+
+    /**
+     * Simple implementation only for SelectHive3QL processor testing.
+     */
+    private class DBCPServiceSimpleImpl extends AbstractControllerService implements Hive3DBCPService {
+
+        @Override
+        public String getIdentifier() {
+            return "dbcp";
+        }
+
+        @Override
+        public Connection getConnection() throws ProcessException {
+            try {
+                Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
+                return DriverManager.getConnection("jdbc:derby:" + DB_LOCATION + ";create=true");
+            } catch (final Exception e) {
+                throw new ProcessException("getConnection failed: " + e);
+            }
+        }
+
+        @Override
+        public String getConnectionURL() {
+            return "jdbc:derby:" + DB_LOCATION + ";create=true";
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/da99f873/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java
new file mode 100644
index 0000000..e1af5a1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.orc;
+
+import org.apache.avro.Schema;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.orc.OrcFile;
+import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
+import org.apache.hadoop.hive.ql.io.orc.Reader;
+import org.apache.hadoop.hive.ql.io.orc.RecordReader;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.log4j.BasicConfigurator;
+import org.apache.nifi.avro.AvroTypeUtil;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.hadoop.exception.FailureException;
+import org.apache.nifi.processors.hadoop.record.HDFSRecordWriter;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.temporal.ChronoField;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiFunction;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.when;
+
+public class PutORCTest {
+
+    private static final String DIRECTORY = "target";
+    private static final String TEST_CONF_PATH = "src/test/resources/core-site.xml";
+
+    private Schema schema;
+    private Configuration testConf;
+    private PutORC proc;
+    private TestRunner testRunner;
+
+    @BeforeClass
+    public static void setupLogging() {
+        BasicConfigurator.configure();
+    }
+
+    @Before
+    public void setup() throws IOException {
+        final String avroSchema = IOUtils.toString(new FileInputStream("src/test/resources/user.avsc"), StandardCharsets.UTF_8);
+        schema = new Schema.Parser().parse(avroSchema);
+
+        testConf = new Configuration();
+        testConf.addResource(new Path(TEST_CONF_PATH));
+
+        proc = new PutORC();
+    }
+
+    private void configure(final PutORC putORC, final int numUsers) throws InitializationException {
+        configure(putORC, numUsers, null);
+    }
+
+    private void configure(final PutORC putORC, final int numUsers, final BiFunction<Integer, MockRecordParser, Void> recordGenerator) throws InitializationException {
+        testRunner = TestRunners.newTestRunner(putORC);
+        testRunner.setProperty(PutORC.HADOOP_CONFIGURATION_RESOURCES, TEST_CONF_PATH);
+        testRunner.setProperty(PutORC.DIRECTORY, DIRECTORY);
+
+        MockRecordParser readerFactory = new MockRecordParser();
+
+        final RecordSchema recordSchema = AvroTypeUtil.createSchema(schema);
+        for (final RecordField recordField : recordSchema.getFields()) {
+            readerFactory.addSchemaField(recordField.getFieldName(), recordField.getDataType().getFieldType(), recordField.isNullable());
+        }
+
+        if (recordGenerator == null) {
+            for (int i = 0; i < numUsers; i++) {
+                readerFactory.addRecord("name" + i, i, "blue" + i, i * 10.0);
+            }
+        } else {
+            recordGenerator.apply(numUsers, readerFactory);
+        }
+
+        testRunner.addControllerService("mock-reader-factory", readerFactory);
+        testRunner.enableControllerService(readerFactory);
+
+        testRunner.setProperty(PutORC.RECORD_READER, "mock-reader-factory");
+    }
+
+    @Test
+    public void testWriteORCWithDefaults() throws IOException, InitializationException {
+        configure(proc, 100);
+
+        final String filename = "testORCWithDefaults-" + System.currentTimeMillis();
+
+        final Map<String, String> flowFileAttributes = new HashMap<>();
+        flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
+
+        testRunner.setProperty(PutORC.HIVE_TABLE_NAME, "myTable");
+
+        testRunner.enqueue("trigger", flowFileAttributes);
+        testRunner.run();
+        testRunner.assertAllFlowFilesTransferred(PutORC.REL_SUCCESS, 1);
+
+        final Path orcFile = new Path(DIRECTORY + "/" + filename);
+
+        // verify the successful flow file has the expected attributes
+        final MockFlowFile mockFlowFile = testRunner.getFlowFilesForRelationship(PutORC.REL_SUCCESS).get(0);
+        mockFlowFile.assertAttributeEquals(PutORC.ABSOLUTE_HDFS_PATH_ATTRIBUTE, orcFile.getParent().toString());
+        mockFlowFile.assertAttributeEquals(CoreAttributes.FILENAME.key(), filename);
+        mockFlowFile.assertAttributeEquals(PutORC.RECORD_COUNT_ATTR, "100");
+        mockFlowFile.assertAttributeEquals(PutORC.HIVE_DDL_ATTRIBUTE,
+                "CREATE EXTERNAL TABLE IF NOT EXISTS myTable (name STRING, favorite_number INT, favorite_color STRING, scale DOUBLE) STORED AS ORC");
+
+        // verify we generated a provenance event
+        final List<ProvenanceEventRecord> provEvents = testRunner.getProvenanceEvents();
+        assertEquals(1, provEvents.size());
+
+        // verify it was a SEND event with the correct URI
+        final ProvenanceEventRecord provEvent = provEvents.get(0);
+        assertEquals(ProvenanceEventType.SEND, provEvent.getEventType());
+        // If it runs with a real HDFS, the protocol will be "hdfs://", but with a local filesystem, just assert the filename.
+        Assert.assertTrue(provEvent.getTransitUri().endsWith(DIRECTORY + "/" + filename));
+
+        // verify the content of the ORC file by reading it back in
+        verifyORCUsers(orcFile, 100);
+
+        // verify we don't have the temp dot file after success
+        final File tempOrcFile = new File(DIRECTORY + "/." + filename);
+        Assert.assertFalse(tempOrcFile.exists());
+
+        // verify we DO have the CRC file after success
+        final File crcAvroORCFile = new File(DIRECTORY + "/." + filename + ".crc");
+        Assert.assertTrue(crcAvroORCFile.exists());
+    }
+
+    @Test
+    public void testWriteORCWithAvroLogicalTypes() throws IOException, InitializationException {
+        final String avroSchema = IOUtils.toString(new FileInputStream("src/test/resources/user_logical_types.avsc"), StandardCharsets.UTF_8);
+        schema = new Schema.Parser().parse(avroSchema);
+        Calendar now = Calendar.getInstance();
+        LocalTime nowTime = LocalTime.now();
+        LocalDateTime nowDateTime = LocalDateTime.now();
+        LocalDate epoch = LocalDate.ofEpochDay(0);
+        LocalDate nowDate = LocalDate.now();
+
+        final int timeMillis = nowTime.get(ChronoField.MILLI_OF_DAY);
+        final Timestamp timestampMillis = Timestamp.valueOf(nowDateTime);
+        final Date dt = Date.valueOf(nowDate);
+        final double dec = 1234.56;
+
+        configure(proc, 10, (numUsers, readerFactory) -> {
+            for (int i = 0; i < numUsers; i++) {
+                readerFactory.addRecord(
+                        i,
+                        timeMillis,
+                        timestampMillis,
+                        dt,
+                        dec);
+            }
+            return null;
+        });
+
+        final String filename = "testORCWithDefaults-" + System.currentTimeMillis();
+
+        final Map<String, String> flowFileAttributes = new HashMap<>();
+        flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
+
+        testRunner.setProperty(PutORC.HIVE_TABLE_NAME, "myTable");
+
+        testRunner.enqueue("trigger", flowFileAttributes);
+        testRunner.run();
+        testRunner.assertAllFlowFilesTransferred(PutORC.REL_SUCCESS, 1);
+
+        final Path orcFile = new Path(DIRECTORY + "/" + filename);
+
+        // verify the successful flow file has the expected attributes
+        final MockFlowFile mockFlowFile = testRunner.getFlowFilesForRelationship(PutORC.REL_SUCCESS).get(0);
+        mockFlowFile.assertAttributeEquals(PutORC.ABSOLUTE_HDFS_PATH_ATTRIBUTE, orcFile.getParent().toString());
+        mockFlowFile.assertAttributeEquals(CoreAttributes.FILENAME.key(), filename);
+        mockFlowFile.assertAttributeEquals(PutORC.RECORD_COUNT_ATTR, "10");
+        // DDL will be created with field names normalized (lowercased, e.g.) for Hive by default
+        mockFlowFile.assertAttributeEquals(PutORC.HIVE_DDL_ATTRIBUTE,
+                "CREATE EXTERNAL TABLE IF NOT EXISTS myTable (id INT, timemillis INT, timestampmillis TIMESTAMP, dt DATE, dec DOUBLE) STORED AS ORC");
+
+        // verify we generated a provenance event
+        final List<ProvenanceEventRecord> provEvents = testRunner.getProvenanceEvents();
+        assertEquals(1, provEvents.size());
+
+        // verify it was a SEND event with the correct URI
+        final ProvenanceEventRecord provEvent = provEvents.get(0);
+        assertEquals(ProvenanceEventType.SEND, provEvent.getEventType());
+        // If it runs with a real HDFS, the protocol will be "hdfs://", but with a local filesystem, just assert the filename.
+        Assert.assertTrue(provEvent.getTransitUri().endsWith(DIRECTORY + "/" + filename));
+
+        // verify the content of the ORC file by reading it back in
+        verifyORCUsers(orcFile, 10, (x, currUser) -> {
+                    assertEquals((int) currUser, ((IntWritable) x.get(0)).get());
+                    assertEquals(timeMillis, ((IntWritable) x.get(1)).get());
+                    assertEquals(timestampMillis, ((TimestampWritable) x.get(2)).getTimestamp());
+                    assertEquals(dt, ((DateWritable) x.get(3)).get());
+                    assertEquals(dec, ((DoubleWritable) x.get(4)).get(), Double.MIN_VALUE);
+                    return null;
+                }
+        );
+
+        // verify we don't have the temp dot file after success
+        final File tempOrcFile = new File(DIRECTORY + "/." + filename);
+        Assert.assertFalse(tempOrcFile.exists());
+
+        // verify we DO have the CRC file after success
+        final File crcAvroORCFile = new File(DIRECTORY + "/." + filename + ".crc");
+        Assert.assertTrue(crcAvroORCFile.exists());
+    }
+
+    @Test
+    public void testValidSchemaWithELShouldBeSuccessful() throws InitializationException {
+        configure(proc, 10);
+
+        final String filename = "testValidSchemaWithELShouldBeSuccessful-" + System.currentTimeMillis();
+
+        // don't provide my.schema as an attribute
+        final Map<String, String> flowFileAttributes = new HashMap<>();
+        flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
+        flowFileAttributes.put("my.schema", schema.toString());
+
+        testRunner.enqueue("trigger", flowFileAttributes);
+        testRunner.run();
+        testRunner.assertAllFlowFilesTransferred(PutORC.REL_SUCCESS, 1);
+    }
+
+    @Test
+    public void testMalformedRecordExceptionFromReaderShouldRouteToFailure() throws InitializationException, IOException, MalformedRecordException, SchemaNotFoundException {
+        configure(proc, 10);
+
+        final org.apache.nifi.serialization.RecordReader recordReader = Mockito.mock(org.apache.nifi.serialization.RecordReader.class);
+        when(recordReader.nextRecord()).thenThrow(new MalformedRecordException("ERROR"));
+
+        final RecordReaderFactory readerFactory = Mockito.mock(RecordReaderFactory.class);
+        when(readerFactory.getIdentifier()).thenReturn("mock-reader-factory");
+        when(readerFactory.createRecordReader(any(FlowFile.class), any(InputStream.class), any(ComponentLog.class))).thenReturn(recordReader);
+
+        testRunner.addControllerService("mock-reader-factory", readerFactory);
+        testRunner.enableControllerService(readerFactory);
+        testRunner.setProperty(PutORC.RECORD_READER, "mock-reader-factory");
+
+        final String filename = "testMalformedRecordExceptionShouldRouteToFailure-" + System.currentTimeMillis();
+
+        final Map<String, String> flowFileAttributes = new HashMap<>();
+        flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
+
+        testRunner.enqueue("trigger", flowFileAttributes);
+        testRunner.run();
+        testRunner.assertAllFlowFilesTransferred(PutORC.REL_FAILURE, 1);
+    }
+
+    @Test
+    public void testIOExceptionCreatingWriterShouldRouteToRetry() throws InitializationException {
+        final PutORC proc = new PutORC() {
+            @Override
+            public HDFSRecordWriter createHDFSRecordWriter(ProcessContext context, FlowFile flowFile, Configuration conf, Path path, RecordSchema schema)
+                    throws IOException {
+                throw new IOException("IOException");
+            }
+        };
+        configure(proc, 0);
+
+        final String filename = "testMalformedRecordExceptionShouldRouteToFailure-" + System.currentTimeMillis();
+
+        final Map<String, String> flowFileAttributes = new HashMap<>();
+        flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
+
+        testRunner.enqueue("trigger", flowFileAttributes);
+        testRunner.run();
+        testRunner.assertAllFlowFilesTransferred(PutORC.REL_RETRY, 1);
+    }
+
+    @Test
+    public void testIOExceptionFromReaderShouldRouteToRetry() throws InitializationException, IOException, MalformedRecordException, SchemaNotFoundException {
+        configure(proc, 10);
+
+        final RecordSet recordSet = Mockito.mock(RecordSet.class);
+        when(recordSet.next()).thenThrow(new IOException("ERROR"));
+
+        final org.apache.nifi.serialization.RecordReader recordReader = Mockito.mock(org.apache.nifi.serialization.RecordReader.class);
+        when(recordReader.createRecordSet()).thenReturn(recordSet);
+        when(recordReader.getSchema()).thenReturn(AvroTypeUtil.createSchema(schema));
+
+        final RecordReaderFactory readerFactory = Mockito.mock(RecordReaderFactory.class);
+        when(readerFactory.getIdentifier()).thenReturn("mock-reader-factory");
+        when(readerFactory.createRecordReader(any(FlowFile.class), any(InputStream.class), any(ComponentLog.class))).thenReturn(recordReader);
+
+        testRunner.addControllerService("mock-reader-factory", readerFactory);
+        testRunner.enableControllerService(readerFactory);
+        testRunner.setProperty(PutORC.RECORD_READER, "mock-reader-factory");
+
+        final String filename = "testMalformedRecordExceptionShouldRouteToFailure-" + System.currentTimeMillis();
+
+        final Map<String, String> flowFileAttributes = new HashMap<>();
+        flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
+
+        testRunner.enqueue("trigger", flowFileAttributes);
+        testRunner.run();
+        testRunner.assertAllFlowFilesTransferred(PutORC.REL_RETRY, 1);
+    }
+
+    @Test
+    public void testIOExceptionRenamingShouldRouteToRetry() throws InitializationException {
+        final PutORC proc = new PutORC() {
+            @Override
+            protected void rename(FileSystem fileSystem, Path srcFile, Path destFile)
+                    throws IOException, InterruptedException, FailureException {
+                throw new IOException("IOException renaming");
+            }
+        };
+
+        configure(proc, 10);
+
+        final String filename = "testIOExceptionRenamingShouldRouteToRetry-" + System.currentTimeMillis();
+
+        final Map<String, String> flowFileAttributes = new HashMap<>();
+        flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
+
+        testRunner.enqueue("trigger", flowFileAttributes);
+        testRunner.run();
+        testRunner.assertAllFlowFilesTransferred(PutORC.REL_RETRY, 1);
+
+        // verify we don't have the temp dot file after success
+        final File tempAvroORCFile = new File(DIRECTORY + "/." + filename);
+        Assert.assertFalse(tempAvroORCFile.exists());
+    }
+
+    private void verifyORCUsers(final Path orcUsers, final int numExpectedUsers) throws IOException {
+        verifyORCUsers(orcUsers, numExpectedUsers, null);
+    }
+
+    private void verifyORCUsers(final Path orcUsers, final int numExpectedUsers, BiFunction<List<Object>, Integer, Void> assertFunction) throws IOException {
+        Reader reader = OrcFile.createReader(orcUsers, OrcFile.readerOptions(testConf));
+        RecordReader recordReader = reader.rows();
+
+        TypeInfo typeInfo =
+                TypeInfoUtils.getTypeInfoFromTypeString("struct<name:string,favorite_number:int,favorite_color:string,scale:double>");
+        StructObjectInspector inspector = (StructObjectInspector)
+                OrcStruct.createObjectInspector(typeInfo);
+
+        int currUser = 0;
+        Object nextRecord = null;
+        while ((nextRecord = recordReader.next(nextRecord)) != null) {
+            Assert.assertNotNull(nextRecord);
+            Assert.assertTrue("Not an OrcStruct", nextRecord instanceof OrcStruct);
+            List<Object> x = inspector.getStructFieldsDataAsList(nextRecord);
+
+            if (assertFunction == null) {
+                assertEquals("name" + currUser, x.get(0).toString());
+                assertEquals(currUser, ((IntWritable) x.get(1)).get());
+                assertEquals("blue" + currUser, x.get(2).toString());
+                assertEquals(10.0 * currUser, ((DoubleWritable) x.get(3)).get(), Double.MIN_VALUE);
+            } else {
+                assertFunction.apply(x, currUser);
+            }
+            currUser++;
+        }
+
+        assertEquals(numExpectedUsers, currUser);
+    }
+
+}
\ No newline at end of file