diff --git llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java index 7ee263d..baaa4d7 100644 --- llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java +++ llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.llap.io.decode; import java.io.IOException; +import java.util.List; import org.apache.hadoop.hive.common.io.encoded.EncodedColumnBatch; import org.apache.hadoop.hive.common.io.encoded.EncodedColumnBatch.ColumnStreamData; @@ -27,7 +28,12 @@ import org.apache.hadoop.hive.llap.io.metadata.OrcFileMetadata; import org.apache.hadoop.hive.llap.io.metadata.OrcStripeMetadata; import org.apache.hadoop.hive.llap.metrics.LlapDaemonQueueMetrics; +import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.orc.CompressionCodec; import org.apache.hadoop.hive.ql.io.orc.encoded.Consumer; @@ -71,6 +77,35 @@ public void setStripeMetadata(OrcStripeMetadata m) { stripes[m.getStripeIx()] = m; } + private static ColumnVector createColumn(OrcProto.Type type, + int batchSize) { + switch (type.getKind()) { + case BOOLEAN: + case BYTE: + case SHORT: + case INT: + case LONG: + case DATE: + return new LongColumnVector(batchSize); + case FLOAT: + case DOUBLE: + return new DoubleColumnVector(batchSize); + case BINARY: + case STRING: + case CHAR: + case VARCHAR: + return new BytesColumnVector(batchSize); + case TIMESTAMP: + return new TimestampColumnVector(batchSize); + case DECIMAL: + return new DecimalColumnVector(batchSize, type.getPrecision(), + type.getScale()); + default: + throw new IllegalArgumentException("LLAP does not support " + + type.getKind()); + } + } + @Override protected void decodeBatch(OrcEncodedColumnBatch batch, Consumer downstreamConsumer) { @@ -112,9 +147,15 @@ protected void decodeBatch(OrcEncodedColumnBatch batch, ColumnVectorBatch cvb = cvbPool.take(); assert cvb.cols.length == batch.getColumnIxs().length; // Must be constant per split. cvb.size = batchSize; - + List types = fileMetadata.getTypes(); + int[] columnMapping = batch.getColumnIxs(); for (int idx = 0; idx < batch.getColumnIxs().length; idx++) { - cvb.cols[idx] = (ColumnVector)columnReaders[idx].nextVector(cvb.cols[idx], batchSize); + if (cvb.cols[idx] == null) { + // skip over the top level struct, but otherwise assume no complex + // types + cvb.cols[idx] = createColumn(types.get(columnMapping[idx]), batchSize); + } + columnReaders[idx].nextVector(cvb.cols[idx], null, batchSize); } // we are done reading a batch, send it to consumer for processing diff --git orc/src/java/org/apache/orc/OrcUtils.java orc/src/java/org/apache/orc/OrcUtils.java index 2e93254..2ebe9a7 100644 --- orc/src/java/org/apache/orc/OrcUtils.java +++ orc/src/java/org/apache/orc/OrcUtils.java @@ -449,4 +449,79 @@ public static int appendOrcTypesRebuildSubtypes(List result, return columnId; } + /** + * Translate the given rootColumn from the list of types to a TypeDescription. + * @param types all of the types + * @param rootColumn translate this type + * @return a new TypeDescription that matches the given rootColumn + */ + public static + TypeDescription convertTypeFromProtobuf(List types, + int rootColumn) { + OrcProto.Type type = types.get(rootColumn); + switch (type.getKind()) { + case BOOLEAN: + return TypeDescription.createBoolean(); + case BYTE: + return TypeDescription.createByte(); + case SHORT: + return TypeDescription.createShort(); + case INT: + return TypeDescription.createInt(); + case LONG: + return TypeDescription.createLong(); + case FLOAT: + return TypeDescription.createFloat(); + case DOUBLE: + return TypeDescription.createDouble(); + case STRING: + return TypeDescription.createString(); + case CHAR: + return TypeDescription.createChar() + .withMaxLength(type.getMaximumLength()); + case VARCHAR: + return TypeDescription.createVarchar() + .withMaxLength(type.getMaximumLength()); + case BINARY: + return TypeDescription.createBinary(); + case TIMESTAMP: + return TypeDescription.createTimestamp(); + case DATE: + return TypeDescription.createDate(); + case DECIMAL: { + TypeDescription result = TypeDescription.createDecimal(); + if (type.hasScale()) { + result.withScale(type.getScale()); + } + if (type.hasPrecision()) { + result.withPrecision(type.getPrecision()); + } + return result; + } + case LIST: + return TypeDescription.createList( + convertTypeFromProtobuf(types, type.getSubtypes(0))); + case MAP: + return TypeDescription.createMap( + convertTypeFromProtobuf(types, type.getSubtypes(0)), + convertTypeFromProtobuf(types, type.getSubtypes(1))); + case STRUCT: { + TypeDescription result = TypeDescription.createStruct(); + for(int f=0; f < type.getSubtypesCount(); ++f) { + result.addField(type.getFieldNames(f), + convertTypeFromProtobuf(types, type.getSubtypes(f))); + } + return result; + } + case UNION: { + TypeDescription result = TypeDescription.createUnion(); + for(int f=0; f < type.getSubtypesCount(); ++f) { + result.addUnionChild( + convertTypeFromProtobuf(types, type.getSubtypes(f))); + } + return result; + } + } + throw new IllegalArgumentException("Unknown ORC type " + type.getKind()); + } } diff --git orc/src/java/org/apache/orc/Reader.java orc/src/java/org/apache/orc/Reader.java index be722b5..62a05e9 100644 --- orc/src/java/org/apache/orc/Reader.java +++ orc/src/java/org/apache/orc/Reader.java @@ -116,9 +116,15 @@ ColumnStatistics[] getStatistics(); /** + * Get the type of rows in this ORC file. + */ + TypeDescription getSchema(); + + /** * Get the list of types contained in the file. The root type is the first * type in the list. * @return the list of flattened types + * @deprecated use getSchema instead */ List getTypes(); diff --git orc/src/java/org/apache/orc/RecordReader.java orc/src/java/org/apache/orc/RecordReader.java index 7229dda..09ba0f0 100644 --- orc/src/java/org/apache/orc/RecordReader.java +++ orc/src/java/org/apache/orc/RecordReader.java @@ -30,13 +30,11 @@ * controlled by the callers. Caller need to look at * VectorizedRowBatch.size of the retunred object to know the batch * size read. - * @param previousBatch a row batch object that can be reused by the reader - * @return the row batch that was read. The batch will have a non-zero row - * count if the pointer isn't at the end of the file + * @param batch a row batch object to read into + * @return were more rows available to read? * @throws java.io.IOException */ - VectorizedRowBatch nextBatch(VectorizedRowBatch previousBatch - ) throws IOException; + boolean nextBatch(VectorizedRowBatch batch) throws IOException; /** * Get the row number of the row that will be returned by the following diff --git orc/src/java/org/apache/orc/TypeDescription.java orc/src/java/org/apache/orc/TypeDescription.java index bd900ac..b8e057e 100644 --- orc/src/java/org/apache/orc/TypeDescription.java +++ orc/src/java/org/apache/orc/TypeDescription.java @@ -61,7 +61,7 @@ LIST("array", false), MAP("map", false), STRUCT("struct", false), - UNION("union", false); + UNION("uniontype", false); Category(String name, boolean isPrimitive) { this.name = name; @@ -258,6 +258,66 @@ public int getId() { return id; } + public TypeDescription clone() { + TypeDescription result = new TypeDescription(category); + result.maxLength = maxLength; + result.precision = precision; + result.scale = scale; + if (fieldNames != null) { + result.fieldNames.addAll(fieldNames); + } + if (children != null) { + for(TypeDescription child: children) { + TypeDescription clone = child.clone(); + clone.parent = result; + result.children.add(clone); + } + } + return result; + } + + @Override + public int hashCode() { + return getId(); + } + + @Override + public boolean equals(Object other) { + if (other == null || other.getClass() != TypeDescription.class) { + return false; + } + if (other == this) { + return true; + } + TypeDescription castOther = (TypeDescription) other; + if (category != castOther.category || + getId() != castOther.getId() || + getMaximumId() != castOther.getMaximumId() || + maxLength != castOther.maxLength || + scale != castOther.scale || + precision != castOther.precision) { + return false; + } + if (children != null) { + if (children.size() != castOther.children.size()) { + return false; + } + for (int i = 0; i < children.size(); ++i) { + if (!children.get(i).equals(castOther.children.get(i))) { + return false; + } + } + } + if (category == Category.STRUCT) { + for(int i=0; i < fieldNames.size(); ++i) { + if (!fieldNames.get(i).equals(castOther.fieldNames.get(i))) { + return false; + } + } + } + return true; + } + /** * Get the maximum id assigned to this type or its children. * The first call will cause all of the the ids in tree to be assigned, so diff --git orc/src/java/org/apache/orc/impl/BitFieldReader.java orc/src/java/org/apache/orc/impl/BitFieldReader.java index 8d9d3cb..dda7355 100644 --- orc/src/java/org/apache/orc/impl/BitFieldReader.java +++ orc/src/java/org/apache/orc/impl/BitFieldReader.java @@ -137,7 +137,7 @@ public void nextVector(LongColumnVector previous, long previousLen) throws IOException { previous.isRepeating = true; for (int i = 0; i < previousLen; i++) { - if (!previous.isNull[i]) { + if (previous.noNulls || !previous.isNull[i]) { previous.vector[i] = next(); } else { // The default value of null for int types in vectorized @@ -150,7 +150,8 @@ public void nextVector(LongColumnVector previous, // when determining the isRepeating flag. if (previous.isRepeating && i > 0 - && ((previous.vector[i - 1] != previous.vector[i]) || (previous.isNull[i - 1] != previous.isNull[i]))) { + && ((previous.vector[0] != previous.vector[i]) || + (previous.isNull[0] != previous.isNull[i]))) { previous.isRepeating = false; } } diff --git orc/src/java/org/apache/orc/impl/IntegerReader.java orc/src/java/org/apache/orc/impl/IntegerReader.java index 7dfd289..8bef0f1 100644 --- orc/src/java/org/apache/orc/impl/IntegerReader.java +++ orc/src/java/org/apache/orc/impl/IntegerReader.java @@ -20,7 +20,7 @@ import java.io.IOException; -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; /** * Interface for reading integers. @@ -57,9 +57,25 @@ /** * Return the next available vector for values. - * @return + * @param column the column being read + * @param data the vector to read into + * @param length the number of numbers to read + * @throws IOException + */ + void nextVector(ColumnVector column, + long[] data, + int length + ) throws IOException; + + /** + * Return the next available vector for values. Does not change the + * value of column.isRepeating. + * @param column the column being read + * @param data the vector to read into + * @param length the number of numbers to read * @throws IOException */ - void nextVector(LongColumnVector previous, final int previousLen) - throws IOException; -} + void nextVector(ColumnVector column, + int[] data, + int length + ) throws IOException;} diff --git orc/src/java/org/apache/orc/impl/RunLengthByteReader.java orc/src/java/org/apache/orc/impl/RunLengthByteReader.java index 380f3391..24bd051 100644 --- orc/src/java/org/apache/orc/impl/RunLengthByteReader.java +++ orc/src/java/org/apache/orc/impl/RunLengthByteReader.java @@ -20,7 +20,7 @@ import java.io.EOFException; import java.io.IOException; -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; /** * A reader that reads a sequence of bytes. A control byte is read before @@ -92,16 +92,16 @@ public byte next() throws IOException { return result; } - public void nextVector(LongColumnVector previous, long previousLen) + public void nextVector(ColumnVector previous, long[] data, long size) throws IOException { previous.isRepeating = true; - for (int i = 0; i < previousLen; i++) { + for (int i = 0; i < size; i++) { if (!previous.isNull[i]) { - previous.vector[i] = next(); + data[i] = next(); } else { // The default value of null for int types in vectorized // processing is 1, so set that if the value is null - previous.vector[i] = 1; + data[i] = 1; } // The default value for nulls in Vectorization for int types is 1 @@ -109,12 +109,36 @@ public void nextVector(LongColumnVector previous, long previousLen) // when determining the isRepeating flag. if (previous.isRepeating && i > 0 - && ((previous.vector[i - 1] != previous.vector[i]) || (previous.isNull[i - 1] != previous.isNull[i]))) { + && ((data[0] != data[i]) || + (previous.isNull[0] != previous.isNull[i]))) { previous.isRepeating = false; } } } + /** + * Read the next size bytes into the data array, skipping over any slots + * where isNull is true. + * @param isNull if non-null, skip any rows where isNull[r] is true + * @param data the array to read into + * @param size the number of elements to read + * @throws IOException + */ + public void nextVector(boolean[] isNull, int[] data, + long size) throws IOException { + if (isNull == null) { + for(int i=0; i < size; ++i) { + data[i] = next(); + } + } else { + for(int i=0; i < size; ++i) { + if (!isNull[i]) { + data[i] = next(); + } + } + } + } + public void seek(PositionProvider index) throws IOException { input.seek(index); int consumed = (int) index.getNext(); diff --git orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java index 0c90cde..b91a263 100644 --- orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java +++ orc/src/java/org/apache/orc/impl/RunLengthIntegerReader.java @@ -20,7 +20,7 @@ import java.io.EOFException; import java.io.IOException; -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; /** * A reader that reads a sequence of integers. @@ -99,15 +99,17 @@ public long next() throws IOException { } @Override - public void nextVector(LongColumnVector previous, final int previousLen) throws IOException { + public void nextVector(ColumnVector previous, + long[] data, + int previousLen) throws IOException { previous.isRepeating = true; for (int i = 0; i < previousLen; i++) { if (!previous.isNull[i]) { - previous.vector[i] = next(); + data[i] = next(); } else { // The default value of null for int type in vectorized // processing is 1, so set that if the value is null - previous.vector[i] = 1; + data[i] = 1; } // The default value for nulls in Vectorization for int types is 1 @@ -115,13 +117,32 @@ public void nextVector(LongColumnVector previous, final int previousLen) throws // when determining the isRepeating flag. if (previous.isRepeating && i > 0 - && (previous.vector[i - 1] != previous.vector[i] || previous.isNull[i - 1] != previous.isNull[i])) { + && (data[0] != data[i] || previous.isNull[0] != previous.isNull[i])) { previous.isRepeating = false; } } } @Override + public void nextVector(ColumnVector vector, + int[] data, + int size) throws IOException { + if (vector.noNulls) { + for(int r=0; r < data.length && r < size; ++r) { + data[r] = (int) next(); + } + } else if (!(vector.isRepeating && vector.isNull[0])) { + for(int r=0; r < data.length && r < size; ++r) { + if (!vector.isNull[r]) { + data[r] = (int) next(); + } else { + data[r] = 1; + } + } + } + } + + @Override public void seek(PositionProvider index) throws IOException { input.seek(index); int consumed = (int) index.getNext(); diff --git orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java index c6d685a..610d9b5 100644 --- orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java +++ orc/src/java/org/apache/orc/impl/RunLengthIntegerReaderV2.java @@ -21,9 +21,9 @@ import java.io.IOException; import java.util.Arrays; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; /** * A reader that reads a sequence of light weight compressed integers. Refer @@ -360,15 +360,17 @@ public void skip(long numValues) throws IOException { } @Override - public void nextVector(LongColumnVector previous, final int previousLen) throws IOException { + public void nextVector(ColumnVector previous, + long[] data, + int previousLen) throws IOException { previous.isRepeating = true; for (int i = 0; i < previousLen; i++) { if (!previous.isNull[i]) { - previous.vector[i] = next(); + data[i] = next(); } else { // The default value of null for int type in vectorized // processing is 1, so set that if the value is null - previous.vector[i] = 1; + data[i] = 1; } // The default value for nulls in Vectorization for int types is 1 @@ -376,10 +378,29 @@ public void nextVector(LongColumnVector previous, final int previousLen) throws // when determining the isRepeating flag. if (previous.isRepeating && i > 0 - && (previous.vector[i - 1] != previous.vector[i] || - previous.isNull[i - 1] != previous.isNull[i])) { + && (data[0] != data[i] || + previous.isNull[0] != previous.isNull[i])) { previous.isRepeating = false; } } } + + @Override + public void nextVector(ColumnVector vector, + int[] data, + int size) throws IOException { + if (vector.noNulls) { + for(int r=0; r < data.length && r < size; ++r) { + data[r] = (int) next(); + } + } else if (!(vector.isRepeating && vector.isNull[0])) { + for(int r=0; r < data.length && r < size; ++r) { + if (!vector.isNull[r]) { + data[r] = (int) next(); + } else { + data[r] = 1; + } + } + } + } } diff --git orc/src/java/org/apache/orc/impl/WriterImpl.java orc/src/java/org/apache/orc/impl/WriterImpl.java index f8afe06..b2966e0 100644 --- orc/src/java/org/apache/orc/impl/WriterImpl.java +++ orc/src/java/org/apache/orc/impl/WriterImpl.java @@ -1693,9 +1693,10 @@ void recordPosition(PositionRecorder recorder) throws IOException { } } + public static long MILLIS_PER_DAY = 24 * 60 * 60 * 1000; + public static long NANOS_PER_MILLI = 1000000; public static final int MILLIS_PER_SECOND = 1000; static final int NANOS_PER_SECOND = 1000000000; - static final int MILLIS_PER_NANO = 1000000; public static final String BASE_TIMESTAMP_STRING = "2015-01-01 00:00:00"; private static class TimestampTreeWriter extends TreeWriter { @@ -2261,32 +2262,36 @@ void writeBatch(ColumnVector vector, int offset, } } else { // write the records in runs of the same tag - byte prevTag = 0; - int currentRun = 0; - boolean started = false; + int[] currentStart = new int[vec.fields.length]; + int[] currentLength = new int[vec.fields.length]; for(int i=0; i < length; ++i) { - if (!vec.isNull[i + offset]) { + // only need to deal with the non-nulls, since the nulls were dealt + // with in the super method. + if (vec.noNulls || !vec.isNull[i + offset]) { byte tag = (byte) vec.tags[offset + i]; tags.write(tag); - if (!started) { - started = true; - currentRun = i; - prevTag = tag; - } else if (tag != prevTag) { - childrenWriters[prevTag].writeBatch(vec.fields[prevTag], - offset + currentRun, i - currentRun); - currentRun = i; - prevTag = tag; + if (currentLength[tag] == 0) { + // start a new sequence + currentStart[tag] = i + offset; + currentLength[tag] = 1; + } else if (currentStart[tag] + currentLength[tag] == i + offset) { + // ok, we are extending the current run for that tag. + currentLength[tag] += 1; + } else { + // otherwise, we need to close off the old run and start a new one + childrenWriters[tag].writeBatch(vec.fields[tag], + currentStart[tag], currentLength[tag]); + currentStart[tag] = i + offset; + currentLength[tag] = 1; } - } else if (started) { - started = false; - childrenWriters[prevTag].writeBatch(vec.fields[prevTag], - offset + currentRun, i - currentRun); } } - if (started) { - childrenWriters[prevTag].writeBatch(vec.fields[prevTag], - offset + currentRun, length - currentRun); + // write out any left over sequences + for(int tag=0; tag < currentStart.length; ++tag) { + if (currentLength[tag] != 0) { + childrenWriters[tag].writeBatch(vec.fields[tag], currentStart[tag], + currentLength[tag]); + } } } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java index 0724191..82a97e0 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java @@ -215,12 +215,9 @@ public VectorizedRowBatch createVectorizedRowBatch(boolean[] columnsToIncludeTru LOG.info("createVectorizedRowBatch columnsToIncludeTruncated " + Arrays.toString(columnsToIncludeTruncated)); int totalColumnCount = rowColumnTypeInfos.length + scratchColumnTypeNames.length; VectorizedRowBatch result = new VectorizedRowBatch(totalColumnCount); - - for (int i = 0; i < columnsToIncludeTruncated.length; i++) { - if (columnsToIncludeTruncated[i]) { - TypeInfo typeInfo = rowColumnTypeInfos[i]; - result.cols[i] = VectorizedBatchUtil.createColumnVector(typeInfo); - } + for (int i = 0; i < dataColumnCount; i++) { + TypeInfo typeInfo = rowColumnTypeInfos[i]; + result.cols[i] = VectorizedBatchUtil.createColumnVector(typeInfo); } for (int i = dataColumnCount; i < dataColumnCount + partitionColumnCount; i++) { @@ -476,8 +473,8 @@ public void addPartitionColsToBatch(VectorizedRowBatch batch, Object[] partition bcv.isNull[0] = true; bcv.isRepeating = true; } else { - bcv.fill(sVal.getBytes()); - bcv.isNull[0] = false; + bcv.setVal(0, sVal.getBytes()); + bcv.isRepeating = true; } } break; diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java index fe0be7b..fcb8ca4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java @@ -301,7 +301,7 @@ public static RecordReader createReaderFromFile(Reader file, /** * Do we have schema on read in the configuration variables? */ - TypeDescription schema = getDesiredRowTypeDescr(conf, /* isAcidRead */ false); + TypeDescription schema = getDesiredRowTypeDescr(conf, false, Integer.MAX_VALUE); Reader.Options options = new Reader.Options().range(offset, length); options.schema(schema); @@ -1743,7 +1743,7 @@ public float getProgress() throws IOException { /** * Do we have schema on read in the configuration variables? */ - TypeDescription schema = getDesiredRowTypeDescr(conf, /* isAcidRead */ true); + TypeDescription schema = getDesiredRowTypeDescr(conf, true, Integer.MAX_VALUE); final Reader reader; final int bucket; @@ -1994,10 +1994,13 @@ void put(Long fileId, FileStatus file, FileMetaInfo fileMetaInfo, Reader orcRead /** * Convert a Hive type property string that contains separated type names into a list of * TypeDescription objects. + * @param hiveTypeProperty the desired types from hive + * @param maxColumns the maximum number of desired columns * @return the list of TypeDescription objects. */ - public static ArrayList typeDescriptionsFromHiveTypeProperty( - String hiveTypeProperty) { + public static ArrayList + typeDescriptionsFromHiveTypeProperty(String hiveTypeProperty, + int maxColumns) { // CONSDIER: We need a type name parser for TypeDescription. @@ -2005,6 +2008,9 @@ void put(Long fileId, FileStatus file, FileMetaInfo fileMetaInfo, Reader orcRead ArrayList typeDescrList =new ArrayList(typeInfoList.size()); for (TypeInfo typeInfo : typeInfoList) { typeDescrList.add(convertTypeInfo(typeInfo)); + if (typeDescrList.size() >= maxColumns) { + break; + } } return typeDescrList; } @@ -2091,8 +2097,18 @@ public static TypeDescription convertTypeInfo(TypeInfo info) { } } - public static TypeDescription getDesiredRowTypeDescr(Configuration conf, boolean isAcidRead) - throws IOException { + /** + * Generate the desired schema for reading the file. + * @param conf the configuration + * @param isAcidRead is this an acid format? + * @param dataColumns the desired number of data columns for vectorized read + * @return the desired schema or null if schema evolution isn't enabled + * @throws IOException + */ + public static TypeDescription getDesiredRowTypeDescr(Configuration conf, + boolean isAcidRead, + int dataColumns + ) throws IOException { String columnNameProperty = null; String columnTypeProperty = null; @@ -2115,8 +2131,10 @@ public static TypeDescription getDesiredRowTypeDescr(Configuration conf, boolean haveSchemaEvolutionProperties = false; } else { schemaEvolutionTypeDescrs = - typeDescriptionsFromHiveTypeProperty(columnTypeProperty); - if (schemaEvolutionTypeDescrs.size() != schemaEvolutionColumnNames.size()) { + typeDescriptionsFromHiveTypeProperty(columnTypeProperty, + dataColumns); + if (schemaEvolutionTypeDescrs.size() != + Math.min(dataColumns, schemaEvolutionColumnNames.size())) { haveSchemaEvolutionProperties = false; } } @@ -2147,8 +2165,9 @@ public static TypeDescription getDesiredRowTypeDescr(Configuration conf, boolean return null; } schemaEvolutionTypeDescrs = - typeDescriptionsFromHiveTypeProperty(columnTypeProperty); - if (schemaEvolutionTypeDescrs.size() != schemaEvolutionColumnNames.size()) { + typeDescriptionsFromHiveTypeProperty(columnTypeProperty, dataColumns); + if (schemaEvolutionTypeDescrs.size() != + Math.min(dataColumns, schemaEvolutionColumnNames.size())) { return null; } @@ -2162,7 +2181,7 @@ public static TypeDescription getDesiredRowTypeDescr(Configuration conf, boolean } columnNum++; } - if (virtualColumnClipNum != -1) { + if (virtualColumnClipNum != -1 && virtualColumnClipNum < dataColumns) { schemaEvolutionColumnNames = Lists.newArrayList(schemaEvolutionColumnNames.subList(0, virtualColumnClipNum)); schemaEvolutionTypeDescrs = Lists.newArrayList(schemaEvolutionTypeDescrs.subList(0, virtualColumnClipNum)); @@ -2179,7 +2198,7 @@ public static TypeDescription getDesiredRowTypeDescr(Configuration conf, boolean // Desired schema does not include virtual columns or partition columns. TypeDescription result = TypeDescription.createStruct(); - for (int i = 0; i < schemaEvolutionColumnNames.size(); i++) { + for (int i = 0; i < schemaEvolutionTypeDescrs.size(); i++) { result.addField(schemaEvolutionColumnNames.get(i), schemaEvolutionTypeDescrs.get(i)); } diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java index f495be2..add19b4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java @@ -447,7 +447,8 @@ private void discoverKeyBounds(Reader reader, this.length = options.getLength(); this.validTxnList = validTxnList; - TypeDescription typeDescr = OrcInputFormat.getDesiredRowTypeDescr(conf, /* isAcidRead */ true); + TypeDescription typeDescr = + OrcInputFormat.getDesiredRowTypeDescr(conf, true, Integer.MAX_VALUE); objectInspector = OrcRecordUpdater.createEventSchema (OrcStruct.createObjectInspector(0, OrcUtils.getOrcTypes(typeDescr))); diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java index a031a92..0bcf9e3 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java @@ -26,6 +26,8 @@ import java.util.List; import java.util.Set; +import org.apache.orc.OrcUtils; +import org.apache.orc.TypeDescription; import org.apache.orc.impl.BufferChunk; import org.apache.orc.ColumnStatistics; import org.apache.orc.impl.ColumnStatisticsImpl; @@ -71,6 +73,7 @@ private final List stripeStats; private final int metadataSize; protected final List types; + private final TypeDescription schema; private final List userMetadata; private final List fileStats; private final List stripes; @@ -243,6 +246,11 @@ public int getRowIndexStride() { return result; } + @Override + public TypeDescription getSchema() { + return schema; + } + /** * Ensure this is an ORC file to prevent users from trying to read text * files or RC files as ORC files. @@ -386,7 +394,9 @@ public ReaderImpl(Path path, OrcFile.ReaderOptions options) throws IOException { this.writerVersion = footerMetaData.writerVersion; this.stripes = convertProtoStripesToStripes(rInfo.footer.getStripesList()); } + this.schema = OrcUtils.convertTypeFromProtobuf(this.types, 0); } + /** * Get the WriterVersion based on the ORC file postscript. * @param writerVersion the integer writer version @@ -668,7 +678,7 @@ public RecordReader rowsOptions(Options options) throws IOException { options.include(include); } return new RecordReaderImpl(this.getStripes(), fileSystem, path, - options, types, codec, bufferSize, rowIndexStride, conf); + options, schema, types, codec, bufferSize, rowIndexStride, conf); } diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java index 3975409..d027f49 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java @@ -27,9 +27,8 @@ import java.util.List; import java.util.Map; -import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.orc.BooleanColumnStatistics; -import org.apache.orc.OrcUtils; +import org.apache.orc.TypeDescription; import org.apache.orc.impl.BufferChunk; import org.apache.orc.ColumnStatistics; import org.apache.orc.impl.ColumnStatisticsImpl; @@ -58,7 +57,6 @@ import org.apache.hadoop.hive.common.io.DiskRangeList; import org.apache.hadoop.hive.common.io.DiskRangeList.CreateHelper; import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.orc.BloomFilterIO; import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; @@ -98,7 +96,6 @@ private final SargApplier sargApp; // an array about which row groups aren't skipped private boolean[] includedRowGroups = null; - private final Configuration conf; private final MetadataReader metadata; private final DataReader dataReader; @@ -145,33 +142,33 @@ protected RecordReaderImpl(List stripes, FileSystem fileSystem, Path path, Reader.Options options, + TypeDescription fileSchema, List types, CompressionCodec codec, int bufferSize, long strideRate, Configuration conf ) throws IOException { - - TreeReaderFactory.TreeReaderSchema treeReaderSchema; + SchemaEvolution treeReaderSchema; + this.included = options.getInclude(); + included[0] = true; if (options.getSchema() == null) { if (LOG.isInfoEnabled()) { LOG.info("Schema on read not provided -- using file schema " + types.toString()); } - treeReaderSchema = new TreeReaderFactory.TreeReaderSchema().fileTypes(types).schemaTypes(types); + treeReaderSchema = new SchemaEvolution(fileSchema, included); } else { // Now that we are creating a record reader for a file, validate that the schema to read // is compatible with the file schema. // - List schemaTypes = OrcUtils.getOrcTypes(options.getSchema()); - treeReaderSchema = SchemaEvolution.validateAndCreate(types, schemaTypes); + treeReaderSchema = new SchemaEvolution(fileSchema, options.getSchema(), + included); } this.path = path; this.codec = codec; this.types = types; this.bufferSize = bufferSize; - this.included = options.getInclude(); - this.conf = conf; this.rowIndexStride = strideRate; this.metadata = new MetadataReaderImpl(fileSystem, path, codec, bufferSize, types.size()); SearchArgument sarg = options.getSearchArgument(); @@ -210,7 +207,8 @@ protected RecordReaderImpl(List stripes, skipCorrupt = OrcConf.SKIP_CORRUPT_DATA.getBoolean(conf); } - reader = TreeReaderFactory.createTreeReader(0, treeReaderSchema, included, skipCorrupt); + reader = TreeReaderFactory.createTreeReader(treeReaderSchema.getReaderSchema(), + treeReaderSchema, included, skipCorrupt); indexes = new OrcProto.RowIndex[types.size()]; bloomFilterIndices = new OrcProto.BloomFilterIndex[types.size()]; advanceToNextRow(reader, 0L, true); @@ -239,7 +237,7 @@ public long getNext() { return metadata.readStripeFooter(stripe); } - static enum Location { + enum Location { BEFORE, MIN, MIDDLE, MAX, AFTER } @@ -1052,31 +1050,26 @@ public Object next(Object previous) throws IOException { } @Override - public VectorizedRowBatch nextBatch(VectorizedRowBatch previous) throws IOException { + public boolean nextBatch(VectorizedRowBatch batch) throws IOException { try { - final VectorizedRowBatch result; if (rowInStripe >= rowCountInStripe) { currentStripe += 1; + if (currentStripe >= stripes.size()) { + batch.size = 0; + return false; + } readStripe(); } - final int batchSize = computeBatchSize(VectorizedRowBatch.DEFAULT_SIZE); + int batchSize = computeBatchSize(batch.getMaxSize()); rowInStripe += batchSize; - if (previous == null) { - ColumnVector[] cols = (ColumnVector[]) reader.nextVector(null, (int) batchSize); - result = new VectorizedRowBatch(cols.length); - result.cols = cols; - } else { - result = previous; - result.selectedInUse = false; - reader.setVectorColumnCount(result.getDataColumnCount()); - reader.nextVector(result.cols, batchSize); - } + reader.setVectorColumnCount(batch.getDataColumnCount()); + reader.nextBatch(batch, batchSize); - result.size = batchSize; + batch.size = (int) batchSize; advanceToNextRow(reader, rowInStripe + rowBaseInStripe, true); - return result; + return batch.size != 0; } catch (IOException e) { // Rethrow exception with file name in log message throw new IOException("Error reading file: " + path, e); diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java index f28ca13..6747691 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java @@ -20,13 +20,12 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.ql.io.orc.TreeReaderFactory.TreeReaderSchema; -import org.apache.orc.OrcProto; -import org.apache.orc.OrcUtils; import org.apache.orc.TypeDescription; /** @@ -34,103 +33,134 @@ * has been schema evolution. */ public class SchemaEvolution { - + private final Map readerToFile; + private final boolean[] included; + private final TypeDescription readerSchema; private static final Log LOG = LogFactory.getLog(SchemaEvolution.class); - public static TreeReaderSchema validateAndCreate(List fileTypes, - List schemaTypes) throws IOException { + public SchemaEvolution(TypeDescription readerSchema, boolean[] included) { + this.included = included; + readerToFile = null; + this.readerSchema = readerSchema; + } - // For ACID, the row is the ROW field in the outer STRUCT. - final boolean isAcid = checkAcidSchema(fileTypes); - final List rowSchema; - int rowSubtype; - if (isAcid) { - rowSubtype = OrcRecordUpdater.ROW + 1; - rowSchema = fileTypes.subList(rowSubtype, fileTypes.size()); + public SchemaEvolution(TypeDescription fileSchema, + TypeDescription readerSchema, + boolean[] included) throws IOException { + readerToFile = new HashMap<>(readerSchema.getMaximumId() + 1); + this.included = included; + if (checkAcidSchema(fileSchema)) { + this.readerSchema = createEventSchema(readerSchema); } else { - rowSubtype = 0; - rowSchema = fileTypes; + this.readerSchema = readerSchema; } + buildMapping(fileSchema, this.readerSchema); + } - // Do checking on the overlap. Additional columns will be defaulted to NULL. - - int numFileColumns = rowSchema.get(0).getSubtypesCount(); - int numDesiredColumns = schemaTypes.get(0).getSubtypesCount(); - - int numReadColumns = Math.min(numFileColumns, numDesiredColumns); - - /** - * Check type promotion. - * - * Currently, we only support integer type promotions that can be done "implicitly". - * That is, we know that using a bigger integer tree reader on the original smaller integer - * column will "just work". - * - * In the future, other type promotions might require type conversion. - */ - // short -> int -> bigint as same integer readers are used for the above types. - - for (int i = 0; i < numReadColumns; i++) { - OrcProto.Type fColType = fileTypes.get(rowSubtype + i); - OrcProto.Type rColType = schemaTypes.get(i); - if (!fColType.getKind().equals(rColType.getKind())) { - - boolean ok = false; - if (fColType.getKind().equals(OrcProto.Type.Kind.SHORT)) { + public TypeDescription getReaderSchema() { + return readerSchema; + } - if (rColType.getKind().equals(OrcProto.Type.Kind.INT) || - rColType.getKind().equals(OrcProto.Type.Kind.LONG)) { - // type promotion possible, converting SHORT to INT/LONG requested type - ok = true; - } - } else if (fColType.getKind().equals(OrcProto.Type.Kind.INT)) { + public TypeDescription getFileType(TypeDescription readerType) { + TypeDescription result; + if (readerToFile == null) { + if (included == null || included[readerType.getId()]) { + result = readerType; + } else { + result = null; + } + } else { + result = readerToFile.get(readerType); + } + return result; + } - if (rColType.getKind().equals(OrcProto.Type.Kind.LONG)) { - // type promotion possible, converting INT to LONG requested type - ok = true; + void buildMapping(TypeDescription fileType, + TypeDescription readerType) throws IOException { + // if the column isn't included, don't map it + if (included != null && !included[readerType.getId()]) { + return; + } + boolean isOk = true; + // check the easy case first + if (fileType.getCategory() == readerType.getCategory()) { + switch (readerType.getCategory()) { + case BOOLEAN: + case BYTE: + case SHORT: + case INT: + case LONG: + case DOUBLE: + case FLOAT: + case STRING: + case TIMESTAMP: + case BINARY: + case DATE: + // these are always a match + break; + case CHAR: + case VARCHAR: + isOk = fileType.getMaxLength() == readerType.getMaxLength(); + break; + case DECIMAL: + // TODO we don't enforce scale and precision checks, but probably should + break; + case UNION: + case MAP: + case LIST: { + // these must be an exact match + List fileChildren = fileType.getChildren(); + List readerChildren = readerType.getChildren(); + if (fileChildren.size() == readerChildren.size()) { + for(int i=0; i < fileChildren.size(); ++i) { + buildMapping(fileChildren.get(i), readerChildren.get(i)); + } + } else { + isOk = false; } + break; } - - if (!ok) { - throw new IOException("ORC does not support type conversion from " + - fColType.getKind().name() + " to " + rColType.getKind().name()); + case STRUCT: { + // allow either side to have fewer fields than the other + List fileChildren = fileType.getChildren(); + List readerChildren = readerType.getChildren(); + int jointSize = Math.min(fileChildren.size(), readerChildren.size()); + for(int i=0; i < jointSize; ++i) { + buildMapping(fileChildren.get(i), readerChildren.get(i)); + } + break; } + default: + throw new IllegalArgumentException("Unknown type " + readerType); } - } - - List fullSchemaTypes; - - if (isAcid) { - fullSchemaTypes = new ArrayList(); - - // This copies the ACID struct type which is subtype = 0. - // It has field names "operation" through "row". - // And we copy the types for all fields EXCEPT ROW (which must be last!). - - for (int i = 0; i < rowSubtype; i++) { - fullSchemaTypes.add(fileTypes.get(i).toBuilder().build()); + } else { + switch (fileType.getCategory()) { + case SHORT: + if (readerType.getCategory() != TypeDescription.Category.INT && + readerType.getCategory() != TypeDescription.Category.LONG) { + isOk = false; + } + break; + case INT: + if (readerType.getCategory() != TypeDescription.Category.LONG) { + isOk = false; + } + break; + default: + isOk = false; } - - // Add the row struct type. - OrcUtils.appendOrcTypesRebuildSubtypes(fullSchemaTypes, schemaTypes, 0); + } + if (isOk) { + readerToFile.put(readerType, fileType); } else { - fullSchemaTypes = schemaTypes; + throw new IOException("ORC does not support type conversion from " + + fileType + " to " + readerType); } - - int innerStructSubtype = rowSubtype; - - // LOG.info("Schema evolution: (fileTypes) " + fileTypes.toString() + - // " (schemaEvolutionTypes) " + schemaEvolutionTypes.toString()); - - return new TreeReaderSchema(). - fileTypes(fileTypes). - schemaTypes(fullSchemaTypes). - innerStructSubtype(innerStructSubtype); } - private static boolean checkAcidSchema(List fileSchema) { - if (fileSchema.get(0).getKind().equals(OrcProto.Type.Kind.STRUCT)) { - List rootFields = fileSchema.get(0).getFieldNamesList(); + private static boolean checkAcidSchema(TypeDescription type) { + if (type.getCategory().equals(TypeDescription.Category.STRUCT)) { + List rootFields = type.getFieldNames(); if (acidEventFieldNames.equals(rootFields)) { return true; } @@ -142,26 +172,14 @@ private static boolean checkAcidSchema(List fileSchema) { * @param typeDescr * @return ORC types for the ACID event based on the row's type description */ - public static List createEventSchema(TypeDescription typeDescr) { - - List result = new ArrayList(); - - OrcProto.Type.Builder type = OrcProto.Type.newBuilder(); - type.setKind(OrcProto.Type.Kind.STRUCT); - type.addAllFieldNames(acidEventFieldNames); - for (int i = 0; i < acidEventFieldNames.size(); i++) { - type.addSubtypes(i + 1); - } - result.add(type.build()); - - // Automatically add all fields except the last (ROW). - for (int i = 0; i < acidEventOrcTypeKinds.size() - 1; i ++) { - type.clear(); - type.setKind(acidEventOrcTypeKinds.get(i)); - result.add(type.build()); - } - - OrcUtils.appendOrcTypesRebuildSubtypes(result, typeDescr); + public static TypeDescription createEventSchema(TypeDescription typeDescr) { + TypeDescription result = TypeDescription.createStruct() + .addField("operation", TypeDescription.createInt()) + .addField("originalTransaction", TypeDescription.createLong()) + .addField("bucket", TypeDescription.createInt()) + .addField("rowId", TypeDescription.createLong()) + .addField("currentTransaction", TypeDescription.createLong()) + .addField("row", typeDescr.clone()); return result; } @@ -174,14 +192,4 @@ private static boolean checkAcidSchema(List fileSchema) { acidEventFieldNames.add("currentTransaction"); acidEventFieldNames.add("row"); } - public static final List acidEventOrcTypeKinds = - new ArrayList(); - static { - acidEventOrcTypeKinds.add(OrcProto.Type.Kind.INT); - acidEventOrcTypeKinds.add(OrcProto.Type.Kind.LONG); - acidEventOrcTypeKinds.add(OrcProto.Type.Kind.INT); - acidEventOrcTypeKinds.add(OrcProto.Type.Kind.LONG); - acidEventOrcTypeKinds.add(OrcProto.Type.Kind.LONG); - acidEventOrcTypeKinds.add(OrcProto.Type.Kind.STRUCT); - } } diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java index 8bb32ea..8ee8cd7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java @@ -24,6 +24,7 @@ import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; @@ -35,9 +36,12 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector; import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector; import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils; +import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr; import org.apache.hadoop.hive.serde2.io.ByteWritable; @@ -56,8 +60,7 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.orc.TypeDescription; import org.apache.orc.impl.BitFieldReader; import org.apache.orc.impl.DynamicByteArray; import org.apache.orc.impl.InStream; @@ -75,60 +78,6 @@ */ public class TreeReaderFactory { - private static final Logger LOG = - LoggerFactory.getLogger(TreeReaderFactory.class); - - public static class TreeReaderSchema { - - /** - * The types in the ORC file. - */ - List fileTypes; - - /** - * The treeReaderSchema that the reader should read as. - */ - List schemaTypes; - - /** - * The subtype of the row STRUCT. Different than 0 for ACID. - */ - int innerStructSubtype; - - public TreeReaderSchema() { - fileTypes = null; - schemaTypes = null; - innerStructSubtype = -1; - } - - public TreeReaderSchema fileTypes(List fileTypes) { - this.fileTypes = fileTypes; - return this; - } - - public TreeReaderSchema schemaTypes(List schemaTypes) { - this.schemaTypes = schemaTypes; - return this; - } - - public TreeReaderSchema innerStructSubtype(int innerStructSubtype) { - this.innerStructSubtype = innerStructSubtype; - return this; - } - - public List getFileTypes() { - return fileTypes; - } - - public List getSchemaTypes() { - return schemaTypes; - } - - public int getInnerStructSubtype() { - return innerStructSubtype; - } - } - public abstract static class TreeReader { protected final int columnId; protected BitFieldReader present = null; @@ -230,36 +179,60 @@ Object next(Object previous) throws IOException { } /** + * Called at the top level to read into the given batch. + * @param batch the batch to read into + * @param batchSize the number of rows to read + * @throws IOException + */ + public void nextBatch(VectorizedRowBatch batch, + int batchSize) throws IOException { + batch.cols[0].reset(); + batch.cols[0].ensureSize(batchSize, false); + nextVector(batch.cols[0], null, batchSize); + } + + /** * Populates the isNull vector array in the previousVector object based on * the present stream values. This function is called from all the child * readers, and they all set the values based on isNull field value. * - * @param previousVector The columnVector object whose isNull value is populated + * @param previous The columnVector object whose isNull value is populated + * @param isNull Whether the each value was null at a higher level. If + * isNull is null, all values are non-null. * @param batchSize Size of the column vector - * @return next column vector * @throws IOException */ - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - ColumnVector result = (ColumnVector) previousVector; - if (present != null) { + public void nextVector(ColumnVector previous, + boolean[] isNull, + int batchSize) throws IOException { + if (present != null || isNull != null) { // Set noNulls and isNull vector of the ColumnVector based on // present stream - result.noNulls = true; + previous.noNulls = true; + boolean allNull = true; for (int i = 0; i < batchSize; i++) { - result.isNull[i] = (present.next() != 1); - if (result.noNulls && result.isNull[i]) { - result.noNulls = false; + if (isNull == null || !isNull[i]) { + if (present != null && present.next() != 1) { + previous.noNulls = false; + previous.isNull[i] = true; + } else { + previous.isNull[i] = false; + allNull = false; + } + } else { + previous.noNulls = false; + previous.isNull[i] = true; } } + previous.isRepeating = !previous.noNulls && allNull; } else { - // There is not present stream, this means that all the values are + // There is no present stream, this means that all the values are // present. - result.noNulls = true; + previous.noNulls = true; for (int i = 0; i < batchSize; i++) { - result.isNull[i] = false; + previous.isNull[i] = false; } } - return previousVector; } public BitFieldReader getPresent() { @@ -267,6 +240,46 @@ public BitFieldReader getPresent() { } } + public static class NullTreeReader extends TreeReader { + + public NullTreeReader(int columnId) throws IOException { + super(columnId); + } + + @Override + public void startStripe(Map streams, + OrcProto.StripeFooter footer) { + // PASS + } + + @Override + void skipRows(long rows) { + // PASS + } + + @Override + public void seek(PositionProvider position) { + // PASS + } + + @Override + public void seek(PositionProvider[] position) { + // PASS + } + + @Override + Object next(Object previous) { + return null; + } + + @Override + public void nextVector(ColumnVector vector, boolean[] isNull, int size) { + vector.noNulls = false; + vector.isNull[0] = true; + vector.isRepeating = true; + } + } + public static class BooleanTreeReader extends TreeReader { protected BitFieldReader reader = null; @@ -322,20 +335,16 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final LongColumnVector result; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + LongColumnVector result = (LongColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); // Read value entries based on isNull entries reader.nextVector(result, batchSize); - return result; } } @@ -387,20 +396,16 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final LongColumnVector result; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final LongColumnVector result = (LongColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; + reader.nextVector(result, result.vector, batchSize); } @Override @@ -473,20 +478,16 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final LongColumnVector result; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final LongColumnVector result = (LongColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; + reader.nextVector(result, result.vector, batchSize); } @Override @@ -559,20 +560,16 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final LongColumnVector result; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final LongColumnVector result = (LongColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; + reader.nextVector(result, result.vector, batchSize); } @Override @@ -646,20 +643,16 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final LongColumnVector result; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final LongColumnVector result = (LongColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; + reader.nextVector(result, result.vector, batchSize); } @Override @@ -719,16 +712,13 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final DoubleColumnVector result; - if (previousVector == null) { - result = new DoubleColumnVector(); - } else { - result = (DoubleColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final DoubleColumnVector result = (DoubleColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); final boolean hasNulls = !result.noNulls; boolean allNulls = hasNulls; @@ -768,7 +758,6 @@ public Object nextVector(Object previousVector, final int batchSize) throws IOEx } result.isRepeating = repeating; } - return result; } @Override @@ -832,16 +821,13 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final DoubleColumnVector result; - if (previousVector == null) { - result = new DoubleColumnVector(); - } else { - result = (DoubleColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final DoubleColumnVector result = (DoubleColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); final boolean hasNulls = !result.noNulls; boolean allNulls = hasNulls; @@ -881,8 +867,6 @@ public Object nextVector(Object previousVector, final int batchSize) throws IOEx } result.isRepeating = repeating; } - - return result; } @Override @@ -974,19 +958,15 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final BytesColumnVector result; - if (previousVector == null) { - result = new BytesColumnVector(); - } else { - result = (BytesColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final BytesColumnVector result = (BytesColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv, result, batchSize); - return result; } @Override @@ -1011,7 +991,6 @@ void skipRows(long items) throws IOException { private final TimeZone readerTimeZone; private TimeZone writerTimeZone; private boolean hasSameTZRules; - private TimestampWritable scratchTimestampWritable; TimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException { this(columnId, null, null, null, null, skipCorrupt); @@ -1115,9 +1094,9 @@ Object next(Object previous) throws IOException { int newNanos = parseNanos(nanos.next()); // fix the rounding when we divided by 1000. if (millis >= 0) { - millis += newNanos / 1000000; + millis += newNanos / WriterImpl.NANOS_PER_MILLI; } else { - millis -= newNanos / 1000000; + millis -= newNanos / WriterImpl.NANOS_PER_MILLI; } long offset = 0; // If reader and writer time zones have different rules, adjust the timezone difference @@ -1144,31 +1123,45 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final TimestampColumnVector result; - if (previousVector == null) { - result = new TimestampColumnVector(); - } else { - result = (TimestampColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + TimestampColumnVector result = (TimestampColumnVector) previousVector; + super.nextVector(previousVector, isNull, batchSize); - result.reset(); - if (scratchTimestampWritable == null) { - scratchTimestampWritable = new TimestampWritable(); - } - Object obj; for (int i = 0; i < batchSize; i++) { - obj = next(scratchTimestampWritable); - if (obj == null) { - result.noNulls = false; - result.isNull[i] = true; - } else { - TimestampWritable writable = (TimestampWritable) obj; - result.set(i, writable.getTimestamp()); + if (result.noNulls || !result.isNull[i]) { + long millis = data.next() + base_timestamp; + int newNanos = parseNanos(nanos.next()); + if (millis < 0 && newNanos != 0) { + millis -= 1; + } + millis *= WriterImpl.MILLIS_PER_SECOND; + long offset = 0; + // If reader and writer time zones have different rules, adjust the timezone difference + // between reader and writer taking day light savings into account. + if (!hasSameTZRules) { + offset = writerTimeZone.getOffset(millis) - readerTimeZone.getOffset(millis); + } + long adjustedMillis = millis + offset; + // Sometimes the reader timezone might have changed after adding the adjustedMillis. + // To account for that change, check for any difference in reader timezone after + // adding adjustedMillis. If so use the new offset (offset at adjustedMillis point of time). + if (!hasSameTZRules && + (readerTimeZone.getOffset(millis) != readerTimeZone.getOffset(adjustedMillis))) { + long newOffset = + writerTimeZone.getOffset(millis) - readerTimeZone.getOffset(adjustedMillis); + adjustedMillis = millis + newOffset; + } + result.time[i] = adjustedMillis; + result.nanos[i] = newNanos; + if (result.isRepeating && i != 0 && + (result.time[0] != result.time[i] || + result.nanos[0] != result.nanos[i])) { + result.isRepeating = false; + } } } - - return result; } private static int parseNanos(long serialized) { @@ -1253,20 +1246,16 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final LongColumnVector result; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final LongColumnVector result = (LongColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; + reader.nextVector(result, result.vector, batchSize); } @Override @@ -1278,7 +1267,7 @@ void skipRows(long items) throws IOException { public static class DecimalTreeReader extends TreeReader { protected InStream valueStream; protected IntegerReader scaleReader = null; - private LongColumnVector scratchScaleVector; + private int[] scratchScaleVector; private final int precision; private final int scale; @@ -1293,7 +1282,7 @@ protected DecimalTreeReader(int columnId, int precision, int scale, InStream pre super(columnId, present); this.precision = precision; this.scale = scale; - this.scratchScaleVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE); + this.scratchScaleVector = new int[VectorizedRowBatch.DEFAULT_SIZE]; this.valueStream = valueStream; if (scaleStream != null && encoding != null) { checkEncoding(encoding); @@ -1352,46 +1341,34 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final DecimalColumnVector result; - if (previousVector == null) { - result = new DecimalColumnVector(precision, scale); - } else { - result = (DecimalColumnVector) previousVector; - } - - // Save the reference for isNull in the scratch vector - boolean[] scratchIsNull = scratchScaleVector.isNull; + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final DecimalColumnVector result = (DecimalColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); + if (batchSize > scratchScaleVector.length) { + scratchScaleVector = new int[(int) batchSize]; + } + scaleReader.nextVector(result, scratchScaleVector, batchSize); // Read value entries based on isNull entries - if (result.isRepeating) { - if (!result.isNull[0]) { + if (result.noNulls) { + for (int r=0; r < batchSize; ++r) { BigInteger bInt = SerializationUtils.readBigInteger(valueStream); - short scaleInData = (short) scaleReader.next(); - HiveDecimal dec = HiveDecimal.create(bInt, scaleInData); - dec = HiveDecimal.enforcePrecisionScale(dec, precision, scale); - result.set(0, dec); + HiveDecimal dec = HiveDecimal.create(bInt, scratchScaleVector[r]); + result.set(r, dec); } - } else { - // result vector has isNull values set, use the same to read scale vector. - scratchScaleVector.isNull = result.isNull; - scaleReader.nextVector(scratchScaleVector, batchSize); - for (int i = 0; i < batchSize; i++) { - if (!result.isNull[i]) { + } else if (!result.isRepeating || !result.isNull[0]) { + for (int r=0; r < batchSize; ++r) { + if (!result.isNull[r]) { BigInteger bInt = SerializationUtils.readBigInteger(valueStream); - short scaleInData = (short) scratchScaleVector.vector[i]; - HiveDecimal dec = HiveDecimal.create(bInt, scaleInData); - dec = HiveDecimal.enforcePrecisionScale(dec, precision, scale); - result.set(i, dec); + HiveDecimal dec = HiveDecimal.create(bInt, scratchScaleVector[r]); + result.set(r, dec); } } } - // Switch back the null vector. - scratchScaleVector.isNull = scratchIsNull; - return result; } @Override @@ -1481,8 +1458,10 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - return reader.nextVector(previousVector, batchSize); + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + reader.nextVector(previousVector, isNull, batchSize); } @Override @@ -1501,7 +1480,7 @@ void skipRows(long items) throws IOException { BytesColumnVector result, final int batchSize) throws IOException { // Read lengths scratchlcv.isNull = result.isNull; // Notice we are replacing the isNull vector here... - lengths.nextVector(scratchlcv, batchSize); + lengths.nextVector(scratchlcv, scratchlcv.vector, batchSize); int totalLength = 0; if (!scratchlcv.isRepeating) { for (int i = 0; i < batchSize; i++) { @@ -1532,31 +1511,35 @@ void skipRows(long items) throws IOException { } // This method has the common code for reading in bytes into a BytesColumnVector. - public static void readOrcByteArrays(InStream stream, IntegerReader lengths, - LongColumnVector scratchlcv, - BytesColumnVector result, final int batchSize) throws IOException { - - byte[] allBytes = commonReadByteArrays(stream, lengths, scratchlcv, result, batchSize); - - // Too expensive to figure out 'repeating' by comparisons. - result.isRepeating = false; - int offset = 0; - if (!scratchlcv.isRepeating) { - for (int i = 0; i < batchSize; i++) { - if (!scratchlcv.isNull[i]) { - result.setRef(i, allBytes, offset, (int) scratchlcv.vector[i]); - offset += scratchlcv.vector[i]; - } else { - result.setRef(i, allBytes, 0, 0); + public static void readOrcByteArrays(InStream stream, + IntegerReader lengths, + LongColumnVector scratchlcv, + BytesColumnVector result, + int batchSize) throws IOException { + if (result.noNulls || !(result.isRepeating && result.isNull[0])) { + byte[] allBytes = commonReadByteArrays(stream, lengths, scratchlcv, + result, (int) batchSize); + + // Too expensive to figure out 'repeating' by comparisons. + result.isRepeating = false; + int offset = 0; + if (!scratchlcv.isRepeating) { + for (int i = 0; i < batchSize; i++) { + if (!scratchlcv.isNull[i]) { + result.setRef(i, allBytes, offset, (int) scratchlcv.vector[i]); + offset += scratchlcv.vector[i]; + } else { + result.setRef(i, allBytes, 0, 0); + } } - } - } else { - for (int i = 0; i < batchSize; i++) { - if (!scratchlcv.isNull[i]) { - result.setRef(i, allBytes, offset, (int) scratchlcv.vector[0]); - offset += scratchlcv.vector[0]; - } else { - result.setRef(i, allBytes, 0, 0); + } else { + for (int i = 0; i < batchSize; i++) { + if (!scratchlcv.isNull[i]) { + result.setRef(i, allBytes, offset, (int) scratchlcv.vector[0]); + offset += scratchlcv.vector[0]; + } else { + result.setRef(i, allBytes, 0, 0); + } } } } @@ -1641,19 +1624,16 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final BytesColumnVector result; - if (previousVector == null) { - result = new BytesColumnVector(); - } else { - result = (BytesColumnVector) previousVector; - } + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final BytesColumnVector result = (BytesColumnVector) previousVector; // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); - BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv, result, batchSize); - return result; + BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv, + result, batchSize); } @Override @@ -1816,18 +1796,15 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final BytesColumnVector result; + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + final BytesColumnVector result = (BytesColumnVector) previousVector; int offset; int length; - if (previousVector == null) { - result = new BytesColumnVector(); - } else { - result = (BytesColumnVector) previousVector; - } // Read present/isNull stream - super.nextVector(result, batchSize); + super.nextVector(result, isNull, batchSize); if (dictionaryBuffer != null) { @@ -1838,7 +1815,8 @@ public Object nextVector(Object previousVector, final int batchSize) throws IOEx // Read string offsets scratchlcv.isNull = result.isNull; - reader.nextVector(scratchlcv, batchSize); + scratchlcv.ensureSize((int) batchSize, false); + reader.nextVector(scratchlcv, scratchlcv.vector, batchSize); if (!scratchlcv.isRepeating) { // The vector has non-repeating strings. Iterate thru the batch @@ -1878,7 +1856,6 @@ public Object nextVector(Object previousVector, final int batchSize) throws IOEx } } } - return result; } int getDictionaryEntryLength(int entry, int offset) { @@ -1936,11 +1913,13 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { // Get the vector of strings from StringTreeReader, then make a 2nd pass to // adjust down the length (right trim and truncate) if necessary. - BytesColumnVector result = (BytesColumnVector) super.nextVector(previousVector, batchSize); - + super.nextVector(previousVector, isNull, batchSize); + BytesColumnVector result = (BytesColumnVector) previousVector; int adjustedDownLen; if (result.isRepeating) { if (result.noNulls || !result.isNull[0]) { @@ -1973,7 +1952,6 @@ public Object nextVector(Object previousVector, final int batchSize) throws IOEx } } } - return result; } } @@ -2010,10 +1988,13 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { // Get the vector of strings from StringTreeReader, then make a 2nd pass to // adjust down the length (truncate) if necessary. - BytesColumnVector result = (BytesColumnVector) super.nextVector(previousVector, batchSize); + super.nextVector(previousVector, isNull, batchSize); + BytesColumnVector result = (BytesColumnVector) previousVector; int adjustedDownLen; if (result.isRepeating) { @@ -2045,62 +2026,26 @@ public Object nextVector(Object previousVector, final int batchSize) throws IOEx } } } - return result; } } protected static class StructTreeReader extends TreeReader { - private final int readColumnCount; - private final int resultColumnCount; protected final TreeReader[] fields; - private final String[] fieldNames; - protected StructTreeReader( - int columnId, - TreeReaderSchema treeReaderSchema, - boolean[] included, - boolean skipCorrupt) throws IOException { + protected StructTreeReader(int columnId, + TypeDescription readerSchema, + SchemaEvolution treeReaderSchema, + boolean[] included, + boolean skipCorrupt) throws IOException { super(columnId); - OrcProto.Type fileStructType = treeReaderSchema.getFileTypes().get(columnId); - - OrcProto.Type schemaStructType = treeReaderSchema.getSchemaTypes().get(columnId); + TypeDescription fileSchema = treeReaderSchema.getFileType(readerSchema); - readColumnCount = Math.min(fileStructType.getFieldNamesCount(), schemaStructType.getFieldNamesCount()); - - if (columnId == treeReaderSchema.getInnerStructSubtype()) { - // If there are more result columns than reader columns, we will default those additional - // columns to NULL. - resultColumnCount = schemaStructType.getFieldNamesCount(); - } else { - resultColumnCount = readColumnCount; - } - - this.fields = new TreeReader[readColumnCount]; - this.fieldNames = new String[readColumnCount]; - - if (included == null) { - for (int i = 0; i < readColumnCount; ++i) { - int subtype = schemaStructType.getSubtypes(i); - this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt); - // Use the treeReaderSchema evolution name since file/reader types may not have the real column name. - this.fieldNames[i] = schemaStructType.getFieldNames(i); - } - } else { - for (int i = 0; i < readColumnCount; ++i) { - int subtype = schemaStructType.getSubtypes(i); - if (subtype >= included.length) { - throw new IOException("subtype " + subtype + " exceeds the included array size " + - included.length + " fileTypes " + treeReaderSchema.getFileTypes().toString() + - " schemaTypes " + treeReaderSchema.getSchemaTypes().toString() + - " innerStructSubtype " + treeReaderSchema.getInnerStructSubtype()); - } - if (included[subtype]) { - this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt); - } - // Use the treeReaderSchema evolution name since file/reader types may not have the real column name. - this.fieldNames[i] = schemaStructType.getFieldNames(i); - } + List childrenTypes = readerSchema.getChildren(); + this.fields = new TreeReader[childrenTypes.size()]; + for (int i = 0; i < fields.length; ++i) { + TypeDescription subtype = childrenTypes.get(i); + this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt); } } @@ -2120,65 +2065,52 @@ Object next(Object previous) throws IOException { OrcStruct result = null; if (valuePresent) { if (previous == null) { - result = new OrcStruct(resultColumnCount); + result = new OrcStruct(fields.length); } else { result = (OrcStruct) previous; // If the input format was initialized with a file with a // different number of fields, the number of fields needs to // be updated to the correct number - if (result.getNumFields() != resultColumnCount) { - result.setNumFields(resultColumnCount); - } + result.setNumFields(fields.length); } - for (int i = 0; i < readColumnCount; ++i) { + for (int i = 0; i < fields.length; ++i) { if (fields[i] != null) { result.setFieldValue(i, fields[i].next(result.getFieldValue(i))); } } - if (resultColumnCount > readColumnCount) { - for (int i = readColumnCount; i < resultColumnCount; ++i) { - // Default new treeReaderSchema evolution fields to NULL. - result.setFieldValue(i, null); - } - } } return result; } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - final ColumnVector[] result; - if (previousVector == null) { - result = new ColumnVector[readColumnCount]; - } else { - result = (ColumnVector[]) previousVector; + public void nextBatch(VectorizedRowBatch batch, + int batchSize) throws IOException { + for(int i=0; i < fields.length && + (vectorColumnCount == -1 || i < vectorColumnCount); ++i) { + batch.cols[i].reset(); + batch.cols[i].ensureSize((int) batchSize, false); + fields[i].nextVector(batch.cols[i], null, batchSize); } + } - // Read all the members of struct as column vectors - for (int i = 0; i < readColumnCount; i++) { - if (fields[i] != null) { - if (result[i] == null) { - result[i] = (ColumnVector) fields[i].nextVector(null, batchSize); - } else { - fields[i].nextVector(result[i], batchSize); - } - } - } + @Override + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + super.nextVector(previousVector, isNull, batchSize); + StructColumnVector result = (StructColumnVector) previousVector; + if (result.noNulls || !(result.isRepeating && result.isNull[0])) { + result.isRepeating = false; - // Default additional treeReaderSchema evolution fields to NULL. - if (vectorColumnCount != -1 && vectorColumnCount > readColumnCount) { - for (int i = readColumnCount; i < vectorColumnCount; ++i) { - ColumnVector colVector = result[i]; - if (colVector != null) { - colVector.isRepeating = true; - colVector.noNulls = false; - colVector.isNull[0] = true; + // Read all the members of struct as column vectors + boolean[] mask = result.noNulls ? null : result.isNull; + for (int f = 0; f < fields.length; f++) { + if (fields[f] != null) { + fields[f].nextVector(result.fields[f], mask, batchSize); } } } - - return result; } @Override @@ -2208,19 +2140,18 @@ void skipRows(long items) throws IOException { protected final TreeReader[] fields; protected RunLengthByteReader tags; - protected UnionTreeReader(int columnId, - TreeReaderSchema treeReaderSchema, - boolean[] included, - boolean skipCorrupt) throws IOException { - super(columnId); - OrcProto.Type type = treeReaderSchema.getSchemaTypes().get(columnId); - int fieldCount = type.getSubtypesCount(); + protected UnionTreeReader(int fileColumn, + TypeDescription readerSchema, + SchemaEvolution treeReaderSchema, + boolean[] included, + boolean skipCorrupt) throws IOException { + super(fileColumn); + List childrenTypes = readerSchema.getChildren(); + int fieldCount = childrenTypes.size(); this.fields = new TreeReader[fieldCount]; for (int i = 0; i < fieldCount; ++i) { - int subtype = type.getSubtypes(i); - if (included == null || included[subtype]) { - this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt); - } + TypeDescription subtype = childrenTypes.get(i); + this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt); } } @@ -2252,9 +2183,25 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previousVector, final int batchSize) throws IOException { - throw new UnsupportedOperationException( - "NextVector is not supported operation for Union type"); + public void nextVector(ColumnVector previousVector, + boolean[] isNull, + int batchSize) throws IOException { + UnionColumnVector result = (UnionColumnVector) previousVector; + super.nextVector(result, isNull, batchSize); + if (result.noNulls || !(result.isRepeating && result.isNull[0])) { + result.isRepeating = false; + tags.nextVector(result.noNulls ? null : result.isNull, result.tags, + batchSize); + boolean[] ignore = new boolean[(int) batchSize]; + for (int f = 0; f < result.fields.length; ++f) { + // build the ignore list for this tag + for (int r = 0; r < batchSize; ++r) { + ignore[r] = (!result.noNulls && result.isNull[r]) || + result.tags[r] != f; + } + fields[f].nextVector(result.fields[f], ignore, batchSize); + } + } } @Override @@ -2288,13 +2235,15 @@ void skipRows(long items) throws IOException { protected final TreeReader elementReader; protected IntegerReader lengths = null; - protected ListTreeReader(int columnId, - TreeReaderSchema treeReaderSchema, - boolean[] included, - boolean skipCorrupt) throws IOException { - super(columnId); - OrcProto.Type type = treeReaderSchema.getSchemaTypes().get(columnId); - elementReader = createTreeReader(type.getSubtypes(0), treeReaderSchema, included, skipCorrupt); + protected ListTreeReader(int fileColumn, + TypeDescription readerSchema, + SchemaEvolution treeReaderSchema, + boolean[] included, + boolean skipCorrupt) throws IOException { + super(fileColumn); + TypeDescription elementType = readerSchema.getChildren().get(0); + elementReader = createTreeReader(elementType, treeReaderSchema, included, + skipCorrupt); } @Override @@ -2335,9 +2284,27 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previous, final int batchSize) throws IOException { - throw new UnsupportedOperationException( - "NextVector is not supported operation for List type"); + public void nextVector(ColumnVector previous, + boolean[] isNull, + int batchSize) throws IOException { + ListColumnVector result = (ListColumnVector) previous; + super.nextVector(result, isNull, batchSize); + // if we have some none-null values, then read them + if (result.noNulls || !(result.isRepeating && result.isNull[0])) { + lengths.nextVector(result, result.lengths, batchSize); + // even with repeating lengths, the list doesn't repeat + result.isRepeating = false; + // build the offsets vector and figure out how many children to read + result.childCount = 0; + for (int r = 0; r < batchSize; ++r) { + if (result.noNulls || !result.isNull[r]) { + result.offsets[r] = result.childCount; + result.childCount += result.lengths[r]; + } + } + result.child.ensureSize(result.childCount, false); + elementReader.nextVector(result.child, null, result.childCount); + } } @Override @@ -2378,24 +2345,16 @@ void skipRows(long items) throws IOException { protected final TreeReader valueReader; protected IntegerReader lengths = null; - protected MapTreeReader(int columnId, - TreeReaderSchema treeReaderSchema, - boolean[] included, - boolean skipCorrupt) throws IOException { - super(columnId); - OrcProto.Type type = treeReaderSchema.getSchemaTypes().get(columnId); - int keyColumn = type.getSubtypes(0); - int valueColumn = type.getSubtypes(1); - if (included == null || included[keyColumn]) { - keyReader = createTreeReader(keyColumn, treeReaderSchema, included, skipCorrupt); - } else { - keyReader = null; - } - if (included == null || included[valueColumn]) { - valueReader = createTreeReader(valueColumn, treeReaderSchema, included, skipCorrupt); - } else { - valueReader = null; - } + protected MapTreeReader(int fileColumn, + TypeDescription readerSchema, + SchemaEvolution treeReaderSchema, + boolean[] included, + boolean skipCorrupt) throws IOException { + super(fileColumn); + TypeDescription keyType = readerSchema.getChildren().get(0); + TypeDescription valueType = readerSchema.getChildren().get(1); + keyReader = createTreeReader(keyType, treeReaderSchema, included, skipCorrupt); + valueReader = createTreeReader(valueType, treeReaderSchema, included, skipCorrupt); } @Override @@ -2429,9 +2388,28 @@ Object next(Object previous) throws IOException { } @Override - public Object nextVector(Object previous, final int batchSize) throws IOException { - throw new UnsupportedOperationException( - "NextVector is not supported operation for Map type"); + public void nextVector(ColumnVector previous, + boolean[] isNull, + int batchSize) throws IOException { + MapColumnVector result = (MapColumnVector) previous; + super.nextVector(result, isNull, batchSize); + if (result.noNulls || !(result.isRepeating && result.isNull[0])) { + lengths.nextVector(result, result.lengths, batchSize); + // even with repeating lengths, the map doesn't repeat + result.isRepeating = false; + // build the offsets vector and figure out how many children to read + result.childCount = 0; + for (int r = 0; r < batchSize; ++r) { + if (result.noNulls || !result.isNull[r]) { + result.offsets[r] = result.childCount; + result.childCount += result.lengths[r]; + } + } + result.keys.ensureSize(result.childCount, false); + result.values.ensureSize(result.childCount, false); + keyReader.nextVector(result.keys, null, result.childCount); + valueReader.nextVector(result.values, null, result.childCount); + } } @Override @@ -2471,61 +2449,61 @@ void skipRows(long items) throws IOException { } } - public static TreeReader createTreeReader(int columnId, - TreeReaderSchema treeReaderSchema, - boolean[] included, - boolean skipCorrupt - ) throws IOException { - OrcProto.Type type = treeReaderSchema.getSchemaTypes().get(columnId); - switch (type.getKind()) { + public static TreeReader createTreeReader(TypeDescription readerType, + SchemaEvolution evolution, + boolean[] included, + boolean skipCorrupt + ) throws IOException { + TypeDescription fileType = evolution.getFileType(readerType); + if (fileType == null || + (included != null && !included[readerType.getId()])) { + return new NullTreeReader(0); + } + switch (readerType.getCategory()) { case BOOLEAN: - return new BooleanTreeReader(columnId); + return new BooleanTreeReader(fileType.getId()); case BYTE: - return new ByteTreeReader(columnId); + return new ByteTreeReader(fileType.getId()); case DOUBLE: - return new DoubleTreeReader(columnId); + return new DoubleTreeReader(fileType.getId()); case FLOAT: - return new FloatTreeReader(columnId); + return new FloatTreeReader(fileType.getId()); case SHORT: - return new ShortTreeReader(columnId); + return new ShortTreeReader(fileType.getId()); case INT: - return new IntTreeReader(columnId); + return new IntTreeReader(fileType.getId()); case LONG: - return new LongTreeReader(columnId, skipCorrupt); + return new LongTreeReader(fileType.getId(), skipCorrupt); case STRING: - return new StringTreeReader(columnId); + return new StringTreeReader(fileType.getId()); case CHAR: - if (!type.hasMaximumLength()) { - throw new IllegalArgumentException("ORC char type has no length specified"); - } - return new CharTreeReader(columnId, type.getMaximumLength()); + return new CharTreeReader(fileType.getId(), readerType.getMaxLength()); case VARCHAR: - if (!type.hasMaximumLength()) { - throw new IllegalArgumentException("ORC varchar type has no length specified"); - } - return new VarcharTreeReader(columnId, type.getMaximumLength()); + return new VarcharTreeReader(fileType.getId(), readerType.getMaxLength()); case BINARY: - return new BinaryTreeReader(columnId); + return new BinaryTreeReader(fileType.getId()); case TIMESTAMP: - return new TimestampTreeReader(columnId, skipCorrupt); + return new TimestampTreeReader(fileType.getId(), skipCorrupt); case DATE: - return new DateTreeReader(columnId); + return new DateTreeReader(fileType.getId()); case DECIMAL: - int precision = - type.hasPrecision() ? type.getPrecision() : HiveDecimal.SYSTEM_DEFAULT_PRECISION; - int scale = type.hasScale() ? type.getScale() : HiveDecimal.SYSTEM_DEFAULT_SCALE; - return new DecimalTreeReader(columnId, precision, scale); + return new DecimalTreeReader(fileType.getId(), readerType.getPrecision(), + readerType.getScale()); case STRUCT: - return new StructTreeReader(columnId, treeReaderSchema, included, skipCorrupt); + return new StructTreeReader(fileType.getId(), readerType, + evolution, included, skipCorrupt); case LIST: - return new ListTreeReader(columnId, treeReaderSchema, included, skipCorrupt); + return new ListTreeReader(fileType.getId(), readerType, + evolution, included, skipCorrupt); case MAP: - return new MapTreeReader(columnId, treeReaderSchema, included, skipCorrupt); + return new MapTreeReader(fileType.getId(), readerType, evolution, + included, skipCorrupt); case UNION: - return new UnionTreeReader(columnId, treeReaderSchema, included, skipCorrupt); + return new UnionTreeReader(fileType.getId(), readerType, + evolution, included, skipCorrupt); default: throw new IllegalArgumentException("Unsupported type " + - type.getKind()); + readerType.getCategory()); } } } diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcInputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcInputFormat.java index 816b52d..897ac76 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcInputFormat.java @@ -71,14 +71,29 @@ OrcInputFormat.raiseAcidTablesMustBeReadWithAcidReaderException(conf); } + rbCtx = Utilities.getVectorizedRowBatchCtx(conf); /** * Do we have schema on read in the configuration variables? */ - TypeDescription schema = OrcInputFormat.getDesiredRowTypeDescr(conf, /* isAcidRead */ false); - List types = file.getTypes(); - Reader.Options options = new Reader.Options(); - options.schema(schema); + int dataColumns = rbCtx.getDataColumnCount(); + TypeDescription schema = + OrcInputFormat.getDesiredRowTypeDescr(conf, false, dataColumns); + if (schema == null) { + schema = file.getSchema(); + // Even if the user isn't doing schema evolution, cut the schema + // to the desired size. + if (schema.getCategory() == TypeDescription.Category.STRUCT && + schema.getChildren().size() > dataColumns) { + schema = schema.clone(); + List children = schema.getChildren(); + for(int c = children.size() - 1; c >= dataColumns; --c) { + children.remove(c); + } + } + } + Reader.Options options = new Reader.Options().schema(schema); + this.offset = fileSplit.getStart(); this.length = fileSplit.getLength(); options.range(offset, length); @@ -87,8 +102,6 @@ this.reader = file.rowsOptions(options); - rbCtx = Utilities.getVectorizedRowBatchCtx(conf); - columnsToIncludeTruncated = rbCtx.getColumnsToIncludeTruncated(conf); int partitionColumnCount = rbCtx.getPartitionColumnCount(); diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java index 70fe803..8e52907 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java @@ -101,8 +101,6 @@ } } - private static final long NANOS_PER_MILLI = 1000000; - /** * Set the value for a given column value within a batch. * @param rowId the row to set diff --git ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java index 2a82092..96af65a 100644 --- ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java +++ ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java @@ -51,11 +51,11 @@ public void testJson() { .addField("f4", TypeDescription.createDouble()) .addField("f5", TypeDescription.createBoolean())) .addField("f6", TypeDescription.createChar().withMaxLength(100)); - assertEquals("struct,f2:struct,f6:char(100)>", + assertEquals("struct,f2:struct,f6:char(100)>", struct.toString()); assertEquals( "{\"category\": \"struct\", \"id\": 0, \"max\": 8, \"fields\": [\n" + - " \"f1\": {\"category\": \"union\", \"id\": 1, \"max\": 3, \"children\": [\n" + + " \"f1\": {\"category\": \"uniontype\", \"id\": 1, \"max\": 3, \"children\": [\n" + " {\"category\": \"tinyint\", \"id\": 2, \"max\": 2},\n" + " {\"category\": \"decimal\", \"id\": 3, \"max\": 3, \"precision\": 20, \"scale\": 10}]},\n" + " \"f2\": {\"category\": \"struct\", \"id\": 4, \"max\": 7, \"fields\": [\n" + diff --git ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java index a82d672..460c925 100644 --- ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java +++ ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java @@ -32,40 +32,14 @@ import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector; import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils; import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory; -import org.apache.hadoop.hive.serde2.io.ByteWritable; import org.apache.hadoop.hive.serde2.io.DateWritable; -import org.apache.hadoop.hive.serde2.io.DoubleWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; -import org.apache.hadoop.hive.serde2.io.ShortWritable; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; -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.ObjectInspectorFactory; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -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.DoubleObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector; -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.io.BooleanWritable; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.FloatWritable; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hive.common.util.HiveTestUtils; import org.apache.orc.BinaryColumnStatistics; @@ -117,6 +91,10 @@ public static class InnerStruct { int int1; Text string1 = new Text(); + InnerStruct(int int1, Text string1) { + this.int1 = int1; + this.string1.set(string1); + } InnerStruct(int int1, String string1) { this.int1 = int1; this.string1.set(string1); @@ -136,50 +114,6 @@ public String toString() { } } - public static class BigRow { - Boolean boolean1; - Byte byte1; - Short short1; - Integer int1; - Long long1; - Float float1; - Double double1; - BytesWritable bytes1; - Text string1; - MiddleStruct middle; - List list = new ArrayList(); - Map map = new HashMap(); - - BigRow(Boolean b1, Byte b2, Short s1, Integer i1, Long l1, Float f1, - Double d1, - BytesWritable b3, String s2, MiddleStruct m1, - List l2, Map m2) { - this.boolean1 = b1; - this.byte1 = b2; - this.short1 = s1; - this.int1 = i1; - this.long1 = l1; - this.float1 = f1; - this.double1 = d1; - this.bytes1 = b3; - if (s2 == null) { - this.string1 = null; - } else { - this.string1 = new Text(s2); - } - this.middle = m1; - this.list = l2; - if (m2 != null) { - this.map = new HashMap(); - for (Map.Entry item : m2.entrySet()) { - this.map.put(new Text(item.getKey()), item.getValue()); - } - } else { - this.map = null; - } - } - } - private static InnerStruct inner(int i, String s) { return new InnerStruct(i, s); } @@ -301,206 +235,115 @@ public void testReadFormat_0_11() throws Exception { assertEquals("count: 7500 hasNull: true min: bye max: hi sum: 0", stats[9].toString()); // check the inspectors - StructObjectInspector readerInspector = (StructObjectInspector) reader - .getObjectInspector(); - assertEquals(ObjectInspector.Category.STRUCT, readerInspector.getCategory()); + TypeDescription schema = reader.getSchema(); + assertEquals(TypeDescription.Category.STRUCT, schema.getCategory()); assertEquals("struct>>,list:array>," + "map:map>,ts:timestamp," - + "decimal1:decimal(38,18)>", readerInspector.getTypeName()); - List fields = readerInspector - .getAllStructFieldRefs(); - BooleanObjectInspector bo = (BooleanObjectInspector) readerInspector - .getStructFieldRef("boolean1").getFieldObjectInspector(); - ByteObjectInspector by = (ByteObjectInspector) readerInspector - .getStructFieldRef("byte1").getFieldObjectInspector(); - ShortObjectInspector sh = (ShortObjectInspector) readerInspector - .getStructFieldRef("short1").getFieldObjectInspector(); - IntObjectInspector in = (IntObjectInspector) readerInspector - .getStructFieldRef("int1").getFieldObjectInspector(); - LongObjectInspector lo = (LongObjectInspector) readerInspector - .getStructFieldRef("long1").getFieldObjectInspector(); - FloatObjectInspector fl = (FloatObjectInspector) readerInspector - .getStructFieldRef("float1").getFieldObjectInspector(); - DoubleObjectInspector dbl = (DoubleObjectInspector) readerInspector - .getStructFieldRef("double1").getFieldObjectInspector(); - BinaryObjectInspector bi = (BinaryObjectInspector) readerInspector - .getStructFieldRef("bytes1").getFieldObjectInspector(); - StringObjectInspector st = (StringObjectInspector) readerInspector - .getStructFieldRef("string1").getFieldObjectInspector(); - StructObjectInspector mid = (StructObjectInspector) readerInspector - .getStructFieldRef("middle").getFieldObjectInspector(); - List midFields = mid.getAllStructFieldRefs(); - ListObjectInspector midli = (ListObjectInspector) midFields.get(0) - .getFieldObjectInspector(); - StructObjectInspector inner = (StructObjectInspector) midli - .getListElementObjectInspector(); - List inFields = inner.getAllStructFieldRefs(); - ListObjectInspector li = (ListObjectInspector) readerInspector - .getStructFieldRef("list").getFieldObjectInspector(); - MapObjectInspector ma = (MapObjectInspector) readerInspector - .getStructFieldRef("map").getFieldObjectInspector(); - TimestampObjectInspector tso = (TimestampObjectInspector) readerInspector - .getStructFieldRef("ts").getFieldObjectInspector(); - HiveDecimalObjectInspector dco = (HiveDecimalObjectInspector) readerInspector - .getStructFieldRef("decimal1").getFieldObjectInspector(); - StringObjectInspector mk = (StringObjectInspector) ma - .getMapKeyObjectInspector(); + + "decimal1:decimal(38,10)>", schema.toString()); + VectorizedRowBatch batch = schema.createRowBatch(); + RecordReader rows = reader.rows(); - Object row = rows.next(null); - assertNotNull(row); + assertEquals(true, rows.nextBatch(batch)); + assertEquals(1024, batch.size); + // check the contents of the first row - assertEquals(false, - bo.get(readerInspector.getStructFieldData(row, fields.get(0)))); - assertEquals(1, - by.get(readerInspector.getStructFieldData(row, fields.get(1)))); - assertEquals(1024, - sh.get(readerInspector.getStructFieldData(row, fields.get(2)))); - assertEquals(65536, - in.get(readerInspector.getStructFieldData(row, fields.get(3)))); - assertEquals(Long.MAX_VALUE, - lo.get(readerInspector.getStructFieldData(row, fields.get(4)))); - assertEquals(1.0, - fl.get(readerInspector.getStructFieldData(row, fields.get(5))), 0.00001); - assertEquals(-15.0, - dbl.get(readerInspector.getStructFieldData(row, fields.get(6))), - 0.00001); - assertEquals(bytes(0, 1, 2, 3, 4), - bi.getPrimitiveWritableObject(readerInspector.getStructFieldData(row, - fields.get(7)))); - assertEquals("hi", st.getPrimitiveJavaObject(readerInspector - .getStructFieldData(row, fields.get(8)))); - List midRow = midli.getList(mid.getStructFieldData( - readerInspector.getStructFieldData(row, fields.get(9)), - midFields.get(0))); - assertNotNull(midRow); - assertEquals(2, midRow.size()); - assertEquals(1, - in.get(inner.getStructFieldData(midRow.get(0), inFields.get(0)))); - assertEquals("bye", st.getPrimitiveJavaObject(inner.getStructFieldData( - midRow.get(0), inFields.get(1)))); - assertEquals(2, - in.get(inner.getStructFieldData(midRow.get(1), inFields.get(0)))); - assertEquals("sigh", st.getPrimitiveJavaObject(inner.getStructFieldData( - midRow.get(1), inFields.get(1)))); - List list = li.getList(readerInspector.getStructFieldData(row, - fields.get(10))); - assertEquals(2, list.size()); - assertEquals(3, - in.get(inner.getStructFieldData(list.get(0), inFields.get(0)))); - assertEquals("good", st.getPrimitiveJavaObject(inner.getStructFieldData( - list.get(0), inFields.get(1)))); - assertEquals(4, - in.get(inner.getStructFieldData(list.get(1), inFields.get(0)))); - assertEquals("bad", st.getPrimitiveJavaObject(inner.getStructFieldData( - list.get(1), inFields.get(1)))); - Map map = ma.getMap(readerInspector.getStructFieldData(row, - fields.get(11))); - assertEquals(0, map.size()); + assertEquals(false, getBoolean(batch, 0)); + assertEquals(1, getByte(batch, 0)); + assertEquals(1024, getShort(batch, 0)); + assertEquals(65536, getInt(batch, 0)); + assertEquals(Long.MAX_VALUE, getLong(batch, 0)); + assertEquals(1.0, getFloat(batch, 0), 0.00001); + assertEquals(-15.0, getDouble(batch, 0), 0.00001); + assertEquals(bytes(0, 1, 2, 3, 4), getBinary(batch, 0)); + assertEquals("hi", getText(batch, 0).toString()); + + StructColumnVector middle = (StructColumnVector) batch.cols[9]; + ListColumnVector midList = (ListColumnVector) middle.fields[0]; + StructColumnVector midListStruct = (StructColumnVector) midList.child; + LongColumnVector midListInt = (LongColumnVector) midListStruct.fields[0]; + BytesColumnVector midListStr = (BytesColumnVector) midListStruct.fields[1]; + ListColumnVector list = (ListColumnVector) batch.cols[10]; + StructColumnVector listStruct = (StructColumnVector) list.child; + LongColumnVector listInts = (LongColumnVector) listStruct.fields[0]; + BytesColumnVector listStrs = (BytesColumnVector) listStruct.fields[1]; + MapColumnVector map = (MapColumnVector) batch.cols[11]; + BytesColumnVector mapKey = (BytesColumnVector) map.keys; + StructColumnVector mapValue = (StructColumnVector) map.values; + LongColumnVector mapValueInts = (LongColumnVector) mapValue.fields[0]; + BytesColumnVector mapValueStrs = (BytesColumnVector) mapValue.fields[1]; + TimestampColumnVector timestamp = (TimestampColumnVector) batch.cols[12]; + DecimalColumnVector decs = (DecimalColumnVector) batch.cols[13]; + + assertEquals(false, middle.isNull[0]); + assertEquals(2, midList.lengths[0]); + int start = (int) midList.offsets[0]; + assertEquals(1, midListInt.vector[start]); + assertEquals("bye", midListStr.toString(start)); + assertEquals(2, midListInt.vector[start + 1]); + assertEquals("sigh", midListStr.toString(start + 1)); + + assertEquals(2, list.lengths[0]); + start = (int) list.offsets[0]; + assertEquals(3, listInts.vector[start]); + assertEquals("good", listStrs.toString(start)); + assertEquals(4, listInts.vector[start + 1]); + assertEquals("bad", listStrs.toString(start + 1)); + assertEquals(0, map.lengths[0]); assertEquals(Timestamp.valueOf("2000-03-12 15:00:00"), - tso.getPrimitiveJavaObject(readerInspector.getStructFieldData(row, - fields.get(12)))); - assertEquals(HiveDecimal.create("12345678.6547456"), - dco.getPrimitiveJavaObject(readerInspector.getStructFieldData(row, - fields.get(13)))); + timestamp.asScratchTimestamp(0)); + assertEquals(new HiveDecimalWritable(HiveDecimal.create("12345678.6547456")), + decs.vector[0]); - // check the contents of second row - assertEquals(true, rows.hasNext()); + // check the contents of row 7499 rows.seekToRow(7499); - row = rows.next(null); - assertEquals(true, - bo.get(readerInspector.getStructFieldData(row, fields.get(0)))); - assertEquals(100, - by.get(readerInspector.getStructFieldData(row, fields.get(1)))); - assertEquals(2048, - sh.get(readerInspector.getStructFieldData(row, fields.get(2)))); - assertEquals(65536, - in.get(readerInspector.getStructFieldData(row, fields.get(3)))); - assertEquals(Long.MAX_VALUE, - lo.get(readerInspector.getStructFieldData(row, fields.get(4)))); - assertEquals(2.0, - fl.get(readerInspector.getStructFieldData(row, fields.get(5))), 0.00001); - assertEquals(-5.0, - dbl.get(readerInspector.getStructFieldData(row, fields.get(6))), - 0.00001); - assertEquals(bytes(), bi.getPrimitiveWritableObject(readerInspector - .getStructFieldData(row, fields.get(7)))); - assertEquals("bye", st.getPrimitiveJavaObject(readerInspector - .getStructFieldData(row, fields.get(8)))); - midRow = midli.getList(mid.getStructFieldData( - readerInspector.getStructFieldData(row, fields.get(9)), - midFields.get(0))); - assertNotNull(midRow); - assertEquals(2, midRow.size()); - assertEquals(1, - in.get(inner.getStructFieldData(midRow.get(0), inFields.get(0)))); - assertEquals("bye", st.getPrimitiveJavaObject(inner.getStructFieldData( - midRow.get(0), inFields.get(1)))); - assertEquals(2, - in.get(inner.getStructFieldData(midRow.get(1), inFields.get(0)))); - assertEquals("sigh", st.getPrimitiveJavaObject(inner.getStructFieldData( - midRow.get(1), inFields.get(1)))); - list = li.getList(readerInspector.getStructFieldData(row, fields.get(10))); - assertEquals(3, list.size()); - assertEquals(100000000, - in.get(inner.getStructFieldData(list.get(0), inFields.get(0)))); - assertEquals("cat", st.getPrimitiveJavaObject(inner.getStructFieldData( - list.get(0), inFields.get(1)))); - assertEquals(-100000, - in.get(inner.getStructFieldData(list.get(1), inFields.get(0)))); - assertEquals("in", st.getPrimitiveJavaObject(inner.getStructFieldData( - list.get(1), inFields.get(1)))); - assertEquals(1234, - in.get(inner.getStructFieldData(list.get(2), inFields.get(0)))); - assertEquals("hat", st.getPrimitiveJavaObject(inner.getStructFieldData( - list.get(2), inFields.get(1)))); - map = ma.getMap(readerInspector.getStructFieldData(row, fields.get(11))); - assertEquals(2, map.size()); - boolean[] found = new boolean[2]; - for(Object key : map.keySet()) { - String str = mk.getPrimitiveJavaObject(key); - if (str.equals("chani")) { - assertEquals(false, found[0]); - assertEquals(5, - in.get(inner.getStructFieldData(map.get(key), inFields.get(0)))); - assertEquals(str, st.getPrimitiveJavaObject(inner.getStructFieldData( - map.get(key), inFields.get(1)))); - found[0] = true; - } else if (str.equals("mauddib")) { - assertEquals(false, found[1]); - assertEquals(1, - in.get(inner.getStructFieldData(map.get(key), inFields.get(0)))); - assertEquals(str, st.getPrimitiveJavaObject(inner.getStructFieldData( - map.get(key), inFields.get(1)))); - found[1] = true; - } else { - throw new IllegalArgumentException("Unknown key " + str); - } - } - assertEquals(true, found[0]); - assertEquals(true, found[1]); + assertEquals(true, rows.nextBatch(batch)); + assertEquals(true, getBoolean(batch, 0)); + assertEquals(100, getByte(batch, 0)); + assertEquals(2048, getShort(batch, 0)); + assertEquals(65536, getInt(batch, 0)); + assertEquals(Long.MAX_VALUE, getLong(batch, 0)); + assertEquals(2.0, getFloat(batch, 0), 0.00001); + assertEquals(-5.0, getDouble(batch, 0), 0.00001); + assertEquals(bytes(), getBinary(batch, 0)); + assertEquals("bye", getText(batch, 0).toString()); + assertEquals(false, middle.isNull[0]); + assertEquals(2, midList.lengths[0]); + start = (int) midList.offsets[0]; + assertEquals(1, midListInt.vector[start]); + assertEquals("bye", midListStr.toString(start)); + assertEquals(2, midListInt.vector[start + 1]); + assertEquals("sigh", midListStr.toString(start + 1)); + assertEquals(3, list.lengths[0]); + start = (int) list.offsets[0]; + assertEquals(100000000, listInts.vector[start]); + assertEquals("cat", listStrs.toString(start)); + assertEquals(-100000, listInts.vector[start + 1]); + assertEquals("in", listStrs.toString(start + 1)); + assertEquals(1234, listInts.vector[start + 2]); + assertEquals("hat", listStrs.toString(start + 2)); + assertEquals(2, map.lengths[0]); + start = (int) map.offsets[0]; + assertEquals("chani", mapKey.toString(start)); + assertEquals(5, mapValueInts.vector[start]); + assertEquals("chani", mapValueStrs.toString(start)); + assertEquals("mauddib", mapKey.toString(start + 1)); + assertEquals(1, mapValueInts.vector[start + 1]); + assertEquals("mauddib", mapValueStrs.toString(start + 1)); assertEquals(Timestamp.valueOf("2000-03-12 15:00:01"), - tso.getPrimitiveJavaObject(readerInspector.getStructFieldData(row, - fields.get(12)))); - assertEquals(HiveDecimal.create("12345678.6547457"), - dco.getPrimitiveJavaObject(readerInspector.getStructFieldData(row, - fields.get(13)))); + timestamp.asScratchTimestamp(0)); + assertEquals(new HiveDecimalWritable(HiveDecimal.create("12345678.6547457")), + decs.vector[0]); // handle the close up - assertEquals(false, rows.hasNext()); + assertEquals(false, rows.nextBatch(batch)); rows.close(); } @Test public void testTimestamp() throws Exception { - ObjectInspector inspector; - synchronized (TestVectorOrcFile.class) { - inspector = ObjectInspectorFactory.getReflectionObjectInspector(Timestamp.class, - ObjectInspectorFactory.ObjectInspectorOptions.JAVA); - } - TypeDescription schema = TypeDescription.createTimestamp(); Writer writer = OrcFile.createWriter(testFilePath, OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000) @@ -533,11 +376,15 @@ public void testTimestamp() throws Exception { Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs)); - RecordReader rows = reader.rows(null); + RecordReader rows = reader.rows(); + batch = reader.getSchema().createRowBatch(); + TimestampColumnVector timestamps = (TimestampColumnVector) batch.cols[0]; int idx = 0; - while (rows.hasNext()) { - Object row = rows.next(null); - assertEquals(tslist.get(idx++).getNanos(), ((TimestampWritable) row).getNanos()); + while (rows.nextBatch(batch)) { + for(int r=0; r < batch.size; ++r) { + assertEquals(tslist.get(idx++).getNanos(), + timestamps.asScratchTimestamp(r).getNanos()); + } } assertEquals(tslist.size(), rows.getRowNumber()); assertEquals(0, writer.getSchema().getMaximumId()); @@ -608,50 +455,28 @@ public void testStringAndBinaryStatistics() throws Exception { stats[2].toString()); // check the inspectors - StructObjectInspector readerInspector = - (StructObjectInspector) reader.getObjectInspector(); - assertEquals(ObjectInspector.Category.STRUCT, - readerInspector.getCategory()); - assertEquals("struct", - readerInspector.getTypeName()); - List fields = - readerInspector.getAllStructFieldRefs(); - BinaryObjectInspector bi = (BinaryObjectInspector) readerInspector. - getStructFieldRef("bytes1").getFieldObjectInspector(); - StringObjectInspector st = (StringObjectInspector) readerInspector. - getStructFieldRef("string1").getFieldObjectInspector(); + batch = reader.getSchema().createRowBatch(); + BytesColumnVector bytes = (BytesColumnVector) batch.cols[0]; + BytesColumnVector strs = (BytesColumnVector) batch.cols[1]; RecordReader rows = reader.rows(); - Object row = rows.next(null); - assertNotNull(row); + assertEquals(true, rows.nextBatch(batch)); + assertEquals(4, batch.size); + // check the contents of the first row - assertEquals(bytes(0,1,2,3,4), bi.getPrimitiveWritableObject( - readerInspector.getStructFieldData(row, fields.get(0)))); - assertEquals("foo", st.getPrimitiveJavaObject(readerInspector. - getStructFieldData(row, fields.get(1)))); + assertEquals(bytes(0,1,2,3,4), getBinary(bytes, 0)); + assertEquals("foo", strs.toString(0)); // check the contents of second row - assertEquals(true, rows.hasNext()); - row = rows.next(row); - assertEquals(bytes(0,1,2,3), bi.getPrimitiveWritableObject( - readerInspector.getStructFieldData(row, fields.get(0)))); - assertEquals("bar", st.getPrimitiveJavaObject(readerInspector. - getStructFieldData(row, fields.get(1)))); + assertEquals(bytes(0,1,2,3), getBinary(bytes, 1)); + assertEquals("bar", strs.toString(1)); // check the contents of third row - assertEquals(true, rows.hasNext()); - row = rows.next(row); - assertEquals(bytes(0,1,2,3,4,5), bi.getPrimitiveWritableObject( - readerInspector.getStructFieldData(row, fields.get(0)))); - assertNull(st.getPrimitiveJavaObject(readerInspector. - getStructFieldData(row, fields.get(1)))); + assertEquals(bytes(0,1,2,3,4,5), getBinary(bytes, 2)); + assertNull(strs.toString(2)); // check the contents of fourth row - assertEquals(true, rows.hasNext()); - row = rows.next(row); - assertNull(bi.getPrimitiveWritableObject( - readerInspector.getStructFieldData(row, fields.get(0)))); - assertEquals("hi", st.getPrimitiveJavaObject(readerInspector. - getStructFieldData(row, fields.get(1)))); + assertNull(getBinary(bytes, 3)); + assertEquals("hi", strs.toString(3)); // handle the close up assertEquals(false, rows.hasNext()); @@ -767,6 +592,19 @@ private static void setInner(StructColumnVector inner, int rowId, } } + private static void checkInner(StructColumnVector inner, int rowId, + int rowInBatch, int i, String value) { + assertEquals("row " + rowId, i, + ((LongColumnVector) inner.fields[0]).vector[rowInBatch]); + if (value != null) { + assertEquals("row " + rowId, value, + ((BytesColumnVector) inner.fields[1]).toString(rowInBatch)); + } else { + assertEquals("row " + rowId, true, inner.fields[1].isNull[rowInBatch]); + assertEquals("row " + rowId, false, inner.fields[1].noNulls); + } + } + private static void setInnerList(ListColumnVector list, int rowId, List value) { if (value != null) { @@ -787,6 +625,23 @@ private static void setInnerList(ListColumnVector list, int rowId, } } + private static void checkInnerList(ListColumnVector list, int rowId, + int rowInBatch, List value) { + if (value != null) { + assertEquals("row " + rowId, value.size(), list.lengths[rowInBatch]); + int start = (int) list.offsets[rowInBatch]; + for (int i = 0; i < list.lengths[rowInBatch]; ++i) { + InnerStruct inner = value.get(i); + checkInner((StructColumnVector) list.child, rowId, i + start, + inner.int1, inner.string1.toString()); + } + list.childCount += value.size(); + } else { + assertEquals("row " + rowId, true, list.isNull[rowInBatch]); + assertEquals("row " + rowId, false, list.noNulls); + } + } + private static void setInnerMap(MapColumnVector map, int rowId, Map value) { if (value != null) { @@ -812,6 +667,24 @@ private static void setInnerMap(MapColumnVector map, int rowId, } } + private static void checkInnerMap(MapColumnVector map, int rowId, + int rowInBatch, + Map value) { + if (value != null) { + assertEquals("row " + rowId, value.size(), map.lengths[rowInBatch]); + int offset = (int) map.offsets[rowInBatch]; + for(int i=0; i < value.size(); ++i) { + String key = ((BytesColumnVector) map.keys).toString(offset + i); + InnerStruct expected = value.get(key); + checkInner((StructColumnVector) map.values, rowId, offset + i, + expected.int1, expected.string1.toString()); + } + } else { + assertEquals("row " + rowId, true, map.isNull[rowId]); + assertEquals("row " + rowId, false, map.noNulls); + } + } + private static void setMiddleStruct(StructColumnVector middle, int rowId, MiddleStruct value) { if (value != null) { @@ -822,6 +695,17 @@ private static void setMiddleStruct(StructColumnVector middle, int rowId, } } + private static void checkMiddleStruct(StructColumnVector middle, int rowId, + int rowInBatch, MiddleStruct value) { + if (value != null) { + checkInnerList((ListColumnVector) middle.fields[0], rowId, rowInBatch, + value.list); + } else { + assertEquals("row " + rowId, true, middle.isNull[rowInBatch]); + assertEquals("row " + rowId, false, middle.noNulls); + } + } + private static void setBigRow(VectorizedRowBatch batch, int rowId, Boolean b1, Byte b2, Short s1, Integer i1, Long l1, Float f1, @@ -853,6 +737,160 @@ private static void setBigRow(VectorizedRowBatch batch, int rowId, setInnerMap((MapColumnVector) batch.cols[11], rowId, m2); } + private static void checkBigRow(VectorizedRowBatch batch, + int rowInBatch, + int rowId, + boolean b1, byte b2, short s1, + int i1, long l1, float f1, + double d1, BytesWritable b3, String s2, + MiddleStruct m1, List l2, + Map m2) { + assertEquals("row " + rowId, b1, getBoolean(batch, rowInBatch)); + assertEquals("row " + rowId, b2, getByte(batch, rowInBatch)); + assertEquals("row " + rowId, s1, getShort(batch, rowInBatch)); + assertEquals("row " + rowId, i1, getInt(batch, rowInBatch)); + assertEquals("row " + rowId, l1, getLong(batch, rowInBatch)); + assertEquals("row " + rowId, f1, getFloat(batch, rowInBatch), 0.0001); + assertEquals("row " + rowId, d1, getDouble(batch, rowInBatch), 0.0001); + if (b3 != null) { + BytesColumnVector bytes = (BytesColumnVector) batch.cols[7]; + assertEquals("row " + rowId, b3.getLength(), bytes.length[rowInBatch]); + for(int i=0; i < b3.getLength(); ++i) { + assertEquals("row " + rowId + " byte " + i, b3.getBytes()[i], + bytes.vector[rowInBatch][bytes.start[rowInBatch] + i]); + } + } else { + assertEquals("row " + rowId, true, batch.cols[7].isNull[rowInBatch]); + assertEquals("row " + rowId, false, batch.cols[7].noNulls); + } + if (s2 != null) { + assertEquals("row " + rowId, s2, getText(batch, rowInBatch).toString()); + } else { + assertEquals("row " + rowId, true, batch.cols[8].isNull[rowInBatch]); + assertEquals("row " + rowId, false, batch.cols[8].noNulls); + } + checkMiddleStruct((StructColumnVector) batch.cols[9], rowId, rowInBatch, + m1); + checkInnerList((ListColumnVector) batch.cols[10], rowId, rowInBatch, l2); + checkInnerMap((MapColumnVector) batch.cols[11], rowId, rowInBatch, m2); + } + + private static boolean getBoolean(VectorizedRowBatch batch, int rowId) { + return ((LongColumnVector) batch.cols[0]).vector[rowId] != 0; + } + + private static byte getByte(VectorizedRowBatch batch, int rowId) { + return (byte) ((LongColumnVector) batch.cols[1]).vector[rowId]; + } + + private static short getShort(VectorizedRowBatch batch, int rowId) { + return (short) ((LongColumnVector) batch.cols[2]).vector[rowId]; + } + + private static int getInt(VectorizedRowBatch batch, int rowId) { + return (int) ((LongColumnVector) batch.cols[3]).vector[rowId]; + } + + private static long getLong(VectorizedRowBatch batch, int rowId) { + return ((LongColumnVector) batch.cols[4]).vector[rowId]; + } + + private static float getFloat(VectorizedRowBatch batch, int rowId) { + return (float) ((DoubleColumnVector) batch.cols[5]).vector[rowId]; + } + + private static double getDouble(VectorizedRowBatch batch, int rowId) { + return ((DoubleColumnVector) batch.cols[6]).vector[rowId]; + } + + private static BytesWritable getBinary(BytesColumnVector column, int rowId) { + if (column.isRepeating) { + rowId = 0; + } + if (column.noNulls || !column.isNull[rowId]) { + return new BytesWritable(Arrays.copyOfRange(column.vector[rowId], + column.start[rowId], column.start[rowId] + column.length[rowId])); + } else { + return null; + } + } + + private static BytesWritable getBinary(VectorizedRowBatch batch, int rowId) { + return getBinary((BytesColumnVector) batch.cols[7], rowId); + } + + private static Text getText(BytesColumnVector vector, int rowId) { + if (vector.isRepeating) { + rowId = 0; + } + if (vector.noNulls || !vector.isNull[rowId]) { + return new Text(Arrays.copyOfRange(vector.vector[rowId], + vector.start[rowId], vector.start[rowId] + vector.length[rowId])); + } else { + return null; + } + } + + private static Text getText(VectorizedRowBatch batch, int rowId) { + return getText((BytesColumnVector) batch.cols[8], rowId); + } + + private static InnerStruct getInner(StructColumnVector vector, + int rowId) { + return new InnerStruct( + (int) ((LongColumnVector) vector.fields[0]).vector[rowId], + getText((BytesColumnVector) vector.fields[1], rowId)); + } + + private static List getList(ListColumnVector cv, + int rowId) { + if (cv.isRepeating) { + rowId = 0; + } + if (cv.noNulls || !cv.isNull[rowId]) { + List result = + new ArrayList((int) cv.lengths[rowId]); + for(long i=cv.offsets[rowId]; + i < cv.offsets[rowId] + cv.lengths[rowId]; ++i) { + result.add(getInner((StructColumnVector) cv.child, (int) i)); + } + return result; + } else { + return null; + } + } + + private static List getMidList(VectorizedRowBatch batch, + int rowId) { + return getList((ListColumnVector) ((StructColumnVector) batch.cols[9]) + .fields[0], rowId); + } + + private static List getList(VectorizedRowBatch batch, + int rowId) { + return getList((ListColumnVector) batch.cols[10], rowId); + } + + private static Map getMap(VectorizedRowBatch batch, + int rowId) { + MapColumnVector cv = (MapColumnVector) batch.cols[11]; + if (cv.isRepeating) { + rowId = 0; + } + if (cv.noNulls || !cv.isNull[rowId]) { + Map result = + new HashMap((int) cv.lengths[rowId]); + for(long i=cv.offsets[rowId]; + i < cv.offsets[rowId] + cv.lengths[rowId]; ++i) { + result.put(getText((BytesColumnVector) cv.keys, (int) i), + getInner((StructColumnVector) cv.values, (int) i)); + } + return result; + } else { + return null; + } + } + private static TypeDescription createInnerSchema() { return TypeDescription.createStruct() .addField("int1", TypeDescription.createInt()) @@ -981,178 +1019,114 @@ public void test1() throws Exception { assertEquals("count: 2 hasNull: false min: bye max: hi sum: 5", stats[9].toString()); - // check the inspectors - StructObjectInspector readerInspector = - (StructObjectInspector) reader.getObjectInspector(); - assertEquals(ObjectInspector.Category.STRUCT, - readerInspector.getCategory()); + // check the schema + TypeDescription readerSchema = reader.getSchema(); + assertEquals(TypeDescription.Category.STRUCT, readerSchema.getCategory()); assertEquals("struct>>,list:array>," + "map:map>>", - readerInspector.getTypeName()); - List fields = - readerInspector.getAllStructFieldRefs(); - BooleanObjectInspector bo = (BooleanObjectInspector) readerInspector. - getStructFieldRef("boolean1").getFieldObjectInspector(); - ByteObjectInspector by = (ByteObjectInspector) readerInspector. - getStructFieldRef("byte1").getFieldObjectInspector(); - ShortObjectInspector sh = (ShortObjectInspector) readerInspector. - getStructFieldRef("short1").getFieldObjectInspector(); - IntObjectInspector in = (IntObjectInspector) readerInspector. - getStructFieldRef("int1").getFieldObjectInspector(); - LongObjectInspector lo = (LongObjectInspector) readerInspector. - getStructFieldRef("long1").getFieldObjectInspector(); - FloatObjectInspector fl = (FloatObjectInspector) readerInspector. - getStructFieldRef("float1").getFieldObjectInspector(); - DoubleObjectInspector dbl = (DoubleObjectInspector) readerInspector. - getStructFieldRef("double1").getFieldObjectInspector(); - BinaryObjectInspector bi = (BinaryObjectInspector) readerInspector. - getStructFieldRef("bytes1").getFieldObjectInspector(); - StringObjectInspector st = (StringObjectInspector) readerInspector. - getStructFieldRef("string1").getFieldObjectInspector(); - StructObjectInspector mid = (StructObjectInspector) readerInspector. - getStructFieldRef("middle").getFieldObjectInspector(); - List midFields = - mid.getAllStructFieldRefs(); - ListObjectInspector midli = - (ListObjectInspector) midFields.get(0).getFieldObjectInspector(); - StructObjectInspector inner = (StructObjectInspector) - midli.getListElementObjectInspector(); - List inFields = inner.getAllStructFieldRefs(); - ListObjectInspector li = (ListObjectInspector) readerInspector. - getStructFieldRef("list").getFieldObjectInspector(); - MapObjectInspector ma = (MapObjectInspector) readerInspector. - getStructFieldRef("map").getFieldObjectInspector(); - StringObjectInspector mk = (StringObjectInspector) - ma.getMapKeyObjectInspector(); + readerSchema.toString()); + List fieldNames = readerSchema.getFieldNames(); + List fieldTypes = readerSchema.getChildren(); + assertEquals("boolean1", fieldNames.get(0)); + assertEquals(TypeDescription.Category.BOOLEAN, fieldTypes.get(0).getCategory()); + assertEquals("byte1", fieldNames.get(1)); + assertEquals(TypeDescription.Category.BYTE, fieldTypes.get(1).getCategory()); + assertEquals("short1", fieldNames.get(2)); + assertEquals(TypeDescription.Category.SHORT, fieldTypes.get(2).getCategory()); + assertEquals("int1", fieldNames.get(3)); + assertEquals(TypeDescription.Category.INT, fieldTypes.get(3).getCategory()); + assertEquals("long1", fieldNames.get(4)); + assertEquals(TypeDescription.Category.LONG, fieldTypes.get(4).getCategory()); + assertEquals("float1", fieldNames.get(5)); + assertEquals(TypeDescription.Category.FLOAT, fieldTypes.get(5).getCategory()); + assertEquals("double1", fieldNames.get(6)); + assertEquals(TypeDescription.Category.DOUBLE, fieldTypes.get(6).getCategory()); + assertEquals("bytes1", fieldNames.get(7)); + assertEquals(TypeDescription.Category.BINARY, fieldTypes.get(7).getCategory()); + assertEquals("string1", fieldNames.get(8)); + assertEquals(TypeDescription.Category.STRING, fieldTypes.get(8).getCategory()); + assertEquals("middle", fieldNames.get(9)); + TypeDescription middle = fieldTypes.get(9); + assertEquals(TypeDescription.Category.STRUCT, middle.getCategory()); + TypeDescription midList = middle.getChildren().get(0); + assertEquals(TypeDescription.Category.LIST, midList.getCategory()); + TypeDescription inner = midList.getChildren().get(0); + assertEquals(TypeDescription.Category.STRUCT, inner.getCategory()); + assertEquals("int1", inner.getFieldNames().get(0)); + assertEquals("string1", inner.getFieldNames().get(1)); + RecordReader rows = reader.rows(); - Object row = rows.next(null); - assertNotNull(row); + // create a new batch + batch = readerSchema.createRowBatch(); + assertEquals(true, rows.nextBatch(batch)); + assertEquals(2, batch.size); + assertEquals(false, rows.hasNext()); + // check the contents of the first row - assertEquals(false, - bo.get(readerInspector.getStructFieldData(row, fields.get(0)))); - assertEquals(1, by.get(readerInspector.getStructFieldData(row, - fields.get(1)))); - assertEquals(1024, sh.get(readerInspector.getStructFieldData(row, - fields.get(2)))); - assertEquals(65536, in.get(readerInspector.getStructFieldData(row, - fields.get(3)))); - assertEquals(Long.MAX_VALUE, lo.get(readerInspector. - getStructFieldData(row, fields.get(4)))); - assertEquals(1.0, fl.get(readerInspector.getStructFieldData(row, - fields.get(5))), 0.00001); - assertEquals(-15.0, dbl.get(readerInspector.getStructFieldData(row, - fields.get(6))), 0.00001); - assertEquals(bytes(0,1,2,3,4), bi.getPrimitiveWritableObject( - readerInspector.getStructFieldData(row, fields.get(7)))); - assertEquals("hi", st.getPrimitiveJavaObject(readerInspector. - getStructFieldData(row, fields.get(8)))); - List midRow = midli.getList(mid.getStructFieldData(readerInspector. - getStructFieldData(row, fields.get(9)), midFields.get(0))); + assertEquals(false, getBoolean(batch, 0)); + assertEquals(1, getByte(batch, 0)); + assertEquals(1024, getShort(batch, 0)); + assertEquals(65536, getInt(batch, 0)); + assertEquals(Long.MAX_VALUE, getLong(batch, 0)); + assertEquals(1.0, getFloat(batch, 0), 0.00001); + assertEquals(-15.0, getDouble(batch, 0), 0.00001); + assertEquals(bytes(0,1,2,3,4), getBinary(batch, 0)); + assertEquals("hi", getText(batch, 0).toString()); + List midRow = getMidList(batch, 0); assertNotNull(midRow); assertEquals(2, midRow.size()); - assertEquals(1, in.get(inner.getStructFieldData(midRow.get(0), - inFields.get(0)))); - assertEquals("bye", st.getPrimitiveJavaObject(inner.getStructFieldData - (midRow.get(0), inFields.get(1)))); - assertEquals(2, in.get(inner.getStructFieldData(midRow.get(1), - inFields.get(0)))); - assertEquals("sigh", st.getPrimitiveJavaObject(inner.getStructFieldData - (midRow.get(1), inFields.get(1)))); - List list = li.getList(readerInspector.getStructFieldData(row, - fields.get(10))); + assertEquals(1, midRow.get(0).int1); + assertEquals("bye", midRow.get(0).string1.toString()); + assertEquals(2, midRow.get(1).int1); + assertEquals("sigh", midRow.get(1).string1.toString()); + List list = getList(batch, 0); assertEquals(2, list.size()); - assertEquals(3, in.get(inner.getStructFieldData(list.get(0), - inFields.get(0)))); - assertEquals("good", st.getPrimitiveJavaObject(inner.getStructFieldData - (list.get(0), inFields.get(1)))); - assertEquals(4, in.get(inner.getStructFieldData(list.get(1), - inFields.get(0)))); - assertEquals("bad", st.getPrimitiveJavaObject(inner.getStructFieldData - (list.get(1), inFields.get(1)))); - Map map = ma.getMap(readerInspector.getStructFieldData(row, - fields.get(11))); + assertEquals(3, list.get(0).int1); + assertEquals("good", list.get(0).string1.toString()); + assertEquals(4, list.get(1).int1); + assertEquals("bad", list.get(1).string1.toString()); + Map map = getMap(batch, 0); assertEquals(0, map.size()); // check the contents of second row - assertEquals(true, rows.hasNext()); - row = rows.next(row); - assertEquals(true, - bo.get(readerInspector.getStructFieldData(row, fields.get(0)))); - assertEquals(100, by.get(readerInspector.getStructFieldData(row, - fields.get(1)))); - assertEquals(2048, sh.get(readerInspector.getStructFieldData(row, - fields.get(2)))); - assertEquals(65536, in.get(readerInspector.getStructFieldData(row, - fields.get(3)))); - assertEquals(Long.MAX_VALUE, lo.get(readerInspector. - getStructFieldData(row, fields.get(4)))); - assertEquals(2.0, fl.get(readerInspector.getStructFieldData(row, - fields.get(5))), 0.00001); - assertEquals(-5.0, dbl.get(readerInspector.getStructFieldData(row, - fields.get(6))), 0.00001); - assertEquals(bytes(), bi.getPrimitiveWritableObject( - readerInspector.getStructFieldData(row, fields.get(7)))); - assertEquals("bye", st.getPrimitiveJavaObject(readerInspector. - getStructFieldData(row, fields.get(8)))); - midRow = midli.getList(mid.getStructFieldData(readerInspector. - getStructFieldData(row, fields.get(9)), midFields.get(0))); + assertEquals(true, getBoolean(batch, 1)); + assertEquals(100, getByte(batch, 1)); + assertEquals(2048, getShort(batch, 1)); + assertEquals(65536, getInt(batch, 1)); + assertEquals(Long.MAX_VALUE, getLong(batch, 1)); + assertEquals(2.0, getFloat(batch, 1), 0.00001); + assertEquals(-5.0, getDouble(batch, 1), 0.00001); + assertEquals(bytes(), getBinary(batch, 1)); + assertEquals("bye", getText(batch, 1).toString()); + midRow = getMidList(batch, 1); assertNotNull(midRow); assertEquals(2, midRow.size()); - assertEquals(1, in.get(inner.getStructFieldData(midRow.get(0), - inFields.get(0)))); - assertEquals("bye", st.getPrimitiveJavaObject(inner.getStructFieldData - (midRow.get(0), inFields.get(1)))); - assertEquals(2, in.get(inner.getStructFieldData(midRow.get(1), - inFields.get(0)))); - assertEquals("sigh", st.getPrimitiveJavaObject(inner.getStructFieldData - (midRow.get(1), inFields.get(1)))); - list = li.getList(readerInspector.getStructFieldData(row, - fields.get(10))); + assertEquals(1, midRow.get(0).int1); + assertEquals("bye", midRow.get(0).string1.toString()); + assertEquals(2, midRow.get(1).int1); + assertEquals("sigh", midRow.get(1).string1.toString()); + list = getList(batch, 1); assertEquals(3, list.size()); - assertEquals(100000000, in.get(inner.getStructFieldData(list.get(0), - inFields.get(0)))); - assertEquals("cat", st.getPrimitiveJavaObject(inner.getStructFieldData - (list.get(0), inFields.get(1)))); - assertEquals(-100000, in.get(inner.getStructFieldData(list.get(1), - inFields.get(0)))); - assertEquals("in", st.getPrimitiveJavaObject(inner.getStructFieldData - (list.get(1), inFields.get(1)))); - assertEquals(1234, in.get(inner.getStructFieldData(list.get(2), - inFields.get(0)))); - assertEquals("hat", st.getPrimitiveJavaObject(inner.getStructFieldData - (list.get(2), inFields.get(1)))); - map = ma.getMap(readerInspector.getStructFieldData(row, - fields.get(11))); + assertEquals(100000000, list.get(0).int1); + assertEquals("cat", list.get(0).string1.toString()); + assertEquals(-100000, list.get(1).int1); + assertEquals("in", list.get(1).string1.toString()); + assertEquals(1234, list.get(2).int1); + assertEquals("hat", list.get(2).string1.toString()); + map = getMap(batch, 1); assertEquals(2, map.size()); - boolean[] found = new boolean[2]; - for(Object key: map.keySet()) { - String str = mk.getPrimitiveJavaObject(key); - if (str.equals("chani")) { - assertEquals(false, found[0]); - assertEquals(5, in.get(inner.getStructFieldData(map.get(key), - inFields.get(0)))); - assertEquals(str, st.getPrimitiveJavaObject( - inner.getStructFieldData(map.get(key), inFields.get(1)))); - found[0] = true; - } else if (str.equals("mauddib")) { - assertEquals(false, found[1]); - assertEquals(1, in.get(inner.getStructFieldData(map.get(key), - inFields.get(0)))); - assertEquals(str, st.getPrimitiveJavaObject( - inner.getStructFieldData(map.get(key), inFields.get(1)))); - found[1] = true; - } else { - throw new IllegalArgumentException("Unknown key " + str); - } - } - assertEquals(true, found[0]); - assertEquals(true, found[1]); + InnerStruct value = map.get(new Text("chani")); + assertEquals(5, value.int1); + assertEquals("chani", value.string1.toString()); + value = map.get(new Text("mauddib")); + assertEquals(1, value.int1); + assertEquals("mauddib", value.string1.toString()); // handle the close up - assertEquals(false, rows.hasNext()); + assertEquals(false, rows.nextBatch(batch)); rows.close(); } @@ -1216,35 +1190,36 @@ public void testColumnProjection() throws Exception { } // check out the types - List types = reader.getTypes(); - assertEquals(3, types.size()); - assertEquals(OrcProto.Type.Kind.STRUCT, types.get(0).getKind()); - assertEquals(2, types.get(0).getSubtypesCount()); - assertEquals(1, types.get(0).getSubtypes(0)); - assertEquals(2, types.get(0).getSubtypes(1)); - assertEquals(OrcProto.Type.Kind.INT, types.get(1).getKind()); - assertEquals(0, types.get(1).getSubtypesCount()); - assertEquals(OrcProto.Type.Kind.STRING, types.get(2).getKind()); - assertEquals(0, types.get(2).getSubtypesCount()); + TypeDescription type = reader.getSchema(); + assertEquals(TypeDescription.Category.STRUCT, type.getCategory()); + assertEquals(2, type.getChildren().size()); + TypeDescription type1 = type.getChildren().get(0); + TypeDescription type2 = type.getChildren().get(1); + assertEquals(TypeDescription.Category.INT, type1.getCategory()); + assertEquals(TypeDescription.Category.STRING, type2.getCategory()); + assertEquals("struct", type.toString()); // read the contents and make sure they match RecordReader rows1 = reader.rows(new boolean[]{true, true, false}); RecordReader rows2 = reader.rows(new boolean[]{true, false, true}); r1 = new Random(1); r2 = new Random(2); - OrcStruct row1 = null; - OrcStruct row2 = null; - for(int i = 0; i < 21000; ++i) { - assertEquals(true, rows1.hasNext()); - assertEquals(true, rows2.hasNext()); - row1 = (OrcStruct) rows1.next(row1); - row2 = (OrcStruct) rows2.next(row2); - assertEquals(r1.nextInt(), ((IntWritable) row1.getFieldValue(0)).get()); - assertEquals(Long.toHexString(r2.nextLong()), - row2.getFieldValue(1).toString()); - } - assertEquals(false, rows1.hasNext()); - assertEquals(false, rows2.hasNext()); + VectorizedRowBatch batch1 = reader.getSchema().createRowBatch(1000); + VectorizedRowBatch batch2 = reader.getSchema().createRowBatch(1000); + for(int i = 0; i < 21000; i += 1000) { + assertEquals(true, rows1.nextBatch(batch1)); + assertEquals(true, rows2.nextBatch(batch2)); + assertEquals(1000, batch1.size); + assertEquals(1000, batch2.size); + for(int j=0; j < 1000; ++j) { + assertEquals(r1.nextInt(), + ((LongColumnVector) batch1.cols[0]).vector[j]); + assertEquals(Long.toHexString(r2.nextLong()), + ((BytesColumnVector) batch2.cols[1]).toString(j)); + } + } + assertEquals(false, rows1.nextBatch(batch1)); + assertEquals(false, rows2.nextBatch(batch2)); rows1.close(); rows2.close(); } @@ -1355,17 +1330,33 @@ public void createOrcDateFile(Path file, int minYear, int maxYear Reader reader = OrcFile.createReader(file, OrcFile.readerOptions(conf)); RecordReader rows = reader.rows(); - OrcStruct row = null; + batch = reader.getSchema().createRowBatch(1000); + TimestampColumnVector times = (TimestampColumnVector) batch.cols[0]; + LongColumnVector dates = (LongColumnVector) batch.cols[1]; for (int year = minYear; year < maxYear; ++year) { + rows.nextBatch(batch); + assertEquals(1000, batch.size); for(int ms = 1000; ms < 2000; ++ms) { - row = (OrcStruct) rows.next(row); - assertEquals(new TimestampWritable - (Timestamp.valueOf(year + "-05-05 12:34:56." + ms)), - row.getFieldValue(0)); - assertEquals(new DateWritable(new Date(year - 1900, 11, 25)), - row.getFieldValue(1)); + StringBuilder buffer = new StringBuilder(); + times.stringifyValue(buffer, ms - 1000); + String expected = Integer.toString(year) + "-05-05 12:34:56."; + // suppress the final zeros on the string by dividing by the largest + // power of 10 that divides evenly. + int roundedMs = ms; + for(int round = 1000; round > 0; round /= 10) { + if (ms % round == 0) { + roundedMs = ms / round; + break; + } + } + expected += roundedMs; + assertEquals(expected, buffer.toString()); + assertEquals(Integer.toString(year) + "-12-25", + new DateWritable((int) dates.vector[ms - 1000]).toString()); } } + rows.nextBatch(batch); + assertEquals(0, batch.size); } @Test @@ -1483,6 +1474,7 @@ public void testUnionAndTimestamp() throws Exception { for(int c=0; c < batch.cols.length; ++c) { batch.cols[c].setRepeating(true); } + ((UnionColumnVector) batch.cols[1]).fields[0].isRepeating = true; setUnion(batch, 0, null, 0, 1732050807, null, null); for(int i=0; i < 5; ++i) { writer.addRowBatch(batch); @@ -1540,83 +1532,115 @@ public void testUnionAndTimestamp() throws Exception { RecordReader rows = reader.rows(); assertEquals(0, rows.getRowNumber()); assertEquals(0.0, rows.getProgress(), 0.000001); - assertEquals(true, rows.hasNext()); - OrcStruct row = (OrcStruct) rows.next(null); - assertEquals(1, rows.getRowNumber()); - ObjectInspector inspector = reader.getObjectInspector(); + + schema = reader.getSchema(); + batch = schema.createRowBatch(74); + assertEquals(0, rows.getRowNumber()); + rows.nextBatch(batch); + assertEquals(74, batch.size); + assertEquals(74, rows.getRowNumber()); + TimestampColumnVector ts = (TimestampColumnVector) batch.cols[0]; + UnionColumnVector union = (UnionColumnVector) batch.cols[1]; + LongColumnVector longs = (LongColumnVector) union.fields[0]; + BytesColumnVector strs = (BytesColumnVector) union.fields[1]; + DecimalColumnVector decs = (DecimalColumnVector) batch.cols[2]; + assertEquals("struct,decimal:decimal(38,18)>", - inspector.getTypeName()); - assertEquals(new TimestampWritable(Timestamp.valueOf("2000-03-12 15:00:00")), - row.getFieldValue(0)); - OrcUnion union = (OrcUnion) row.getFieldValue(1); - assertEquals(0, union.getTag()); - assertEquals(new IntWritable(42), union.getObject()); - assertEquals(new HiveDecimalWritable(HiveDecimal.create("12345678.6547456")), - row.getFieldValue(2)); - row = (OrcStruct) rows.next(row); - assertEquals(2, rows.getRowNumber()); - assertEquals(new TimestampWritable(Timestamp.valueOf("2000-03-20 12:00:00.123456789")), - row.getFieldValue(0)); - assertEquals(1, union.getTag()); - assertEquals(new Text("hello"), union.getObject()); - assertEquals(new HiveDecimalWritable(HiveDecimal.create("-5643.234")), - row.getFieldValue(2)); - row = (OrcStruct) rows.next(row); - assertEquals(null, row.getFieldValue(0)); - assertEquals(null, row.getFieldValue(1)); - assertEquals(null, row.getFieldValue(2)); - row = (OrcStruct) rows.next(row); - assertEquals(null, row.getFieldValue(0)); - union = (OrcUnion) row.getFieldValue(1); - assertEquals(0, union.getTag()); - assertEquals(null, union.getObject()); - assertEquals(null, row.getFieldValue(2)); - row = (OrcStruct) rows.next(row); - assertEquals(null, row.getFieldValue(0)); - assertEquals(1, union.getTag()); - assertEquals(null, union.getObject()); - assertEquals(null, row.getFieldValue(2)); - row = (OrcStruct) rows.next(row); - assertEquals(new TimestampWritable(Timestamp.valueOf("1970-01-01 00:00:00")), - row.getFieldValue(0)); - assertEquals(new IntWritable(200000), union.getObject()); - assertEquals(new HiveDecimalWritable(HiveDecimal.create("10000000000000000000")), - row.getFieldValue(2)); + schema.toString()); + assertEquals("2000-03-12 15:00:00.0", ts.asScratchTimestamp(0).toString()); + assertEquals(0, union.tags[0]); + assertEquals(42, longs.vector[0]); + assertEquals("12345678.6547456", decs.vector[0].toString()); + + assertEquals("2000-03-20 12:00:00.123456789", ts.asScratchTimestamp(1).toString()); + assertEquals(1, union.tags[1]); + assertEquals("hello", strs.toString(1)); + assertEquals("-5643.234", decs.vector[1].toString()); + + assertEquals(false, ts.noNulls); + assertEquals(false, union.noNulls); + assertEquals(false, decs.noNulls); + assertEquals(true, ts.isNull[2]); + assertEquals(true, union.isNull[2]); + assertEquals(true, decs.isNull[2]); + + assertEquals(true, ts.isNull[3]); + assertEquals(false, union.isNull[3]); + assertEquals(0, union.tags[3]); + assertEquals(true, longs.isNull[3]); + assertEquals(true, decs.isNull[3]); + + assertEquals(true, ts.isNull[4]); + assertEquals(false, union.isNull[4]); + assertEquals(1, union.tags[4]); + assertEquals(true, strs.isNull[4]); + assertEquals(true, decs.isNull[4]); + + assertEquals(false, ts.isNull[5]); + assertEquals("1970-01-01 00:00:00.0", ts.asScratchTimestamp(5).toString()); + assertEquals(false, union.isNull[5]); + assertEquals(0, union.tags[5]); + assertEquals(false, longs.isNull[5]); + assertEquals(200000, longs.vector[5]); + assertEquals(false, decs.isNull[5]); + assertEquals("10000000000000000000", decs.vector[5].toString()); + rand = new Random(42); for(int i=1970; i < 2038; ++i) { - row = (OrcStruct) rows.next(row); - assertEquals(new TimestampWritable(Timestamp.valueOf(i + "-05-05 12:34:56." + i)), - row.getFieldValue(0)); + int row = 6 + i - 1970; + assertEquals(Timestamp.valueOf(i + "-05-05 12:34:56." + i), + ts.asScratchTimestamp(row)); if ((i & 1) == 0) { - assertEquals(0, union.getTag()); - assertEquals(new IntWritable(i*i), union.getObject()); + assertEquals(0, union.tags[row]); + assertEquals(i*i, longs.vector[row]); } else { - assertEquals(1, union.getTag()); - assertEquals(new Text(Integer.toString(i * i)), union.getObject()); + assertEquals(1, union.tags[row]); + assertEquals(Integer.toString(i * i), strs.toString(row)); } assertEquals(new HiveDecimalWritable(HiveDecimal.create(new BigInteger(64, rand), - rand.nextInt(18))), row.getFieldValue(2)); - } - for(int i=0; i < 5000; ++i) { - row = (OrcStruct) rows.next(row); - assertEquals(new IntWritable(1732050807), union.getObject()); - } - row = (OrcStruct) rows.next(row); - assertEquals(new IntWritable(0), union.getObject()); - row = (OrcStruct) rows.next(row); - assertEquals(new IntWritable(10), union.getObject()); - row = (OrcStruct) rows.next(row); - assertEquals(new IntWritable(138), union.getObject()); - assertEquals(false, rows.hasNext()); + rand.nextInt(18))), decs.vector[row]); + } + + // rebuild the row batch, so that we can read by 1000 rows + batch = schema.createRowBatch(1000); + ts = (TimestampColumnVector) batch.cols[0]; + union = (UnionColumnVector) batch.cols[1]; + longs = (LongColumnVector) union.fields[0]; + strs = (BytesColumnVector) union.fields[1]; + decs = (DecimalColumnVector) batch.cols[2]; + + for(int i=0; i < 5; ++i) { + rows.nextBatch(batch); + assertEquals("batch " + i, 1000, batch.size); + assertEquals("batch " + i, false, union.isRepeating); + assertEquals("batch " + i, true, union.noNulls); + for(int r=0; r < batch.size; ++r) { + assertEquals("bad tag at " + i + "." +r, 0, union.tags[r]); + } + assertEquals("batch " + i, true, longs.isRepeating); + assertEquals("batch " + i, 1732050807, longs.vector[0]); + } + + rows.nextBatch(batch); + assertEquals(3, batch.size); + assertEquals(0, union.tags[0]); + assertEquals(0, longs.vector[0]); + assertEquals(0, union.tags[1]); + assertEquals(10, longs.vector[1]); + assertEquals(0, union.tags[2]); + assertEquals(138, longs.vector[2]); + + rows.nextBatch(batch); + assertEquals(0, batch.size); assertEquals(1.0, rows.getProgress(), 0.00001); assertEquals(reader.getNumberOfRows(), rows.getRowNumber()); rows.seekToRow(1); - row = (OrcStruct) rows.next(row); - assertEquals(new TimestampWritable(Timestamp.valueOf("2000-03-20 12:00:00.123456789")), - row.getFieldValue(0)); - assertEquals(1, union.getTag()); - assertEquals(new Text("hello"), union.getObject()); - assertEquals(new HiveDecimalWritable(HiveDecimal.create("-5643.234")), row.getFieldValue(2)); + rows.nextBatch(batch); + assertEquals(1000, batch.size); + assertEquals(Timestamp.valueOf("2000-03-20 12:00:00.123456789"), ts.asScratchTimestamp(0)); + assertEquals(1, union.tags[0]); + assertEquals("hello", strs.toString(0)); + assertEquals(new HiveDecimalWritable(HiveDecimal.create("-5643.234")), decs.vector[0]); rows.close(); } @@ -1647,17 +1671,22 @@ public void testSnappy() throws Exception { writer.close(); Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs)); + assertEquals(CompressionKind.SNAPPY, reader.getCompressionKind()); RecordReader rows = reader.rows(); + batch = reader.getSchema().createRowBatch(1000); rand = new Random(12); - OrcStruct row = null; - for(int i=0; i < 10000; ++i) { - assertEquals(true, rows.hasNext()); - row = (OrcStruct) rows.next(row); - assertEquals(rand.nextInt(), ((IntWritable) row.getFieldValue(0)).get()); - assertEquals(Integer.toHexString(rand.nextInt()), - row.getFieldValue(1).toString()); + LongColumnVector longs = (LongColumnVector) batch.cols[0]; + BytesColumnVector strs = (BytesColumnVector) batch.cols[1]; + for(int b=0; b < 10; ++b) { + rows.nextBatch(batch); + assertEquals(1000, batch.size); + for(int r=0; r < batch.size; ++r) { + assertEquals(rand.nextInt(), longs.vector[r]); + assertEquals(Integer.toHexString(rand.nextInt()), strs.toString(r)); + } } - assertEquals(false, rows.hasNext()); + rows.nextBatch(batch); + assertEquals(0, batch.size); rows.close(); } @@ -1697,18 +1726,23 @@ public void testWithoutIndex() throws Exception { assertEquals(0, stripe.getIndexLength()); RecordReader rows = reader.rows(); rand = new Random(24); - OrcStruct row = null; - for(int i=0; i < 10000; ++i) { - int intVal = rand.nextInt(); - String strVal = Integer.toBinaryString(rand.nextInt()); - for(int j=0; j < 5; ++j) { - assertEquals(true, rows.hasNext()); - row = (OrcStruct) rows.next(row); - assertEquals(intVal, ((IntWritable) row.getFieldValue(0)).get()); - assertEquals(strVal, row.getFieldValue(1).toString()); + batch = reader.getSchema().createRowBatch(1000); + LongColumnVector longs = (LongColumnVector) batch.cols[0]; + BytesColumnVector strs = (BytesColumnVector) batch.cols[1]; + for(int i=0; i < 50; ++i) { + rows.nextBatch(batch); + assertEquals("batch " + i, 1000, batch.size); + for(int j=0; j < 200; ++j) { + int intVal = rand.nextInt(); + String strVal = Integer.toBinaryString(rand.nextInt()); + for (int k = 0; k < 5; ++k) { + assertEquals(intVal, longs.vector[j * 5 + k]); + assertEquals(strVal, strs.toString(j * 5 + k)); + } } } - assertEquals(false, rows.hasNext()); + rows.nextBatch(batch); + assertEquals(0, batch.size); rows.close(); } @@ -1772,34 +1806,18 @@ public void testSeek() throws Exception { assertEquals(1000, colIndex.getEntry(0).getStatistics().getNumberOfValues()); } - OrcStruct row = null; - for(int i=COUNT-1; i >= 0; --i) { - rows.seekToRow(i); - row = (OrcStruct) rows.next(row); - BigRow expected = createRandomRow(intValues, doubleValues, - stringValues, byteValues, words, i); - assertEquals(expected.boolean1.booleanValue(), - ((BooleanWritable) row.getFieldValue(0)).get()); - assertEquals(expected.byte1.byteValue(), - ((ByteWritable) row.getFieldValue(1)).get()); - assertEquals(expected.short1.shortValue(), - ((ShortWritable) row.getFieldValue(2)).get()); - assertEquals(expected.int1.intValue(), - ((IntWritable) row.getFieldValue(3)).get()); - assertEquals(expected.long1.longValue(), - ((LongWritable) row.getFieldValue(4)).get()); - assertEquals(expected.float1, - ((FloatWritable) row.getFieldValue(5)).get(), 0.0001); - assertEquals(expected.double1, - ((DoubleWritable) row.getFieldValue(6)).get(), 0.0001); - assertEquals(expected.bytes1, row.getFieldValue(7)); - assertEquals(expected.string1, row.getFieldValue(8)); - List expectedList = expected.middle.list; - List actualList = - (List) ((OrcStruct) row.getFieldValue(9)).getFieldValue(0); - compareList(expectedList, actualList, "middle list " + i); - compareList(expected.list, (List) row.getFieldValue(10), - "list " + i); + batch = reader.getSchema().createRowBatch(); + int nextRowInBatch = -1; + for(int i=COUNT-1; i >= 0; --i, --nextRowInBatch) { + // if we have consumed the previous batch read a new one + if (nextRowInBatch < 0) { + long base = Math.max(i - 1023, 0); + rows.seekToRow(base); + assertEquals("row " + i, true, rows.nextBatch(batch)); + nextRowInBatch = batch.size - 1; + } + checkRandomRow(batch, intValues, doubleValues, + stringValues, byteValues, words, i, nextRowInBatch); } rows.close(); Iterator stripeIterator = @@ -1825,41 +1843,20 @@ public void testSeek() throws Exception { .range(offsetOfStripe2, offsetOfStripe4 - offsetOfStripe2) .include(columns)); rows.seekToRow(lastRowOfStripe2); - for(int i = 0; i < 2; ++i) { - row = (OrcStruct) rows.next(row); - BigRow expected = createRandomRow(intValues, doubleValues, - stringValues, byteValues, words, - (int) (lastRowOfStripe2 + i)); - - assertEquals(expected.long1.longValue(), - ((LongWritable) row.getFieldValue(4)).get()); - assertEquals(expected.string1, row.getFieldValue(8)); - } + // we only want two rows + batch = reader.getSchema().createRowBatch(2); + assertEquals(true, rows.nextBatch(batch)); + assertEquals(1, batch.size); + assertEquals(intValues[(int) lastRowOfStripe2], getLong(batch, 0)); + assertEquals(stringValues[(int) lastRowOfStripe2], + getText(batch, 0).toString()); + assertEquals(true, rows.nextBatch(batch)); + assertEquals(intValues[(int) lastRowOfStripe2 + 1], getLong(batch, 0)); + assertEquals(stringValues[(int) lastRowOfStripe2 + 1], + getText(batch, 0).toString()); rows.close(); } - private void compareInner(InnerStruct expect, - OrcStruct actual, - String context) throws Exception { - if (expect == null || actual == null) { - assertEquals(context, null, expect); - assertEquals(context, null, actual); - } else { - assertEquals(context, expect.int1, - ((IntWritable) actual.getFieldValue(0)).get()); - assertEquals(context, expect.string1, actual.getFieldValue(1)); - } - } - - private void compareList(List expect, - List actual, - String context) throws Exception { - assertEquals(context, expect.size(), actual.size()); - for(int j=0; j < expect.size(); ++j) { - compareInner(expect.get(j), actual.get(j), context + " at " + j); - } - } - private void appendRandomRow(VectorizedRowBatch batch, long[] intValues, double[] doubleValues, String[] stringValues, @@ -1874,17 +1871,18 @@ private void appendRandomRow(VectorizedRowBatch batch, new MiddleStruct(inner, inner2), list(), map(inner, inner2)); } - private BigRow createRandomRow(long[] intValues, double[] doubleValues, - String[] stringValues, - BytesWritable[] byteValues, - String[] words, int i) { + private void checkRandomRow(VectorizedRowBatch batch, + long[] intValues, double[] doubleValues, + String[] stringValues, + BytesWritable[] byteValues, + String[] words, int i, int rowInBatch) { InnerStruct inner = new InnerStruct((int) intValues[i], stringValues[i]); InnerStruct inner2 = new InnerStruct((int) (intValues[i] >> 32), words[i % words.length] + "-x"); - return new BigRow((intValues[i] & 1) == 0, (byte) intValues[i], + checkBigRow(batch, rowInBatch, i, (intValues[i] & 1) == 0, (byte) intValues[i], (short) intValues[i], (int) intValues[i], intValues[i], - (float) doubleValues[i], doubleValues[i], byteValues[i],stringValues[i], - new MiddleStruct(inner, inner2), list(), map(inner,inner2)); + (float) doubleValues[i], doubleValues[i], byteValues[i], stringValues[i], + new MiddleStruct(inner, inner2), list(), map(inner, inner2)); } private static class MyMemoryManager extends MemoryManager { @@ -2045,15 +2043,19 @@ public void testPredicatePushdown() throws Exception { .range(0L, Long.MAX_VALUE) .include(new boolean[]{true, true, true}) .searchArgument(sarg, new String[]{null, "int1", "string1"})); + batch = reader.getSchema().createRowBatch(2000); + LongColumnVector ints = (LongColumnVector) batch.cols[0]; + BytesColumnVector strs = (BytesColumnVector) batch.cols[1]; + assertEquals(1000L, rows.getRowNumber()); - OrcStruct row = null; + assertEquals(true, rows.nextBatch(batch)); + assertEquals(1000, batch.size); + for(int i=1000; i < 2000; ++i) { - assertTrue(rows.hasNext()); - row = (OrcStruct) rows.next(row); - assertEquals(300 * i, ((IntWritable) row.getFieldValue(0)).get()); - assertEquals(Integer.toHexString(10*i), row.getFieldValue(1).toString()); + assertEquals(300 * i, ints.vector[i - 1000]); + assertEquals(Integer.toHexString(10*i), strs.toString(i - 1000)); } - assertTrue(!rows.hasNext()); + assertEquals(false, rows.nextBatch(batch)); assertEquals(3500, rows.getRowNumber()); // look through the file with no rows selected @@ -2082,40 +2084,26 @@ public void testPredicatePushdown() throws Exception { .range(0L, Long.MAX_VALUE) .include(new boolean[]{true, true, true}) .searchArgument(sarg, new String[]{null, "int1", "string1"})); - row = null; + assertEquals(0, rows.getRowNumber()); + assertEquals(true, rows.nextBatch(batch)); + assertEquals(1000, batch.size); + assertEquals(3000, rows.getRowNumber()); for(int i=0; i < 1000; ++i) { - assertTrue(rows.hasNext()); - assertEquals(i, rows.getRowNumber()); - row = (OrcStruct) rows.next(row); - assertEquals(300 * i, ((IntWritable) row.getFieldValue(0)).get()); - assertEquals(Integer.toHexString(10*i), row.getFieldValue(1).toString()); + assertEquals(300 * i, ints.vector[i]); + assertEquals(Integer.toHexString(10*i), strs.toString(i)); } + + assertEquals(true, rows.nextBatch(batch)); + assertEquals(500, batch.size); + assertEquals(3500, rows.getRowNumber()); for(int i=3000; i < 3500; ++i) { - assertTrue(rows.hasNext()); - assertEquals(i, rows.getRowNumber()); - row = (OrcStruct) rows.next(row); - assertEquals(300 * i, ((IntWritable) row.getFieldValue(0)).get()); - assertEquals(Integer.toHexString(10*i), row.getFieldValue(1).toString()); + assertEquals(300 * i, ints.vector[i - 3000]); + assertEquals(Integer.toHexString(10*i), strs.toString(i - 3000)); } - assertTrue(!rows.hasNext()); + assertEquals(false, rows.nextBatch(batch)); assertEquals(3500, rows.getRowNumber()); } - private static String pad(String value, int length) { - if (value.length() == length) { - return value; - } else if (value.length() > length) { - return value.substring(0, length); - } else { - StringBuilder buf = new StringBuilder(); - buf.append(value); - for(int i=0; i < length - value.length(); ++i) { - buf.append(' '); - } - return buf.toString(); - } - } - /** * Test all of the types that have distinct ORC writers using the vectorized * writer with different combinations of repeating and null values. @@ -2232,8 +2220,7 @@ public void testRepeating() throws Exception { ((LongColumnVector) batch.cols[6]).vector[r] = new DateWritable(new Date(111, 6, 1)).getDays() + r; - Timestamp ts = new Timestamp(115, 9, 23, 10, 11, 59, 999999999); - ts.setTime(ts.getTime() + r * 1000); + Timestamp ts = new Timestamp(115, 9, 25, 10, 11, 59 + r, 999999999); ((TimestampColumnVector) batch.cols[7]).set(r, ts); ((DecimalColumnVector) batch.cols[8]).vector[r] = new HiveDecimalWritable("1.234567"); @@ -2302,118 +2289,125 @@ public void testRepeating() throws Exception { assertEquals(14813, ((StringColumnStatistics) stats[12]).getSum()); RecordReader rows = reader.rows(); - OrcStruct row = null; + batch = reader.getSchema().createRowBatch(1024); + BytesColumnVector bins = (BytesColumnVector) batch.cols[0]; + LongColumnVector bools = (LongColumnVector) batch.cols[1]; + LongColumnVector bytes = (LongColumnVector) batch.cols[2]; + LongColumnVector longs = (LongColumnVector) batch.cols[3]; + DoubleColumnVector floats = (DoubleColumnVector) batch.cols[4]; + DoubleColumnVector doubles = (DoubleColumnVector) batch.cols[5]; + LongColumnVector dates = (LongColumnVector) batch.cols[6]; + TimestampColumnVector times = (TimestampColumnVector) batch.cols[7]; + DecimalColumnVector decs = (DecimalColumnVector) batch.cols[8]; + BytesColumnVector strs = (BytesColumnVector) batch.cols[9]; + BytesColumnVector chars = (BytesColumnVector) batch.cols[10]; + BytesColumnVector vcs = (BytesColumnVector) batch.cols[11]; + StructColumnVector structs = (StructColumnVector) batch.cols[12]; + UnionColumnVector unions = (UnionColumnVector) batch.cols[13]; + ListColumnVector lists = (ListColumnVector) batch.cols[14]; + MapColumnVector maps = (MapColumnVector) batch.cols[15]; + LongColumnVector structInts = (LongColumnVector) structs.fields[0]; + LongColumnVector unionInts = (LongColumnVector) unions.fields[1]; + LongColumnVector listInts = (LongColumnVector) lists.child; + BytesColumnVector mapKeys = (BytesColumnVector) maps.keys; + BytesColumnVector mapValues = (BytesColumnVector) maps.values; + + assertEquals(true, rows.nextBatch(batch)); + assertEquals(1024, batch.size); // read the 1024 nulls - for(int r=0; r < 1024; ++r) { - assertEquals(true, rows.hasNext()); - row = (OrcStruct) rows.next(row); - for(int f=0; f < row.getNumFields(); ++f) { - assertEquals("non-null on row " + r + " field " + f, - null, row.getFieldValue(f)); - } + for(int f=0; f < batch.cols.length; ++f) { + assertEquals("field " + f, + true, batch.cols[f].isRepeating); + assertEquals("field " + f, + false, batch.cols[f].noNulls); + assertEquals("field " + f, + true, batch.cols[f].isNull[0]); } // read the 1024 repeat values + assertEquals(true, rows.nextBatch(batch)); + assertEquals(1024, batch.size); for(int r=0; r < 1024; ++r) { - assertEquals(true, rows.hasNext()); - row = (OrcStruct) rows.next(row); - assertEquals("row " + r, "48 6f 72 74 6f 6e", - row.getFieldValue(0).toString()); - assertEquals("row " + r, "true", row.getFieldValue(1).toString()); - assertEquals("row " + r, "-126", row.getFieldValue(2).toString()); - assertEquals("row " + r, "1311768467463790320", - row.getFieldValue(3).toString()); - assertEquals("row " + r, "1.125", row.getFieldValue(4).toString()); - assertEquals("row " + r, "9.765625E-4", row.getFieldValue(5).toString()); - assertEquals("row " + r, "2011-07-01", row.getFieldValue(6).toString()); + assertEquals("row " + r, "Horton", bins.toString(r)); + assertEquals("row " + r, 1, bools.vector[r]); + assertEquals("row " + r, -126, bytes.vector[r]); + assertEquals("row " + r, 1311768467463790320L, longs.vector[r]); + assertEquals("row " + r, 1.125, floats.vector[r], 0.00001); + assertEquals("row " + r, 9.765625E-4, doubles.vector[r], 0.000001); + assertEquals("row " + r, "2011-07-01", + new DateWritable((int) dates.vector[r]).toString()); assertEquals("row " + r, "2015-10-23 10:11:59.999999999", - row.getFieldValue(7).toString()); - assertEquals("row " + r, "1.234567", row.getFieldValue(8).toString()); - assertEquals("row " + r, "Echelon", row.getFieldValue(9).toString()); - assertEquals("row " + r, "Juggernaut", row.getFieldValue(10).toString()); - assertEquals("row " + r, "Dreadnaugh", row.getFieldValue(11).toString()); - assertEquals("row " + r, "{123}", row.getFieldValue(12).toString()); - assertEquals("row " + r, "union(1, 1234)", - row.getFieldValue(13).toString()); - assertEquals("row " + r, "[31415, 31415, 31415]", - row.getFieldValue(14).toString()); - assertEquals("row " + r, "{ORC=fast, Hive=fast, LLAP=fast}", - row.getFieldValue(15).toString()); + times.asScratchTimestamp(r).toString()); + assertEquals("row " + r, "1.234567", decs.vector[r].toString()); + assertEquals("row " + r, "Echelon", strs.toString(r)); + assertEquals("row " + r, "Juggernaut", chars.toString(r)); + assertEquals("row " + r, "Dreadnaugh", vcs.toString(r)); + assertEquals("row " + r, 123, structInts.vector[r]); + assertEquals("row " + r, 1, unions.tags[r]); + assertEquals("row " + r, 1234, unionInts.vector[r]); + assertEquals("row " + r, 3, lists.lengths[r]); + assertEquals("row " + r, true, listInts.isRepeating); + assertEquals("row " + r, 31415, listInts.vector[0]); + assertEquals("row " + r, 3, maps.lengths[r]); + assertEquals("row " + r, "ORC", mapKeys.toString((int) maps.offsets[r])); + assertEquals("row " + r, "Hive", mapKeys.toString((int) maps.offsets[r] + 1)); + assertEquals("row " + r, "LLAP", mapKeys.toString((int) maps.offsets[r] + 2)); + assertEquals("row " + r, "fast", mapValues.toString((int) maps.offsets[r])); + assertEquals("row " + r, "fast", mapValues.toString((int) maps.offsets[r] + 1)); + assertEquals("row " + r, "fast", mapValues.toString((int) maps.offsets[r] + 2)); } // read the second set of 1024 nulls - for(int r=0; r < 1024; ++r) { - assertEquals(true, rows.hasNext()); - row = (OrcStruct) rows.next(row); - for(int f=0; f < row.getNumFields(); ++f) { - assertEquals("non-null on row " + r + " field " + f, - null, row.getFieldValue(f)); - } + assertEquals(true, rows.nextBatch(batch)); + assertEquals(1024, batch.size); + for(int f=0; f < batch.cols.length; ++f) { + assertEquals("field " + f, + true, batch.cols[f].isRepeating); + assertEquals("field " + f, + false, batch.cols[f].noNulls); + assertEquals("field " + f, + true, batch.cols[f].isNull[0]); } + + assertEquals(true, rows.nextBatch(batch)); + assertEquals(1024, batch.size); for(int r=0; r < 1024; ++r) { - assertEquals(true, rows.hasNext()); - row = (OrcStruct) rows.next(row); - byte[] hex = Integer.toHexString(r).getBytes(); - StringBuilder expected = new StringBuilder(); - for(int i=0; i < hex.length; ++i) { - if (i != 0) { - expected.append(' '); - } - expected.append(Integer.toHexString(hex[i])); - } - assertEquals("row " + r, expected.toString(), - row.getFieldValue(0).toString()); - assertEquals("row " + r, r % 2 == 1 ? "true" : "false", - row.getFieldValue(1).toString()); - assertEquals("row " + r, Integer.toString((byte) (r % 255)), - row.getFieldValue(2).toString()); - assertEquals("row " + r, Long.toString(31415L * r), - row.getFieldValue(3).toString()); - assertEquals("row " + r, Float.toString(1.125F * r), - row.getFieldValue(4).toString()); - assertEquals("row " + r, Double.toString(0.0009765625 * r), - row.getFieldValue(5).toString()); - assertEquals("row " + r, new Date(111, 6, 1 + r).toString(), - row.getFieldValue(6).toString()); - Timestamp ts = new Timestamp(115, 9, 23, 10, 11, 59, 999999999); - ts.setTime(ts.getTime() + r * 1000); + String hex = Integer.toHexString(r); + + assertEquals("row " + r, hex, bins.toString(r)); + assertEquals("row " + r, r % 2 == 1 ? 1 : 0, bools.vector[r]); + assertEquals("row " + r, (byte) (r % 255), bytes.vector[r]); + assertEquals("row " + r, 31415L * r, longs.vector[r]); + assertEquals("row " + r, 1.125F * r, floats.vector[r], 0.0001); + assertEquals("row " + r, 0.0009765625 * r, doubles.vector[r], 0.000001); + assertEquals("row " + r, new DateWritable(new Date(111, 6, 1 + r)), + new DateWritable((int) dates.vector[r])); assertEquals("row " + r, - ts.toString(), - row.getFieldValue(7).toString()); - assertEquals("row " + r, "1.234567", row.getFieldValue(8).toString()); - assertEquals("row " + r, Integer.toString(r), - row.getFieldValue(9).toString()); - assertEquals("row " + r, pad(Integer.toHexString(r), 10), - row.getFieldValue(10).toString()); - assertEquals("row " + r, Integer.toHexString(r * 128), - row.getFieldValue(11).toString()); - assertEquals("row " + r, "{" + Integer.toString(r + 13) + "}", - row.getFieldValue(12).toString()); - assertEquals("row " + r, "union(1, " + Integer.toString(r + 42) + ")", - row.getFieldValue(13).toString()); - assertEquals("row " + r, "[31415, 31416, 31417]", - row.getFieldValue(14).toString()); - expected = new StringBuilder(); - expected.append('{'); - expected.append(Integer.toHexString(3 * r)); - expected.append('='); - expected.append(3 * r); - expected.append(", "); - expected.append(Integer.toHexString(3 * r + 1)); - expected.append('='); - expected.append(3 * r + 1); - expected.append(", "); - expected.append(Integer.toHexString(3 * r + 2)); - expected.append('='); - expected.append(3 * r + 2); - expected.append('}'); - assertEquals("row " + r, expected.toString(), - row.getFieldValue(15).toString()); + new Timestamp(115, 9, 25, 10, 11, 59 + r, 999999999), + times.asScratchTimestamp(r)); + assertEquals("row " + r, "1.234567", decs.vector[r].toString()); + assertEquals("row " + r, Integer.toString(r), strs.toString(r)); + assertEquals("row " + r, Integer.toHexString(r), chars.toString(r)); + assertEquals("row " + r, Integer.toHexString(r * 128), vcs.toString(r)); + assertEquals("row " + r, r + 13, structInts.vector[r]); + assertEquals("row " + r, 1, unions.tags[r]); + assertEquals("row " + r, r + 42, unionInts.vector[r]); + assertEquals("row " + r, 3, lists.lengths[r]); + assertEquals("row " + r, 31415, listInts.vector[(int) lists.offsets[r]]); + assertEquals("row " + r, 31416, listInts.vector[(int) lists.offsets[r] + 1]); + assertEquals("row " + r, 31417, listInts.vector[(int) lists.offsets[r] + 2]); + assertEquals("row " + r, 3, maps.lengths[3]); + assertEquals("row " + r, Integer.toHexString(3 * r), mapKeys.toString((int) maps.offsets[r])); + assertEquals("row " + r, Integer.toString(3 * r), mapValues.toString((int) maps.offsets[r])); + assertEquals("row " + r, Integer.toHexString(3 * r + 1), mapKeys.toString((int) maps.offsets[r] + 1)); + assertEquals("row " + r, Integer.toString(3 * r + 1), mapValues.toString((int) maps.offsets[r] + 1)); + assertEquals("row " + r, Integer.toHexString(3 * r + 2), mapKeys.toString((int) maps.offsets[r] + 2)); + assertEquals("row " + r, Integer.toString(3 * r + 2), mapValues.toString((int) maps.offsets[r] + 2)); } // should have no more rows - assertEquals(false, rows.hasNext()); + assertEquals(false, rows.nextBatch(batch)); } private static String makeString(BytesColumnVector vector, int row) { @@ -2455,7 +2449,8 @@ public void testStringPadding() throws Exception { Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf)); RecordReader rows = reader.rows(); - batch = rows.nextBatch(null); + batch = reader.getSchema().createRowBatch(); + assertEquals(true, rows.nextBatch(batch)); assertEquals(4, batch.size); // ORC currently trims the output strings. See HIVE-12286 assertEquals("", @@ -2504,19 +2499,20 @@ public void testNonDictionaryRepeatingString() throws Exception { Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf)); RecordReader rows = reader.rows(); - batch = rows.nextBatch(null); + batch = reader.getSchema().createRowBatch(); + assertEquals(true, rows.nextBatch(batch)); assertEquals(1024, batch.size); for(int r=0; r < 1024; ++r) { assertEquals(Integer.toString(r * 10001), makeString((BytesColumnVector) batch.cols[0], r)); } - batch = rows.nextBatch(batch); + assertEquals(true, rows.nextBatch(batch)); assertEquals(1024, batch.size); for(int r=0; r < 1024; ++r) { assertEquals("Halloween", makeString((BytesColumnVector) batch.cols[0], r)); } - assertEquals(false, rows.hasNext()); + assertEquals(false, rows.nextBatch(batch)); } @Test @@ -2541,18 +2537,21 @@ public void testStructs() throws Exception { Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf)); RecordReader rows = reader.rows(); - OrcStruct row = null; + batch = reader.getSchema().createRowBatch(); + rows.nextBatch(batch); + assertEquals(1024, batch.size); + StructColumnVector inner = (StructColumnVector) batch.cols[0]; + LongColumnVector vec = (LongColumnVector) inner.fields[0]; for(int r=0; r < 1024; ++r) { - assertEquals(true, rows.hasNext()); - row = (OrcStruct) rows.next(row); - OrcStruct inner = (OrcStruct) row.getFieldValue(0); if (r < 200 || (r >= 400 && r < 600) || r >= 800) { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, true, inner.isNull[r]); } else { - assertEquals("row " + r, "{" + r + "}", inner.toString()); + assertEquals("row " + r, false, inner.isNull[r]); + assertEquals("row " + r, r, vec.vector[r]); } } - assertEquals(false, rows.hasNext()); + rows.nextBatch(batch); + assertEquals(0, batch.size); } /** @@ -2595,28 +2594,38 @@ public void testUnions() throws Exception { Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf)); RecordReader rows = reader.rows(); - OrcStruct row = null; + batch = reader.getSchema().createRowBatch(1024); + UnionColumnVector union = (UnionColumnVector) batch.cols[0]; + LongColumnVector ints = (LongColumnVector) union.fields[0]; + LongColumnVector longs = (LongColumnVector) union.fields[1]; + assertEquals(true, rows.nextBatch(batch)); + assertEquals(1024, batch.size); for(int r=0; r < 1024; ++r) { - assertEquals(true, rows.hasNext()); - row = (OrcStruct) rows.next(row); - OrcUnion inner = (OrcUnion) row.getFieldValue(0); if (r < 200) { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, true, union.isNull[r]); } else if (r < 300) { - assertEquals("row " + r, "union(0, " + r +")", inner.toString()); + assertEquals("row " + r, false, union.isNull[r]); + assertEquals("row " + r, 0, union.tags[r]); + assertEquals("row " + r, r, ints.vector[r]); } else if (r < 400) { - assertEquals("row " + r, "union(1, " + -r +")", inner.toString()); + assertEquals("row " + r, false, union.isNull[r]); + assertEquals("row " + r, 1, union.tags[r]); + assertEquals("row " + r, -r, longs.vector[r]); } else if (r < 600) { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, true, union.isNull[r]); } else if (r < 800) { - assertEquals("row " + r, "union(1, " + -r +")", inner.toString()); + assertEquals("row " + r, false, union.isNull[r]); + assertEquals("row " + r, 1, union.tags[r]); + assertEquals("row " + r, -r, longs.vector[r]); } else if (r < 1000) { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, true, union.isNull[r]); } else { - assertEquals("row " + r, "union(1, " + -r +")", inner.toString()); + assertEquals("row " + r, false, union.isNull[r]); + assertEquals("row " + r, 1, union.tags[r]); + assertEquals("row " + r, -r, longs.vector[r]); } } - assertEquals(false, rows.hasNext()); + assertEquals(false, rows.nextBatch(batch)); } /** @@ -2663,31 +2672,33 @@ public void testLists() throws Exception { Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf)); RecordReader rows = reader.rows(); - OrcStruct row = null; + batch = reader.getSchema().createRowBatch(1024); + list = (ListColumnVector) batch.cols[0]; + rows.nextBatch(batch); + assertEquals(1024, batch.size); for(int r=0; r < 1024; ++r) { - assertEquals(true, rows.hasNext()); - row = (OrcStruct) rows.next(row); - List inner = (List) row.getFieldValue(0); + StringBuilder actual = new StringBuilder(); + list.stringifyValue(actual, r); if (r < 200) { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, "null", actual.toString()); } else if (r < 300) { assertEquals("row " + r, "[" + ((r - 200) * 10) + "]", - inner.toString()); + actual.toString()); } else if (r < 400) { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, "null", actual.toString()); } else if (r < 500) { assertEquals("row " + r, "[" + ((r - 300) * 10) + "]", - inner.toString()); + actual.toString()); } else if (r < 600) { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, "null", actual.toString()); } else if (r < 700) { assertEquals("row " + r, "[" + (10 * r) + ", " + (10 * (r + 1)) + "]", - inner.toString()); + actual.toString()); } else { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, "null", actual.toString()); } } - assertEquals(false, rows.hasNext()); + assertEquals(false, rows.nextBatch(batch)); } /** @@ -2736,31 +2747,36 @@ public void testMaps() throws Exception { Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf)); RecordReader rows = reader.rows(); - OrcStruct row = null; + batch = reader.getSchema().createRowBatch(); + map = (MapColumnVector) batch.cols[0]; + rows.nextBatch(batch); + assertEquals(1024, batch.size); for(int r=0; r < 1024; ++r) { - assertEquals(true, rows.hasNext()); - row = (OrcStruct) rows.next(row); - Map inner = (Map) row.getFieldValue(0); + StringBuilder buffer = new StringBuilder(); + map.stringifyValue(buffer, r); + String actual = buffer.toString(); if (r < 200) { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, "null", actual); } else if (r < 300) { - assertEquals("row " + r, "{" + (r - 200) + "=" + ((r - 200) * 10) + "}", - inner.toString()); + assertEquals("row " + r, "[{\"key\": " + (r - 200) + + ", \"value\": " + ((r - 200) * 10) + "}]", + actual); } else if (r < 400) { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, "null", actual); } else if (r < 500) { - assertEquals("row " + r, "{" + (r - 300) + "=" + ((r - 300) * 10) + "}", - inner.toString()); + assertEquals("row " + r, "[{\"key\": " + (r - 300) + + ", \"value\": " + ((r - 300) * 10) + "}]", actual); } else if (r < 600) { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, "null", actual); } else if (r < 700) { - assertEquals("row " + r, "{" + r + "=" + (r * 10) + ", " + - (r + 1) + "=" + (10 * (r + 1)) + "}", - inner.toString()); + assertEquals("row " + r, "[{\"key\": " + r + ", \"value\": " + (r * 10) + + "}, {\"key\": " + (r + 1) + ", \"value\": " + (10 * (r + 1)) + + "}]", actual); } else { - assertEquals("row " + r, null, inner); + assertEquals("row " + r, "null", actual); } } - assertEquals(false, rows.hasNext()); + rows.nextBatch(batch); + assertEquals(0, batch.size); } } diff --git ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java index adb52f0..a52b3ef 100644 --- ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java +++ ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java @@ -151,12 +151,11 @@ private void checkVectorizedReader() throws Exception { OrcFile.readerOptions(conf)); RecordReaderImpl vrr = (RecordReaderImpl) vreader.rows(); RecordReaderImpl rr = (RecordReaderImpl) reader.rows(); - VectorizedRowBatch batch = null; + VectorizedRowBatch batch = reader.getSchema().createRowBatch(); OrcStruct row = null; // Check Vectorized ORC reader against ORC row reader - while (vrr.hasNext()) { - batch = vrr.nextBatch(batch); + while (vrr.nextBatch(batch)) { for (int i = 0; i < batch.size; i++) { row = (OrcStruct) rr.next(row); for (int j = 0; j < batch.cols.length; j++) { @@ -239,6 +238,6 @@ private void checkVectorizedReader() throws Exception { Assert.assertEquals(false, batch.cols[8].noNulls); Assert.assertEquals(false, batch.cols[9].noNulls); } - Assert.assertEquals(false, rr.hasNext()); + Assert.assertEquals(false, rr.nextBatch(batch)); } } diff --git storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java index 99744cd..f915a7e 100644 --- storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java +++ storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java @@ -338,6 +338,17 @@ public void init() { initBuffer(0); } + public String toString(int row) { + if (isRepeating) { + row = 0; + } + if (noNulls || !isNull[row]) { + return new String(vector[row], start[row], length[row]); + } else { + return null; + } + } + @Override public void stringifyValue(StringBuilder buffer, int row) { if (isRepeating) { diff --git storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java index c0dd5ed..d971339 100644 --- storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java +++ storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java @@ -392,4 +392,4 @@ public void stringifyValue(StringBuilder buffer, int row) { buffer.append("null"); } } -} \ No newline at end of file +} diff --git storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/UnionColumnVector.java storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/UnionColumnVector.java index 298d588..0c61243 100644 --- storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/UnionColumnVector.java +++ storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/UnionColumnVector.java @@ -18,8 +18,6 @@ package org.apache.hadoop.hive.ql.exec.vector; -import java.util.Arrays; - /** * The representation of a vectorized column of struct objects. *