From b1462dbca1da37837ba665049fc71d97d6017a99 Mon Sep 17 00:00:00 2001 From: Matt Burgess Date: Tue, 26 Jul 2016 23:25:11 -0400 Subject: [PATCH 1/3] NIFI-1663: Add ConvertAvroToORC processor --- .../nifi-hive-processors/pom.xml | 9 - .../hadoop/hive/ql/io/orc/NiFiOrcUtils.java | 466 +++ .../hive/ql/io/orc/OrcFlowFileWriter.java | 2650 +++++++++++++++++ .../processors/hive/ConvertAvroToORC.java | 289 ++ .../org.apache.nifi.processor.Processor | 1 + .../processors/hive/TestConvertAvroToORC.java | 214 ++ .../processors/hive/TestPutHiveStreaming.java | 10 +- .../nifi/util/orc/TestNiFiOrcUtils.java | 371 +++ 8 files changed, 3996 insertions(+), 14 deletions(-) create mode 100644 nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java create mode 100644 nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java create mode 100644 nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java create mode 100644 nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java create mode 100644 nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java 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 9ef6fc431e34..02face9127c3 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 @@ -44,10 +44,6 @@ hive-jdbc ${hive.version} - - org.apache.hive - hive-orc - org.apache.hadoop hadoop-common @@ -191,11 +187,6 @@ nifi-mock test - - org.slf4j - slf4j-simple - test - junit junit diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java new file mode 100644 index 000000000000..78430bf9c6fc --- /dev/null +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java @@ -0,0 +1,466 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.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.conf.HiveConf; +import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO; +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.MapWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_DEFAULT_BLOCK_PADDING; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_DEFAULT_BLOCK_SIZE; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_WRITE_FORMAT; + +/** + * 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) { + if (o != null) { + if (typeInfo instanceof UnionTypeInfo) { + OrcUnion union = new OrcUnion(); + // Need to find which of the union types correspond to the primitive object + TypeInfo objectTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector( + ObjectInspectorFactory.getReflectionObjectInspector(o.getClass(), ObjectInspectorFactory.ObjectInspectorOptions.JAVA)); + List 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)); + } 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 int[]) { + int[] intArray = (int[]) o; + return Arrays.stream(intArray) + .mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("int"), element)) + .collect(Collectors.toList()); + } + if (o instanceof long[]) { + long[] longArray = (long[]) o; + return Arrays.stream(longArray) + .mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("bigint"), element)) + .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)) + .collect(Collectors.toList()); + } + if (o instanceof double[]) { + double[] doubleArray = (double[]) o; + return Arrays.stream(doubleArray) + .mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("double"), element)) + .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)) + .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)).collect(Collectors.toList()); + } + if (o instanceof List) { + return o; + } + if (o instanceof Map) { + MapWritable mapWritable = new MapWritable(); + TypeInfo keyInfo = ((MapTypeInfo) typeInfo).getMapKeyTypeInfo(); + TypeInfo valueInfo = ((MapTypeInfo) typeInfo).getMapKeyTypeInfo(); + // 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); + Object valueObject = convertToORCObject(valueInfo, value); + if (keyObject == null + || !(keyObject instanceof Writable) + || !(valueObject instanceof Writable) + ) { + throw new IllegalArgumentException("Maps may only contain Writable types, and the key cannot be null"); + } + mapWritable.put((Writable) keyObject, (Writable) valueObject); + }); + return mapWritable; + } + + } + 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 + */ + public static OrcStruct createOrcStruct(TypeInfo typeInfo, Object... objs) { + SettableStructObjectInspector oi = (SettableStructObjectInspector) OrcStruct + .createObjectInspector(typeInfo); + List fields = (List) 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) { + 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 hiveColumns = new ArrayList<>(); + List fields = avroSchema.getFields(); + if (fields != null) { + hiveColumns.addAll( + fields.stream().map(field -> field.name() + " " + getHiveTypeFromAvroType(field.schema())).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) throws IllegalArgumentException { + Schema.Type fieldType = fieldSchema.getType(); + + switch (fieldType) { + case INT: + case LONG: + case BOOLEAN: + case BYTES: + case DOUBLE: + case FLOAT: + case STRING: + return getPrimitiveOrcTypeFromPrimitiveAvroType(fieldType); + + case UNION: + List unionFieldSchemas = fieldSchema.getTypes(); + + if (unionFieldSchemas != null) { + // Ignore null types in union + List orcFields = unionFieldSchemas.stream().filter( + unionFieldSchema -> !Schema.Type.NULL.equals(unionFieldSchema.getType())) + .map(NiFiOrcUtils::getOrcField) + .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())); + + case MAP: + return TypeInfoFactory.getMapTypeInfo( + getPrimitiveOrcTypeFromPrimitiveAvroType(Schema.Type.STRING), + getOrcField(fieldSchema.getValueType())); + + case RECORD: + List avroFields = fieldSchema.getFields(); + if (avroFields != null) { + List orcFieldNames = new ArrayList<>(avroFields.size()); + List orcFields = new ArrayList<>(avroFields.size()); + avroFields.forEach(avroField -> { + String fieldName = avroField.name(); + orcFieldNames.add(fieldName); + orcFields.add(getOrcField(avroField.schema())); + }); + 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) { + if (avroSchema == null) { + throw new IllegalArgumentException("Avro schema is null"); + } + + Schema.Type avroType = avroSchema.getType(); + + switch (avroType) { + case INT: + return "INT"; + case LONG: + return "BIGINT"; + case BOOLEAN: + return "BOOLEAN"; + case BYTES: + return "BINARY"; + case DOUBLE: + return "DOUBLE"; + case FLOAT: + return "FLOAT"; + case STRING: + case ENUM: + return "STRING"; + case UNION: + List unionFieldSchemas = avroSchema.getTypes(); + if (unionFieldSchemas != null) { + List 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)); + } + } + // 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"; + case ARRAY: + return "ARRAY<" + getHiveTypeFromAvroType(avroSchema.getElementType()) + ">"; + case RECORD: + List recordFields = avroSchema.getFields(); + if (recordFields != null) { + List hiveFields = recordFields.stream().map( + recordField -> recordField.name() + ":" + getHiveTypeFromAvroType(recordField.schema())).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 OrcFlowFileWriter createWriter(OutputStream flowFileOutputStream, + Path path, + Configuration conf, + TypeInfo orcSchema, + long stripeSize, + CompressionKind compress, + int bufferSize) throws IOException { + + int rowIndexStride = HiveConf.getIntVar(conf, HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE); + + boolean addBlockPadding = HiveConf.getBoolVar(conf, HIVE_ORC_DEFAULT_BLOCK_PADDING); + + String versionName = HiveConf.getVar(conf, HIVE_ORC_WRITE_FORMAT); + OrcFile.Version versionValue = (versionName == null) + ? OrcFile.Version.CURRENT + : OrcFile.Version.byName(versionName); + + OrcFile.EncodingStrategy encodingStrategy; + String enString = conf.get(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname); + if (enString == null) { + encodingStrategy = OrcFile.EncodingStrategy.SPEED; + } else { + encodingStrategy = OrcFile.EncodingStrategy.valueOf(enString); + } + + OrcFile.CompressionStrategy compressionStrategy; + String compString = conf.get(HiveConf.ConfVars.HIVE_ORC_COMPRESSION_STRATEGY.varname); + if (compString == null) { + compressionStrategy = OrcFile.CompressionStrategy.SPEED; + } else { + compressionStrategy = OrcFile.CompressionStrategy.valueOf(compString); + } + + float paddingTolerance; + paddingTolerance = conf.getFloat(HiveConf.ConfVars.HIVE_ORC_BLOCK_PADDING_TOLERANCE.varname, + HiveConf.ConfVars.HIVE_ORC_BLOCK_PADDING_TOLERANCE.defaultFloatVal); + + long blockSizeValue = HiveConf.getLongVar(conf, HIVE_ORC_DEFAULT_BLOCK_SIZE); + + double bloomFilterFpp = BloomFilterIO.DEFAULT_FPP; + + ObjectInspector inspector = OrcStruct.createObjectInspector(orcSchema); + + return new OrcFlowFileWriter(flowFileOutputStream, + path, + conf, + inspector, + stripeSize, + compress, + bufferSize, + rowIndexStride, + getMemoryManager(conf), + addBlockPadding, + versionValue, + null, // no callback + encodingStrategy, + compressionStrategy, + paddingTolerance, + blockSizeValue, + null, // no Bloom Filter column names + bloomFilterFpp); + } + + private static MemoryManager memoryManager = null; + + private static synchronized MemoryManager getMemoryManager(Configuration conf) { + if (memoryManager == null) { + memoryManager = new MemoryManager(conf); + } + return memoryManager; + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java new file mode 100644 index 000000000000..a98092a6132e --- /dev/null +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java @@ -0,0 +1,2650 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 static com.google.common.base.Preconditions.checkArgument; + +import java.io.IOException; +import java.io.OutputStream; +import java.lang.management.ManagementFactory; +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.TimeZone; +import java.util.TreeMap; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.JavaUtils; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.ql.io.IOConstants; +import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO; +import org.apache.hadoop.hive.ql.io.orc.CompressionCodec.Modifier; +import org.apache.hadoop.hive.ql.io.orc.OrcFile.CompressionStrategy; +import org.apache.hadoop.hive.ql.io.orc.OrcFile.EncodingStrategy; +import org.apache.hadoop.hive.ql.io.orc.OrcProto.RowIndexEntry; +import org.apache.hadoop.hive.ql.io.orc.OrcProto.StripeStatistics; +import org.apache.hadoop.hive.ql.io.orc.OrcProto.Type; +import org.apache.hadoop.hive.ql.io.orc.OrcProto.UserMetadataItem; +import org.apache.hadoop.hive.ql.util.JavaDataModel; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Text; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import com.google.common.primitives.Longs; +import com.google.protobuf.ByteString; +import com.google.protobuf.CodedOutputStream; +import org.apache.nifi.stream.io.ByteCountingOutputStream; + +/** + * An ORC file writer. The file is divided into stripes, which is the natural + * unit of work when reading. Each stripe is buffered in memory until the + * memory reaches the stripe size and then it is written out broken down by + * columns. Each column is written by a TreeWriter that is specific to that + * type of column. TreeWriters may have children TreeWriters that handle the + * sub-types. Each of the TreeWriters writes the column's data as a set of + * streams. + *

+ * This class is synchronized so that multi-threaded access is ok. In + * particular, because the MemoryManager is shared between writers, this class + * assumes that checkMemory may be called from a separate thread. + */ +public class OrcFlowFileWriter implements Writer, MemoryManager.Callback { + + private static final Log LOG = LogFactory.getLog(WriterImpl.class); + + private static final int HDFS_BUFFER_SIZE = 256 * 1024; + private static final int MIN_ROW_INDEX_STRIDE = 1000; + + // threshold above which buffer size will be automatically resized + private static final int COLUMN_COUNT_THRESHOLD = 1000; + + private final Path path; + private final long defaultStripeSize; + private long adjustedStripeSize; + private final int rowIndexStride; + private final CompressionKind compress; + private final CompressionCodec codec; + private final boolean addBlockPadding; + private final int bufferSize; + private final long blockSize; + private final float paddingTolerance; + // the streams that make up the current stripe + private final Map streams = new TreeMap<>(); + + private final OutputStream flowFileOutputStream; + + private ByteCountingOutputStream rawWriter = null; + // the compressed metadata information outStream + private OutStream writer = null; + // a protobuf outStream around streamFactory + private CodedOutputStream protobufWriter = null; + private long headerLength; + private int columnCount; + private long rowCount = 0; + private long rowsInStripe = 0; + private long rawDataSize = 0; + private int rowsInIndex = 0; + private int stripesAtLastFlush = -1; + private final List stripes = new ArrayList<>(); + private final Map userMetadata = new TreeMap<>(); + private final StreamFactory streamFactory = new StreamFactory(); + private final TreeWriter treeWriter; + private final boolean buildIndex; + private final MemoryManager memoryManager; + private final OrcFile.Version version; + private final Configuration conf; + private final OrcFile.WriterCallback callback; + private final OrcFile.WriterContext callbackContext; + private final OrcFile.EncodingStrategy encodingStrategy; + private final OrcFile.CompressionStrategy compressionStrategy; + private final boolean[] bloomFilterColumns; + private final double bloomFilterFpp; + + + public OrcFlowFileWriter(OutputStream flowFileOutputStream, + Path path, + Configuration conf, + ObjectInspector inspector, + long stripeSize, + CompressionKind compress, + int bufferSize, + int rowIndexStride, + MemoryManager memoryManager, + boolean addBlockPadding, + OrcFile.Version version, + OrcFile.WriterCallback callback, + EncodingStrategy encodingStrategy, + CompressionStrategy compressionStrategy, + float paddingTolerance, + long blockSizeValue, + String bloomFilterColumnNames, + double bloomFilterFpp) throws IOException { + this.flowFileOutputStream = flowFileOutputStream; + this.path = path; + this.conf = conf; + this.callback = callback; + callbackContext = (callback != null) ? () -> OrcFlowFileWriter.this : null; + this.adjustedStripeSize = stripeSize; + this.defaultStripeSize = stripeSize; + this.version = version; + this.encodingStrategy = encodingStrategy; + this.compressionStrategy = compressionStrategy; + this.addBlockPadding = addBlockPadding; + this.blockSize = blockSizeValue; + this.paddingTolerance = paddingTolerance; + this.compress = compress; + this.rowIndexStride = rowIndexStride; + this.memoryManager = memoryManager; + buildIndex = rowIndexStride > 0; + codec = createCodec(compress); + String allColumns = conf.get(IOConstants.COLUMNS); + if (allColumns == null) { + allColumns = getColumnNamesFromInspector(inspector); + } + this.bufferSize = getEstimatedBufferSize(allColumns, bufferSize); + if (version == OrcFile.Version.V_0_11) { + /* do not write bloom filters for ORC v11 */ + this.bloomFilterColumns = + OrcUtils.includeColumns(null, allColumns, inspector); + } else { + this.bloomFilterColumns = + OrcUtils.includeColumns(bloomFilterColumnNames, allColumns, inspector); + } + this.bloomFilterFpp = bloomFilterFpp; + treeWriter = createTreeWriter(inspector, streamFactory, false); + if (buildIndex && rowIndexStride < MIN_ROW_INDEX_STRIDE) { + throw new IllegalArgumentException("Row stride must be at least " + + MIN_ROW_INDEX_STRIDE); + } + + // ensure that we are able to handle callbacks before we register ourselves + memoryManager.addWriter(path, stripeSize, this); + } + + private String getColumnNamesFromInspector(ObjectInspector inspector) { + List fieldNames = Lists.newArrayList(); + Joiner joiner = Joiner.on(","); + if (inspector instanceof StructObjectInspector) { + StructObjectInspector soi = (StructObjectInspector) inspector; + List fields = soi.getAllStructFieldRefs(); + fieldNames.addAll(fields.stream().map((Function) StructField::getFieldName).collect(Collectors.toList())); + } + return joiner.join(fieldNames); + } + + @VisibleForTesting + int getEstimatedBufferSize(int bs) { + return getEstimatedBufferSize(conf.get(IOConstants.COLUMNS), bs); + } + + int getEstimatedBufferSize(String colNames, int bs) { + long availableMem = getMemoryAvailableForORC(); + if (colNames != null) { + final int numCols = colNames.split(",").length; + if (numCols > COLUMN_COUNT_THRESHOLD) { + // In BufferedStream, there are 3 outstream buffers (compressed, + // uncompressed and overflow) and list of previously compressed buffers. + // Since overflow buffer is rarely used, lets consider only 2 allocation. + // Also, initially, the list of compression buffers will be empty. + final int outStreamBuffers = codec == null ? 1 : 2; + + // max possible streams per column is 5. For string columns, there is + // ROW_INDEX, PRESENT, DATA, LENGTH, DICTIONARY_DATA streams. + final int maxStreams = 5; + + // Lets assume 10% memory for holding dictionary in memory and other + // object allocations + final long miscAllocation = (long) (0.1f * availableMem); + + // compute the available memory + final long remainingMem = availableMem - miscAllocation; + + int estBufferSize = (int) (remainingMem / (maxStreams * outStreamBuffers * numCols)); + estBufferSize = getClosestBufferSize(estBufferSize, bs); + if (estBufferSize > bs) { + estBufferSize = bs; + } + + LOG.info("WIDE TABLE - Number of columns: " + numCols + " Chosen compression buffer size: " + estBufferSize); + return estBufferSize; + } + } + return bs; + } + + private int getClosestBufferSize(int estBufferSize, int bs) { + final int kb4 = 4 * 1024; + final int kb8 = 8 * 1024; + final int kb16 = 16 * 1024; + final int kb32 = 32 * 1024; + final int kb64 = 64 * 1024; + final int kb128 = 128 * 1024; + final int kb256 = 256 * 1024; + if (estBufferSize <= kb4) { + return kb4; + } else if (estBufferSize > kb4 && estBufferSize <= kb8) { + return kb8; + } else if (estBufferSize > kb8 && estBufferSize <= kb16) { + return kb16; + } else if (estBufferSize > kb16 && estBufferSize <= kb32) { + return kb32; + } else if (estBufferSize > kb32 && estBufferSize <= kb64) { + return kb64; + } else if (estBufferSize > kb64 && estBufferSize <= kb128) { + return kb128; + } else { + return kb256; + } + } + + // the assumption is only one ORC writer open at a time, which holds true for + // most of the cases. HIVE-6455 forces single writer case. + private long getMemoryAvailableForORC() { + HiveConf.ConfVars poolVar = HiveConf.ConfVars.HIVE_ORC_FILE_MEMORY_POOL; + double maxLoad = conf.getFloat(poolVar.varname, poolVar.defaultFloatVal); + long totalMemoryPool = Math.round(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax() * maxLoad); + return totalMemoryPool; + } + + public static CompressionCodec createCodec(CompressionKind kind) { + switch (kind) { + case NONE: + return null; + case ZLIB: + return new ZlibCodec(); + case SNAPPY: + return new SnappyCodec(); + case LZO: + try { + Class lzo = + (Class) + JavaUtils.loadClass("org.apache.hadoop.hive.ql.io.orc.LzoCodec"); + return lzo.newInstance(); + } catch (ClassNotFoundException e) { + throw new IllegalArgumentException("LZO is not available.", e); + } catch (InstantiationException e) { + throw new IllegalArgumentException("Problem initializing LZO", e); + } catch (IllegalAccessException e) { + throw new IllegalArgumentException("Insufficient access to LZO", e); + } + default: + throw new IllegalArgumentException("Unknown compression codec: " + + kind); + } + } + + @Override + public synchronized boolean checkMemory(double newScale) throws IOException { + long limit = Math.round(adjustedStripeSize * newScale); + long size = estimateStripeSize(); + if (LOG.isDebugEnabled()) { + LOG.debug("ORC writer " + path + " size = " + size + " limit = " + + limit); + } + if (size > limit) { + flushStripe(); + return true; + } + return false; + } + + /** + * This class is used to hold the contents of streams as they are buffered. + * The TreeWriters write to the outStream and the codec compresses the + * data as buffers fill up and stores them in the output list. When the + * stripe is being written, the whole stream is written to the file. + */ + private class BufferedStream implements OutStream.OutputReceiver { + private final OutStream outStream; + private final List output = new ArrayList(); + + BufferedStream(String name, int bufferSize, + CompressionCodec codec) throws IOException { + outStream = new OutStream(name, bufferSize, codec, this); + } + + /** + * Receive a buffer from the compression codec. + * + * @param buffer the buffer to save + * @throws IOException if an error occurs while receiving a buffer + */ + @Override + public void output(ByteBuffer buffer) { + output.add(buffer); + } + + /** + * Get the number of bytes in buffers that are allocated to this stream. + * + * @return number of bytes in buffers + */ + public long getBufferSize() { + long result = 0; + for (ByteBuffer buf : output) { + result += buf.capacity(); + } + return outStream.getBufferSize() + result; + } + + /** + * Flush the stream to the codec. + * + * @throws IOException if an error occurs while flushing the stream + */ + public void flush() throws IOException { + outStream.flush(); + } + + /** + * Clear all of the buffers. + * + * @throws IOException if an error occurs while clearing the buffers + */ + public void clear() throws IOException { + outStream.clear(); + output.clear(); + } + + /** + * Check the state of suppress flag in output stream + * + * @return value of suppress flag + */ + public boolean isSuppressed() { + return outStream.isSuppressed(); + } + + /** + * Get the number of bytes that will be written to the output. Assumes + * the stream has already been flushed. + * + * @return the number of bytes + */ + public long getOutputSize() { + long result = 0; + for (ByteBuffer buffer : output) { + result += buffer.remaining(); + } + return result; + } + + /** + * Write the saved compressed buffers to the OutputStream. + * + * @param out the stream to write to + * @throws IOException if an error occurs during write + */ + void spillTo(OutputStream out) throws IOException { + for (ByteBuffer buffer : output) { + out.write(buffer.array(), buffer.arrayOffset() + buffer.position(), + buffer.remaining()); + } + } + + @Override + public String toString() { + return outStream.toString(); + } + } + + /** + * An output receiver that writes the ByteBuffers to the output stream + * as they are received. + */ + private class DirectStream implements OutStream.OutputReceiver { + private final OutputStream output; + + DirectStream(OutputStream output) { + this.output = output; + } + + @Override + public void output(ByteBuffer buffer) throws IOException { + output.write(buffer.array(), buffer.arrayOffset() + buffer.position(), + buffer.remaining()); + } + } + + private static class RowIndexPositionRecorder implements PositionRecorder { + private final OrcProto.RowIndexEntry.Builder builder; + + RowIndexPositionRecorder(OrcProto.RowIndexEntry.Builder builder) { + this.builder = builder; + } + + @Override + public void addPosition(long position) { + builder.addPositions(position); + } + } + + /** + * Interface from the Writer to the TreeWriters. This limits the visibility + * that the TreeWriters have into the Writer. + */ + private class StreamFactory { + /** + * Create a stream to store part of a column. + * + * @param column the column id for the stream + * @param kind the kind of stream + * @return The output outStream that the section needs to be written to. + * @throws IOException if an error occurs while creating the stream + */ + public OutStream createStream(int column, + OrcProto.Stream.Kind kind + ) throws IOException { + final StreamName name = new StreamName(column, kind); + final EnumSet modifiers; + + switch (kind) { + case BLOOM_FILTER: + case DATA: + case DICTIONARY_DATA: + if (getCompressionStrategy() == CompressionStrategy.SPEED) { + modifiers = EnumSet.of(Modifier.FAST, Modifier.TEXT); + } else { + modifiers = EnumSet.of(Modifier.DEFAULT, Modifier.TEXT); + } + break; + case LENGTH: + case DICTIONARY_COUNT: + case PRESENT: + case ROW_INDEX: + case SECONDARY: + // easily compressed using the fastest modes + modifiers = EnumSet.of(Modifier.FASTEST, Modifier.BINARY); + break; + default: + LOG.warn("Missing ORC compression modifiers for " + kind); + modifiers = null; + break; + } + + BufferedStream result = streams.get(name); + if (result == null) { + result = new BufferedStream(name.toString(), bufferSize, + codec == null ? codec : codec.modify(modifiers)); + streams.put(name, result); + } + return result.outStream; + } + + /** + * Get the next column id. + * + * @return a number from 0 to the number of columns - 1 + */ + public int getNextColumnId() { + return columnCount++; + } + + /** + * Get the current column id. After creating all tree writers this count should tell how many + * columns (including columns within nested complex objects) are created in total. + * + * @return current column id + */ + public int getCurrentColumnId() { + return columnCount; + } + + /** + * Get the stride rate of the row index. + */ + public int getRowIndexStride() { + return rowIndexStride; + } + + /** + * Should be building the row index. + * + * @return true if we are building the index + */ + public boolean buildIndex() { + return buildIndex; + } + + /** + * Is the ORC file compressed? + * + * @return are the streams compressed + */ + public boolean isCompressed() { + return codec != null; + } + + /** + * Get the encoding strategy to use. + * + * @return encoding strategy + */ + public EncodingStrategy getEncodingStrategy() { + return encodingStrategy; + } + + /** + * Get the compression strategy to use. + * + * @return compression strategy + */ + public CompressionStrategy getCompressionStrategy() { + return compressionStrategy; + } + + /** + * Get the bloom filter columns + * + * @return bloom filter columns + */ + public boolean[] getBloomFilterColumns() { + return bloomFilterColumns; + } + + /** + * Get bloom filter false positive percentage. + * + * @return fpp + */ + public double getBloomFilterFPP() { + return bloomFilterFpp; + } + + /** + * Get the writer's configuration. + * + * @return configuration + */ + public Configuration getConfiguration() { + return conf; + } + + /** + * Get the version of the file to write. + */ + public OrcFile.Version getVersion() { + return version; + } + } + + /** + * The parent class of all of the writers for each column. Each column + * is written by an instance of this class. The compound types (struct, + * list, map, and union) have children tree writers that write the children + * types. + */ + private abstract static class TreeWriter { + protected final int id; + protected final ObjectInspector inspector; + private final BitFieldWriter isPresent; + private final boolean isCompressed; + protected final ColumnStatisticsImpl indexStatistics; + protected final ColumnStatisticsImpl stripeColStatistics; + private final ColumnStatisticsImpl fileStatistics; + protected TreeWriter[] childrenWriters; + protected final RowIndexPositionRecorder rowIndexPosition; + private final OrcProto.RowIndex.Builder rowIndex; + private final OrcProto.RowIndexEntry.Builder rowIndexEntry; + private final PositionedOutputStream rowIndexStream; + private final PositionedOutputStream bloomFilterStream; + protected final BloomFilterIO bloomFilter; + protected final boolean createBloomFilter; + private final OrcProto.BloomFilterIndex.Builder bloomFilterIndex; + private final OrcProto.BloomFilter.Builder bloomFilterEntry; + private boolean foundNulls; + private OutStream isPresentOutStream; + private final List stripeStatsBuilders; + + /** + * Create a tree writer. + * + * @param columnId the column id of the column to write + * @param inspector the object inspector to use + * @param streamFactory limited access to the Writer's data. + * @param nullable can the value be null? + * @throws IOException if an error occurs during creation + */ + TreeWriter(int columnId, ObjectInspector inspector, + StreamFactory streamFactory, + boolean nullable) throws IOException { + this.isCompressed = streamFactory.isCompressed(); + this.id = columnId; + this.inspector = inspector; + if (nullable) { + isPresentOutStream = streamFactory.createStream(id, + OrcProto.Stream.Kind.PRESENT); + isPresent = new BitFieldWriter(isPresentOutStream, 1); + } else { + isPresent = null; + } + this.foundNulls = false; + createBloomFilter = streamFactory.getBloomFilterColumns()[columnId]; + indexStatistics = ColumnStatisticsImpl.create(inspector); + stripeColStatistics = ColumnStatisticsImpl.create(inspector); + fileStatistics = ColumnStatisticsImpl.create(inspector); + childrenWriters = new TreeWriter[0]; + rowIndex = OrcProto.RowIndex.newBuilder(); + rowIndexEntry = OrcProto.RowIndexEntry.newBuilder(); + rowIndexPosition = new RowIndexPositionRecorder(rowIndexEntry); + stripeStatsBuilders = Lists.newArrayList(); + if (streamFactory.buildIndex()) { + rowIndexStream = streamFactory.createStream(id, OrcProto.Stream.Kind.ROW_INDEX); + } else { + rowIndexStream = null; + } + if (createBloomFilter) { + bloomFilterEntry = OrcProto.BloomFilter.newBuilder(); + bloomFilterIndex = OrcProto.BloomFilterIndex.newBuilder(); + bloomFilterStream = streamFactory.createStream(id, OrcProto.Stream.Kind.BLOOM_FILTER); + bloomFilter = new BloomFilterIO(streamFactory.getRowIndexStride(), + streamFactory.getBloomFilterFPP()); + } else { + bloomFilterEntry = null; + bloomFilterIndex = null; + bloomFilterStream = null; + bloomFilter = null; + } + } + + protected OrcProto.RowIndex.Builder getRowIndex() { + return rowIndex; + } + + protected ColumnStatisticsImpl getStripeStatistics() { + return stripeColStatistics; + } + + protected ColumnStatisticsImpl getFileStatistics() { + return fileStatistics; + } + + protected OrcProto.RowIndexEntry.Builder getRowIndexEntry() { + return rowIndexEntry; + } + + IntegerWriter createIntegerWriter(PositionedOutputStream output, + boolean signed, boolean isDirectV2, + StreamFactory writer) { + if (isDirectV2) { + boolean alignedBitpacking = false; + if (writer.getEncodingStrategy().equals(EncodingStrategy.SPEED)) { + alignedBitpacking = true; + } + return new RunLengthIntegerWriterV2(output, signed, alignedBitpacking); + } else { + return new RunLengthIntegerWriter(output, signed); + } + } + + boolean isNewWriteFormat(StreamFactory writer) { + return writer.getVersion() != OrcFile.Version.V_0_11; + } + + /** + * Add a new value to the column. + * + * @param obj The value to write to the column + * @throws IOException if an error occurs during add + */ + void write(Object obj) throws IOException { + if (obj != null) { + indexStatistics.increment(); + } else { + indexStatistics.setNull(); + } + if (isPresent != null) { + isPresent.write(obj == null ? 0 : 1); + if (obj == null) { + foundNulls = true; + } + } + } + + private void removeIsPresentPositions() { + for (int i = 0; i < rowIndex.getEntryCount(); ++i) { + RowIndexEntry.Builder entry = rowIndex.getEntryBuilder(i); + List positions = entry.getPositionsList(); + // bit streams use 3 positions if uncompressed, 4 if compressed + positions = positions.subList(isCompressed ? 4 : 3, positions.size()); + entry.clearPositions(); + entry.addAllPositions(positions); + } + } + + /** + * Write the stripe out to the file. + * + * @param builder the stripe footer that contains the information about the + * layout of the stripe. The TreeWriter is required to update + * the footer with its information. + * @param requiredIndexEntries the number of index entries that are + * required. this is to check to make sure the + * row index is well formed. + * @throws IOException if an error occurs during write + */ + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + if (isPresent != null) { + isPresent.flush(); + + // if no nulls are found in a stream, then suppress the stream + if (!foundNulls) { + isPresentOutStream.suppress(); + // since isPresent bitstream is suppressed, update the index to + // remove the positions of the isPresent stream + if (rowIndexStream != null) { + removeIsPresentPositions(); + } + } + } + + // merge stripe-level column statistics to file statistics and write it to + // stripe statistics + OrcProto.StripeStatistics.Builder stripeStatsBuilder = OrcProto.StripeStatistics.newBuilder(); + writeStripeStatistics(stripeStatsBuilder, this); + stripeStatsBuilders.add(stripeStatsBuilder); + + // reset the flag for next stripe + foundNulls = false; + + builder.addColumns(getEncoding()); + builder.setWriterTimezone(TimeZone.getDefault().getID()); + if (rowIndexStream != null) { + if (rowIndex.getEntryCount() != requiredIndexEntries) { + throw new IllegalArgumentException("Column has wrong number of " + + "index entries found: " + rowIndex.getEntryCount() + " expected: " + + requiredIndexEntries); + } + rowIndex.build().writeTo(rowIndexStream); + rowIndexStream.flush(); + } + rowIndex.clear(); + rowIndexEntry.clear(); + + // write the bloom filter to out stream + if (bloomFilterStream != null) { + bloomFilterIndex.build().writeTo(bloomFilterStream); + bloomFilterStream.flush(); + bloomFilterIndex.clear(); + bloomFilterEntry.clear(); + } + } + + private void writeStripeStatistics(OrcProto.StripeStatistics.Builder builder, + TreeWriter treeWriter) { + treeWriter.fileStatistics.merge(treeWriter.stripeColStatistics); + builder.addColStats(treeWriter.stripeColStatistics.serialize().build()); + treeWriter.stripeColStatistics.reset(); + for (TreeWriter child : treeWriter.getChildrenWriters()) { + writeStripeStatistics(builder, child); + } + } + + TreeWriter[] getChildrenWriters() { + return childrenWriters; + } + + /** + * Get the encoding for this column. + * + * @return the information about the encoding of this column + */ + OrcProto.ColumnEncoding getEncoding() { + return OrcProto.ColumnEncoding.newBuilder().setKind( + OrcProto.ColumnEncoding.Kind.DIRECT).build(); + } + + /** + * Create a row index entry with the previous location and the current + * index statistics. Also merges the index statistics into the file + * statistics before they are cleared. Finally, it records the start of the + * next index and ensures all of the children columns also create an entry. + * + * @throws IOException if an error occurs during create + */ + void createRowIndexEntry() throws IOException { + stripeColStatistics.merge(indexStatistics); + rowIndexEntry.setStatistics(indexStatistics.serialize()); + indexStatistics.reset(); + rowIndex.addEntry(rowIndexEntry); + rowIndexEntry.clear(); + addBloomFilterEntry(); + recordPosition(rowIndexPosition); + for (TreeWriter child : childrenWriters) { + child.createRowIndexEntry(); + } + } + + void addBloomFilterEntry() { + if (createBloomFilter) { + bloomFilterEntry.setNumHashFunctions(bloomFilter.getNumHashFunctions()); + bloomFilterEntry.addAllBitset(Longs.asList(bloomFilter.getBitSet())); + bloomFilterIndex.addBloomFilter(bloomFilterEntry.build()); + bloomFilter.reset(); + bloomFilterEntry.clear(); + } + } + + /** + * Record the current position in each of this column's streams. + * + * @param recorder where should the locations be recorded + * @throws IOException if an error occurs during position recording + */ + void recordPosition(PositionRecorder recorder) throws IOException { + if (isPresent != null) { + isPresent.getPosition(recorder); + } + } + + /** + * Estimate how much memory the writer is consuming excluding the streams. + * + * @return the number of bytes. + */ + long estimateMemory() { + long result = 0; + for (TreeWriter child : childrenWriters) { + result += child.estimateMemory(); + } + return result; + } + + /** + * Handle the top level object write. + * + * This default method is used for all types except structs, which are the + * typical case. VectorizedRowBatch assumes the top level object is a + * struct, so we use the first column for all other types. + * @param batch the batch to write from + * @param offset the row to start on + * @param length the number of rows to write + * @throws IOException if an error occurs during write + */ + void writeRootBatch(VectorizedRowBatch batch, int offset, + int length) throws IOException { + writeBatch(batch.cols[0], offset, length); + } + + /** + * Write the values from the given vector from offset for length elements. + * @param vector the vector to write from + * @param offset the first value from the vector to write + * @param length the number of values from the vector to write + * @throws IOException if an error occurs during write + */ + void writeBatch(ColumnVector vector, int offset, + int length) throws IOException { + if (vector.noNulls) { + if (isPresent != null) { + for (int i = 0; i < length; ++i) { + isPresent.write(1); + indexStatistics.increment(); + } + } + } else { + if (vector.isRepeating) { + boolean isNull = vector.isNull[0]; + if (isPresent != null) { + for (int i = 0; i < length; ++i) { + isPresent.write(isNull ? 0 : 1); + } + } + if (isNull) { + foundNulls = true; + indexStatistics.setNull(); + } else { + indexStatistics.increment(); + } + } else { + // count the number of non-null values + int nonNullCount = 0; + for(int i = 0; i < length; ++i) { + boolean isNull = vector.isNull[i + offset]; + if (!isNull) { + nonNullCount += 1; + } + if (isPresent != null) { + isPresent.write(isNull ? 0 : 1); + } + } + indexStatistics.increment(); + if (nonNullCount != length) { + foundNulls = true; + indexStatistics.setNull(); + } + } + } + } + } + + private static class BooleanTreeWriter extends TreeWriter { + private final BitFieldWriter writer; + + BooleanTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + PositionedOutputStream out = writer.createStream(id, + OrcProto.Stream.Kind.DATA); + this.writer = new BitFieldWriter(out, 1); + recordPosition(rowIndexPosition); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + boolean val = ((BooleanObjectInspector) inspector).get(obj); + indexStatistics.updateBoolean(val); + writer.write(val ? 1 : 0); + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + writer.flush(); + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + writer.getPosition(recorder); + } + } + + private static class ByteTreeWriter extends TreeWriter { + private final RunLengthByteWriter writer; + + ByteTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + this.writer = new RunLengthByteWriter(writer.createStream(id, + OrcProto.Stream.Kind.DATA)); + recordPosition(rowIndexPosition); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + byte val = ((ByteObjectInspector) inspector).get(obj); + indexStatistics.updateInteger(val); + if (createBloomFilter) { + bloomFilter.addLong(val); + } + writer.write(val); + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + writer.flush(); + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + writer.getPosition(recorder); + } + } + + private static class IntegerTreeWriter extends TreeWriter { + private final IntegerWriter writer; + private final ShortObjectInspector shortInspector; + private final IntObjectInspector intInspector; + private final LongObjectInspector longInspector; + private boolean isDirectV2 = true; + + IntegerTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + OutStream out = writer.createStream(id, + OrcProto.Stream.Kind.DATA); + this.isDirectV2 = isNewWriteFormat(writer); + this.writer = createIntegerWriter(out, true, isDirectV2, writer); + if (inspector instanceof IntObjectInspector) { + intInspector = (IntObjectInspector) inspector; + shortInspector = null; + longInspector = null; + } else { + intInspector = null; + if (inspector instanceof LongObjectInspector) { + longInspector = (LongObjectInspector) inspector; + shortInspector = null; + } else { + shortInspector = (ShortObjectInspector) inspector; + longInspector = null; + } + } + recordPosition(rowIndexPosition); + } + + @Override + OrcProto.ColumnEncoding getEncoding() { + if (isDirectV2) { + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build(); + } + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build(); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + long val; + if (intInspector != null) { + val = intInspector.get(obj); + } else if (longInspector != null) { + val = longInspector.get(obj); + } else { + val = shortInspector.get(obj); + } + indexStatistics.updateInteger(val); + if (createBloomFilter) { + // integers are converted to longs in column statistics and during SARG evaluation + bloomFilter.addLong(val); + } + writer.write(val); + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + writer.flush(); + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + writer.getPosition(recorder); + } + } + + private static class FloatTreeWriter extends TreeWriter { + private final PositionedOutputStream stream; + private final SerializationUtils utils; + + FloatTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + this.stream = writer.createStream(id, + OrcProto.Stream.Kind.DATA); + this.utils = new SerializationUtils(); + recordPosition(rowIndexPosition); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + float val = ((FloatObjectInspector) inspector).get(obj); + indexStatistics.updateDouble(val); + if (createBloomFilter) { + // floats are converted to doubles in column statistics and during SARG evaluation + bloomFilter.addDouble(val); + } + utils.writeFloat(stream, val); + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + stream.flush(); + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + stream.getPosition(recorder); + } + } + + private static class DoubleTreeWriter extends TreeWriter { + private final PositionedOutputStream stream; + private final SerializationUtils utils; + + DoubleTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + this.stream = writer.createStream(id, + OrcProto.Stream.Kind.DATA); + this.utils = new SerializationUtils(); + recordPosition(rowIndexPosition); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + double val = ((DoubleObjectInspector) inspector).get(obj); + indexStatistics.updateDouble(val); + if (createBloomFilter) { + bloomFilter.addDouble(val); + } + utils.writeDouble(stream, val); + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + stream.flush(); + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + stream.getPosition(recorder); + } + } + + private static class StringTreeWriter extends TreeWriter { + private static final int INITIAL_DICTIONARY_SIZE = 4096; + private final OutStream stringOutput; + private final IntegerWriter lengthOutput; + private final IntegerWriter rowOutput; + private final StringRedBlackTree dictionary = + new StringRedBlackTree(INITIAL_DICTIONARY_SIZE); + private final DynamicIntArray rows = new DynamicIntArray(); + private final PositionedOutputStream directStreamOutput; + private final IntegerWriter directLengthOutput; + private final List savedRowIndex = + new ArrayList(); + private final boolean buildIndex; + private final List rowIndexValueCount = new ArrayList(); + // If the number of keys in a dictionary is greater than this fraction of + //the total number of non-null rows, turn off dictionary encoding + private final float dictionaryKeySizeThreshold; + private boolean useDictionaryEncoding = true; + private boolean isDirectV2 = true; + private boolean doneDictionaryCheck; + private final boolean strideDictionaryCheck; + + StringTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + this.isDirectV2 = isNewWriteFormat(writer); + stringOutput = writer.createStream(id, + OrcProto.Stream.Kind.DICTIONARY_DATA); + lengthOutput = createIntegerWriter(writer.createStream(id, + OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer); + rowOutput = createIntegerWriter(writer.createStream(id, + OrcProto.Stream.Kind.DATA), false, isDirectV2, writer); + recordPosition(rowIndexPosition); + rowIndexValueCount.add(0L); + buildIndex = writer.buildIndex(); + directStreamOutput = writer.createStream(id, OrcProto.Stream.Kind.DATA); + directLengthOutput = createIntegerWriter(writer.createStream(id, + OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer); + dictionaryKeySizeThreshold = writer.getConfiguration().getFloat( + HiveConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.varname, + HiveConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.defaultFloatVal); + strideDictionaryCheck = writer.getConfiguration().getBoolean( + HiveConf.ConfVars.HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK.varname, + HiveConf.ConfVars.HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK.defaultBoolVal); + doneDictionaryCheck = false; + } + + /** + * Method to retrieve text values from the value object, which can be overridden + * by subclasses. + * + * @param obj value + * @return Text text value from obj + */ + Text getTextValue(Object obj) { + return ((StringObjectInspector) inspector).getPrimitiveWritableObject(obj); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + Text val = getTextValue(obj); + if (useDictionaryEncoding || !strideDictionaryCheck) { + rows.add(dictionary.add(val)); + } else { + // write data and length + directStreamOutput.write(val.getBytes(), 0, val.getLength()); + directLengthOutput.write(val.getLength()); + } + indexStatistics.updateString(val); + if (createBloomFilter) { + bloomFilter.addBytes(val.getBytes(), val.getLength()); + } + } + } + + private boolean checkDictionaryEncoding() { + if (!doneDictionaryCheck) { + // Set the flag indicating whether or not to use dictionary encoding + // based on whether or not the fraction of distinct keys over number of + // non-null rows is less than the configured threshold + float ratio = rows.size() > 0 ? (float) (dictionary.size()) / rows.size() : 0.0f; + useDictionaryEncoding = !isDirectV2 || ratio <= dictionaryKeySizeThreshold; + doneDictionaryCheck = true; + } + return useDictionaryEncoding; + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + // if rows in stripe is less than dictionaryCheckAfterRows, dictionary + // checking would not have happened. So do it again here. + checkDictionaryEncoding(); + + if (useDictionaryEncoding) { + flushDictionary(); + } else { + // flushout any left over entries from dictionary + if (rows.size() > 0) { + flushDictionary(); + } + + // suppress the stream for every stripe if dictionary is disabled + stringOutput.suppress(); + } + + // we need to build the rowindex before calling super, since it + // writes it out. + super.writeStripe(builder, requiredIndexEntries); + stringOutput.flush(); + lengthOutput.flush(); + rowOutput.flush(); + directStreamOutput.flush(); + directLengthOutput.flush(); + // reset all of the fields to be ready for the next stripe. + dictionary.clear(); + savedRowIndex.clear(); + rowIndexValueCount.clear(); + recordPosition(rowIndexPosition); + rowIndexValueCount.add(0L); + + if (!useDictionaryEncoding) { + // record the start positions of first index stride of next stripe i.e + // beginning of the direct streams when dictionary is disabled + recordDirectStreamPosition(); + } + } + + private void flushDictionary() throws IOException { + final int[] dumpOrder = new int[dictionary.size()]; + + if (useDictionaryEncoding) { + // Write the dictionary by traversing the red-black tree writing out + // the bytes and lengths; and creating the map from the original order + // to the final sorted order. + + dictionary.visit(new StringRedBlackTree.Visitor() { + private int currentId = 0; + + @Override + public void visit(StringRedBlackTree.VisitorContext context + ) throws IOException { + context.writeBytes(stringOutput); + lengthOutput.write(context.getLength()); + dumpOrder[context.getOriginalPosition()] = currentId++; + } + }); + } else { + // for direct encoding, we don't want the dictionary data stream + stringOutput.suppress(); + } + int length = rows.size(); + int rowIndexEntry = 0; + OrcProto.RowIndex.Builder rowIndex = getRowIndex(); + Text text = new Text(); + // write the values translated into the dump order. + for (int i = 0; i <= length; ++i) { + // now that we are writing out the row values, we can finalize the + // row index + if (buildIndex) { + while (i == rowIndexValueCount.get(rowIndexEntry) && rowIndexEntry < savedRowIndex.size()) { + OrcProto.RowIndexEntry.Builder base = + savedRowIndex.get(rowIndexEntry++).toBuilder(); + if (useDictionaryEncoding) { + rowOutput.getPosition(new RowIndexPositionRecorder(base)); + } else { + PositionRecorder posn = new RowIndexPositionRecorder(base); + directStreamOutput.getPosition(posn); + directLengthOutput.getPosition(posn); + } + rowIndex.addEntry(base.build()); + } + } + if (i != length) { + if (useDictionaryEncoding) { + rowOutput.write(dumpOrder[rows.get(i)]); + } else { + dictionary.getText(text, rows.get(i)); + directStreamOutput.write(text.getBytes(), 0, text.getLength()); + directLengthOutput.write(text.getLength()); + } + } + } + rows.clear(); + } + + @Override + OrcProto.ColumnEncoding getEncoding() { + // Returns the encoding used for the last call to writeStripe + if (useDictionaryEncoding) { + if (isDirectV2) { + return OrcProto.ColumnEncoding.newBuilder().setKind( + OrcProto.ColumnEncoding.Kind.DICTIONARY_V2).setDictionarySize(dictionary.size()).build(); + } + return OrcProto.ColumnEncoding.newBuilder().setKind( + OrcProto.ColumnEncoding.Kind.DICTIONARY).setDictionarySize(dictionary.size()).build(); + } else { + if (isDirectV2) { + return OrcProto.ColumnEncoding.newBuilder().setKind( + OrcProto.ColumnEncoding.Kind.DIRECT_V2).build(); + } + return OrcProto.ColumnEncoding.newBuilder().setKind( + OrcProto.ColumnEncoding.Kind.DIRECT).build(); + } + } + + /** + * This method doesn't call the super method, because unlike most of the + * other TreeWriters, this one can't record the position in the streams + * until the stripe is being flushed. Therefore it saves all of the entries + * and augments them with the final information as the stripe is written. + * + * @throws IOException if an error occurs on create + */ + @Override + void createRowIndexEntry() throws IOException { + getStripeStatistics().merge(indexStatistics); + OrcProto.RowIndexEntry.Builder rowIndexEntry = getRowIndexEntry(); + rowIndexEntry.setStatistics(indexStatistics.serialize()); + indexStatistics.reset(); + OrcProto.RowIndexEntry base = rowIndexEntry.build(); + savedRowIndex.add(base); + rowIndexEntry.clear(); + addBloomFilterEntry(); + recordPosition(rowIndexPosition); + rowIndexValueCount.add(Long.valueOf(rows.size())); + if (strideDictionaryCheck) { + checkDictionaryEncoding(); + } + if (!useDictionaryEncoding) { + if (rows.size() > 0) { + flushDictionary(); + // just record the start positions of next index stride + recordDirectStreamPosition(); + } else { + // record the start positions of next index stride + recordDirectStreamPosition(); + getRowIndex().addEntry(base); + } + } + } + + private void recordDirectStreamPosition() throws IOException { + directStreamOutput.getPosition(rowIndexPosition); + directLengthOutput.getPosition(rowIndexPosition); + } + + @Override + long estimateMemory() { + return rows.getSizeInBytes() + dictionary.getSizeInBytes(); + } + } + + /** + * Under the covers, char is written to ORC the same way as string. + */ + private static class CharTreeWriter extends StringTreeWriter { + + CharTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + } + + /** + * Override base class implementation to support char values. + */ + @Override + Text getTextValue(Object obj) { + return (((HiveCharObjectInspector) inspector) + .getPrimitiveWritableObject(obj)).getTextValue(); + } + } + + /** + * Under the covers, varchar is written to ORC the same way as string. + */ + private static class VarcharTreeWriter extends StringTreeWriter { + + VarcharTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + } + + /** + * Override base class implementation to support varchar values. + */ + @Override + Text getTextValue(Object obj) { + return (((HiveVarcharObjectInspector) inspector) + .getPrimitiveWritableObject(obj)).getTextValue(); + } + } + + private static class BinaryTreeWriter extends TreeWriter { + private final PositionedOutputStream stream; + private final IntegerWriter length; + private boolean isDirectV2 = true; + + BinaryTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + this.stream = writer.createStream(id, + OrcProto.Stream.Kind.DATA); + this.isDirectV2 = isNewWriteFormat(writer); + this.length = createIntegerWriter(writer.createStream(id, + OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer); + recordPosition(rowIndexPosition); + } + + @Override + OrcProto.ColumnEncoding getEncoding() { + if (isDirectV2) { + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build(); + } + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build(); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + BytesWritable val = + ((BinaryObjectInspector) inspector).getPrimitiveWritableObject(obj); + stream.write(val.getBytes(), 0, val.getLength()); + length.write(val.getLength()); + indexStatistics.updateBinary(val); + if (createBloomFilter) { + bloomFilter.addBytes(val.getBytes(), val.getLength()); + } + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + stream.flush(); + length.flush(); + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + stream.getPosition(recorder); + length.getPosition(recorder); + } + } + + static final int MILLIS_PER_SECOND = 1000; + static final String BASE_TIMESTAMP_STRING = "2015-01-01 00:00:00"; + + private static class TimestampTreeWriter extends TreeWriter { + private final IntegerWriter seconds; + private final IntegerWriter nanos; + private final boolean isDirectV2; + private final long base_timestamp; + + TimestampTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + this.isDirectV2 = isNewWriteFormat(writer); + this.seconds = createIntegerWriter(writer.createStream(id, + OrcProto.Stream.Kind.DATA), true, isDirectV2, writer); + this.nanos = createIntegerWriter(writer.createStream(id, + OrcProto.Stream.Kind.SECONDARY), false, isDirectV2, writer); + recordPosition(rowIndexPosition); + // for unit tests to set different time zones + this.base_timestamp = Timestamp.valueOf(BASE_TIMESTAMP_STRING).getTime() / MILLIS_PER_SECOND; + } + + @Override + OrcProto.ColumnEncoding getEncoding() { + if (isDirectV2) { + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build(); + } + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build(); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + Timestamp val = + ((TimestampObjectInspector) inspector).getPrimitiveJavaObject(obj); + indexStatistics.updateTimestamp(val); + seconds.write((val.getTime() / MILLIS_PER_SECOND) - base_timestamp); + nanos.write(formatNanos(val.getNanos())); + if (createBloomFilter) { + bloomFilter.addLong(val.getTime()); + } + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + seconds.flush(); + nanos.flush(); + recordPosition(rowIndexPosition); + } + + private static long formatNanos(int nanos) { + if (nanos == 0) { + return 0; + } else if (nanos % 100 != 0) { + return ((long) nanos) << 3; + } else { + nanos /= 100; + int trailingZeros = 1; + while (nanos % 10 == 0 && trailingZeros < 7) { + nanos /= 10; + trailingZeros += 1; + } + return ((long) nanos) << 3 | trailingZeros; + } + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + seconds.getPosition(recorder); + nanos.getPosition(recorder); + } + } + + private static class DateTreeWriter extends TreeWriter { + private final IntegerWriter writer; + private final boolean isDirectV2; + + DateTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + OutStream out = writer.createStream(id, + OrcProto.Stream.Kind.DATA); + this.isDirectV2 = isNewWriteFormat(writer); + this.writer = createIntegerWriter(out, true, isDirectV2, writer); + recordPosition(rowIndexPosition); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + // Using the Writable here as it's used directly for writing as well as for stats. + DateWritable val = ((DateObjectInspector) inspector).getPrimitiveWritableObject(obj); + indexStatistics.updateDate(val); + writer.write(val.getDays()); + if (createBloomFilter) { + bloomFilter.addLong(val.getDays()); + } + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + writer.flush(); + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + writer.getPosition(recorder); + } + + @Override + OrcProto.ColumnEncoding getEncoding() { + if (isDirectV2) { + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build(); + } + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build(); + } + } + + private static class DecimalTreeWriter extends TreeWriter { + private final PositionedOutputStream valueStream; + private final IntegerWriter scaleStream; + private final boolean isDirectV2; + + DecimalTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + this.isDirectV2 = isNewWriteFormat(writer); + valueStream = writer.createStream(id, OrcProto.Stream.Kind.DATA); + this.scaleStream = createIntegerWriter(writer.createStream(id, + OrcProto.Stream.Kind.SECONDARY), true, isDirectV2, writer); + recordPosition(rowIndexPosition); + } + + @Override + OrcProto.ColumnEncoding getEncoding() { + if (isDirectV2) { + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build(); + } + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build(); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + HiveDecimal decimal = ((HiveDecimalObjectInspector) inspector).getPrimitiveJavaObject(obj); + if (decimal == null) { + return; + } + SerializationUtils.writeBigInteger(valueStream, + decimal.unscaledValue()); + scaleStream.write(decimal.scale()); + indexStatistics.updateDecimal(decimal); + if (createBloomFilter) { + bloomFilter.addString(decimal.toString()); + } + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + valueStream.flush(); + scaleStream.flush(); + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + valueStream.getPosition(recorder); + scaleStream.getPosition(recorder); + } + } + + private static class StructTreeWriter extends TreeWriter { + private final List fields; + + StructTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + StructObjectInspector structObjectInspector = + (StructObjectInspector) inspector; + fields = structObjectInspector.getAllStructFieldRefs(); + childrenWriters = new TreeWriter[fields.size()]; + for (int i = 0; i < childrenWriters.length; ++i) { + childrenWriters[i] = createTreeWriter( + fields.get(i).getFieldObjectInspector(), writer, true); + } + recordPosition(rowIndexPosition); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + StructObjectInspector insp = (StructObjectInspector) inspector; + for (int i = 0; i < fields.size(); ++i) { + StructField field = fields.get(i); + TreeWriter writer = childrenWriters[i]; + writer.write(insp.getStructFieldData(obj, field)); + } + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + for (TreeWriter child : childrenWriters) { + child.writeStripe(builder, requiredIndexEntries); + } + recordPosition(rowIndexPosition); + } + } + + private static class ListTreeWriter extends TreeWriter { + private final IntegerWriter lengths; + private final boolean isDirectV2; + + ListTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + this.isDirectV2 = isNewWriteFormat(writer); + ListObjectInspector listObjectInspector = (ListObjectInspector) inspector; + childrenWriters = new TreeWriter[1]; + childrenWriters[0] = + createTreeWriter(listObjectInspector.getListElementObjectInspector(), + writer, true); + lengths = createIntegerWriter(writer.createStream(columnId, + OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer); + recordPosition(rowIndexPosition); + } + + @Override + OrcProto.ColumnEncoding getEncoding() { + if (isDirectV2) { + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build(); + } + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build(); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + ListObjectInspector insp = (ListObjectInspector) inspector; + int len = insp.getListLength(obj); + lengths.write(len); + if (createBloomFilter) { + bloomFilter.addLong(len); + } + for (int i = 0; i < len; ++i) { + childrenWriters[0].write(insp.getListElement(obj, i)); + } + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + lengths.flush(); + for (TreeWriter child : childrenWriters) { + child.writeStripe(builder, requiredIndexEntries); + } + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + lengths.getPosition(recorder); + } + } + + private static class MapTreeWriter extends TreeWriter { + private final IntegerWriter lengths; + private final boolean isDirectV2; + + MapTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + this.isDirectV2 = isNewWriteFormat(writer); + MapObjectInspector insp = (MapObjectInspector) inspector; + childrenWriters = new TreeWriter[2]; + childrenWriters[0] = + createTreeWriter(insp.getMapKeyObjectInspector(), writer, true); + childrenWriters[1] = + createTreeWriter(insp.getMapValueObjectInspector(), writer, true); + lengths = createIntegerWriter(writer.createStream(columnId, + OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer); + recordPosition(rowIndexPosition); + } + + @Override + OrcProto.ColumnEncoding getEncoding() { + if (isDirectV2) { + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build(); + } + return OrcProto.ColumnEncoding.newBuilder() + .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build(); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + MapObjectInspector insp = (MapObjectInspector) inspector; + // this sucks, but it will have to do until we can get a better + // accessor in the MapObjectInspector. + Map valueMap = insp.getMap(obj); + lengths.write(valueMap.size()); + if (createBloomFilter) { + bloomFilter.addLong(valueMap.size()); + } + for (Map.Entry entry : valueMap.entrySet()) { + childrenWriters[0].write(entry.getKey()); + childrenWriters[1].write(entry.getValue()); + } + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + lengths.flush(); + for (TreeWriter child : childrenWriters) { + child.writeStripe(builder, requiredIndexEntries); + } + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + lengths.getPosition(recorder); + } + } + + private static class UnionTreeWriter extends TreeWriter { + private final RunLengthByteWriter tags; + + UnionTreeWriter(int columnId, + ObjectInspector inspector, + StreamFactory writer, + boolean nullable) throws IOException { + super(columnId, inspector, writer, nullable); + UnionObjectInspector insp = (UnionObjectInspector) inspector; + List choices = insp.getObjectInspectors(); + childrenWriters = new TreeWriter[choices.size()]; + for (int i = 0; i < childrenWriters.length; ++i) { + childrenWriters[i] = createTreeWriter(choices.get(i), writer, true); + } + tags = + new RunLengthByteWriter(writer.createStream(columnId, + OrcProto.Stream.Kind.DATA)); + recordPosition(rowIndexPosition); + } + + @Override + void write(Object obj) throws IOException { + super.write(obj); + if (obj != null) { + UnionObjectInspector insp = (UnionObjectInspector) inspector; + byte tag = insp.getTag(obj); + tags.write(tag); + if (createBloomFilter) { + bloomFilter.addLong(tag); + } + childrenWriters[tag].write(insp.getField(obj)); + } + } + + @Override + void writeStripe(OrcProto.StripeFooter.Builder builder, + int requiredIndexEntries) throws IOException { + super.writeStripe(builder, requiredIndexEntries); + tags.flush(); + for (TreeWriter child : childrenWriters) { + child.writeStripe(builder, requiredIndexEntries); + } + recordPosition(rowIndexPosition); + } + + @Override + void recordPosition(PositionRecorder recorder) throws IOException { + super.recordPosition(recorder); + tags.getPosition(recorder); + } + } + + private static TreeWriter createTreeWriter(ObjectInspector inspector, + StreamFactory streamFactory, + boolean nullable) throws IOException { + switch (inspector.getCategory()) { + case PRIMITIVE: + switch (((PrimitiveObjectInspector) inspector).getPrimitiveCategory()) { + case BOOLEAN: + return new BooleanTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case BYTE: + return new ByteTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case SHORT: + case INT: + case LONG: + return new IntegerTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case FLOAT: + return new FloatTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case DOUBLE: + return new DoubleTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case STRING: + return new StringTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case CHAR: + return new CharTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case VARCHAR: + return new VarcharTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case BINARY: + return new BinaryTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case TIMESTAMP: + return new TimestampTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case DATE: + return new DateTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + case DECIMAL: + return new DecimalTreeWriter(streamFactory.getNextColumnId(), + inspector, streamFactory, nullable); + default: + throw new IllegalArgumentException("Bad primitive category " + + ((PrimitiveObjectInspector) inspector).getPrimitiveCategory()); + } + case STRUCT: + return new StructTreeWriter(streamFactory.getNextColumnId(), inspector, + streamFactory, nullable); + case MAP: + return new MapTreeWriter(streamFactory.getNextColumnId(), inspector, + streamFactory, nullable); + case LIST: + return new ListTreeWriter(streamFactory.getNextColumnId(), inspector, + streamFactory, nullable); + case UNION: + return new UnionTreeWriter(streamFactory.getNextColumnId(), inspector, + streamFactory, nullable); + default: + throw new IllegalArgumentException("Bad category: " + + inspector.getCategory()); + } + } + + private static void writeTypes(OrcProto.Footer.Builder builder, + TreeWriter treeWriter) { + OrcProto.Type.Builder type = OrcProto.Type.newBuilder(); + switch (treeWriter.inspector.getCategory()) { + case PRIMITIVE: + switch (((PrimitiveObjectInspector) treeWriter.inspector).getPrimitiveCategory()) { + case BOOLEAN: + type.setKind(OrcProto.Type.Kind.BOOLEAN); + break; + case BYTE: + type.setKind(OrcProto.Type.Kind.BYTE); + break; + case SHORT: + type.setKind(OrcProto.Type.Kind.SHORT); + break; + case INT: + type.setKind(OrcProto.Type.Kind.INT); + break; + case LONG: + type.setKind(OrcProto.Type.Kind.LONG); + break; + case FLOAT: + type.setKind(OrcProto.Type.Kind.FLOAT); + break; + case DOUBLE: + type.setKind(OrcProto.Type.Kind.DOUBLE); + break; + case STRING: + type.setKind(OrcProto.Type.Kind.STRING); + break; + case CHAR: + // The char length needs to be written to file and should be available + // from the object inspector + CharTypeInfo charTypeInfo = (CharTypeInfo) ((PrimitiveObjectInspector) treeWriter.inspector).getTypeInfo(); + type.setKind(Type.Kind.CHAR); + type.setMaximumLength(charTypeInfo.getLength()); + break; + case VARCHAR: + // The varchar length needs to be written to file and should be available + // from the object inspector + VarcharTypeInfo typeInfo = (VarcharTypeInfo) ((PrimitiveObjectInspector) treeWriter.inspector).getTypeInfo(); + type.setKind(Type.Kind.VARCHAR); + type.setMaximumLength(typeInfo.getLength()); + break; + case BINARY: + type.setKind(OrcProto.Type.Kind.BINARY); + break; + case TIMESTAMP: + type.setKind(OrcProto.Type.Kind.TIMESTAMP); + break; + case DATE: + type.setKind(OrcProto.Type.Kind.DATE); + break; + case DECIMAL: + DecimalTypeInfo decTypeInfo = (DecimalTypeInfo) ((PrimitiveObjectInspector) treeWriter.inspector).getTypeInfo(); + type.setKind(OrcProto.Type.Kind.DECIMAL); + type.setPrecision(decTypeInfo.precision()); + type.setScale(decTypeInfo.scale()); + break; + default: + throw new IllegalArgumentException("Unknown primitive category: " + + ((PrimitiveObjectInspector) treeWriter.inspector).getPrimitiveCategory()); + } + break; + case LIST: + type.setKind(OrcProto.Type.Kind.LIST); + type.addSubtypes(treeWriter.childrenWriters[0].id); + break; + case MAP: + type.setKind(OrcProto.Type.Kind.MAP); + type.addSubtypes(treeWriter.childrenWriters[0].id); + type.addSubtypes(treeWriter.childrenWriters[1].id); + break; + case STRUCT: + type.setKind(OrcProto.Type.Kind.STRUCT); + for (TreeWriter child : treeWriter.childrenWriters) { + type.addSubtypes(child.id); + } + for (StructField field : ((StructTreeWriter) treeWriter).fields) { + type.addFieldNames(field.getFieldName()); + } + break; + case UNION: + type.setKind(OrcProto.Type.Kind.UNION); + for (TreeWriter child : treeWriter.childrenWriters) { + type.addSubtypes(child.id); + } + break; + default: + throw new IllegalArgumentException("Unknown category: " + treeWriter.inspector.getCategory()); + } + builder.addTypes(type); + for (TreeWriter child : treeWriter.childrenWriters) { + writeTypes(builder, child); + } + } + + @VisibleForTesting + public OutputStream getStream() throws IOException { + if (rawWriter == null) { + rawWriter = new ByteCountingOutputStream(flowFileOutputStream); + rawWriter.write(OrcFile.MAGIC.getBytes()); + headerLength = rawWriter.getBytesWritten(); + writer = new OutStream("metadata", bufferSize, codec, + new DirectStream(rawWriter)); + protobufWriter = CodedOutputStream.newInstance(writer); + } + return rawWriter; + } + + private void createRowIndexEntry() throws IOException { + treeWriter.createRowIndexEntry(); + rowsInIndex = 0; + } + + private void flushStripe() throws IOException { + getStream(); + if (buildIndex && rowsInIndex != 0) { + createRowIndexEntry(); + } + if (rowsInStripe != 0) { + if (callback != null) { + callback.preStripeWrite(callbackContext); + } + // finalize the data for the stripe + int requiredIndexEntries = rowIndexStride == 0 ? 0 : + (int) ((rowsInStripe + rowIndexStride - 1) / rowIndexStride); + OrcProto.StripeFooter.Builder builder = + OrcProto.StripeFooter.newBuilder(); + treeWriter.writeStripe(builder, requiredIndexEntries); + long indexSize = 0; + long dataSize = 0; + for (Map.Entry pair : streams.entrySet()) { + BufferedStream stream = pair.getValue(); + if (!stream.isSuppressed()) { + stream.flush(); + StreamName name = pair.getKey(); + long streamSize = pair.getValue().getOutputSize(); + builder.addStreams(OrcProto.Stream.newBuilder() + .setColumn(name.getColumn()) + .setKind(name.getKind()) + .setLength(streamSize)); + if (StreamName.Area.INDEX == name.getArea()) { + indexSize += streamSize; + } else { + dataSize += streamSize; + } + } + } + OrcProto.StripeFooter footer = builder.build(); + + // Do we need to pad the file so the stripe doesn't straddle a block + // boundary? + long start = rawWriter.getBytesWritten(); + final long currentStripeSize = indexSize + dataSize + footer.getSerializedSize(); + final long available = blockSize - (start % blockSize); + final long overflow = currentStripeSize - adjustedStripeSize; + final float availRatio = (float) available / (float) defaultStripeSize; + + if (availRatio > 0.0f && availRatio < 1.0f + && availRatio > paddingTolerance) { + // adjust default stripe size to fit into remaining space, also adjust + // the next stripe for correction based on the current stripe size + // and user specified padding tolerance. Since stripe size can overflow + // the default stripe size we should apply this correction to avoid + // writing portion of last stripe to next hdfs block. + float correction = overflow > 0 ? (float) overflow + / (float) adjustedStripeSize : 0.0f; + + // correction should not be greater than user specified padding + // tolerance + correction = correction > paddingTolerance ? paddingTolerance + : correction; + + // adjust next stripe size based on current stripe estimate correction + adjustedStripeSize = (long) ((1.0f - correction) * (availRatio * defaultStripeSize)); + } else if (availRatio >= 1.0) { + adjustedStripeSize = defaultStripeSize; + } + + if (availRatio < paddingTolerance && addBlockPadding) { + long padding = blockSize - (start % blockSize); + byte[] pad = new byte[(int) Math.min(HDFS_BUFFER_SIZE, padding)]; + LOG.info(String.format("Padding ORC by %d bytes (<= %.2f * %d)", + padding, availRatio, defaultStripeSize)); + start += padding; + while (padding > 0) { + int writeLen = (int) Math.min(padding, pad.length); + rawWriter.write(pad, 0, writeLen); + padding -= writeLen; + } + adjustedStripeSize = defaultStripeSize; + } else if (currentStripeSize < blockSize + && (start % blockSize) + currentStripeSize > blockSize) { + // even if you don't pad, reset the default stripe size when crossing a + // block boundary + adjustedStripeSize = defaultStripeSize; + } + + // write out the data streams + for (Map.Entry pair : streams.entrySet()) { + BufferedStream stream = pair.getValue(); + if (!stream.isSuppressed()) { + stream.spillTo(rawWriter); + } + stream.clear(); + } + footer.writeTo(protobufWriter); + protobufWriter.flush(); + writer.flush(); + long footerLength = rawWriter.getBytesWritten() - start - dataSize - indexSize; + OrcProto.StripeInformation dirEntry = + OrcProto.StripeInformation.newBuilder() + .setOffset(start) + .setNumberOfRows(rowsInStripe) + .setIndexLength(indexSize) + .setDataLength(dataSize) + .setFooterLength(footerLength).build(); + stripes.add(dirEntry); + rowCount += rowsInStripe; + rowsInStripe = 0; + } + } + + private long computeRawDataSize() { + long result = 0; + for (TreeWriter child : treeWriter.getChildrenWriters()) { + result += getRawDataSizeFromInspectors(child, child.inspector); + } + return result; + } + + private long getRawDataSizeFromInspectors(TreeWriter child, ObjectInspector oi) { + long total = 0; + switch (oi.getCategory()) { + case PRIMITIVE: + total += getRawDataSizeFromPrimitives(child, oi); + break; + case LIST: + case MAP: + case UNION: + case STRUCT: + for (TreeWriter tw : child.childrenWriters) { + total += getRawDataSizeFromInspectors(tw, tw.inspector); + } + break; + default: + LOG.debug("Unknown object inspector category."); + break; + } + return total; + } + + private long getRawDataSizeFromPrimitives(TreeWriter child, ObjectInspector oi) { + long result = 0; + long numVals = child.fileStatistics.getNumberOfValues(); + switch (((PrimitiveObjectInspector) oi).getPrimitiveCategory()) { + case BOOLEAN: + case BYTE: + case SHORT: + case INT: + case FLOAT: + return numVals * JavaDataModel.get().primitive1(); + case LONG: + case DOUBLE: + return numVals * JavaDataModel.get().primitive2(); + case STRING: + case VARCHAR: + case CHAR: + // ORC strings are converted to java Strings. so use JavaDataModel to + // compute the overall size of strings + child = (StringTreeWriter) child; + StringColumnStatistics scs = (StringColumnStatistics) child.fileStatistics; + numVals = numVals == 0 ? 1 : numVals; + int avgStringLen = (int) (scs.getSum() / numVals); + return numVals * JavaDataModel.get().lengthForStringOfLength(avgStringLen); + case DECIMAL: + return numVals * JavaDataModel.get().lengthOfDecimal(); + case DATE: + return numVals * JavaDataModel.get().lengthOfDate(); + case BINARY: + // get total length of binary blob + BinaryColumnStatistics bcs = (BinaryColumnStatistics) child.fileStatistics; + return bcs.getSum(); + case TIMESTAMP: + return numVals * JavaDataModel.get().lengthOfTimestamp(); + default: + LOG.debug("Unknown primitive category."); + break; + } + + return result; + } + + private OrcProto.CompressionKind writeCompressionKind(CompressionKind kind) { + switch (kind) { + case NONE: + return OrcProto.CompressionKind.NONE; + case ZLIB: + return OrcProto.CompressionKind.ZLIB; + case SNAPPY: + return OrcProto.CompressionKind.SNAPPY; + case LZO: + return OrcProto.CompressionKind.LZO; + default: + throw new IllegalArgumentException("Unknown compression " + kind); + } + } + + private void writeFileStatistics(OrcProto.Footer.Builder builder, + TreeWriter writer) throws IOException { + builder.addStatistics(writer.fileStatistics.serialize()); + for (TreeWriter child : writer.getChildrenWriters()) { + writeFileStatistics(builder, child); + } + } + + private int writeMetadata() throws IOException { + getStream(); + OrcProto.Metadata.Builder builder = OrcProto.Metadata.newBuilder(); + for (OrcProto.StripeStatistics.Builder ssb : treeWriter.stripeStatsBuilders) { + builder.addStripeStats(ssb.build()); + } + + long startPosn = rawWriter.getBytesWritten(); + OrcProto.Metadata metadata = builder.build(); + metadata.writeTo(protobufWriter); + protobufWriter.flush(); + writer.flush(); + return (int) (rawWriter.getBytesWritten() - startPosn); + } + + private int writeFooter(long bodyLength) throws IOException { + getStream(); + OrcProto.Footer.Builder builder = OrcProto.Footer.newBuilder(); + builder.setContentLength(bodyLength); + builder.setHeaderLength(headerLength); + builder.setNumberOfRows(rowCount); + builder.setRowIndexStride(rowIndexStride); + // populate raw data size + rawDataSize = computeRawDataSize(); + // serialize the types + writeTypes(builder, treeWriter); + // add the stripe information + for (OrcProto.StripeInformation stripe : stripes) { + builder.addStripes(stripe); + } + // add the column statistics + writeFileStatistics(builder, treeWriter); + // add all of the user metadata + for (Map.Entry entry : userMetadata.entrySet()) { + builder.addMetadata(OrcProto.UserMetadataItem.newBuilder() + .setName(entry.getKey()).setValue(entry.getValue())); + } + long startPosn = rawWriter.getBytesWritten(); + OrcProto.Footer footer = builder.build(); + footer.writeTo(protobufWriter); + protobufWriter.flush(); + writer.flush(); + return (int) (rawWriter.getBytesWritten() - startPosn); + } + + private int writePostScript(int footerLength, int metadataLength) throws IOException { + OrcProto.PostScript.Builder builder = + OrcProto.PostScript.newBuilder() + .setCompression(writeCompressionKind(compress)) + .setFooterLength(footerLength) + .setMetadataLength(metadataLength) + .setMagic(OrcFile.MAGIC) + .addVersion(version.getMajor()) + .addVersion(version.getMinor()) + .setWriterVersion(OrcFile.WriterVersion.HIVE_8732.getId()); + if (compress != CompressionKind.NONE) { + builder.setCompressionBlockSize(bufferSize); + } + OrcProto.PostScript ps = builder.build(); + // need to write this uncompressed + long startPosn = rawWriter.getBytesWritten(); + ps.writeTo(rawWriter); + long length = rawWriter.getBytesWritten() - startPosn; + if (length > 255) { + throw new IllegalArgumentException("PostScript too large at " + length); + } + return (int) length; + } + + private long estimateStripeSize() { + long result = 0; + for (BufferedStream stream : streams.values()) { + result += stream.getBufferSize(); + } + result += treeWriter.estimateMemory(); + return result; + } + + @Override + public synchronized void addUserMetadata(String name, ByteBuffer value) { + userMetadata.put(name, ByteString.copyFrom(value)); + } + + @Override + public void addRow(Object row) throws IOException { + synchronized (this) { + treeWriter.write(row); + rowsInStripe += 1; + if (buildIndex) { + rowsInIndex += 1; + + if (rowsInIndex >= rowIndexStride) { + createRowIndexEntry(); + } + } + } + memoryManager.addedRow(); + } + + public void addRowBatch(VectorizedRowBatch batch) throws IOException { + if (buildIndex) { + // Batch the writes up to the rowIndexStride so that we can get the + // right size indexes. + int posn = 0; + while (posn < batch.size) { + int chunkSize = Math.min(batch.size - posn, + rowIndexStride - rowsInIndex); + treeWriter.writeRootBatch(batch, posn, chunkSize); + posn += chunkSize; + rowsInIndex += chunkSize; + rowsInStripe += chunkSize; + if (rowsInIndex >= rowIndexStride) { + createRowIndexEntry(); + } + } + } else { + rowsInStripe += batch.size; + treeWriter.writeRootBatch(batch, 0, batch.size); + } + memoryManager.addedRow(); + } + + + @Override + public void close() throws IOException { + if (callback != null) { + callback.preFooterWrite(callbackContext); + } + // remove us from the memory manager so that we don't get any callbacks + memoryManager.removeWriter(path); + // actually close the file + flushStripe(); + int metadataLength = writeMetadata(); + int footerLength = writeFooter(rawWriter.getBytesWritten() - metadataLength); + rawWriter.write(writePostScript(footerLength, metadataLength)); + rawWriter.close(); + + } + + /** + * Raw data size will be compute when writing the file footer. Hence raw data + * size value will be available only after closing the writer. + */ + @Override + public long getRawDataSize() { + return rawDataSize; + } + + /** + * Row count gets updated when flushing the stripes. To get accurate row + * count call this method after writer is closed. + */ + @Override + public long getNumberOfRows() { + return rowCount; + } + + @Override + public long writeIntermediateFooter() throws IOException { + // flush any buffered rows + flushStripe(); + // write a footer + if (stripesAtLastFlush != stripes.size()) { + if (callback != null) { + callback.preFooterWrite(callbackContext); + } + int metaLength = writeMetadata(); + int footLength = writeFooter(rawWriter.getBytesWritten() - metaLength); + rawWriter.write(writePostScript(footLength, metaLength)); + stripesAtLastFlush = stripes.size(); + rawWriter.flush(); + } + return rawWriter.getBytesWritten(); + } + + @Override + public void appendStripe(byte[] stripe, int offset, int length, + StripeInformation stripeInfo, + OrcProto.StripeStatistics stripeStatistics) throws IOException { + checkArgument(stripe != null, "Stripe must not be null"); + checkArgument(length <= stripe.length, + "Specified length must not be greater specified array length"); + checkArgument(stripeInfo != null, "Stripe information must not be null"); + checkArgument(stripeStatistics != null, + "Stripe statistics must not be null"); + + getStream(); + long start = rawWriter.getBytesWritten(); + long stripeLen = length; + long availBlockSpace = blockSize - (start % blockSize); + + // see if stripe can fit in the current hdfs block, else pad the remaining + // space in the block + if (stripeLen < blockSize && stripeLen > availBlockSpace && addBlockPadding) { + byte[] pad = new byte[(int) Math.min(HDFS_BUFFER_SIZE, availBlockSpace)]; + LOG.info(String.format("Padding ORC by %d bytes while merging..", + availBlockSpace)); + start += availBlockSpace; + while (availBlockSpace > 0) { + int writeLen = (int) Math.min(availBlockSpace, pad.length); + rawWriter.write(pad, 0, writeLen); + availBlockSpace -= writeLen; + } + } + + rawWriter.write(stripe); + rowsInStripe = stripeStatistics.getColStats(0).getNumberOfValues(); + rowCount += rowsInStripe; + + // since we have already written the stripe, just update stripe statistics + treeWriter.stripeStatsBuilders.add(stripeStatistics.toBuilder()); + + // update file level statistics + updateFileStatistics(stripeStatistics); + + // update stripe information + OrcProto.StripeInformation dirEntry = OrcProto.StripeInformation + .newBuilder() + .setOffset(start) + .setNumberOfRows(rowsInStripe) + .setIndexLength(stripeInfo.getIndexLength()) + .setDataLength(stripeInfo.getDataLength()) + .setFooterLength(stripeInfo.getFooterLength()) + .build(); + stripes.add(dirEntry); + + // reset it after writing the stripe + rowsInStripe = 0; + } + + private void updateFileStatistics(OrcProto.StripeStatistics stripeStatistics) { + List cs = stripeStatistics.getColStatsList(); + List allWriters = getAllColumnTreeWriters(treeWriter); + for (int i = 0; i < allWriters.size(); i++) { + allWriters.get(i).fileStatistics.merge(ColumnStatisticsImpl.deserialize(cs.get(i))); + } + } + + private List getAllColumnTreeWriters(TreeWriter rootTreeWriter) { + List result = Lists.newArrayList(); + getAllColumnTreeWritersImpl(rootTreeWriter, result); + return result; + } + + private void getAllColumnTreeWritersImpl(TreeWriter tw, + List result) { + result.add(tw); + for (TreeWriter child : tw.childrenWriters) { + getAllColumnTreeWritersImpl(child, result); + } + } + + @Override + public void appendUserMetadata(List userMetadata) { + if (userMetadata != null) { + for (UserMetadataItem item : userMetadata) { + this.userMetadata.put(item.getName(), item.getValue()); + } + } + } +} diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java new file mode 100644 index 000000000000..6100bd112cba --- /dev/null +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java @@ -0,0 +1,289 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +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.OrcFlowFileWriter; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +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.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.util.hive.HiveJdbcCommon; +import org.apache.nifi.util.hive.HiveUtils; +import org.apache.hadoop.hive.ql.io.orc.NiFiOrcUtils; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +/** + * The ConvertAvroToORC processor takes an Avro-formatted flow file as input and converts it into ORC format. + */ +@SideEffectFree +@SupportsBatching +@Tags({"avro", "orc", "hive", "convert"}) +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +@CapabilityDescription("Converts an Avro record into ORC file format. This processor provides a direct mapping of an Avro record to an ORC record, such " + + "that the resulting ORC file will have the same hierarchical structure as the Avro document. If an incoming FlowFile contains a stream of " + + "multiple Avro records, the resultant FlowFile will contain a ORC file containing all of the Avro records. If an incoming FlowFile does " + + "not contain any records, an empty ORC file is the output.") +@WritesAttributes({ + @WritesAttribute(attribute = "mime.type", description = "Sets the mime type to application/octet-stream"), + @WritesAttribute(attribute = "filename", description = "Sets the filename to the existing filename with the extension replaced by / added to by .orc"), + @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the ORC file."), + @WritesAttribute(attribute = "hive.ddl", description = "Creates a partial Hive DDL statement for creating a table in Hive from this ORC file. " + + "This can be used in ReplaceText for setting the content to the DDL. To make it valid DDL, add \"LOCATION ''\", where " + + "the path is the directory that contains this ORC file on HDFS. For example, ConvertAvroToORC can send flow files to a PutHDFS processor to send the file to " + + "HDFS, then to a 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.") +}) +public class ConvertAvroToORC extends AbstractProcessor { + + // Attributes + public static final String ORC_MIME_TYPE = "application/octet-stream"; + public static final String HIVE_DDL_ATTRIBUTE = "hive.ddl"; + public static final String RECORD_COUNT_ATTRIBUTE = "record.count"; + + + // Properties + public static final PropertyDescriptor ORC_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder() + .name("orc-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("orc-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("100 KB") + .build(); + + public static final PropertyDescriptor BUFFER_SIZE = new PropertyDescriptor.Builder() + .name("orc-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(); + + public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder() + .name("orc-compression-type") + .displayName("Compression Type") + .required(true) + .allowableValues("NONE", "ZLIB", "SNAPPY", "LZO") + .defaultValue("NONE") + .build(); + + public static final PropertyDescriptor HIVE_TABLE_NAME = new PropertyDescriptor.Builder() + .name("orc-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 PutHiveQL 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(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .build(); + + // Relationships + static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile is routed to this relationship after it has been converted to ORC format.") + .build(); + static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFile is routed to this relationship if it cannot be parsed as Avro or cannot be converted to ORC for any reason") + .build(); + + private final static List propertyDescriptors; + private final static Set relationships; + + private volatile Configuration orcConfig; + + /* + * Will ensure that the list of property descriptors is built only once. + * Will also create a Set of relationships + */ + static { + List _propertyDescriptors = new ArrayList<>(); + _propertyDescriptors.add(ORC_CONFIGURATION_RESOURCES); + _propertyDescriptors.add(STRIPE_SIZE); + _propertyDescriptors.add(BUFFER_SIZE); + _propertyDescriptors.add(COMPRESSION_TYPE); + _propertyDescriptors.add(HIVE_TABLE_NAME); + propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors); + + Set _relationships = new HashSet<>(); + _relationships.add(REL_SUCCESS); + _relationships.add(REL_FAILURE); + relationships = Collections.unmodifiableSet(_relationships); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return propertyDescriptors; + } + + @Override + public Set getRelationships() { + return relationships; + } + + @OnScheduled + public void setup(ProcessContext context) { + boolean confFileProvided = context.getProperty(ORC_CONFIGURATION_RESOURCES).isSet(); + if (confFileProvided) { + final String configFiles = context.getProperty(ORC_CONFIGURATION_RESOURCES).getValue(); + orcConfig = HiveJdbcCommon.getConfigurationFromFiles(configFiles); + } + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + try { + long startTime = System.currentTimeMillis(); + 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 AtomicReference hiveAvroSchema = new AtomicReference<>(null); + final AtomicInteger totalRecordCount = new AtomicInteger(0); + final String fileName = flowFile.getAttribute(CoreAttributes.FILENAME.key()); + flowFile = session.write(flowFile, (rawIn, rawOut) -> { + try (final InputStream in = new BufferedInputStream(rawIn); + final OutputStream out = new BufferedOutputStream(rawOut); + final DataFileStream reader = new DataFileStream<>(in, new GenericDatumReader<>())) { + + // Create ORC schema from Avro schema + Schema avroSchema = reader.getSchema(); + + TypeInfo orcSchema = NiFiOrcUtils.getOrcField(avroSchema); + + if (orcConfig == null) { + orcConfig = new Configuration(); + } + + OrcFlowFileWriter orcWriter = NiFiOrcUtils.createWriter( + out, + new Path(fileName), + orcConfig, + orcSchema, + stripeSize, + compressionType, + bufferSize); + try { + + int recordCount = 0; + GenericRecord currRecord = null; + while (reader.hasNext()) { + currRecord = reader.next(currRecord); + List fields = currRecord.getSchema().getFields(); + if (fields != null) { + Object[] row = new Object[fields.size()]; + for (int i = 0; i < fields.size(); i++) { + Schema.Field field = fields.get(i); + Schema fieldSchema = field.schema(); + Object o = currRecord.get(field.name()); + try { + row[i] = NiFiOrcUtils.convertToORCObject(NiFiOrcUtils.getOrcField(fieldSchema), o); + } catch (ArrayIndexOutOfBoundsException aioobe) { + getLogger().error("Index out of bounds at record {} for column {}, type {}, and object {}", + new Object[]{recordCount, i, fieldSchema.getType().getName(), o.toString()}, + aioobe); + throw new IOException(aioobe); + } + } + orcWriter.addRow(NiFiOrcUtils.createOrcStruct(orcSchema, row)); + recordCount++; + } + } + hiveAvroSchema.set(avroSchema); + totalRecordCount.set(recordCount); + } finally { + // finished writing this record, close the writer (which will flush to the flow file) + orcWriter.close(); + } + } + }); + + final String hiveTableName = context.getProperty(HIVE_TABLE_NAME).isSet() + ? context.getProperty(HIVE_TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue() + : NiFiOrcUtils.normalizeHiveTableName(hiveAvroSchema.get().getFullName()); + String hiveDDL = NiFiOrcUtils.generateHiveDDL(hiveAvroSchema.get(), hiveTableName); + // Add attributes and transfer to success + flowFile = session.putAttribute(flowFile, RECORD_COUNT_ATTRIBUTE, Integer.toString(totalRecordCount.get())); + flowFile = session.putAttribute(flowFile, HIVE_DDL_ATTRIBUTE, hiveDDL); + StringBuilder newFilename = new StringBuilder(); + int extensionIndex = fileName.lastIndexOf("."); + if (extensionIndex != -1) { + newFilename.append(fileName.substring(0, extensionIndex)); + } else { + newFilename.append(fileName); + } + newFilename.append(".orc"); + flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), ORC_MIME_TYPE); + flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), newFilename.toString()); + session.transfer(flowFile, REL_SUCCESS); + session.getProvenanceReporter().modifyContent(flowFile, "Converted "+totalRecordCount.get()+" records", System.currentTimeMillis() - startTime); + + } catch (final ProcessException pe) { + getLogger().error("Failed to convert {} from Avro to ORC due to {}; transferring to failure", new Object[]{flowFile, pe}); + session.transfer(flowFile, REL_FAILURE); + } + } +} diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor index 55ee005f7187..6f5c1217c9be 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -12,6 +12,7 @@ # WITHOUT 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.ConvertAvroToORC org.apache.nifi.processors.hive.SelectHiveQL org.apache.nifi.processors.hive.PutHiveQL org.apache.nifi.processors.hive.PutHiveStreaming diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java new file mode 100644 index 000000000000..ad40198d24a9 --- /dev/null +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.io.DatumWriter; +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.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.io.DoubleWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.apache.nifi.util.orc.TestNiFiOrcUtils; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.FileOutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.TreeMap; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + + +/** + * Unit tests for ConvertAvroToORC processor + */ +public class TestConvertAvroToORC { + + private ConvertAvroToORC processor; + private TestRunner runner; + + @Before + public void setUp() throws Exception { + processor = new ConvertAvroToORC(); + runner = TestRunners.newTestRunner(processor); + } + + @Test + public void test_Setup() throws Exception { + + } + + @Test + public void test_onTrigger_primitive_record() throws Exception { + GenericData.Record record = TestNiFiOrcUtils.buildPrimitiveAvroRecord(10, 20L, true, 30.0f, 40, StandardCharsets.UTF_8.encode("Hello"), "World"); + + DatumWriter writer = new GenericDatumWriter<>(record.getSchema()); + DataFileWriter fileWriter = new DataFileWriter<>(writer); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + fileWriter.create(record.getSchema(), out); + fileWriter.append(record); + // Put another record in + record = TestNiFiOrcUtils.buildPrimitiveAvroRecord(1, 2L, false, 3.0f, 4L, StandardCharsets.UTF_8.encode("I am"), "another record"); + fileWriter.append(record); + // And one more + record = TestNiFiOrcUtils.buildPrimitiveAvroRecord(100, 200L, true, 300.0f, 400L, StandardCharsets.UTF_8.encode("Me"), "too!"); + fileWriter.append(record); + fileWriter.flush(); + fileWriter.close(); + out.close(); + Map attributes = new HashMap() {{ + put(CoreAttributes.FILENAME.key(), "test.avro"); + }}; + runner.enqueue(out.toByteArray(), attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ConvertAvroToORC.REL_SUCCESS, 1); + + // Write the flow file out to disk, since the ORC Reader needs a path + MockFlowFile resultFlowFile = runner.getFlowFilesForRelationship(ConvertAvroToORC.REL_SUCCESS).get(0); + assertEquals("CREATE EXTERNAL TABLE IF NOT EXISTS test_record (int INT, long BIGINT, boolean BOOLEAN, float FLOAT, double DOUBLE, bytes BINARY, string STRING)" + + " STORED AS ORC", resultFlowFile.getAttribute(ConvertAvroToORC.HIVE_DDL_ATTRIBUTE)); + assertEquals("3", resultFlowFile.getAttribute(ConvertAvroToORC.RECORD_COUNT_ATTRIBUTE)); + assertEquals("test.orc", resultFlowFile.getAttribute(CoreAttributes.FILENAME.key())); + byte[] resultContents = runner.getContentAsByteArray(resultFlowFile); + FileOutputStream fos = new FileOutputStream("target/test1.orc"); + fos.write(resultContents); + fos.flush(); + fos.close(); + + Configuration conf = new Configuration(); + FileSystem fs = FileSystem.getLocal(conf); + Reader reader = OrcFile.createReader(new Path("target/test1.orc"), OrcFile.readerOptions(conf).filesystem(fs)); + RecordReader rows = reader.rows(); + Object o = rows.next(null); + assertNotNull(o); + assertTrue(o instanceof OrcStruct); + TypeInfo resultSchema = TestNiFiOrcUtils.buildPrimitiveOrcSchema(); + StructObjectInspector inspector = (StructObjectInspector) OrcStruct.createObjectInspector(resultSchema); + + // Check some fields in the first row + Object intFieldObject = inspector.getStructFieldData(o, inspector.getStructFieldRef("int")); + assertTrue(intFieldObject instanceof IntWritable); + assertEquals(10, ((IntWritable) intFieldObject).get()); + Object stringFieldObject = inspector.getStructFieldData(o, inspector.getStructFieldRef("string")); + assertTrue(stringFieldObject instanceof Text); + assertEquals("World", stringFieldObject.toString()); + + } + + @Test + public void test_onTrigger_complex_record() throws Exception { + + Map mapData1 = new TreeMap() {{ + put("key1", 1.0); + put("key2", 2.0); + }}; + + GenericData.Record record = TestNiFiOrcUtils.buildComplexAvroRecord(10, mapData1, "DEF", 3.0f, Arrays.asList(10, 20)); + + DatumWriter writer = new GenericDatumWriter<>(record.getSchema()); + DataFileWriter fileWriter = new DataFileWriter<>(writer); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + fileWriter.create(record.getSchema(), out); + fileWriter.append(record); + + // Put another record in + Map mapData2 = new TreeMap() {{ + put("key1", 3.0); + put("key2", 4.0); + }}; + + record = TestNiFiOrcUtils.buildComplexAvroRecord(null, mapData2, "XYZ", 4L, Arrays.asList(100, 200)); + fileWriter.append(record); + + fileWriter.flush(); + fileWriter.close(); + out.close(); + + Map attributes = new HashMap() {{ + put(CoreAttributes.FILENAME.key(), "test"); + }}; + runner.enqueue(out.toByteArray(), attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ConvertAvroToORC.REL_SUCCESS, 1); + + // Write the flow file out to disk, since the ORC Reader needs a path + MockFlowFile resultFlowFile = runner.getFlowFilesForRelationship(ConvertAvroToORC.REL_SUCCESS).get(0); + assertEquals("CREATE EXTERNAL TABLE IF NOT EXISTS complex_record " + + "(myInt INT, myMap MAP, myEnum STRING, myLongOrFloat UNIONTYPE, myIntList ARRAY)" + + " STORED AS ORC", resultFlowFile.getAttribute(ConvertAvroToORC.HIVE_DDL_ATTRIBUTE)); + assertEquals("2", resultFlowFile.getAttribute(ConvertAvroToORC.RECORD_COUNT_ATTRIBUTE)); + assertEquals("test.orc", resultFlowFile.getAttribute(CoreAttributes.FILENAME.key())); + byte[] resultContents = runner.getContentAsByteArray(resultFlowFile); + FileOutputStream fos = new FileOutputStream("target/test1.orc"); + fos.write(resultContents); + fos.flush(); + fos.close(); + + Configuration conf = new Configuration(); + FileSystem fs = FileSystem.getLocal(conf); + Reader reader = OrcFile.createReader(new Path("target/test1.orc"), OrcFile.readerOptions(conf).filesystem(fs)); + RecordReader rows = reader.rows(); + Object o = rows.next(null); + assertNotNull(o); + assertTrue(o instanceof OrcStruct); + TypeInfo resultSchema = TestNiFiOrcUtils.buildComplexOrcSchema(); + StructObjectInspector inspector = (StructObjectInspector) OrcStruct.createObjectInspector(resultSchema); + + // Check some fields in the first row + Object intFieldObject = inspector.getStructFieldData(o, inspector.getStructFieldRef("myInt")); + assertTrue(intFieldObject instanceof IntWritable); + assertEquals(10, ((IntWritable) intFieldObject).get()); + + // This is pretty awkward and messy. The map object is a Map (not a MapWritable) but the keys are writables (in this case Text) + // and so are the values (DoubleWritables in this case). + Object mapFieldObject = inspector.getStructFieldData(o, inspector.getStructFieldRef("myMap")); + assertTrue(mapFieldObject instanceof Map); + Map map = (Map) mapFieldObject; + Object mapValue = map.get(new Text("key1")); + assertNotNull(mapValue); + assertTrue(mapValue instanceof DoubleWritable); + assertEquals(1.0, ((DoubleWritable) mapValue).get(), Double.MIN_VALUE); + + mapValue = map.get(new Text("key2")); + assertNotNull(mapValue); + assertTrue(mapValue instanceof DoubleWritable); + assertEquals(2.0, ((DoubleWritable) mapValue).get(), Double.MIN_VALUE); + + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveStreaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveStreaming.java index 3876e74c24a2..f95a37766e54 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveStreaming.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveStreaming.java @@ -684,12 +684,12 @@ public void abort() throws StreamingException, TxnBatchFailure, InterruptedExcep @Override protected void closeConnection() throws InterruptedException { - // TODO + // Empty } @Override protected void commitTxn() throws CommitFailure, InterruptedException { - // TODO + // Empty } @Override @@ -700,17 +700,17 @@ protected TransactionBatch nextTxnBatch(RecordWriter recordWriter) throws Interr @Override protected void closeTxnBatch() throws InterruptedException { - // TODO + // Empty } @Override protected void abortTxn() throws InterruptedException { - // TODO + // Empty } @Override protected void nextTxn(boolean rollToNext) throws StreamingException, InterruptedException, TxnBatchFailure { - // TODO + // Empty } } diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java new file mode 100644 index 000000000000..556723c13e6f --- /dev/null +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java @@ -0,0 +1,371 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.hadoop.hive.ql.io.orc.NiFiOrcUtils; +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.MapWritable; +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() throws Exception { + // 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 fields = testSchema.getFields(); + for (int i = 0; i < fields.size(); i++) { + assertEquals(expectedTypes[i], NiFiOrcUtils.getOrcField(fields.get(i).schema())); + } + + } + + @Test + public void test_getOrcField_union_optional_type() throws Exception { + final SchemaBuilder.FieldAssembler 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()); + assertEquals(TypeInfoCreator.createBoolean(), orcType); + } + + @Test + public void test_getOrcField_union() throws Exception { + final SchemaBuilder.FieldAssembler 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()); + assertEquals( + TypeInfoFactory.getUnionTypeInfo(Arrays.asList( + TypeInfoCreator.createInt(), + TypeInfoCreator.createBoolean())), + orcType); + } + + @Test + public void test_getOrcField_map() throws Exception { + final SchemaBuilder.FieldAssembler 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()); + assertEquals( + TypeInfoFactory.getMapTypeInfo( + TypeInfoCreator.createString(), + TypeInfoCreator.createDouble()), + orcType); + } + + @Test + public void test_getOrcField_nested_map() throws Exception { + final SchemaBuilder.FieldAssembler 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()); + assertEquals( + TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(), + TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(), TypeInfoCreator.createDouble())), + orcType); + } + + @Test + public void test_getOrcField_array() throws Exception { + final SchemaBuilder.FieldAssembler 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()); + assertEquals( + TypeInfoFactory.getListTypeInfo(TypeInfoCreator.createLong()), + orcType); + } + + @Test + public void test_getOrcField_complex_array() throws Exception { + final SchemaBuilder.FieldAssembler 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()); + assertEquals( + TypeInfoFactory.getListTypeInfo(TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(), TypeInfoCreator.createFloat())), + orcType); + } + + @Test + public void test_getOrcField_record() throws Exception { + final SchemaBuilder.FieldAssembler 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(); + TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema); + 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() throws Exception { + final SchemaBuilder.FieldAssembler 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()); + assertEquals(TypeInfoCreator.createString(), orcType); + } + + @Test + public void test_getPrimitiveOrcTypeFromPrimitiveAvroType() throws Exception { + // Expected ORC types + TypeInfo[] expectedTypes = { + TypeInfoCreator.createInt(), + TypeInfoCreator.createLong(), + TypeInfoCreator.createBoolean(), + TypeInfoCreator.createFloat(), + TypeInfoCreator.createDouble(), + TypeInfoCreator.createBinary(), + TypeInfoCreator.createString(), + }; + + Schema testSchema = buildPrimitiveAvroSchema(); + List 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() throws Exception { + Schema.Type nonPrimitiveType = Schema.Type.ARRAY; + NiFiOrcUtils.getPrimitiveOrcTypeFromPrimitiveAvroType(nonPrimitiveType); + } + + @Test + public void test_getWritable() throws Exception { + assertTrue(NiFiOrcUtils.convertToORCObject(null, 1) instanceof IntWritable); + assertTrue(NiFiOrcUtils.convertToORCObject(null, 1L) instanceof LongWritable); + assertTrue(NiFiOrcUtils.convertToORCObject(null, 1.0f) instanceof FloatWritable); + assertTrue(NiFiOrcUtils.convertToORCObject(null, 1.0) instanceof DoubleWritable); + assertTrue(NiFiOrcUtils.convertToORCObject(null, new int[]{1, 2, 3}) instanceof List); + assertTrue(NiFiOrcUtils.convertToORCObject(null, Arrays.asList(1, 2, 3)) instanceof List); + Map map = new HashMap<>(); + map.put("Hello", 1.0f); + map.put("World", 2.0f); + + Object writable = NiFiOrcUtils.convertToORCObject(TypeInfoUtils.getTypeInfoFromTypeString("map"), map); + assertTrue(writable instanceof MapWritable); + MapWritable mapWritable = (MapWritable) writable; + mapWritable.forEach((key, value) -> { + assertTrue(key instanceof Text); + assertTrue(value instanceof FloatWritable); + }); + } + + @Test + public void test_getHiveTypeFromAvroType_primitive() throws Exception { + // Expected ORC types + String[] expectedTypes = { + "INT", + "BIGINT", + "BOOLEAN", + "FLOAT", + "DOUBLE", + "BINARY", + "STRING", + }; + + Schema testSchema = buildPrimitiveAvroSchema(); + List fields = testSchema.getFields(); + for (int i = 0; i < fields.size(); i++) { + assertEquals(expectedTypes[i], NiFiOrcUtils.getHiveTypeFromAvroType(fields.get(i).schema())); + } + } + + @Test + public void test_getHiveTypeFromAvroType_complex() throws Exception { + // Expected ORC types + String[] expectedTypes = { + "INT", + "MAP", + "STRING", + "UNIONTYPE", + "ARRAY" + }; + + Schema testSchema = buildComplexAvroSchema(); + List fields = testSchema.getFields(); + for (int i = 0; i < fields.size(); i++) { + assertEquals(expectedTypes[i], NiFiOrcUtils.getHiveTypeFromAvroType(fields.get(i).schema())); + } + + assertEquals("STRUCT, myEnum:STRING, myLongOrFloat:UNIONTYPE, myIntList:ARRAY>", + NiFiOrcUtils.getHiveTypeFromAvroType(testSchema)); + } + + @Test + public void test_generateHiveDDL_primitive() throws Exception { + Schema avroSchema = buildPrimitiveAvroSchema(); + String ddl = NiFiOrcUtils.generateHiveDDL(avroSchema, "myHiveTable"); + 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() throws Exception { + Schema avroSchema = buildComplexAvroSchema(); + String ddl = NiFiOrcUtils.generateHiveDDL(avroSchema, "myHiveTable"); + assertEquals("CREATE EXTERNAL TABLE IF NOT EXISTS myHiveTable " + + "(myInt INT, myMap MAP, myEnum STRING, myLongOrFloat UNIONTYPE, myIntList ARRAY)" + + " STORED AS ORC", ddl); + } + + + ////////////////// + // Helper methods + ////////////////// + + public static Schema buildPrimitiveAvroSchema() { + // Build a fake Avro record with all primitive types + final SchemaBuilder.FieldAssembler 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 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 m, String e, Object unionVal, List 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,myEnum:string,myLongOrFloat:uniontype,myIntList:array>"); + } + + + 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 From e2ac392d1de1221ae7ff308cf27034595c669575 Mon Sep 17 00:00:00 2001 From: Matt Burgess Date: Thu, 4 Aug 2016 09:59:21 -0400 Subject: [PATCH 2/3] NIFI-1663: Updated NifiOrcUtils with review comments --- .../hadoop/hive/ql/io/orc/NiFiOrcUtils.java | 14 ++++++++--- .../nifi/util/orc/TestNiFiOrcUtils.java | 23 +++++++++++++++++++ 2 files changed, 34 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java index 78430bf9c6fc..e936e045e496 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java @@ -165,7 +165,8 @@ public static Object convertToORCObject(TypeInfo typeInfo, Object o) { }); return mapWritable; } - + // The object is not null but isn't of any recognized type either, so throw an exception + throw new IllegalArgumentException("Object of type " + o.getClass().getName() + " cannot be converted to an ORC object"); } return null; } @@ -179,12 +180,19 @@ public static Object convertToORCObject(TypeInfo typeInfo, Object o) { * @return an OrcStruct containing the specified objects for the specified schema */ public static OrcStruct createOrcStruct(TypeInfo typeInfo, Object... objs) { + if (objs == null) { + throw new IllegalArgumentException("No objects provided to createOrcStruct"); + } SettableStructObjectInspector oi = (SettableStructObjectInspector) OrcStruct .createObjectInspector(typeInfo); List fields = (List) oi.getAllStructFieldRefs(); + int numFields = fields.size(); + if (numFields != objs.length) { + throw new IllegalArgumentException("Number of fields in ORC struct " + numFields + " does not match number of provided objects " + objs.length); + } OrcStruct result = (OrcStruct) oi.create(); - result.setNumFields(fields.size()); - for (int i = 0; i < fields.size(); i++) { + result.setNumFields(numFields); + for (int i = 0; i < numFields; i++) { oi.setStructFieldData(result, fields.get(i), objs[i]); } return result; diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java index 556723c13e6f..9a7e96252578 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java @@ -21,6 +21,9 @@ import org.apache.avro.SchemaBuilder; import org.apache.avro.generic.GenericData; import org.apache.hadoop.hive.ql.io.orc.NiFiOrcUtils; +import org.apache.hadoop.hive.ql.io.orc.OrcStruct; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; @@ -269,6 +272,26 @@ public void test_generateHiveDDL_complex() throws Exception { + " STORED AS ORC", ddl); } + @Test + public void test_createOrcStruct_primitive() throws Exception { + TypeInfo primitiveOrcSchema = buildPrimitiveOrcSchema(); + OrcStruct orcStruct = NiFiOrcUtils.createOrcStruct(primitiveOrcSchema, 4, 5L, Boolean.TRUE, 2.0f, 3.0, ByteBuffer.allocate(100), "a string"); + ObjectInspector objectInspector = OrcStruct.createObjectInspector(primitiveOrcSchema); + assertTrue(objectInspector instanceof SettableStructObjectInspector); + SettableStructObjectInspector orcStructInspector = (SettableStructObjectInspector) objectInspector; + assertEquals(4, orcStructInspector.getStructFieldData(orcStruct, orcStructInspector.getStructFieldRef("int"))); + assertEquals(5L, orcStructInspector.getStructFieldData(orcStruct, orcStructInspector.getStructFieldRef("long"))); + assertTrue((boolean) orcStructInspector.getStructFieldData(orcStruct, orcStructInspector.getStructFieldRef("boolean"))); + assertEquals(2.0f, orcStructInspector.getStructFieldData(orcStruct, orcStructInspector.getStructFieldRef("float"))); + assertEquals(3.0, orcStructInspector.getStructFieldData(orcStruct, orcStructInspector.getStructFieldRef("double"))); + assertEquals("a string", orcStructInspector.getStructFieldData(orcStruct, orcStructInspector.getStructFieldRef("string"))); + } + + @Test(expected = IllegalArgumentException.class) + public void test_createOrcStruct_primitive_not_enough_objects() throws Exception { + TypeInfo primitiveOrcSchema = buildPrimitiveOrcSchema(); + NiFiOrcUtils.createOrcStruct(primitiveOrcSchema, 1); + } ////////////////// // Helper methods From dc6e40f795db218261992a1a1b42565cfda58de8 Mon Sep 17 00:00:00 2001 From: Matt Burgess Date: Mon, 8 Aug 2016 20:50:02 -0400 Subject: [PATCH 3/3] NIFI-1663: Added support to ConvertAvroToORC for nested records, added unit tests --- .../hadoop/hive/ql/io/orc/NiFiOrcUtils.java | 68 +++++----- .../processors/hive/ConvertAvroToORC.java | 7 +- .../hive/ql/io}/orc/TestNiFiOrcUtils.java | 77 +++++++++-- .../processors/hive/TestConvertAvroToORC.java | 128 +++++++++++++++++- 4 files changed, 233 insertions(+), 47 deletions(-) rename nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/{nifi/util => hadoop/hive/ql/io}/orc/TestNiFiOrcUtils.java (83%) diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java index e936e045e496..f1ad6e97aff3 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java @@ -30,6 +30,7 @@ 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.PrimitiveTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; @@ -67,12 +68,23 @@ public class NiFiOrcUtils { public static Object convertToORCObject(TypeInfo typeInfo, Object o) { if (o != null) { if (typeInfo instanceof UnionTypeInfo) { - OrcUnion union = new OrcUnion(); - // Need to find which of the union types correspond to the primitive object - TypeInfo objectTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector( - ObjectInspectorFactory.getReflectionObjectInspector(o.getClass(), ObjectInspectorFactory.ObjectInspectorOptions.JAVA)); + + // Check the union types, if its not a primitive type then we can't (yet) support unions containing them. List unionTypeInfos = ((UnionTypeInfo) typeInfo).getAllUnionObjectTypeInfos(); + for (TypeInfo info : unionTypeInfos) { + if (!(info instanceof PrimitiveTypeInfo)) { + throw new IllegalArgumentException("Elements in unions must be primitive types, instead found " + info.getTypeName()); + } + } + OrcUnion union = new OrcUnion(); + + // If this is an Avro Utf8 object, use its String representation so we get a good type to match against the union type + Class objClass = (o instanceof Utf8) ? String.class : o.getClass(); + ObjectInspector objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(objClass, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + TypeInfo objectTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(objectInspector); + + // Need to find which of the union types correspond to the primitive object int index = 0; while (index < unionTypeInfos.size() && !unionTypeInfos.get(index).equals(objectTypeInfo)) { index++; @@ -146,10 +158,22 @@ public static Object convertToORCObject(TypeInfo typeInfo, Object o) { if (o instanceof List) { return o; } + if (o instanceof GenericData.Record) { + // For records we need to recursively convert the fields to ORC/Writable fields and create an OrcStruct from them + GenericData.Record record = (GenericData.Record) o; + int numFields = record.getSchema().getFields().size(); + Object[] fieldObjects = new Object[numFields]; + for (int i = 0; i < numFields; i++) { + Schema objSchema = record.getSchema().getFields().get(i).schema(); + fieldObjects[i] = convertToORCObject(getOrcField(objSchema), record.get(i)); + } + + return createOrcStruct(typeInfo, fieldObjects); + } if (o instanceof Map) { MapWritable mapWritable = new MapWritable(); TypeInfo keyInfo = ((MapTypeInfo) typeInfo).getMapKeyTypeInfo(); - TypeInfo valueInfo = ((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) -> { @@ -224,7 +248,13 @@ public static String generateHiveDDL(Schema avroSchema, String tableName) { public static TypeInfo getOrcField(Schema fieldSchema) throws IllegalArgumentException { + if (fieldSchema == null) { + throw new IllegalArgumentException("Field schema cannot be null"); + } Schema.Type fieldType = fieldSchema.getType(); + if (fieldType == null) { + throw new IllegalArgumentException("Field schema has no type"); + } switch (fieldType) { case INT: @@ -287,34 +317,6 @@ public static TypeInfo getOrcField(Schema fieldSchema) throws IllegalArgumentExc } - 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"); diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java index 6100bd112cba..7881065fd6f1 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java @@ -70,7 +70,8 @@ @CapabilityDescription("Converts an Avro record into ORC file format. This processor provides a direct mapping of an Avro record to an ORC record, such " + "that the resulting ORC file will have the same hierarchical structure as the Avro document. If an incoming FlowFile contains a stream of " + "multiple Avro records, the resultant FlowFile will contain a ORC file containing all of the Avro records. If an incoming FlowFile does " - + "not contain any records, an empty ORC file is the output.") + + "not contain any records, an empty ORC file is the output. Please note that not all complex/nested structures are supported (such as maps of " + + "unions, unions of lists, etc.)") @WritesAttributes({ @WritesAttribute(attribute = "mime.type", description = "Sets the mime type to application/octet-stream"), @WritesAttribute(attribute = "filename", description = "Sets the filename to the existing filename with the extension replaced by / added to by .orc"), @@ -119,6 +120,8 @@ public class ConvertAvroToORC extends AbstractProcessor { public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder() .name("orc-compression-type") .displayName("Compression Type") + .description("The type of compression used to reduce the size of the outgoing ORC file. Note that some types (LZO, e.g.) may not be available " + + "on the system, in which case an error will occur if such a type is selected.") .required(true) .allowableValues("NONE", "ZLIB", "SNAPPY", "LZO") .defaultValue("NONE") @@ -279,7 +282,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), ORC_MIME_TYPE); flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), newFilename.toString()); session.transfer(flowFile, REL_SUCCESS); - session.getProvenanceReporter().modifyContent(flowFile, "Converted "+totalRecordCount.get()+" records", System.currentTimeMillis() - startTime); + session.getProvenanceReporter().modifyContent(flowFile, "Converted " + totalRecordCount.get() + " records", System.currentTimeMillis() - startTime); } catch (final ProcessException pe) { getLogger().error("Failed to convert {} from Avro to ORC due to {}; transferring to failure", new Object[]{flowFile, pe}); diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/hadoop/hive/ql/io/orc/TestNiFiOrcUtils.java similarity index 83% rename from nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java rename to nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/hadoop/hive/ql/io/orc/TestNiFiOrcUtils.java index 9a7e96252578..b6b9cecb81ea 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/util/orc/TestNiFiOrcUtils.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/hadoop/hive/ql/io/orc/TestNiFiOrcUtils.java @@ -14,16 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.util.orc; +package org.apache.hadoop.hive.ql.io.orc; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; import org.apache.avro.generic.GenericData; -import org.apache.hadoop.hive.ql.io.orc.NiFiOrcUtils; -import org.apache.hadoop.hive.ql.io.orc.OrcStruct; +import org.apache.avro.util.Utf8; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; @@ -118,6 +118,39 @@ public void test_getOrcField_nested_map() throws Exception { orcType); } + @Test(expected = IllegalArgumentException.class) + public void test_convertToORCObject_map_of_unions() throws Exception { + final SchemaBuilder.FieldAssembler builder = SchemaBuilder.record("testRecord").namespace("any.data").fields(); + builder.name("map").type().map().values().unionOf().stringType().and().intType().endUnion().noDefault(); + Schema testSchema = builder.endRecord(); + TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("map").schema()); + assertEquals( + TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(), + TypeInfoFactory.getUnionTypeInfo(Arrays.asList(TypeInfoCreator.createString(), TypeInfoCreator.createInt()))), + orcType); + + Map record1 = new HashMap<>(); + record1.put("record1", new Utf8("Hello")); + NiFiOrcUtils.convertToORCObject(orcType, record1); + } + + @Test(expected = IllegalArgumentException.class) + public void test_convertToORCObject_union_of_complex_types() throws Exception { + final SchemaBuilder.FieldAssembler builder = SchemaBuilder.record("testRecord").namespace("any.data").fields(); + builder.name("union").type().unionOf().map().values().intType().and().floatType().endUnion().noDefault(); + Schema testSchema = builder.endRecord(); + TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("union").schema()); + assertEquals( + TypeInfoFactory.getUnionTypeInfo(Arrays.asList( + TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(), TypeInfoCreator.createInt()), + TypeInfoCreator.createFloat())), + orcType); + + Map record1 = new HashMap<>(); + record1.put("record1", new Utf8("Hello")); + NiFiOrcUtils.convertToORCObject(orcType, record1); + } + @Test public void test_getOrcField_array() throws Exception { final SchemaBuilder.FieldAssembler builder = SchemaBuilder.record("testRecord").namespace("any.data").fields(); @@ -293,6 +326,28 @@ public void test_createOrcStruct_primitive_not_enough_objects() throws Exception NiFiOrcUtils.createOrcStruct(primitiveOrcSchema, 1); } + @Test + public void test_createOrcStruct_map_with_union() { + + final SchemaBuilder.FieldAssembler builder = SchemaBuilder.record("complex.record").namespace("any.data").fields(); + builder.name("myMap").type().map().values().unionOf().floatType().and().stringType().endUnion().noDefault(); + final Schema schema = builder.endRecord(); + + TypeInfo orcSchema = TypeInfoUtils.getTypeInfoFromTypeString("struct>>"); + + Map myMap = new HashMap<>(); + myMap.put("x", "hello"); + myMap.put("y", 1.0f); + + GenericData.Record record = new GenericData.Record(schema); + record.put("myMap", myMap); + OrcStruct orcStruct = NiFiOrcUtils.createOrcStruct(orcSchema, myMap); + StructObjectInspector inspector = (StructObjectInspector) OrcStruct.createObjectInspector(NiFiOrcUtils.getOrcField(schema)); + Map resultMap = (Map) inspector.getStructFieldData(orcStruct, inspector.getStructFieldRef("myMap")); + assertEquals("hello", resultMap.get("x")); + assertEquals(1.0f, (float) resultMap.get("y"), Float.MIN_VALUE); + } + ////////////////// // Helper methods ////////////////// @@ -362,32 +417,32 @@ public static TypeInfo buildComplexOrcSchema() { } - private static class TypeInfoCreator { - static TypeInfo createInt() { + public static class TypeInfoCreator { + public static TypeInfo createInt() { return TypeInfoFactory.getPrimitiveTypeInfo("int"); } - static TypeInfo createLong() { + public static TypeInfo createLong() { return TypeInfoFactory.getPrimitiveTypeInfo("bigint"); } - static TypeInfo createBoolean() { + public static TypeInfo createBoolean() { return TypeInfoFactory.getPrimitiveTypeInfo("boolean"); } - static TypeInfo createFloat() { + public static TypeInfo createFloat() { return TypeInfoFactory.getPrimitiveTypeInfo("float"); } - static TypeInfo createDouble() { + public static TypeInfo createDouble() { return TypeInfoFactory.getPrimitiveTypeInfo("double"); } - static TypeInfo createBinary() { + public static TypeInfo createBinary() { return TypeInfoFactory.getPrimitiveTypeInfo("binary"); } - static TypeInfo createString() { + public static TypeInfo createString() { return TypeInfoFactory.getPrimitiveTypeInfo("string"); } } diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java index ad40198d24a9..8c0c196ba99f 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java @@ -16,6 +16,7 @@ */ package org.apache.nifi.processors.hive; +import org.apache.avro.Schema; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumWriter; @@ -23,12 +24,15 @@ 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.NiFiOrcUtils; 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.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.io.DoubleWritable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; @@ -36,7 +40,7 @@ import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; -import org.apache.nifi.util.orc.TestNiFiOrcUtils; +import org.apache.hadoop.hive.ql.io.orc.TestNiFiOrcUtils; import org.junit.Before; import org.junit.Test; @@ -45,6 +49,7 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -209,6 +214,127 @@ record = TestNiFiOrcUtils.buildComplexAvroRecord(null, mapData2, "XYZ", 4L, Arra assertNotNull(mapValue); assertTrue(mapValue instanceof DoubleWritable); assertEquals(2.0, ((DoubleWritable) mapValue).get(), Double.MIN_VALUE); + } + + @Test + public void test_array_of_records() throws Exception { + + Schema testSchema = new Schema.Parser().parse("{\n" + + " \"namespace\" : \"org.apache.nifi\",\n" + + " \"name\" : \"outer_record\",\n" + + " \"type\" : \"record\",\n" + + " \"fields\" : [ {\n" + + " \"name\" : \"records\",\n" + + " \"type\" : {\n" + + " \"type\" : \"array\",\n" + + " \"items\" : {\n" + + " \"type\" : \"record\",\n" + + " \"name\" : \"records_44\",\n" + + " \"fields\" : [ {\n" + + " \"name\" : \"name\",\n" + + " \"type\" : \"string\"\n" + + " }, {\n" + + " \"name\" : \"age\",\n" + + " \"type\" : \"double\"\n" + + " } ]\n" + + " }\n" + + " }\n" + + " } ]\n" + + "}\n"); + + // Verify the ORC schema (generated from the Avro schema) is correct + TypeInfo arrayType = NiFiOrcUtils.getOrcField(testSchema.getField("records").schema()); + assertEquals( + TypeInfoFactory.getListTypeInfo( + TypeInfoFactory.getStructTypeInfo( + Arrays.asList("name", "age"), + Arrays.asList(TestNiFiOrcUtils.TypeInfoCreator.createString(), TestNiFiOrcUtils.TypeInfoCreator.createDouble()))), + arrayType); + + // Create records to put in an array + GenericData.Record record1 = new GenericData.Record(testSchema.getField("records").schema().getElementType()); + record1.put("name", "X1"); + record1.put("age", 2.0); + + GenericData.Record record2 = new GenericData.Record(testSchema.getField("records").schema().getElementType()); + record2.put("name", "X2"); + record2.put("age", 4.0); + + // Create an array then a record to contain them + GenericData.Array array = new GenericData.Array<>(2, testSchema.getField("records").schema()); + array.add(record1); + array.add(record2); + GenericData.Record record = new GenericData.Record(testSchema); + record.put(0, array); + + // Write the Avro record to a byte stream (to become the flow file contents) + DatumWriter writer = new GenericDatumWriter<>(record.getSchema()); + DataFileWriter fileWriter = new DataFileWriter<>(writer); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + fileWriter.create(record.getSchema(), out); + fileWriter.append(record); + fileWriter.flush(); + fileWriter.close(); + out.close(); + + Map attributes = new HashMap() {{ + put(CoreAttributes.FILENAME.key(), "test"); + }}; + runner.enqueue(out.toByteArray(), attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ConvertAvroToORC.REL_SUCCESS, 1); + + // Write the flow file out to disk, since the ORC Reader needs a path + MockFlowFile resultFlowFile = runner.getFlowFilesForRelationship(ConvertAvroToORC.REL_SUCCESS).get(0); + assertEquals("CREATE EXTERNAL TABLE IF NOT EXISTS org_apache_nifi_outer_record (records ARRAY>) STORED AS ORC", + resultFlowFile.getAttribute(ConvertAvroToORC.HIVE_DDL_ATTRIBUTE)); + assertEquals("1", resultFlowFile.getAttribute(ConvertAvroToORC.RECORD_COUNT_ATTRIBUTE)); + assertEquals("test.orc", resultFlowFile.getAttribute(CoreAttributes.FILENAME.key())); + byte[] resultContents = runner.getContentAsByteArray(resultFlowFile); + FileOutputStream fos = new FileOutputStream("target/test1.orc"); + fos.write(resultContents); + fos.flush(); + fos.close(); + + Configuration conf = new Configuration(); + FileSystem fs = FileSystem.getLocal(conf); + Reader reader = OrcFile.createReader(new Path("target/test1.orc"), OrcFile.readerOptions(conf).filesystem(fs)); + RecordReader rows = reader.rows(); + Object o = rows.next(null); + assertNotNull(o); + assertTrue(o instanceof OrcStruct); + + // Validate the structure and contents + StructObjectInspector inspector = (StructObjectInspector) OrcStruct.createObjectInspector(NiFiOrcUtils.getOrcField(testSchema)); + + // Check some fields in the first row + Object arrayFieldObj = inspector.getStructFieldData(o, inspector.getStructFieldRef("records")); + assertTrue(arrayFieldObj instanceof List); + List arrayField = (List) arrayFieldObj; + assertEquals(2, arrayField.size()); + + // Look at each inner record + StructObjectInspector recordInspector = (StructObjectInspector) OrcStruct.createObjectInspector(((ListTypeInfo) arrayType).getListElementTypeInfo()); + + Object record1Obj = arrayField.get(0); + assertNotNull(record1Obj); + assertTrue(record1Obj instanceof OrcStruct); + Object nameFieldObject = recordInspector.getStructFieldData(record1Obj, recordInspector.getStructFieldRef("name")); + assertTrue(nameFieldObject instanceof Text); + assertEquals("X1", nameFieldObject.toString()); + Object ageFieldObject = recordInspector.getStructFieldData(record1Obj, recordInspector.getStructFieldRef("age")); + assertTrue(ageFieldObject instanceof DoubleWritable); + assertEquals(2.0, ((DoubleWritable) ageFieldObject).get(), Double.MIN_VALUE); + Object record2Obj = arrayField.get(1); + assertNotNull(record2Obj); + assertTrue(record2Obj instanceof OrcStruct); + nameFieldObject = recordInspector.getStructFieldData(record2Obj, recordInspector.getStructFieldRef("name")); + assertTrue(nameFieldObject instanceof Text); + assertEquals("X2", nameFieldObject.toString()); + ageFieldObject = recordInspector.getStructFieldData(record2Obj, recordInspector.getStructFieldRef("age")); + assertTrue(ageFieldObject instanceof DoubleWritable); + assertEquals(4.0, ((DoubleWritable) ageFieldObject).get(), Double.MIN_VALUE); } } \ No newline at end of file