diff --git common/src/java/org/apache/hadoop/hive/common/DiskRangeInfo.java common/src/java/org/apache/hadoop/hive/common/DiskRangeInfo.java new file mode 100644 index 0000000..5157d38 --- /dev/null +++ common/src/java/org/apache/hadoop/hive/common/DiskRangeInfo.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.common; + +import java.util.List; + +import com.google.common.collect.Lists; + +/** + * Disk range information class containing disk ranges and total length. + */ +public class DiskRangeInfo { + List diskRanges; + long totalLength; + + public DiskRangeInfo() { + this.diskRanges = Lists.newArrayList(); + this.totalLength = 0; + } + + public void addDiskRange(DiskRange diskRange) { + diskRanges.add(diskRange); + totalLength += diskRange.getLength(); + } + + public List getDiskRanges() { + return diskRanges; + } + + public long getTotalLength() { + return totalLength; + } +} + 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 b712075..257727a 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 @@ -30,14 +30,13 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.hadoop.hive.ql.io.orc.CompressionCodec; -import org.apache.hadoop.hive.ql.io.orc.EncodedRecordReaderImplFactory; +import org.apache.hadoop.hive.ql.io.orc.EncodedTreeReaderFactory; import org.apache.hadoop.hive.ql.io.orc.OrcProto; import org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl; -import org.apache.hadoop.hive.ql.io.orc.RecordReaderImplFactory; import org.apache.hadoop.hive.ql.io.orc.WriterImpl; public class OrcEncodedDataConsumer extends EncodedDataConsumer { - private RecordReaderImplFactory.TreeReader[] columnReaders; + private EncodedTreeReaderFactory.TreeReader[] columnReaders; private int previousStripeIndex = -1; private OrcFileMetadata fileMetadata; // We assume one request is only for one file. private CompressionCodec codec; @@ -83,7 +82,7 @@ protected void decodeBatch(EncodedColumnBatch batch, int batchSize = VectorizedRowBatch.DEFAULT_SIZE; int numCols = batch.columnIxs.length; if (columnReaders == null || !sameStripe) { - this.columnReaders = EncodedRecordReaderImplFactory.createEncodedTreeReader(numCols, + this.columnReaders = EncodedTreeReaderFactory.createEncodedTreeReader(numCols, fileMetadata.getTypes(), stripeMetadata.getEncodings(), batch, codec, skipCorrupt); } else { repositionInStreams(this.columnReaders, batch, sameStripe, numCols, stripeMetadata); @@ -116,7 +115,7 @@ protected void decodeBatch(EncodedColumnBatch batch, } } - private void repositionInStreams(RecordReaderImplFactory.TreeReader[] columnReaders, + private void repositionInStreams(EncodedTreeReaderFactory.TreeReader[] columnReaders, EncodedColumnBatch batch, boolean sameStripe, int numCols, OrcStripeMetadata stripeMetadata) throws IOException { for (int i = 0; i < numCols; i++) { diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/EncodedRecordReaderImplFactory.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/EncodedRecordReaderImplFactory.java deleted file mode 100644 index fe605a9..0000000 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/EncodedRecordReaderImplFactory.java +++ /dev/null @@ -1,2033 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.ql.io.orc; - -import java.io.IOException; -import java.util.List; - -import org.apache.hadoop.hive.common.DiskRange; -import org.apache.hadoop.hive.llap.io.api.EncodedColumnBatch; -import org.apache.hadoop.hive.llap.io.api.orc.OrcBatchKey; - -import com.google.common.collect.Lists; - -/** - * - */ -public class EncodedRecordReaderImplFactory extends RecordReaderImplFactory { - - protected static class TimestampStreamReader extends TimestampTreeReader { - private boolean isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _secondsStream; - private SettableUncompressedStream _nanosStream; - - private TimestampStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, SettableUncompressedStream nanos, boolean isFileCompressed, - OrcProto.ColumnEncoding encoding, boolean skipCorrupt) throws IOException { - super(columnId, present, data, nanos, encoding, skipCorrupt); - this.isFileCompressed = isFileCompressed; - this._presentStream = present; - this._secondsStream = data; - this._nanosStream = nanos; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_secondsStream.available() > 0) { - if (isFileCompressed) { - index.getNext(); - } - data.seek(index); - } - - if (_nanosStream.available() > 0) { - if (isFileCompressed) { - index.getNext(); - } - nanos.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_secondsStream != null) { - List secondsDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, secondsDiskRanges); - _secondsStream.setBuffers(secondsDiskRanges, length); - } - if (_nanosStream != null) { - List nanosDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(secondaryStreamBuffer, nanosDiskRanges); - _nanosStream.setBuffers(nanosDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private EncodedColumnBatch.StreamBuffer nanosStream; - private CompressionCodec compressionCodec; - private OrcProto.ColumnEncoding columnEncoding; - private boolean skipCorrupt; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setSecondsStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setNanosStream(EncodedColumnBatch.StreamBuffer secondaryStream) { - this.nanosStream = secondaryStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { - this.columnEncoding = encoding; - return this; - } - - public StreamReaderBuilder skipCorrupt(boolean skipCorrupt) { - this.skipCorrupt = skipCorrupt; - return this; - } - - public TimestampStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - SettableUncompressedStream nanos = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.SECONDARY.name(), - fileId, nanosStream); - - boolean isFileCompressed = compressionCodec != null; - return new TimestampStreamReader(columnIndex, present, data, nanos, - isFileCompressed, columnEncoding, skipCorrupt); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - } - - protected static class StringStreamReader extends StringTreeReader { - private boolean _isFileCompressed; - private boolean _isDictionaryEncoding; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - private SettableUncompressedStream _lengthStream; - private SettableUncompressedStream _dictionaryStream; - - private StringStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, SettableUncompressedStream length, - SettableUncompressedStream dictionary, - boolean isFileCompressed, OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, present, data, length, dictionary, encoding); - this._isDictionaryEncoding = dictionary != null; - this._isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - this._lengthStream = length; - this._dictionaryStream = dictionary; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - reader.present.seek(index); - } - - if (_isDictionaryEncoding) { - // DICTIONARY encoding - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - ((StringDictionaryTreeReader) reader).reader.seek(index); - } - } else { - // DIRECT encoding - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - ((StringDirectTreeReader) reader).stream.seek(index); - } - - if (_lengthStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - ((StringDirectTreeReader) reader).lengths.seek(index); - } - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - if (!_isDictionaryEncoding) { - if (_lengthStream != null) { - List lengthDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(lengthsStreamBuffer, lengthDiskRanges); - _lengthStream.setBuffers(lengthDiskRanges, length); - } - } - - // set these streams only if the stripe is different - if (!sameStripe && _isDictionaryEncoding) { - if (_lengthStream != null) { - List lengthDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(lengthsStreamBuffer, lengthDiskRanges); - _lengthStream.setBuffers(lengthDiskRanges, length); - } - if (_dictionaryStream != null) { - List dictionaryDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dictionaryStreamBuffer, dictionaryDiskRanges); - _dictionaryStream.setBuffers(dictionaryDiskRanges, length); - } - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private EncodedColumnBatch.StreamBuffer dictionaryStream; - private EncodedColumnBatch.StreamBuffer lengthStream; - private CompressionCodec compressionCodec; - private OrcProto.ColumnEncoding columnEncoding; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setLengthStream(EncodedColumnBatch.StreamBuffer lengthStream) { - this.lengthStream = lengthStream; - return this; - } - - public StreamReaderBuilder setDictionaryStream(EncodedColumnBatch.StreamBuffer dictStream) { - this.dictionaryStream = dictStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { - this.columnEncoding = encoding; - return this; - } - - public StringStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - SettableUncompressedStream length = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.LENGTH.name(), fileId, - lengthStream); - - SettableUncompressedStream dictionary = StreamUtils.createLlapInStream( - OrcProto.Stream.Kind.DICTIONARY_DATA.name(), fileId, dictionaryStream); - - boolean isFileCompressed = compressionCodec != null; - return new StringStreamReader(columnIndex, present, data, length, dictionary, - isFileCompressed, columnEncoding); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - - } - - protected static class ShortStreamReader extends ShortTreeReader { - private boolean isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - - private ShortStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, boolean isFileCompressed, - OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, present, data, encoding); - this.isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (isFileCompressed) { - index.getNext(); - } - reader.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private CompressionCodec compressionCodec; - private OrcProto.ColumnEncoding columnEncoding; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { - this.columnEncoding = encoding; - return this; - } - - public ShortStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - boolean isFileCompressed = compressionCodec != null; - return new ShortStreamReader(columnIndex, present, data, isFileCompressed, - columnEncoding); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - } - - protected static class LongStreamReader extends LongTreeReader { - private boolean _isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - - private LongStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, boolean isFileCompressed, - OrcProto.ColumnEncoding encoding, boolean skipCorrupt) throws IOException { - super(columnId, present, data, encoding, skipCorrupt); - this._isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - reader.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private CompressionCodec compressionCodec; - private OrcProto.ColumnEncoding columnEncoding; - private boolean skipCorrupt; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { - this.columnEncoding = encoding; - return this; - } - - public StreamReaderBuilder skipCorrupt(boolean skipCorrupt) { - this.skipCorrupt = skipCorrupt; - return this; - } - - public LongStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - boolean isFileCompressed = compressionCodec != null; - return new LongStreamReader(columnIndex, present, data, isFileCompressed, - columnEncoding, skipCorrupt); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - } - - protected static class IntStreamReader extends IntTreeReader { - private boolean _isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - - private IntStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, boolean isFileCompressed, - OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, present, data, encoding); - this._isFileCompressed = isFileCompressed; - this._dataStream = data; - this._presentStream = present; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - reader.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private CompressionCodec compressionCodec; - private OrcProto.ColumnEncoding columnEncoding; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { - this.columnEncoding = encoding; - return this; - } - - public IntStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - boolean isFileCompressed = compressionCodec != null; - return new IntStreamReader(columnIndex, present, data, isFileCompressed, - columnEncoding); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - - } - - protected static class FloatStreamReader extends FloatTreeReader { - private boolean _isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - - private FloatStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, boolean isFileCompressed) throws IOException { - super(columnId, present, data); - this._isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - stream.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private CompressionCodec compressionCodec; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public FloatStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - boolean isFileCompressed = compressionCodec != null; - return new FloatStreamReader(columnIndex, present, data, isFileCompressed); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - - } - - protected static class DoubleStreamReader extends DoubleTreeReader { - private boolean _isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - - private DoubleStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, boolean isFileCompressed) throws IOException { - super(columnId, present, data); - this._isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - stream.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private CompressionCodec compressionCodec; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public DoubleStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - boolean isFileCompressed = compressionCodec != null; - return new DoubleStreamReader(columnIndex, present, data, isFileCompressed); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - } - - protected static class DecimalStreamReader extends DecimalTreeReader { - private boolean _isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _valueStream; - private SettableUncompressedStream _scaleStream; - - private DecimalStreamReader(int columnId, int precision, int scale, - SettableUncompressedStream presentStream, - SettableUncompressedStream valueStream, SettableUncompressedStream scaleStream, - boolean isFileCompressed, - OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, precision, scale, presentStream, valueStream, scaleStream, encoding); - this._isFileCompressed = isFileCompressed; - this._presentStream = presentStream; - this._valueStream = valueStream; - this._scaleStream = scaleStream; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_valueStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - valueStream.seek(index); - } - - if (_scaleStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - scaleReader.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_valueStream != null) { - List valueDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, valueDiskRanges); - _valueStream.setBuffers(valueDiskRanges, length); - } - if (_scaleStream != null) { - List scaleDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(secondaryStreamBuffer, scaleDiskRanges); - _scaleStream.setBuffers(scaleDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer valueStream; - private EncodedColumnBatch.StreamBuffer scaleStream; - private int scale; - private int precision; - private CompressionCodec compressionCodec; - private OrcProto.ColumnEncoding columnEncoding; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPrecision(int precision) { - this.precision = precision; - return this; - } - - public StreamReaderBuilder setScale(int scale) { - this.scale = scale; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setValueStream(EncodedColumnBatch.StreamBuffer valueStream) { - this.valueStream = valueStream; - return this; - } - - public StreamReaderBuilder setScaleStream(EncodedColumnBatch.StreamBuffer scaleStream) { - this.scaleStream = scaleStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { - this.columnEncoding = encoding; - return this; - } - - public DecimalStreamReader build() throws IOException { - SettableUncompressedStream presentInStream = StreamUtils.createLlapInStream( - OrcProto.Stream.Kind.PRESENT.name(), fileId, presentStream); - - SettableUncompressedStream valueInStream = StreamUtils.createLlapInStream( - OrcProto.Stream.Kind.DATA.name(), fileId, valueStream); - - SettableUncompressedStream scaleInStream = StreamUtils.createLlapInStream( - OrcProto.Stream.Kind.SECONDARY.name(), fileId, scaleStream); - - boolean isFileCompressed = compressionCodec != null; - return new DecimalStreamReader(columnIndex, precision, scale, presentInStream, - valueInStream, - scaleInStream, isFileCompressed, columnEncoding); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - } - - protected static class DateStreamReader extends DateTreeReader { - private boolean isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - - private DateStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, boolean isFileCompressed, - OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, present, data, encoding); - this.isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (isFileCompressed) { - index.getNext(); - } - reader.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private CompressionCodec compressionCodec; - private OrcProto.ColumnEncoding columnEncoding; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { - this.columnEncoding = encoding; - return this; - } - - public DateStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - boolean isFileCompressed = compressionCodec != null; - return new DateStreamReader(columnIndex, present, data, isFileCompressed, - columnEncoding); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - } - - protected static class CharStreamReader extends CharTreeReader { - private boolean _isFileCompressed; - private boolean _isDictionaryEncoding; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - private SettableUncompressedStream _lengthStream; - private SettableUncompressedStream _dictionaryStream; - - private CharStreamReader(int columnId, int maxLength, - SettableUncompressedStream present, SettableUncompressedStream data, - SettableUncompressedStream length, SettableUncompressedStream dictionary, - boolean isFileCompressed, OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, maxLength, present, data, length, - dictionary, encoding); - this._isDictionaryEncoding = dictionary != null; - this._isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - this._lengthStream = length; - this._dictionaryStream = dictionary; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - reader.present.seek(index); - } - - if (_isDictionaryEncoding) { - // DICTIONARY encoding - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - ((StringDictionaryTreeReader) reader).reader.seek(index); - } - } else { - // DIRECT encoding - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - ((StringDirectTreeReader) reader).stream.seek(index); - } - - if (_lengthStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - ((StringDirectTreeReader) reader).lengths.seek(index); - } - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - if (!_isDictionaryEncoding) { - if (_lengthStream != null) { - List lengthDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(lengthsStreamBuffer, lengthDiskRanges); - _lengthStream.setBuffers(lengthDiskRanges, length); - } - } - - // set these streams only if the stripe is different - if (!sameStripe && _isDictionaryEncoding) { - if (_lengthStream != null) { - List lengthDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(lengthsStreamBuffer, lengthDiskRanges); - _lengthStream.setBuffers(lengthDiskRanges, length); - } - if (_dictionaryStream != null) { - List dictionaryDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dictionaryStreamBuffer, dictionaryDiskRanges); - _dictionaryStream.setBuffers(dictionaryDiskRanges, length); - } - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private int maxLength; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private EncodedColumnBatch.StreamBuffer dictionaryStream; - private EncodedColumnBatch.StreamBuffer lengthStream; - private CompressionCodec compressionCodec; - private OrcProto.ColumnEncoding columnEncoding; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setMaxLength(int maxLength) { - this.maxLength = maxLength; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setLengthStream(EncodedColumnBatch.StreamBuffer lengthStream) { - this.lengthStream = lengthStream; - return this; - } - - public StreamReaderBuilder setDictionaryStream(EncodedColumnBatch.StreamBuffer dictStream) { - this.dictionaryStream = dictStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { - this.columnEncoding = encoding; - return this; - } - - public CharStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - SettableUncompressedStream length = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.LENGTH.name(), fileId, - lengthStream); - - SettableUncompressedStream dictionary = StreamUtils.createLlapInStream( - OrcProto.Stream.Kind.DICTIONARY_DATA.name(), fileId, dictionaryStream); - - boolean isFileCompressed = compressionCodec != null; - return new CharStreamReader(columnIndex, maxLength, present, data, length, - dictionary, isFileCompressed, columnEncoding); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - - } - - protected static class VarcharStreamReader extends VarcharTreeReader { - private boolean _isFileCompressed; - private boolean _isDictionaryEncoding; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - private SettableUncompressedStream _lengthStream; - private SettableUncompressedStream _dictionaryStream; - - private VarcharStreamReader(int columnId, int maxLength, - SettableUncompressedStream present, SettableUncompressedStream data, - SettableUncompressedStream length, SettableUncompressedStream dictionary, - boolean isFileCompressed, OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, maxLength, present, data, length, - dictionary, encoding); - this._isDictionaryEncoding = dictionary != null; - this._isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - this._lengthStream = length; - this._dictionaryStream = dictionary; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - reader.present.seek(index); - } - - if (_isDictionaryEncoding) { - // DICTIONARY encoding - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - ((StringDictionaryTreeReader) reader).reader.seek(index); - } - } else { - // DIRECT encoding - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - ((StringDirectTreeReader) reader).stream.seek(index); - } - - if (_lengthStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - ((StringDirectTreeReader) reader).lengths.seek(index); - } - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - if (!_isDictionaryEncoding) { - if (_lengthStream != null) { - List lengthDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(lengthsStreamBuffer, lengthDiskRanges); - _lengthStream.setBuffers(lengthDiskRanges, length); - } - } - - // set these streams only if the stripe is different - if (!sameStripe && _isDictionaryEncoding) { - if (_lengthStream != null) { - List lengthDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(lengthsStreamBuffer, lengthDiskRanges); - _lengthStream.setBuffers(lengthDiskRanges, length); - } - if (_dictionaryStream != null) { - List dictionaryDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dictionaryStreamBuffer, dictionaryDiskRanges); - _dictionaryStream.setBuffers(dictionaryDiskRanges, length); - } - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private int maxLength; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private EncodedColumnBatch.StreamBuffer dictionaryStream; - private EncodedColumnBatch.StreamBuffer lengthStream; - private CompressionCodec compressionCodec; - private OrcProto.ColumnEncoding columnEncoding; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setMaxLength(int maxLength) { - this.maxLength = maxLength; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setLengthStream(EncodedColumnBatch.StreamBuffer lengthStream) { - this.lengthStream = lengthStream; - return this; - } - - public StreamReaderBuilder setDictionaryStream(EncodedColumnBatch.StreamBuffer dictStream) { - this.dictionaryStream = dictStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { - this.columnEncoding = encoding; - return this; - } - - public VarcharStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - SettableUncompressedStream length = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.LENGTH.name(), fileId, - lengthStream); - - SettableUncompressedStream dictionary = StreamUtils.createLlapInStream( - OrcProto.Stream.Kind.DICTIONARY_DATA.name(), fileId, dictionaryStream); - - boolean isFileCompressed = compressionCodec != null; - return new VarcharStreamReader(columnIndex, maxLength, present, data, length, - dictionary, isFileCompressed, columnEncoding); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - - } - - protected static class ByteStreamReader extends ByteTreeReader { - private boolean _isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - - private ByteStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, boolean isFileCompressed) throws IOException { - super(columnId, present, data); - this._isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - reader.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private CompressionCodec compressionCodec; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public ByteStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - boolean isFileCompressed = compressionCodec != null; - return new ByteStreamReader(columnIndex, present, data, isFileCompressed); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - } - - protected static class BinaryStreamReader extends BinaryTreeReader { - private boolean _isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - private SettableUncompressedStream _lengthsStream; - - private BinaryStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, SettableUncompressedStream length, - boolean isFileCompressed, - OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, present, data, length, encoding); - this._isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - this._lengthsStream = length; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - stream.seek(index); - } - - if (lengths != null && _lengthsStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - lengths.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - if (_lengthsStream != null) { - List lengthDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(lengthsStreamBuffer, lengthDiskRanges); - _lengthsStream.setBuffers(lengthDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private EncodedColumnBatch.StreamBuffer lengthStream; - private CompressionCodec compressionCodec; - private OrcProto.ColumnEncoding columnEncoding; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setLengthStream(EncodedColumnBatch.StreamBuffer secondaryStream) { - this.lengthStream = secondaryStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { - this.columnEncoding = encoding; - return this; - } - - public BinaryStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils.createLlapInStream( - OrcProto.Stream.Kind.PRESENT.name(), fileId, presentStream); - - SettableUncompressedStream data = StreamUtils.createLlapInStream( - OrcProto.Stream.Kind.DATA.name(), fileId, dataStream); - - SettableUncompressedStream length = StreamUtils.createLlapInStream( - OrcProto.Stream.Kind.LENGTH.name(), fileId, lengthStream); - - boolean isFileCompressed = compressionCodec != null; - return new BinaryStreamReader(columnIndex, present, data, length, isFileCompressed, - columnEncoding); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - } - - protected static class BooleanStreamReader extends BooleanTreeReader { - private boolean _isFileCompressed; - private SettableUncompressedStream _presentStream; - private SettableUncompressedStream _dataStream; - - private BooleanStreamReader(int columnId, SettableUncompressedStream present, - SettableUncompressedStream data, boolean isFileCompressed) throws IOException { - super(columnId, present, data); - this._isFileCompressed = isFileCompressed; - this._presentStream = present; - this._dataStream = data; - } - - @Override - public void seek(PositionProvider index) throws IOException { - if (present != null) { - if (_isFileCompressed) { - index.getNext(); - } - present.seek(index); - } - - // data stream could be empty stream or already reached end of stream before present stream. - // This can happen if all values in stream are nulls or last row group values are all null. - if (_dataStream.available() > 0) { - if (_isFileCompressed) { - index.getNext(); - } - reader.seek(index); - } - } - - @Override - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - super.setBuffers(buffers, sameStripe); - if (_presentStream != null) { - List presentDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(presentStreamBuffer, presentDiskRanges); - _presentStream.setBuffers(presentDiskRanges, length); - } - if (_dataStream != null) { - List dataDiskRanges = Lists.newArrayList(); - long length = StreamUtils.createDiskRanges(dataStreamBuffer, dataDiskRanges); - _dataStream.setBuffers(dataDiskRanges, length); - } - } - - public static class StreamReaderBuilder { - private Long fileId; - private int columnIndex; - private EncodedColumnBatch.StreamBuffer presentStream; - private EncodedColumnBatch.StreamBuffer dataStream; - private CompressionCodec compressionCodec; - - public StreamReaderBuilder setFileId(Long fileId) { - this.fileId = fileId; - return this; - } - - public StreamReaderBuilder setColumnIndex(int columnIndex) { - this.columnIndex = columnIndex; - return this; - } - - public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { - this.presentStream = presentStream; - return this; - } - - public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { - this.dataStream = dataStream; - return this; - } - - public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { - this.compressionCodec = compressionCodec; - return this; - } - - public BooleanStreamReader build() throws IOException { - SettableUncompressedStream present = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.PRESENT.name(), - fileId, presentStream); - - SettableUncompressedStream data = StreamUtils - .createLlapInStream(OrcProto.Stream.Kind.DATA.name(), fileId, - dataStream); - - boolean isFileCompressed = compressionCodec != null; - return new BooleanStreamReader(columnIndex, present, data, isFileCompressed); - } - } - - public static StreamReaderBuilder builder() { - return new StreamReaderBuilder(); - } - } - - public static TreeReader[] createEncodedTreeReader(int numCols, - List types, - List encodings, - EncodedColumnBatch batch, - CompressionCodec codec, boolean skipCorrupt) throws IOException { - long file = batch.batchKey.file; - TreeReader[] treeReaders = new TreeReader[numCols]; - for (int i = 0; i < numCols; i++) { - int columnIndex = batch.columnIxs[i]; - EncodedColumnBatch.StreamBuffer[] streamBuffers = batch.columnData[i]; - OrcProto.Type columnType = types.get(columnIndex); - - // EncodedColumnBatch is already decompressed, we don't really need to pass codec. - // But we need to know if the original data is compressed or not. This is used to skip - // positions in row index properly. If the file is originally compressed, - // then 1st position (compressed offset) in row index should be skipped to get - // uncompressed offset, else 1st position should not be skipped. - // TODO: there should be a better way to do this, code just needs to be modified - OrcProto.ColumnEncoding columnEncoding = encodings.get(columnIndex); - - // stream buffers are arranged in enum order of stream kind - EncodedColumnBatch.StreamBuffer present = null; - EncodedColumnBatch.StreamBuffer data = null; - EncodedColumnBatch.StreamBuffer dictionary = null; - EncodedColumnBatch.StreamBuffer lengths = null; - EncodedColumnBatch.StreamBuffer secondary = null; - for (EncodedColumnBatch.StreamBuffer streamBuffer : streamBuffers) { - switch (streamBuffer.streamKind) { - case 0: - // PRESENT stream - present = streamBuffer; - break; - case 1: - // DATA stream - data = streamBuffer; - break; - case 2: - // LENGTH stream - lengths = streamBuffer; - break; - case 3: - // DICTIONARY_DATA stream - dictionary = streamBuffer; - break; - case 5: - // SECONDARY stream - secondary = streamBuffer; - break; - default: - throw new IOException("Unexpected stream kind: " + streamBuffer.streamKind); - } - } - - switch (columnType.getKind()) { - case BINARY: - treeReaders[i] = BinaryStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setDataStream(data) - .setLengthStream(lengths) - .setCompressionCodec(codec) - .setColumnEncoding(columnEncoding) - .build(); - break; - case BOOLEAN: - treeReaders[i] = BooleanStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setDataStream(data) - .setCompressionCodec(codec) - .build(); - break; - case BYTE: - treeReaders[i] = ByteStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setDataStream(data) - .setCompressionCodec(codec) - .build(); - break; - case SHORT: - treeReaders[i] = ShortStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setDataStream(data) - .setCompressionCodec(codec) - .setColumnEncoding(columnEncoding) - .build(); - break; - case INT: - treeReaders[i] = IntStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setDataStream(data) - .setCompressionCodec(codec) - .setColumnEncoding(columnEncoding) - .build(); - break; - case LONG: - treeReaders[i] = LongStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setDataStream(data) - .setCompressionCodec(codec) - .setColumnEncoding(columnEncoding) - .skipCorrupt(skipCorrupt) - .build(); - break; - case FLOAT: - treeReaders[i] = FloatStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setDataStream(data) - .setCompressionCodec(codec) - .build(); - break; - case DOUBLE: - treeReaders[i] = DoubleStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setDataStream(data) - .setCompressionCodec(codec) - .build(); - break; - case CHAR: - treeReaders[i] = CharStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setMaxLength(columnType.getMaximumLength()) - .setPresentStream(present) - .setDataStream(data) - .setLengthStream(lengths) - .setDictionaryStream(dictionary) - .setCompressionCodec(codec) - .setColumnEncoding(columnEncoding) - .build(); - break; - case VARCHAR: - treeReaders[i] = VarcharStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setMaxLength(columnType.getMaximumLength()) - .setPresentStream(present) - .setDataStream(data) - .setLengthStream(lengths) - .setDictionaryStream(dictionary) - .setCompressionCodec(codec) - .setColumnEncoding(columnEncoding) - .build(); - break; - case STRING: - treeReaders[i] = StringStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setDataStream(data) - .setLengthStream(lengths) - .setDictionaryStream(dictionary) - .setCompressionCodec(codec) - .setColumnEncoding(columnEncoding) - .build(); - break; - case DECIMAL: - treeReaders[i] = DecimalStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPrecision(columnType.getPrecision()) - .setScale(columnType.getScale()) - .setPresentStream(present) - .setValueStream(data) - .setScaleStream(secondary) - .setCompressionCodec(codec) - .setColumnEncoding(columnEncoding) - .build(); - break; - case TIMESTAMP: - treeReaders[i] = TimestampStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setSecondsStream(data) - .setNanosStream(secondary) - .setCompressionCodec(codec) - .setColumnEncoding(columnEncoding) - .skipCorrupt(skipCorrupt) - .build(); - break; - case DATE: - treeReaders[i] = DateStreamReader.builder() - .setFileId(file) - .setColumnIndex(columnIndex) - .setPresentStream(present) - .setDataStream(data) - .setCompressionCodec(codec) - .setColumnEncoding(columnEncoding) - .build(); - break; - default: - throw new UnsupportedOperationException("Data type not supported yet! " + columnType); - } - } - - return treeReaders; - } -} diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/EncodedTreeReaderFactory.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/EncodedTreeReaderFactory.java new file mode 100644 index 0000000..9e2d281 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/EncodedTreeReaderFactory.java @@ -0,0 +1,1950 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.io.orc; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hive.llap.io.api.EncodedColumnBatch; +import org.apache.hadoop.hive.llap.io.api.orc.OrcBatchKey; + +/** + * + */ +public class EncodedTreeReaderFactory extends TreeReaderFactory { + + protected static class TimestampStreamReader extends TimestampTreeReader { + private boolean isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _secondsStream; + private SettableUncompressedStream _nanosStream; + + private TimestampStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, SettableUncompressedStream nanos, boolean isFileCompressed, + OrcProto.ColumnEncoding encoding, boolean skipCorrupt) throws IOException { + super(columnId, present, data, nanos, encoding, skipCorrupt); + this.isFileCompressed = isFileCompressed; + this._presentStream = present; + this._secondsStream = data; + this._nanosStream = nanos; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_secondsStream.available() > 0) { + if (isFileCompressed) { + index.getNext(); + } + data.seek(index); + } + + if (_nanosStream.available() > 0) { + if (isFileCompressed) { + index.getNext(); + } + nanos.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_secondsStream != null) { + _secondsStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + if (_nanosStream != null) { + _nanosStream.setBuffers(StreamUtils.createDiskRangeInfo(secondaryStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private EncodedColumnBatch.StreamBuffer nanosStream; + private CompressionCodec compressionCodec; + private OrcProto.ColumnEncoding columnEncoding; + private boolean skipCorrupt; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setSecondsStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setNanosStream(EncodedColumnBatch.StreamBuffer secondaryStream) { + this.nanosStream = secondaryStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { + this.columnEncoding = encoding; + return this; + } + + public StreamReaderBuilder skipCorrupt(boolean skipCorrupt) { + this.skipCorrupt = skipCorrupt; + return this; + } + + public TimestampStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + SettableUncompressedStream nanos = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.SECONDARY.name(), + fileId, nanosStream); + + boolean isFileCompressed = compressionCodec != null; + return new TimestampStreamReader(columnIndex, present, data, nanos, + isFileCompressed, columnEncoding, skipCorrupt); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + } + + protected static class StringStreamReader extends StringTreeReader { + private boolean _isFileCompressed; + private boolean _isDictionaryEncoding; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + private SettableUncompressedStream _lengthStream; + private SettableUncompressedStream _dictionaryStream; + + private StringStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, SettableUncompressedStream length, + SettableUncompressedStream dictionary, + boolean isFileCompressed, OrcProto.ColumnEncoding encoding) throws IOException { + super(columnId, present, data, length, dictionary, encoding); + this._isDictionaryEncoding = dictionary != null; + this._isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + this._lengthStream = length; + this._dictionaryStream = dictionary; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + reader.present.seek(index); + } + + if (_isDictionaryEncoding) { + // DICTIONARY encoding + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + ((StringDictionaryTreeReader) reader).reader.seek(index); + } + } else { + // DIRECT encoding + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + ((StringDirectTreeReader) reader).stream.seek(index); + } + + if (_lengthStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + ((StringDirectTreeReader) reader).lengths.seek(index); + } + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + if (!_isDictionaryEncoding) { + if (_lengthStream != null) { + _lengthStream.setBuffers(StreamUtils.createDiskRangeInfo(lengthsStreamBuffer)); + } + } + + // set these streams only if the stripe is different + if (!sameStripe && _isDictionaryEncoding) { + if (_lengthStream != null) { + _lengthStream.setBuffers(StreamUtils.createDiskRangeInfo(lengthsStreamBuffer)); + } + if (_dictionaryStream != null) { + _dictionaryStream.setBuffers(StreamUtils.createDiskRangeInfo(dictionaryStreamBuffer)); + } + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private EncodedColumnBatch.StreamBuffer dictionaryStream; + private EncodedColumnBatch.StreamBuffer lengthStream; + private CompressionCodec compressionCodec; + private OrcProto.ColumnEncoding columnEncoding; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setLengthStream(EncodedColumnBatch.StreamBuffer lengthStream) { + this.lengthStream = lengthStream; + return this; + } + + public StreamReaderBuilder setDictionaryStream(EncodedColumnBatch.StreamBuffer dictStream) { + this.dictionaryStream = dictStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { + this.columnEncoding = encoding; + return this; + } + + public StringStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + SettableUncompressedStream length = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.LENGTH.name(), fileId, + lengthStream); + + SettableUncompressedStream dictionary = StreamUtils.createSettableUncompressedStream( + OrcProto.Stream.Kind.DICTIONARY_DATA.name(), fileId, dictionaryStream); + + boolean isFileCompressed = compressionCodec != null; + return new StringStreamReader(columnIndex, present, data, length, dictionary, + isFileCompressed, columnEncoding); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + + } + + protected static class ShortStreamReader extends ShortTreeReader { + private boolean isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + + private ShortStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, boolean isFileCompressed, + OrcProto.ColumnEncoding encoding) throws IOException { + super(columnId, present, data, encoding); + this.isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (isFileCompressed) { + index.getNext(); + } + reader.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private CompressionCodec compressionCodec; + private OrcProto.ColumnEncoding columnEncoding; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { + this.columnEncoding = encoding; + return this; + } + + public ShortStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + boolean isFileCompressed = compressionCodec != null; + return new ShortStreamReader(columnIndex, present, data, isFileCompressed, + columnEncoding); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + } + + protected static class LongStreamReader extends LongTreeReader { + private boolean _isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + + private LongStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, boolean isFileCompressed, + OrcProto.ColumnEncoding encoding, boolean skipCorrupt) throws IOException { + super(columnId, present, data, encoding, skipCorrupt); + this._isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + reader.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private CompressionCodec compressionCodec; + private OrcProto.ColumnEncoding columnEncoding; + private boolean skipCorrupt; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { + this.columnEncoding = encoding; + return this; + } + + public StreamReaderBuilder skipCorrupt(boolean skipCorrupt) { + this.skipCorrupt = skipCorrupt; + return this; + } + + public LongStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + boolean isFileCompressed = compressionCodec != null; + return new LongStreamReader(columnIndex, present, data, isFileCompressed, + columnEncoding, skipCorrupt); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + } + + protected static class IntStreamReader extends IntTreeReader { + private boolean _isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + + private IntStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, boolean isFileCompressed, + OrcProto.ColumnEncoding encoding) throws IOException { + super(columnId, present, data, encoding); + this._isFileCompressed = isFileCompressed; + this._dataStream = data; + this._presentStream = present; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + reader.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private CompressionCodec compressionCodec; + private OrcProto.ColumnEncoding columnEncoding; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { + this.columnEncoding = encoding; + return this; + } + + public IntStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + boolean isFileCompressed = compressionCodec != null; + return new IntStreamReader(columnIndex, present, data, isFileCompressed, + columnEncoding); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + + } + + protected static class FloatStreamReader extends FloatTreeReader { + private boolean _isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + + private FloatStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, boolean isFileCompressed) throws IOException { + super(columnId, present, data); + this._isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + stream.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private CompressionCodec compressionCodec; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public FloatStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + boolean isFileCompressed = compressionCodec != null; + return new FloatStreamReader(columnIndex, present, data, isFileCompressed); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + + } + + protected static class DoubleStreamReader extends DoubleTreeReader { + private boolean _isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + + private DoubleStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, boolean isFileCompressed) throws IOException { + super(columnId, present, data); + this._isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + stream.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private CompressionCodec compressionCodec; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public DoubleStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + boolean isFileCompressed = compressionCodec != null; + return new DoubleStreamReader(columnIndex, present, data, isFileCompressed); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + } + + protected static class DecimalStreamReader extends DecimalTreeReader { + private boolean _isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _valueStream; + private SettableUncompressedStream _scaleStream; + + private DecimalStreamReader(int columnId, int precision, int scale, + SettableUncompressedStream presentStream, + SettableUncompressedStream valueStream, SettableUncompressedStream scaleStream, + boolean isFileCompressed, + OrcProto.ColumnEncoding encoding) throws IOException { + super(columnId, precision, scale, presentStream, valueStream, scaleStream, encoding); + this._isFileCompressed = isFileCompressed; + this._presentStream = presentStream; + this._valueStream = valueStream; + this._scaleStream = scaleStream; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_valueStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + valueStream.seek(index); + } + + if (_scaleStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + scaleReader.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_valueStream != null) { + _valueStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + if (_scaleStream != null) { + _scaleStream.setBuffers(StreamUtils.createDiskRangeInfo(secondaryStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer valueStream; + private EncodedColumnBatch.StreamBuffer scaleStream; + private int scale; + private int precision; + private CompressionCodec compressionCodec; + private OrcProto.ColumnEncoding columnEncoding; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPrecision(int precision) { + this.precision = precision; + return this; + } + + public StreamReaderBuilder setScale(int scale) { + this.scale = scale; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setValueStream(EncodedColumnBatch.StreamBuffer valueStream) { + this.valueStream = valueStream; + return this; + } + + public StreamReaderBuilder setScaleStream(EncodedColumnBatch.StreamBuffer scaleStream) { + this.scaleStream = scaleStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { + this.columnEncoding = encoding; + return this; + } + + public DecimalStreamReader build() throws IOException { + SettableUncompressedStream presentInStream = StreamUtils.createSettableUncompressedStream( + OrcProto.Stream.Kind.PRESENT.name(), fileId, presentStream); + + SettableUncompressedStream valueInStream = StreamUtils.createSettableUncompressedStream( + OrcProto.Stream.Kind.DATA.name(), fileId, valueStream); + + SettableUncompressedStream scaleInStream = StreamUtils.createSettableUncompressedStream( + OrcProto.Stream.Kind.SECONDARY.name(), fileId, scaleStream); + + boolean isFileCompressed = compressionCodec != null; + return new DecimalStreamReader(columnIndex, precision, scale, presentInStream, + valueInStream, + scaleInStream, isFileCompressed, columnEncoding); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + } + + protected static class DateStreamReader extends DateTreeReader { + private boolean isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + + private DateStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, boolean isFileCompressed, + OrcProto.ColumnEncoding encoding) throws IOException { + super(columnId, present, data, encoding); + this.isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (isFileCompressed) { + index.getNext(); + } + reader.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private CompressionCodec compressionCodec; + private OrcProto.ColumnEncoding columnEncoding; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { + this.columnEncoding = encoding; + return this; + } + + public DateStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + boolean isFileCompressed = compressionCodec != null; + return new DateStreamReader(columnIndex, present, data, isFileCompressed, + columnEncoding); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + } + + protected static class CharStreamReader extends CharTreeReader { + private boolean _isFileCompressed; + private boolean _isDictionaryEncoding; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + private SettableUncompressedStream _lengthStream; + private SettableUncompressedStream _dictionaryStream; + + private CharStreamReader(int columnId, int maxLength, + SettableUncompressedStream present, SettableUncompressedStream data, + SettableUncompressedStream length, SettableUncompressedStream dictionary, + boolean isFileCompressed, OrcProto.ColumnEncoding encoding) throws IOException { + super(columnId, maxLength, present, data, length, + dictionary, encoding); + this._isDictionaryEncoding = dictionary != null; + this._isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + this._lengthStream = length; + this._dictionaryStream = dictionary; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + reader.present.seek(index); + } + + if (_isDictionaryEncoding) { + // DICTIONARY encoding + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + ((StringDictionaryTreeReader) reader).reader.seek(index); + } + } else { + // DIRECT encoding + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + ((StringDirectTreeReader) reader).stream.seek(index); + } + + if (_lengthStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + ((StringDirectTreeReader) reader).lengths.seek(index); + } + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + if (!_isDictionaryEncoding) { + if (_lengthStream != null) { + _lengthStream.setBuffers(StreamUtils.createDiskRangeInfo(lengthsStreamBuffer)); + } + } + + // set these streams only if the stripe is different + if (!sameStripe && _isDictionaryEncoding) { + if (_lengthStream != null) { + _lengthStream.setBuffers(StreamUtils.createDiskRangeInfo(lengthsStreamBuffer)); + } + if (_dictionaryStream != null) { + _dictionaryStream.setBuffers(StreamUtils.createDiskRangeInfo(dictionaryStreamBuffer)); + } + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private int maxLength; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private EncodedColumnBatch.StreamBuffer dictionaryStream; + private EncodedColumnBatch.StreamBuffer lengthStream; + private CompressionCodec compressionCodec; + private OrcProto.ColumnEncoding columnEncoding; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setMaxLength(int maxLength) { + this.maxLength = maxLength; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setLengthStream(EncodedColumnBatch.StreamBuffer lengthStream) { + this.lengthStream = lengthStream; + return this; + } + + public StreamReaderBuilder setDictionaryStream(EncodedColumnBatch.StreamBuffer dictStream) { + this.dictionaryStream = dictStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { + this.columnEncoding = encoding; + return this; + } + + public CharStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + SettableUncompressedStream length = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.LENGTH.name(), fileId, + lengthStream); + + SettableUncompressedStream dictionary = StreamUtils.createSettableUncompressedStream( + OrcProto.Stream.Kind.DICTIONARY_DATA.name(), fileId, dictionaryStream); + + boolean isFileCompressed = compressionCodec != null; + return new CharStreamReader(columnIndex, maxLength, present, data, length, + dictionary, isFileCompressed, columnEncoding); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + + } + + protected static class VarcharStreamReader extends VarcharTreeReader { + private boolean _isFileCompressed; + private boolean _isDictionaryEncoding; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + private SettableUncompressedStream _lengthStream; + private SettableUncompressedStream _dictionaryStream; + + private VarcharStreamReader(int columnId, int maxLength, + SettableUncompressedStream present, SettableUncompressedStream data, + SettableUncompressedStream length, SettableUncompressedStream dictionary, + boolean isFileCompressed, OrcProto.ColumnEncoding encoding) throws IOException { + super(columnId, maxLength, present, data, length, + dictionary, encoding); + this._isDictionaryEncoding = dictionary != null; + this._isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + this._lengthStream = length; + this._dictionaryStream = dictionary; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + reader.present.seek(index); + } + + if (_isDictionaryEncoding) { + // DICTIONARY encoding + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + ((StringDictionaryTreeReader) reader).reader.seek(index); + } + } else { + // DIRECT encoding + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + ((StringDirectTreeReader) reader).stream.seek(index); + } + + if (_lengthStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + ((StringDirectTreeReader) reader).lengths.seek(index); + } + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + if (!_isDictionaryEncoding) { + if (_lengthStream != null) { + _lengthStream.setBuffers(StreamUtils.createDiskRangeInfo(lengthsStreamBuffer)); + } + } + + // set these streams only if the stripe is different + if (!sameStripe && _isDictionaryEncoding) { + if (_lengthStream != null) { + _lengthStream.setBuffers(StreamUtils.createDiskRangeInfo(lengthsStreamBuffer)); + } + if (_dictionaryStream != null) { + _dictionaryStream.setBuffers(StreamUtils.createDiskRangeInfo(dictionaryStreamBuffer)); + } + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private int maxLength; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private EncodedColumnBatch.StreamBuffer dictionaryStream; + private EncodedColumnBatch.StreamBuffer lengthStream; + private CompressionCodec compressionCodec; + private OrcProto.ColumnEncoding columnEncoding; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setMaxLength(int maxLength) { + this.maxLength = maxLength; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setLengthStream(EncodedColumnBatch.StreamBuffer lengthStream) { + this.lengthStream = lengthStream; + return this; + } + + public StreamReaderBuilder setDictionaryStream(EncodedColumnBatch.StreamBuffer dictStream) { + this.dictionaryStream = dictStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { + this.columnEncoding = encoding; + return this; + } + + public VarcharStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + SettableUncompressedStream length = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.LENGTH.name(), fileId, + lengthStream); + + SettableUncompressedStream dictionary = StreamUtils.createSettableUncompressedStream( + OrcProto.Stream.Kind.DICTIONARY_DATA.name(), fileId, dictionaryStream); + + boolean isFileCompressed = compressionCodec != null; + return new VarcharStreamReader(columnIndex, maxLength, present, data, length, + dictionary, isFileCompressed, columnEncoding); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + + } + + protected static class ByteStreamReader extends ByteTreeReader { + private boolean _isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + + private ByteStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, boolean isFileCompressed) throws IOException { + super(columnId, present, data); + this._isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + reader.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private CompressionCodec compressionCodec; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public ByteStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + boolean isFileCompressed = compressionCodec != null; + return new ByteStreamReader(columnIndex, present, data, isFileCompressed); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + } + + protected static class BinaryStreamReader extends BinaryTreeReader { + private boolean _isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + private SettableUncompressedStream _lengthsStream; + + private BinaryStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, SettableUncompressedStream length, + boolean isFileCompressed, + OrcProto.ColumnEncoding encoding) throws IOException { + super(columnId, present, data, length, encoding); + this._isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + this._lengthsStream = length; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + stream.seek(index); + } + + if (lengths != null && _lengthsStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + lengths.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + if (_lengthsStream != null) { + _lengthsStream.setBuffers(StreamUtils.createDiskRangeInfo(lengthsStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private EncodedColumnBatch.StreamBuffer lengthStream; + private CompressionCodec compressionCodec; + private OrcProto.ColumnEncoding columnEncoding; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setLengthStream(EncodedColumnBatch.StreamBuffer secondaryStream) { + this.lengthStream = secondaryStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) { + this.columnEncoding = encoding; + return this; + } + + public BinaryStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils.createSettableUncompressedStream( + OrcProto.Stream.Kind.PRESENT.name(), fileId, presentStream); + + SettableUncompressedStream data = StreamUtils.createSettableUncompressedStream( + OrcProto.Stream.Kind.DATA.name(), fileId, dataStream); + + SettableUncompressedStream length = StreamUtils.createSettableUncompressedStream( + OrcProto.Stream.Kind.LENGTH.name(), fileId, lengthStream); + + boolean isFileCompressed = compressionCodec != null; + return new BinaryStreamReader(columnIndex, present, data, length, isFileCompressed, + columnEncoding); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + } + + protected static class BooleanStreamReader extends BooleanTreeReader { + private boolean _isFileCompressed; + private SettableUncompressedStream _presentStream; + private SettableUncompressedStream _dataStream; + + private BooleanStreamReader(int columnId, SettableUncompressedStream present, + SettableUncompressedStream data, boolean isFileCompressed) throws IOException { + super(columnId, present, data); + this._isFileCompressed = isFileCompressed; + this._presentStream = present; + this._dataStream = data; + } + + @Override + public void seek(PositionProvider index) throws IOException { + if (present != null) { + if (_isFileCompressed) { + index.getNext(); + } + present.seek(index); + } + + // data stream could be empty stream or already reached end of stream before present stream. + // This can happen if all values in stream are nulls or last row group values are all null. + if (_dataStream.available() > 0) { + if (_isFileCompressed) { + index.getNext(); + } + reader.seek(index); + } + } + + @Override + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + super.setBuffers(buffers, sameStripe); + if (_presentStream != null) { + _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(presentStreamBuffer)); + } + if (_dataStream != null) { + _dataStream.setBuffers(StreamUtils.createDiskRangeInfo(dataStreamBuffer)); + } + } + + public static class StreamReaderBuilder { + private Long fileId; + private int columnIndex; + private EncodedColumnBatch.StreamBuffer presentStream; + private EncodedColumnBatch.StreamBuffer dataStream; + private CompressionCodec compressionCodec; + + public StreamReaderBuilder setFileId(Long fileId) { + this.fileId = fileId; + return this; + } + + public StreamReaderBuilder setColumnIndex(int columnIndex) { + this.columnIndex = columnIndex; + return this; + } + + public StreamReaderBuilder setPresentStream(EncodedColumnBatch.StreamBuffer presentStream) { + this.presentStream = presentStream; + return this; + } + + public StreamReaderBuilder setDataStream(EncodedColumnBatch.StreamBuffer dataStream) { + this.dataStream = dataStream; + return this; + } + + public StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) { + this.compressionCodec = compressionCodec; + return this; + } + + public BooleanStreamReader build() throws IOException { + SettableUncompressedStream present = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(), + fileId, presentStream); + + SettableUncompressedStream data = StreamUtils + .createSettableUncompressedStream(OrcProto.Stream.Kind.DATA.name(), fileId, + dataStream); + + boolean isFileCompressed = compressionCodec != null; + return new BooleanStreamReader(columnIndex, present, data, isFileCompressed); + } + } + + public static StreamReaderBuilder builder() { + return new StreamReaderBuilder(); + } + } + + public static TreeReader[] createEncodedTreeReader(int numCols, + List types, + List encodings, + EncodedColumnBatch batch, + CompressionCodec codec, boolean skipCorrupt) throws IOException { + long file = batch.batchKey.file; + TreeReader[] treeReaders = new TreeReader[numCols]; + for (int i = 0; i < numCols; i++) { + int columnIndex = batch.columnIxs[i]; + EncodedColumnBatch.StreamBuffer[] streamBuffers = batch.columnData[i]; + OrcProto.Type columnType = types.get(columnIndex); + + // EncodedColumnBatch is already decompressed, we don't really need to pass codec. + // But we need to know if the original data is compressed or not. This is used to skip + // positions in row index properly. If the file is originally compressed, + // then 1st position (compressed offset) in row index should be skipped to get + // uncompressed offset, else 1st position should not be skipped. + // TODO: there should be a better way to do this, code just needs to be modified + OrcProto.ColumnEncoding columnEncoding = encodings.get(columnIndex); + + // stream buffers are arranged in enum order of stream kind + EncodedColumnBatch.StreamBuffer present = null; + EncodedColumnBatch.StreamBuffer data = null; + EncodedColumnBatch.StreamBuffer dictionary = null; + EncodedColumnBatch.StreamBuffer lengths = null; + EncodedColumnBatch.StreamBuffer secondary = null; + for (EncodedColumnBatch.StreamBuffer streamBuffer : streamBuffers) { + switch (streamBuffer.streamKind) { + case 0: + // PRESENT stream + present = streamBuffer; + break; + case 1: + // DATA stream + data = streamBuffer; + break; + case 2: + // LENGTH stream + lengths = streamBuffer; + break; + case 3: + // DICTIONARY_DATA stream + dictionary = streamBuffer; + break; + case 5: + // SECONDARY stream + secondary = streamBuffer; + break; + default: + throw new IOException("Unexpected stream kind: " + streamBuffer.streamKind); + } + } + + switch (columnType.getKind()) { + case BINARY: + treeReaders[i] = BinaryStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setDataStream(data) + .setLengthStream(lengths) + .setCompressionCodec(codec) + .setColumnEncoding(columnEncoding) + .build(); + break; + case BOOLEAN: + treeReaders[i] = BooleanStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setDataStream(data) + .setCompressionCodec(codec) + .build(); + break; + case BYTE: + treeReaders[i] = ByteStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setDataStream(data) + .setCompressionCodec(codec) + .build(); + break; + case SHORT: + treeReaders[i] = ShortStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setDataStream(data) + .setCompressionCodec(codec) + .setColumnEncoding(columnEncoding) + .build(); + break; + case INT: + treeReaders[i] = IntStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setDataStream(data) + .setCompressionCodec(codec) + .setColumnEncoding(columnEncoding) + .build(); + break; + case LONG: + treeReaders[i] = LongStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setDataStream(data) + .setCompressionCodec(codec) + .setColumnEncoding(columnEncoding) + .skipCorrupt(skipCorrupt) + .build(); + break; + case FLOAT: + treeReaders[i] = FloatStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setDataStream(data) + .setCompressionCodec(codec) + .build(); + break; + case DOUBLE: + treeReaders[i] = DoubleStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setDataStream(data) + .setCompressionCodec(codec) + .build(); + break; + case CHAR: + treeReaders[i] = CharStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setMaxLength(columnType.getMaximumLength()) + .setPresentStream(present) + .setDataStream(data) + .setLengthStream(lengths) + .setDictionaryStream(dictionary) + .setCompressionCodec(codec) + .setColumnEncoding(columnEncoding) + .build(); + break; + case VARCHAR: + treeReaders[i] = VarcharStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setMaxLength(columnType.getMaximumLength()) + .setPresentStream(present) + .setDataStream(data) + .setLengthStream(lengths) + .setDictionaryStream(dictionary) + .setCompressionCodec(codec) + .setColumnEncoding(columnEncoding) + .build(); + break; + case STRING: + treeReaders[i] = StringStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setDataStream(data) + .setLengthStream(lengths) + .setDictionaryStream(dictionary) + .setCompressionCodec(codec) + .setColumnEncoding(columnEncoding) + .build(); + break; + case DECIMAL: + treeReaders[i] = DecimalStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPrecision(columnType.getPrecision()) + .setScale(columnType.getScale()) + .setPresentStream(present) + .setValueStream(data) + .setScaleStream(secondary) + .setCompressionCodec(codec) + .setColumnEncoding(columnEncoding) + .build(); + break; + case TIMESTAMP: + treeReaders[i] = TimestampStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setSecondsStream(data) + .setNanosStream(secondary) + .setCompressionCodec(codec) + .setColumnEncoding(columnEncoding) + .skipCorrupt(skipCorrupt) + .build(); + break; + case DATE: + treeReaders[i] = DateStreamReader.builder() + .setFileId(file) + .setColumnIndex(columnIndex) + .setPresentStream(present) + .setDataStream(data) + .setCompressionCodec(codec) + .setColumnEncoding(columnEncoding) + .build(); + break; + default: + throw new UnsupportedOperationException("Data type not supported yet! " + columnType); + } + } + + return treeReaders; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImplFactory.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImplFactory.java deleted file mode 100644 index ff2984a..0000000 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImplFactory.java +++ /dev/null @@ -1,2373 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.ql.io.orc; - -import java.io.EOFException; -import java.io.IOException; -import java.math.BigInteger; -import java.sql.Timestamp; -import java.text.ParseException; -import java.text.SimpleDateFormat; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.TimeZone; - -import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hadoop.hive.llap.io.api.EncodedColumnBatch; -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.TimestampUtils; -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; -import org.apache.hadoop.hive.serde2.io.DateWritable; -import org.apache.hadoop.hive.serde2.io.DoubleWritable; -import org.apache.hadoop.hive.serde2.io.HiveCharWritable; -import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; -import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; -import org.apache.hadoop.hive.serde2.io.ShortWritable; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; -import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils; -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; - -/** - * - */ -public class RecordReaderImplFactory { - - public abstract static class TreeReader { - protected final int columnId; - protected BitFieldReader present = null; - protected boolean valuePresent = false; - protected EncodedColumnBatch.StreamBuffer presentStreamBuffer = null; - protected EncodedColumnBatch.StreamBuffer dataStreamBuffer = null; - protected EncodedColumnBatch.StreamBuffer dictionaryStreamBuffer = null; - protected EncodedColumnBatch.StreamBuffer lengthsStreamBuffer = null; - protected EncodedColumnBatch.StreamBuffer secondaryStreamBuffer = null; - - TreeReader(int columnId) throws IOException { - this(columnId, null); - } - - TreeReader(int columnId, InStream in) throws IOException { - this.columnId = columnId; - if (in == null) { - present = null; - valuePresent = true; - } else { - present = new BitFieldReader(in, 1); - } - } - - public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) - throws IOException { - // stream buffers are arranged in enum order of stream kind - for (EncodedColumnBatch.StreamBuffer streamBuffer : buffers) { - switch(streamBuffer.streamKind) { - case 0: - // PRESENT stream - presentStreamBuffer = streamBuffer; - break; - case 1: - // DATA stream - dataStreamBuffer = streamBuffer; - break; - case 2: - // LENGTH stream - lengthsStreamBuffer = streamBuffer; - break; - case 3: - // DICTIONARY_DATA stream - dictionaryStreamBuffer = streamBuffer; - break; - case 5: - // SECONDARY stream - secondaryStreamBuffer = streamBuffer; - break; - default: - throw new IOException("Unexpected stream kind: " + streamBuffer.streamKind); - } - } - } - - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - IntegerReader createIntegerReader(OrcProto.ColumnEncoding.Kind kind, - InStream in, - boolean signed, boolean skipCorrupt) throws IOException { - switch (kind) { - case DIRECT_V2: - case DICTIONARY_V2: - return new RunLengthIntegerReaderV2(in, signed, skipCorrupt); - case DIRECT: - case DICTIONARY: - return new RunLengthIntegerReader(in, signed); - default: - throw new IllegalArgumentException("Unknown encoding " + kind); - } - } - - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - checkEncoding(stripeFooter.getColumnsList().get(columnId)); - InStream in = streams.get(new StreamName(columnId, - OrcProto.Stream.Kind.PRESENT)); - if (in == null) { - present = null; - valuePresent = true; - } else { - present = new BitFieldReader(in, 1); - } - } - - /** - * Seek to the given position. - * - * @param index the indexes loaded from the file - * @throws IOException - */ - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - public void seek(PositionProvider index) throws IOException { - if (present != null) { - present.seek(index); - } - } - - long countNonNulls(long rows) throws IOException { - if (present != null) { - long result = 0; - for (long c = 0; c < rows; ++c) { - if (present.next() == 1) { - result += 1; - } - } - return result; - } else { - return rows; - } - } - - abstract void skipRows(long rows) throws IOException; - - Object next(Object previous) throws IOException { - if (present != null) { - valuePresent = present.next() == 1; - } - return previous; - } - - /** - * 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 batchSize Size of the column vector - * @return - * @throws IOException - */ - public Object nextVector(Object previousVector, long batchSize) throws IOException { - ColumnVector result = (ColumnVector) previousVector; - if (present != null) { - // Set noNulls and isNull vector of the ColumnVector based on - // present stream - result.noNulls = true; - for (int i = 0; i < batchSize; i++) { - result.isNull[i] = (present.next() != 1); - if (result.noNulls && result.isNull[i]) { - result.noNulls = false; - } - } - } else { - // There is not present stream, this means that all the values are - // present. - result.noNulls = true; - for (int i = 0; i < batchSize; i++) { - result.isNull[i] = false; - } - } - return previousVector; - } - } - - - protected static class BooleanTreeReader extends TreeReader { - protected BitFieldReader reader = null; - - BooleanTreeReader(int columnId) throws IOException { - this(columnId, null, null); - } - - BooleanTreeReader(int columnId, InStream present, InStream data) throws IOException { - super(columnId, present); - if (data != null) { - reader = new BitFieldReader(data, 1); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - reader = new BitFieldReader(streams.get(new StreamName(columnId, - OrcProto.Stream.Kind.DATA)), 1); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - reader.seek(index); - } - - @Override - void skipRows(long items) throws IOException { - reader.skip(countNonNulls(items)); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - BooleanWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new BooleanWritable(); - } else { - result = (BooleanWritable) previous; - } - result.set(reader.next() == 1); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - LongColumnVector result = null; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; - } - } - - - protected static class ByteTreeReader extends TreeReader { - protected RunLengthByteReader reader = null; - - ByteTreeReader(int columnId) throws IOException { - this(columnId, null, null); - } - - ByteTreeReader(int columnId, InStream present, InStream data) throws IOException { - super(columnId, present); - this.reader = new RunLengthByteReader(data); - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - reader = new RunLengthByteReader(streams.get(new StreamName(columnId, - OrcProto.Stream.Kind.DATA))); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - reader.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - ByteWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new ByteWritable(); - } else { - result = (ByteWritable) previous; - } - result.set(reader.next()); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - LongColumnVector result = null; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; - } - - @Override - void skipRows(long items) throws IOException { - reader.skip(countNonNulls(items)); - } - } - - protected static class ShortTreeReader extends TreeReader { - protected IntegerReader reader = null; - - ShortTreeReader(int columnId) throws IOException { - this(columnId, null, null, null); - } - - ShortTreeReader(int columnId, InStream present, InStream data, - OrcProto.ColumnEncoding encoding) - throws IOException { - super(columnId, present); - if (data != null && encoding != null) { - checkEncoding(encoding); - this.reader = createIntegerReader(encoding.getKind(), data, true, false); - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) && - (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - StreamName name = new StreamName(columnId, - OrcProto.Stream.Kind.DATA); - reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(name), true, false); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - reader.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - ShortWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new ShortWritable(); - } else { - result = (ShortWritable) previous; - } - result.set((short) reader.next()); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - LongColumnVector result = null; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; - } - - @Override - void skipRows(long items) throws IOException { - reader.skip(countNonNulls(items)); - } - } - - protected static class IntTreeReader extends TreeReader { - protected IntegerReader reader = null; - - IntTreeReader(int columnId) throws IOException { - this(columnId, null, null, null); - } - - IntTreeReader(int columnId, InStream present, InStream data, - OrcProto.ColumnEncoding encoding) - throws IOException { - super(columnId, present); - if (data != null && encoding != null) { - checkEncoding(encoding); - this.reader = createIntegerReader(encoding.getKind(), data, true, false); - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) && - (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - StreamName name = new StreamName(columnId, - OrcProto.Stream.Kind.DATA); - // TODO: stripeFooter.getColumnsList()? - reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(name), true, false); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - reader.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - IntWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new IntWritable(); - } else { - result = (IntWritable) previous; - } - result.set((int) reader.next()); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - LongColumnVector result = null; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; - } - - @Override - void skipRows(long items) throws IOException { - reader.skip(countNonNulls(items)); - } - } - - protected static class LongTreeReader extends TreeReader { - protected IntegerReader reader = null; - - LongTreeReader(int columnId, boolean skipCorrupt) throws IOException { - this(columnId, null, null, null, skipCorrupt); - } - - LongTreeReader(int columnId, InStream present, InStream data, - OrcProto.ColumnEncoding encoding, - boolean skipCorrupt) - throws IOException { - super(columnId, present); - if (data != null && encoding != null) { - checkEncoding(encoding); - this.reader = createIntegerReader(encoding.getKind(), data, true, skipCorrupt); - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) && - (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - StreamName name = new StreamName(columnId, - OrcProto.Stream.Kind.DATA); - reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(name), true, false); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - reader.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - LongWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new LongWritable(); - } else { - result = (LongWritable) previous; - } - result.set(reader.next()); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - LongColumnVector result = null; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; - } - - @Override - void skipRows(long items) throws IOException { - reader.skip(countNonNulls(items)); - } - } - - protected static class FloatTreeReader extends TreeReader { - protected InStream stream; - private final SerializationUtils utils; - - FloatTreeReader(int columnId) throws IOException { - this(columnId, null, null); - } - - FloatTreeReader(int columnId, InStream present, InStream data) throws IOException { - super(columnId, present); - this.utils = new SerializationUtils(); - this.stream = data; - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - StreamName name = new StreamName(columnId, - OrcProto.Stream.Kind.DATA); - stream = streams.get(name); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - stream.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - FloatWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new FloatWritable(); - } else { - result = (FloatWritable) previous; - } - result.set(utils.readFloat(stream)); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - DoubleColumnVector result = null; - if (previousVector == null) { - result = new DoubleColumnVector(); - } else { - result = (DoubleColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - // Read value entries based on isNull entries - for (int i = 0; i < batchSize; i++) { - if (!result.isNull[i]) { - result.vector[i] = utils.readFloat(stream); - } else { - - // If the value is not present then set NaN - result.vector[i] = Double.NaN; - } - } - - // Set isRepeating flag - result.isRepeating = true; - for (int i = 0; (i < batchSize - 1 && result.isRepeating); i++) { - if (result.vector[i] != result.vector[i + 1]) { - result.isRepeating = false; - } - } - return result; - } - - @Override - protected void skipRows(long items) throws IOException { - items = countNonNulls(items); - for (int i = 0; i < items; ++i) { - utils.readFloat(stream); - } - } - } - - protected static class DoubleTreeReader extends TreeReader { - protected InStream stream; - private final SerializationUtils utils; - - DoubleTreeReader(int columnId) throws IOException { - this(columnId, null, null); - } - - DoubleTreeReader(int columnId, InStream present, InStream data) throws IOException { - super(columnId, present); - this.utils = new SerializationUtils(); - this.stream = data; - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - StreamName name = - new StreamName(columnId, - OrcProto.Stream.Kind.DATA); - stream = streams.get(name); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - stream.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - DoubleWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new DoubleWritable(); - } else { - result = (DoubleWritable) previous; - } - result.set(utils.readDouble(stream)); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - DoubleColumnVector result = null; - if (previousVector == null) { - result = new DoubleColumnVector(); - } else { - result = (DoubleColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - // Read value entries based on isNull entries - for (int i = 0; i < batchSize; i++) { - if (!result.isNull[i]) { - result.vector[i] = utils.readDouble(stream); - } else { - // If the value is not present then set NaN - result.vector[i] = Double.NaN; - } - } - - // Set isRepeating flag - result.isRepeating = true; - for (int i = 0; (i < batchSize - 1 && result.isRepeating); i++) { - if (result.vector[i] != result.vector[i + 1]) { - result.isRepeating = false; - } - } - return result; - } - - @Override - void skipRows(long items) throws IOException { - items = countNonNulls(items); - stream.skip(items * 8); - } - } - - - protected static class BinaryTreeReader extends TreeReader { - protected InStream stream; - protected IntegerReader lengths = null; - - protected final LongColumnVector scratchlcv; - - BinaryTreeReader(int columnId) throws IOException { - this(columnId, null, null, null, null); - } - - BinaryTreeReader(int columnId, InStream present, InStream data, InStream length, - OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, present); - scratchlcv = new LongColumnVector(); - this.stream = data; - if (length != null && encoding != null) { - checkEncoding(encoding); - this.lengths = createIntegerReader(encoding.getKind(), length, false, false); - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) && - (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - StreamName name = new StreamName(columnId, - OrcProto.Stream.Kind.DATA); - stream = streams.get(name); - lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(new StreamName(columnId, OrcProto.Stream.Kind.LENGTH)), false, false); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - stream.seek(index); - lengths.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - BytesWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new BytesWritable(); - } else { - result = (BytesWritable) previous; - } - int len = (int) lengths.next(); - result.setSize(len); - int offset = 0; - while (len > 0) { - int written = stream.read(result.getBytes(), offset, len); - if (written < 0) { - throw new EOFException("Can't finish byte read from " + stream); - } - len -= written; - offset += written; - } - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - BytesColumnVector result = null; - if (previousVector == null) { - result = new BytesColumnVector(); - } else { - result = (BytesColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv, result, batchSize); - return result; - } - - @Override - void skipRows(long items) throws IOException { - items = countNonNulls(items); - long lengthToSkip = 0; - for (int i = 0; i < items; ++i) { - lengthToSkip += lengths.next(); - } - stream.skip(lengthToSkip); - } - } - - protected static class TimestampTreeReader extends TreeReader { - protected IntegerReader data = null; - protected IntegerReader nanos = null; - protected final boolean skipCorrupt; - protected Map baseTimestampMap; - protected long base_timestamp; - protected final TimeZone readerTimeZone; - protected TimeZone writerTimeZone; - protected boolean hasSameTZRules; - - TimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException { - this(columnId, null, null, null, null, skipCorrupt); - } - - TimestampTreeReader(int columnId, InStream presentStream, InStream dataStream, - InStream nanosStream, OrcProto.ColumnEncoding encoding, boolean skipCorrupt) - throws IOException { - super(columnId, presentStream); - this.skipCorrupt = skipCorrupt; - this.baseTimestampMap = new HashMap<>(); - this.readerTimeZone = TimeZone.getDefault(); - this.writerTimeZone = readerTimeZone; - this.hasSameTZRules = writerTimeZone.hasSameRules(readerTimeZone); - this.base_timestamp = getBaseTimestamp(readerTimeZone.getID()); - if (encoding != null) { - checkEncoding(encoding); - - if (dataStream != null) { - this.data = createIntegerReader(encoding.getKind(), dataStream, true, skipCorrupt); - } - - if (nanosStream != null) { - this.nanos = createIntegerReader(encoding.getKind(), nanosStream, false, skipCorrupt); - } - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) && - (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - data = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(new StreamName(columnId, - OrcProto.Stream.Kind.DATA)), true, skipCorrupt); - nanos = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(new StreamName(columnId, - OrcProto.Stream.Kind.SECONDARY)), false, skipCorrupt); - base_timestamp = getBaseTimestamp(stripeFooter.getWriterTimezone()); - } - - private long getBaseTimestamp(String timeZoneId) throws IOException { - // to make sure new readers read old files in the same way - if (timeZoneId == null || timeZoneId.isEmpty()) { - timeZoneId = readerTimeZone.getID(); - } - - if (!baseTimestampMap.containsKey(timeZoneId)) { - writerTimeZone = TimeZone.getTimeZone(timeZoneId); - hasSameTZRules = writerTimeZone.hasSameRules(readerTimeZone); - SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - sdf.setTimeZone(writerTimeZone); - try { - long epoch = - sdf.parse(WriterImpl.BASE_TIMESTAMP_STRING).getTime() / WriterImpl.MILLIS_PER_SECOND; - baseTimestampMap.put(timeZoneId, epoch); - return epoch; - } catch (ParseException e) { - throw new IOException("Unable to create base timestamp", e); - } finally { - sdf.setTimeZone(readerTimeZone); - } - } - - return baseTimestampMap.get(timeZoneId); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - data.seek(index); - nanos.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - TimestampWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new TimestampWritable(); - } else { - result = (TimestampWritable) previous; - } - long millis = (data.next() + base_timestamp) * WriterImpl.MILLIS_PER_SECOND; - int newNanos = parseNanos(nanos.next()); - // fix the rounding when we divided by 1000. - if (millis >= 0) { - millis += newNanos / 1000000; - } else { - millis -= newNanos / 1000000; - } - 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; - Timestamp ts = new Timestamp(adjustedMillis); - // 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; - ts.setTime(adjustedMillis); - } - ts.setNanos(newNanos); - result.set(ts); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - LongColumnVector result = null; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } - - result.reset(); - Object obj = null; - for (int i = 0; i < batchSize; i++) { - obj = next(obj); - if (obj == null) { - result.noNulls = false; - result.isNull[i] = true; - } else { - TimestampWritable writable = (TimestampWritable) obj; - Timestamp timestamp = writable.getTimestamp(); - result.vector[i] = TimestampUtils.getTimeNanoSec(timestamp); - } - } - - return result; - } - - private static int parseNanos(long serialized) { - int zeros = 7 & (int) serialized; - int result = (int) (serialized >>> 3); - if (zeros != 0) { - for (int i = 0; i <= zeros; ++i) { - result *= 10; - } - } - return result; - } - - @Override - void skipRows(long items) throws IOException { - items = countNonNulls(items); - data.skip(items); - nanos.skip(items); - } - } - - protected static class DateTreeReader extends TreeReader { - protected IntegerReader reader = null; - - DateTreeReader(int columnId) throws IOException { - this(columnId, null, null, null); - } - - DateTreeReader(int columnId, InStream present, InStream data, - OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, present); - if (data != null && encoding != null) { - checkEncoding(encoding); - reader = createIntegerReader(encoding.getKind(), data, true, false); - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) && - (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - StreamName name = new StreamName(columnId, - OrcProto.Stream.Kind.DATA); - reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(name), true, false); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - reader.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - DateWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new DateWritable(); - } else { - result = (DateWritable) previous; - } - result.set((int) reader.next()); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - LongColumnVector result = null; - if (previousVector == null) { - result = new LongColumnVector(); - } else { - result = (LongColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - // Read value entries based on isNull entries - reader.nextVector(result, batchSize); - return result; - } - - @Override - void skipRows(long items) throws IOException { - reader.skip(countNonNulls(items)); - } - } - - protected static class DecimalTreeReader extends TreeReader { - protected InStream valueStream; - protected IntegerReader scaleReader = null; - private LongColumnVector scratchScaleVector; - - private final int precision; - private final int scale; - - DecimalTreeReader(int columnId, int precision, int scale) throws IOException { - this(columnId, precision, scale, null, null, null, null); - } - - DecimalTreeReader(int columnId, int precision, int scale, InStream present, - InStream valueStream, InStream scaleStream, OrcProto.ColumnEncoding encoding) - throws IOException { - super(columnId, present); - this.precision = precision; - this.scale = scale; - this.scratchScaleVector = new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE); - this.valueStream = valueStream; - if (scaleStream != null && encoding != null) { - checkEncoding(encoding); - this.scaleReader = createIntegerReader(encoding.getKind(), scaleStream, true, false); - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) && - (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - valueStream = streams.get(new StreamName(columnId, - OrcProto.Stream.Kind.DATA)); - scaleReader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(new StreamName(columnId, OrcProto.Stream.Kind.SECONDARY)), true, false); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - valueStream.seek(index); - scaleReader.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - HiveDecimalWritable result = null; - if (valuePresent) { - if (previous == null) { - result = new HiveDecimalWritable(); - } else { - result = (HiveDecimalWritable) previous; - } - result.set(HiveDecimal.create(SerializationUtils.readBigInteger(valueStream), - (int) scaleReader.next())); - return HiveDecimalUtils.enforcePrecisionScale(result, precision, scale); - } - return null; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - DecimalColumnVector result = null; - 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; - - // Read present/isNull stream - super.nextVector(result, batchSize); - - // Read value entries based on isNull entries - if (result.isRepeating) { - if (!result.isNull[0]) { - BigInteger bInt = SerializationUtils.readBigInteger(valueStream); - short scaleInData = (short) scaleReader.next(); - HiveDecimal dec = HiveDecimal.create(bInt, scaleInData); - dec = HiveDecimalUtils.enforcePrecisionScale(dec, precision, scale); - result.set(0, 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]) { - BigInteger bInt = SerializationUtils.readBigInteger(valueStream); - short scaleInData = (short) scratchScaleVector.vector[i]; - HiveDecimal dec = HiveDecimal.create(bInt, scaleInData); - dec = HiveDecimalUtils.enforcePrecisionScale(dec, precision, scale); - result.set(i, dec); - } - } - } - // Switch back the null vector. - scratchScaleVector.isNull = scratchIsNull; - return result; - } - - @Override - void skipRows(long items) throws IOException { - items = countNonNulls(items); - for (int i = 0; i < items; i++) { - SerializationUtils.readBigInteger(valueStream); - } - scaleReader.skip(items); - } - } - - /** - * A tree reader that will read string columns. At the start of the - * stripe, it creates an internal reader based on whether a direct or - * dictionary encoding was used. - */ - protected static class StringTreeReader extends TreeReader { - protected TreeReader reader; - - StringTreeReader(int columnId) throws IOException { - super(columnId); - } - - StringTreeReader(int columnId, InStream present, InStream data, InStream length, - InStream dictionary, OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, present); - if (encoding != null) { - switch (encoding.getKind()) { - case DIRECT: - case DIRECT_V2: - reader = new StringDirectTreeReader(columnId, present, data, length, - encoding.getKind()); - break; - case DICTIONARY: - case DICTIONARY_V2: - reader = new StringDictionaryTreeReader(columnId, present, data, length, dictionary, - encoding); - break; - default: - throw new IllegalArgumentException("Unsupported encoding " + - encoding.getKind()); - } - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - reader.checkEncoding(encoding); - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - // For each stripe, checks the encoding and initializes the appropriate - // reader - switch (stripeFooter.getColumnsList().get(columnId).getKind()) { - case DIRECT: - case DIRECT_V2: - reader = new StringDirectTreeReader(columnId); - break; - case DICTIONARY: - case DICTIONARY_V2: - reader = new StringDictionaryTreeReader(columnId); - break; - default: - throw new IllegalArgumentException("Unsupported encoding " + - stripeFooter.getColumnsList().get(columnId).getKind()); - } - reader.startStripe(streams, stripeFooter); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - reader.seek(index); - } - - @Override - public void seek(PositionProvider index) throws IOException { - reader.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - return reader.next(previous); - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - return reader.nextVector(previousVector, batchSize); - } - - @Override - void skipRows(long items) throws IOException { - reader.skipRows(items); - } - } - - // This class collects together very similar methods for reading an ORC vector of byte arrays and - // creating the BytesColumnVector. - // - public static class BytesColumnVectorUtil { - - private static byte[] commonReadByteArrays(InStream stream, IntegerReader lengths, - LongColumnVector scratchlcv, - BytesColumnVector result, long batchSize) throws IOException { - // Read lengths - scratchlcv.isNull = result.isNull; // Notice we are replacing the isNull vector here... - lengths.nextVector(scratchlcv, batchSize); - int totalLength = 0; - if (!scratchlcv.isRepeating) { - for (int i = 0; i < batchSize; i++) { - if (!scratchlcv.isNull[i]) { - totalLength += (int) scratchlcv.vector[i]; - } - } - } else { - if (!scratchlcv.isNull[0]) { - totalLength = (int) (batchSize * scratchlcv.vector[0]); - } - } - - // Read all the strings for this batch - byte[] allBytes = new byte[totalLength]; - int offset = 0; - int len = totalLength; - while (len > 0) { - int bytesRead = stream.read(allBytes, offset, len); - if (bytesRead < 0) { - throw new EOFException("Can't finish byte read of " + len + "/" + totalLength - + " bytes for batchSize " + batchSize + " from [" + stream + "]"); - } - len -= bytesRead; - offset += bytesRead; - } - - return allBytes; - } - - // 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, long 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); - } - } - } 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); - } - } - } - } - } - - /** - * A reader for string columns that are direct encoded in the current - * stripe. - */ - protected static class StringDirectTreeReader extends TreeReader { - protected InStream stream; - protected IntegerReader lengths; - private final LongColumnVector scratchlcv; - - StringDirectTreeReader(int columnId) throws IOException { - this(columnId, null, null, null, null); - } - - StringDirectTreeReader(int columnId, InStream present, InStream data, InStream length, - OrcProto.ColumnEncoding.Kind encoding) throws IOException { - super(columnId, present); - this.scratchlcv = new LongColumnVector(); - this.stream = data; - if (length != null && encoding != null) { - this.lengths = createIntegerReader(encoding, length, false, false); - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT && - encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - StreamName name = new StreamName(columnId, - OrcProto.Stream.Kind.DATA); - stream = streams.get(name); - lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(new StreamName(columnId, OrcProto.Stream.Kind.LENGTH)), - false, false); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - stream.seek(index); - lengths.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - Text result = null; - if (valuePresent) { - if (previous == null) { - result = new Text(); - } else { - result = (Text) previous; - } - int len = (int) lengths.next(); - int offset = 0; - byte[] bytes = new byte[len]; - while (len > 0) { - int written = stream.read(bytes, offset, len); - if (written < 0) { - throw new EOFException("Can't finish byte read from " + stream); - } - len -= written; - offset += written; - } - result.set(bytes); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - BytesColumnVector result = null; - if (previousVector == null) { - result = new BytesColumnVector(); - } else { - result = (BytesColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv, result, batchSize); - return result; - } - - @Override - void skipRows(long items) throws IOException { - items = countNonNulls(items); - long lengthToSkip = 0; - for (int i = 0; i < items; ++i) { - lengthToSkip += lengths.next(); - } - stream.skip(lengthToSkip); - } - } - - /** - * A reader for string columns that are dictionary encoded in the current - * stripe. - */ - protected static class StringDictionaryTreeReader extends TreeReader { - private DynamicByteArray dictionaryBuffer; - private int[] dictionaryOffsets; - protected IntegerReader reader; - - private byte[] dictionaryBufferInBytesCache = null; - private final LongColumnVector scratchlcv; - - StringDictionaryTreeReader(int columnId) throws IOException { - this(columnId, null, null, null, null, null); - } - - StringDictionaryTreeReader(int columnId, InStream present, InStream data, - InStream length, InStream dictionary, OrcProto.ColumnEncoding encoding) - throws IOException { - super(columnId, present); - scratchlcv = new LongColumnVector(); - if (data != null && encoding != null) { - this.reader = createIntegerReader(encoding.getKind(), data, false, false); - } - - if (dictionary != null && encoding != null) { - readDictionaryStream(dictionary); - } - - if (length != null && encoding != null) { - readDictionaryLengthStream(length, encoding); - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DICTIONARY && - encoding.getKind() != OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - - // read the dictionary blob - StreamName name = new StreamName(columnId, - OrcProto.Stream.Kind.DICTIONARY_DATA); - InStream in = streams.get(name); - readDictionaryStream(in); - - // read the lengths - name = new StreamName(columnId, OrcProto.Stream.Kind.LENGTH); - in = streams.get(name); - readDictionaryLengthStream(in, stripeFooter.getColumnsList().get(columnId)); - - // set up the row reader - name = new StreamName(columnId, OrcProto.Stream.Kind.DATA); - reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(name), false, false); - } - - private void readDictionaryLengthStream(InStream in, OrcProto.ColumnEncoding encoding) - throws IOException { - int dictionarySize = encoding.getDictionarySize(); - if (in != null) { // Guard against empty LENGTH stream. - IntegerReader lenReader = createIntegerReader(encoding.getKind(), in, false, false); - int offset = 0; - if (dictionaryOffsets == null || - dictionaryOffsets.length < dictionarySize + 1) { - dictionaryOffsets = new int[dictionarySize + 1]; - } - for (int i = 0; i < dictionarySize; ++i) { - dictionaryOffsets[i] = offset; - offset += (int) lenReader.next(); - } - dictionaryOffsets[dictionarySize] = offset; - in.close(); - } - - } - - private void readDictionaryStream(InStream in) throws IOException { - if (in != null) { // Guard against empty dictionary stream. - if (in.available() > 0) { - dictionaryBuffer = new DynamicByteArray(64, in.available()); - dictionaryBuffer.readAll(in); - // Since its start of strip invalidate the cache. - dictionaryBufferInBytesCache = null; - } - in.close(); - } else { - dictionaryBuffer = null; - } - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - public void seek(PositionProvider index) throws IOException { - super.seek(index); - reader.seek(index); - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - Text result = null; - if (valuePresent) { - int entry = (int) reader.next(); - if (previous == null) { - result = new Text(); - } else { - result = (Text) previous; - } - int offset = dictionaryOffsets[entry]; - int length = getDictionaryEntryLength(entry, offset); - // If the column is just empty strings, the size will be zero, - // so the buffer will be null, in that case just return result - // as it will default to empty - if (dictionaryBuffer != null) { - dictionaryBuffer.setText(result, offset, length); - } else { - result.clear(); - } - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - BytesColumnVector result = null; - int offset = 0, length = 0; - if (previousVector == null) { - result = new BytesColumnVector(); - } else { - result = (BytesColumnVector) previousVector; - } - - // Read present/isNull stream - super.nextVector(result, batchSize); - - if (dictionaryBuffer != null) { - - // Load dictionaryBuffer into cache. - if (dictionaryBufferInBytesCache == null) { - dictionaryBufferInBytesCache = dictionaryBuffer.get(); - } - - // Read string offsets - scratchlcv.isNull = result.isNull; - reader.nextVector(scratchlcv, batchSize); - if (!scratchlcv.isRepeating) { - - // The vector has non-repeating strings. Iterate thru the batch - // and set strings one by one - for (int i = 0; i < batchSize; i++) { - if (!scratchlcv.isNull[i]) { - offset = dictionaryOffsets[(int) scratchlcv.vector[i]]; - length = getDictionaryEntryLength((int) scratchlcv.vector[i], offset); - result.setRef(i, dictionaryBufferInBytesCache, offset, length); - } else { - // If the value is null then set offset and length to zero (null string) - result.setRef(i, dictionaryBufferInBytesCache, 0, 0); - } - } - } else { - // If the value is repeating then just set the first value in the - // vector and set the isRepeating flag to true. No need to iterate thru and - // set all the elements to the same value - offset = dictionaryOffsets[(int) scratchlcv.vector[0]]; - length = getDictionaryEntryLength((int) scratchlcv.vector[0], offset); - result.setRef(0, dictionaryBufferInBytesCache, offset, length); - } - result.isRepeating = scratchlcv.isRepeating; - } else { - // Entire stripe contains null strings. - result.isRepeating = true; - result.noNulls = false; - result.isNull[0] = true; - result.setRef(0, "".getBytes(), 0, 0); - } - return result; - } - - int getDictionaryEntryLength(int entry, int offset) { - int length = 0; - // if it isn't the last entry, subtract the offsets otherwise use - // the buffer length. - if (entry < dictionaryOffsets.length - 1) { - length = dictionaryOffsets[entry + 1] - offset; - } else { - length = dictionaryBuffer.size() - offset; - } - return length; - } - - @Override - void skipRows(long items) throws IOException { - reader.skip(countNonNulls(items)); - } - } - - protected static class CharTreeReader extends StringTreeReader { - int maxLength; - - CharTreeReader(int columnId, int maxLength) throws IOException { - this(columnId, maxLength, null, null, null, null, null); - } - - CharTreeReader(int columnId, int maxLength, InStream present, InStream data, - InStream length, InStream dictionary, OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, present, data, length, dictionary, encoding); - this.maxLength = maxLength; - } - - @Override - Object next(Object previous) throws IOException { - HiveCharWritable result = null; - if (previous == null) { - result = new HiveCharWritable(); - } else { - result = (HiveCharWritable) previous; - } - // Use the string reader implementation to populate the internal Text value - Object textVal = super.next(result.getTextValue()); - if (textVal == null) { - return null; - } - // result should now hold the value that was read in. - // enforce char length - result.enforceMaxLength(maxLength); - return result; - } - - @Override - public Object nextVector(Object previousVector, long 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); - - int adjustedDownLen; - if (result.isRepeating) { - if (result.noNulls || !result.isNull[0]) { - adjustedDownLen = StringExpr - .rightTrimAndTruncate(result.vector[0], result.start[0], result.length[0], maxLength); - if (adjustedDownLen < result.length[0]) { - result.setRef(0, result.vector[0], result.start[0], adjustedDownLen); - } - } - } else { - if (result.noNulls) { - for (int i = 0; i < batchSize; i++) { - adjustedDownLen = StringExpr - .rightTrimAndTruncate(result.vector[i], result.start[i], result.length[i], - maxLength); - if (adjustedDownLen < result.length[i]) { - result.setRef(i, result.vector[i], result.start[i], adjustedDownLen); - } - } - } else { - for (int i = 0; i < batchSize; i++) { - if (!result.isNull[i]) { - adjustedDownLen = StringExpr - .rightTrimAndTruncate(result.vector[i], result.start[i], result.length[i], - maxLength); - if (adjustedDownLen < result.length[i]) { - result.setRef(i, result.vector[i], result.start[i], adjustedDownLen); - } - } - } - } - } - return result; - } - } - - protected static class VarcharTreeReader extends StringTreeReader { - int maxLength; - - VarcharTreeReader(int columnId, int maxLength) throws IOException { - this(columnId, maxLength, null, null, null, null, null); - } - - VarcharTreeReader(int columnId, int maxLength, InStream present, InStream data, - InStream length, InStream dictionary, OrcProto.ColumnEncoding encoding) throws IOException { - super(columnId, present, data, length, dictionary, encoding); - this.maxLength = maxLength; - } - - @Override - Object next(Object previous) throws IOException { - HiveVarcharWritable result = null; - if (previous == null) { - result = new HiveVarcharWritable(); - } else { - result = (HiveVarcharWritable) previous; - } - // Use the string reader implementation to populate the internal Text value - Object textVal = super.next(result.getTextValue()); - if (textVal == null) { - return null; - } - // result should now hold the value that was read in. - // enforce varchar length - result.enforceMaxLength(maxLength); - return result; - } - - @Override - public Object nextVector(Object previousVector, long 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); - - int adjustedDownLen; - if (result.isRepeating) { - if (result.noNulls || !result.isNull[0]) { - adjustedDownLen = StringExpr - .truncate(result.vector[0], result.start[0], result.length[0], maxLength); - if (adjustedDownLen < result.length[0]) { - result.setRef(0, result.vector[0], result.start[0], adjustedDownLen); - } - } - } else { - if (result.noNulls) { - for (int i = 0; i < batchSize; i++) { - adjustedDownLen = StringExpr - .truncate(result.vector[i], result.start[i], result.length[i], maxLength); - if (adjustedDownLen < result.length[i]) { - result.setRef(i, result.vector[i], result.start[i], adjustedDownLen); - } - } - } else { - for (int i = 0; i < batchSize; i++) { - if (!result.isNull[i]) { - adjustedDownLen = StringExpr - .truncate(result.vector[i], result.start[i], result.length[i], maxLength); - if (adjustedDownLen < result.length[i]) { - result.setRef(i, result.vector[i], result.start[i], adjustedDownLen); - } - } - } - } - } - return result; - } - } - - - protected static class StructTreeReader extends TreeReader { - protected final TreeReader[] fields; - protected final String[] fieldNames; - - StructTreeReader(int columnId, - List types, - boolean[] included, - boolean skipCorrupt) throws IOException { - super(columnId); - OrcProto.Type type = types.get(columnId); - int fieldCount = type.getFieldNamesCount(); - this.fields = new TreeReader[fieldCount]; - this.fieldNames = new String[fieldCount]; - for (int i = 0; i < fieldCount; ++i) { - int subtype = type.getSubtypes(i); - if (included == null || included[subtype]) { - this.fields[i] = createTreeReader(subtype, types, included, skipCorrupt); - } - this.fieldNames[i] = type.getFieldNames(i); - } - } - - @Override - void seek(PositionProvider[] index) throws IOException { - super.seek(index); - for (TreeReader kid : fields) { - if (kid != null) { - kid.seek(index); - } - } - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - OrcStruct result = null; - if (valuePresent) { - if (previous == null) { - 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() != fields.length) { - result.setNumFields(fields.length); - } - } - for (int i = 0; i < fields.length; ++i) { - if (fields[i] != null) { - result.setFieldValue(i, fields[i].next(result.getFieldValue(i))); - } - } - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - ColumnVector[] result = null; - if (previousVector == null) { - result = new ColumnVector[fields.length]; - } else { - result = (ColumnVector[]) previousVector; - } - - // Read all the members of struct as column vectors - for (int i = 0; i < fields.length; i++) { - if (fields[i] != null) { - if (result[i] == null) { - result[i] = (ColumnVector) fields[i].nextVector(null, batchSize); - } else { - fields[i].nextVector(result[i], batchSize); - } - } - } - return result; - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - for (TreeReader field : fields) { - if (field != null) { - field.startStripe(streams, stripeFooter); - } - } - } - - @Override - void skipRows(long items) throws IOException { - items = countNonNulls(items); - for (TreeReader field : fields) { - if (field != null) { - field.skipRows(items); - } - } - } - } - - protected static class UnionTreeReader extends TreeReader { - protected final TreeReader[] fields; - protected RunLengthByteReader tags; - - UnionTreeReader(int columnId, - List types, - boolean[] included, - boolean skipCorrupt) throws IOException { - super(columnId); - OrcProto.Type type = types.get(columnId); - int fieldCount = type.getSubtypesCount(); - 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, types, included, skipCorrupt); - } - } - } - - @Override - void seek(PositionProvider[] index) throws IOException { - super.seek(index); - tags.seek(index[columnId]); - for (TreeReader kid : fields) { - kid.seek(index); - } - } - - @Override - Object next(Object previous) throws IOException { - super.next(previous); - OrcUnion result = null; - if (valuePresent) { - if (previous == null) { - result = new OrcUnion(); - } else { - result = (OrcUnion) previous; - } - byte tag = tags.next(); - Object previousVal = result.getObject(); - result.set(tag, fields[tag].next(tag == result.getTag() ? - previousVal : null)); - } - return result; - } - - @Override - public Object nextVector(Object previousVector, long batchSize) throws IOException { - throw new UnsupportedOperationException( - "NextVector is not supported operation for Union type"); - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - tags = new RunLengthByteReader(streams.get(new StreamName(columnId, - OrcProto.Stream.Kind.DATA))); - for (TreeReader field : fields) { - if (field != null) { - field.startStripe(streams, stripeFooter); - } - } - } - - @Override - void skipRows(long items) throws IOException { - items = countNonNulls(items); - long[] counts = new long[fields.length]; - for (int i = 0; i < items; ++i) { - counts[tags.next()] += 1; - } - for (int i = 0; i < counts.length; ++i) { - fields[i].skipRows(counts[i]); - } - } - } - - protected static class ListTreeReader extends TreeReader { - protected final TreeReader elementReader; - protected IntegerReader lengths = null; - - ListTreeReader(int columnId, - List types, - boolean[] included, - boolean skipCorrupt) throws IOException { - super(columnId); - OrcProto.Type type = types.get(columnId); - elementReader = createTreeReader(type.getSubtypes(0), types, included, skipCorrupt); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - super.seek(index); - lengths.seek(index[columnId]); - elementReader.seek(index); - } - - @Override - @SuppressWarnings("unchecked") - Object next(Object previous) throws IOException { - super.next(previous); - List result = null; - if (valuePresent) { - if (previous == null) { - result = new ArrayList(); - } else { - result = (ArrayList) previous; - } - int prevLength = result.size(); - int length = (int) lengths.next(); - // extend the list to the new length - for (int i = prevLength; i < length; ++i) { - result.add(null); - } - // read the new elements into the array - for (int i = 0; i < length; i++) { - result.set(i, elementReader.next(i < prevLength ? - result.get(i) : null)); - } - // remove any extra elements - for (int i = prevLength - 1; i >= length; --i) { - result.remove(i); - } - } - return result; - } - - @Override - public Object nextVector(Object previous, long batchSize) throws IOException { - throw new UnsupportedOperationException( - "NextVector is not supported operation for List type"); - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) && - (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(new StreamName(columnId, - OrcProto.Stream.Kind.LENGTH)), false, false); - if (elementReader != null) { - elementReader.startStripe(streams, stripeFooter); - } - } - - @Override - void skipRows(long items) throws IOException { - items = countNonNulls(items); - long childSkip = 0; - for (long i = 0; i < items; ++i) { - childSkip += lengths.next(); - } - elementReader.skipRows(childSkip); - } - } - - protected static class MapTreeReader extends TreeReader { - protected final TreeReader keyReader; - protected final TreeReader valueReader; - protected IntegerReader lengths = null; - - MapTreeReader(int columnId, - List types, - boolean[] included, - boolean skipCorrupt) throws IOException { - super(columnId); - OrcProto.Type type = types.get(columnId); - int keyColumn = type.getSubtypes(0); - int valueColumn = type.getSubtypes(1); - if (included == null || included[keyColumn]) { - keyReader = createTreeReader(keyColumn, types, included, skipCorrupt); - } else { - keyReader = null; - } - if (included == null || included[valueColumn]) { - valueReader = createTreeReader(valueColumn, types, included, skipCorrupt); - } else { - valueReader = null; - } - } - - @Override - void seek(PositionProvider[] index) throws IOException { - super.seek(index); - lengths.seek(index[columnId]); - keyReader.seek(index); - valueReader.seek(index); - } - - @Override - @SuppressWarnings("unchecked") - Object next(Object previous) throws IOException { - super.next(previous); - Map result = null; - if (valuePresent) { - if (previous == null) { - result = new LinkedHashMap(); - } else { - result = (LinkedHashMap) previous; - } - // for now just clear and create new objects - result.clear(); - int length = (int) lengths.next(); - // read the new elements into the array - for (int i = 0; i < length; i++) { - result.put(keyReader.next(null), valueReader.next(null)); - } - } - return result; - } - - @Override - public Object nextVector(Object previous, long batchSize) throws IOException { - throw new UnsupportedOperationException( - "NextVector is not supported operation for Map type"); - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) && - (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(new StreamName(columnId, - OrcProto.Stream.Kind.LENGTH)), false, false); - if (keyReader != null) { - keyReader.startStripe(streams, stripeFooter); - } - if (valueReader != null) { - valueReader.startStripe(streams, stripeFooter); - } - } - - @Override - void skipRows(long items) throws IOException { - items = countNonNulls(items); - long childSkip = 0; - for (long i = 0; i < items; ++i) { - childSkip += lengths.next(); - } - keyReader.skipRows(childSkip); - valueReader.skipRows(childSkip); - } - } - - public static TreeReader createTreeReader(int columnId, - List types, - boolean[] included, - boolean skipCorrupt - ) throws IOException { - OrcProto.Type type = types.get(columnId); - switch (type.getKind()) { - case BOOLEAN: - return new BooleanTreeReader(columnId); - case BYTE: - return new ByteTreeReader(columnId); - case DOUBLE: - return new DoubleTreeReader(columnId); - case FLOAT: - return new FloatTreeReader(columnId); - case SHORT: - return new ShortTreeReader(columnId); - case INT: - return new IntTreeReader(columnId); - case LONG: - return new LongTreeReader(columnId, skipCorrupt); - case STRING: - return new StringTreeReader(columnId); - case CHAR: - if (!type.hasMaximumLength()) { - throw new IllegalArgumentException("ORC char type has no length specified"); - } - return new CharTreeReader(columnId, type.getMaximumLength()); - case VARCHAR: - if (!type.hasMaximumLength()) { - throw new IllegalArgumentException("ORC varchar type has no length specified"); - } - return new VarcharTreeReader(columnId, type.getMaximumLength()); - case BINARY: - return new BinaryTreeReader(columnId); - case TIMESTAMP: - return new TimestampTreeReader(columnId, skipCorrupt); - case DATE: - return new DateTreeReader(columnId); - 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); - case STRUCT: - return new StructTreeReader(columnId, types, included, skipCorrupt); - case LIST: - return new ListTreeReader(columnId, types, included, skipCorrupt); - case MAP: - return new MapTreeReader(columnId, types, included, skipCorrupt); - case UNION: - return new UnionTreeReader(columnId, types, included, skipCorrupt); - default: - throw new IllegalArgumentException("Unsupported type " + - type.getKind()); - } - } -} diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/SettableUncompressedStream.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/SettableUncompressedStream.java index 88398bc..3643fa2 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/SettableUncompressedStream.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/SettableUncompressedStream.java @@ -20,6 +20,7 @@ import java.util.List; import org.apache.hadoop.hive.common.DiskRange; +import org.apache.hadoop.hive.common.DiskRangeInfo; /** * An uncompressed stream whose underlying byte buffer can be set. @@ -31,7 +32,7 @@ public SettableUncompressedStream(Long fileId, String name, super(fileId, name, input, length); } - public void setBuffers(List input, long length) { - reset(input, length); + public void setBuffers(DiskRangeInfo diskRangeInfo) { + reset(diskRangeInfo.getDiskRanges(), diskRangeInfo.getTotalLength()); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/StreamUtils.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/StreamUtils.java index a456e0d..174f952 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/StreamUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/StreamUtils.java @@ -19,56 +19,49 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.util.List; -import org.apache.hadoop.hive.common.DiskRange; +import org.apache.hadoop.hive.common.DiskRangeInfo; import org.apache.hadoop.hive.llap.io.api.EncodedColumnBatch; import org.apache.hadoop.hive.llap.io.api.cache.LlapMemoryBuffer; -import com.google.common.collect.Lists; - /** * Stream utility. */ public class StreamUtils { /** - * Create LlapInStream from stream buffer. + * Create SettableUncompressedStream from stream buffer. * * @param streamName - stream name * @param fileId - file id * @param streamBuffer - stream buffer - * @return - LlapInStream + * @return - SettableUncompressedStream * @throws IOException */ - public static SettableUncompressedStream createLlapInStream(String streamName, Long fileId, - EncodedColumnBatch.StreamBuffer streamBuffer) throws IOException { + public static SettableUncompressedStream createSettableUncompressedStream(String streamName, + Long fileId, EncodedColumnBatch.StreamBuffer streamBuffer) throws IOException { if (streamBuffer == null) { return null; } - List diskRanges = Lists.newArrayList(); - long totalLength = createDiskRanges(streamBuffer, diskRanges); - return new SettableUncompressedStream(fileId, streamName, diskRanges, totalLength); + DiskRangeInfo diskRangeInfo = createDiskRangeInfo(streamBuffer); + return new SettableUncompressedStream(fileId, streamName, diskRangeInfo.getDiskRanges(), + diskRangeInfo.getTotalLength()); } /** * Converts stream buffers to disk ranges. * @param streamBuffer - stream buffer - * @param diskRanges - initial empty list of disk ranges * @return - total length of disk ranges */ - // TODO: unnecessary - public static long createDiskRanges(EncodedColumnBatch.StreamBuffer streamBuffer, - List diskRanges) { - long totalLength = 0; + public static DiskRangeInfo createDiskRangeInfo(EncodedColumnBatch.StreamBuffer streamBuffer) { + DiskRangeInfo diskRangeInfo = new DiskRangeInfo(); + long offset = 0; for (LlapMemoryBuffer memoryBuffer : streamBuffer.cacheBuffers) { ByteBuffer buffer = memoryBuffer.getByteBufferDup(); - RecordReaderImpl.BufferChunk bufferChunk = new RecordReaderImpl.BufferChunk(buffer, - totalLength); - diskRanges.add(bufferChunk); - totalLength += buffer.remaining(); + diskRangeInfo.addDiskRange(new RecordReaderImpl.BufferChunk(buffer, offset)); + offset += buffer.remaining(); } - return totalLength; + return diskRangeInfo; } } 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 3c2a44f..2266662 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 @@ -31,6 +31,7 @@ import java.util.TimeZone; import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.llap.io.api.EncodedColumnBatch; 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; @@ -60,10 +61,15 @@ */ public class TreeReaderFactory { - protected abstract static class TreeReader { + public abstract static class TreeReader { protected final int columnId; protected BitFieldReader present = null; protected boolean valuePresent = false; + protected EncodedColumnBatch.StreamBuffer presentStreamBuffer = null; + protected EncodedColumnBatch.StreamBuffer dataStreamBuffer = null; + protected EncodedColumnBatch.StreamBuffer dictionaryStreamBuffer = null; + protected EncodedColumnBatch.StreamBuffer lengthsStreamBuffer = null; + protected EncodedColumnBatch.StreamBuffer secondaryStreamBuffer = null; TreeReader(int columnId) throws IOException { this(columnId, null); @@ -131,6 +137,37 @@ public void seek(PositionProvider index) throws IOException { } } + public void setBuffers(EncodedColumnBatch.StreamBuffer[] buffers, boolean sameStripe) + throws IOException { + // stream buffers are arranged in enum order of stream kind + for (EncodedColumnBatch.StreamBuffer streamBuffer : buffers) { + switch (streamBuffer.streamKind) { + case 0: + // PRESENT stream + presentStreamBuffer = streamBuffer; + break; + case 1: + // DATA stream + dataStreamBuffer = streamBuffer; + break; + case 2: + // LENGTH stream + lengthsStreamBuffer = streamBuffer; + break; + case 3: + // DICTIONARY_DATA stream + dictionaryStreamBuffer = streamBuffer; + break; + case 5: + // SECONDARY stream + secondaryStreamBuffer = streamBuffer; + break; + default: + throw new IOException("Unexpected stream kind: " + streamBuffer.streamKind); + } + } + } + protected long countNonNulls(long rows) throws IOException { if (present != null) { long result = 0;