private void readPageV1(DataPageV1 page) { ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); ValuesReader dlReader = page.getDlEncoding().getValuesReader(descriptor, DEFINITION_LEVEL); this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader); this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader); try { BytesInput bytes = page.getBytes(); LOG.debug("page size " + bytes.size() + " bytes and " + pageValueCount + " records"); ByteBufferInputStream in = bytes.toInputStream(); LOG.debug("reading repetition levels at " + in.position()); rlReader.initFromPage(pageValueCount, in); LOG.debug("reading definition levels at " + in.position()); dlReader.initFromPage(pageValueCount, in); LOG.debug("reading data at " + in.position()); initDataReader(page.getValueEncoding(), in, page.getValueCount()); } catch (IOException e) { throw new ParquetDecodingException("could not read page " + page + " in col " + descriptor, e); } }
@Override public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException { this.in = in; if (fixedWidth) { // initialize for repetition and definition levels if (readLength) { int length = readIntLittleEndian(); this.in = in.sliceStream(length); } } else { // initialize for values if (in.available() > 0) { init(in.read()); } } if (bitWidth == 0) { // 0 bit width, treat this as an RLE run of valueCount number of 0's. this.mode = MODE.RLE; this.currentCount = valueCount; this.currentValue = 0; } else { this.currentCount = 0; } }
private ByteBuffer getBuffer(int length) { try { return in.slice(length).order(ByteOrder.LITTLE_ENDIAN); } catch (IOException e) { throw new ParquetDecodingException("Failed to read " + length + " bytes", e); } }
@Override public void initFromPage(int valueCount, ByteBufferInputStream stream) throws IOException { LOG.debug("init from page at offset {} for length {}", stream.position(), (stream.available() - stream.position())); this.in = stream.remainingStream(); } }
@Override public int readInteger() { ++ decodedPosition; if (decodedPosition == decoded.length) { try { if (in.available() < bitWidth) { // unpack8Values needs at least bitWidth bytes to read from, // We have to fill in 0 byte at the end of encoded bytes. byte[] tempEncode = new byte[bitWidth]; in.read(tempEncode, 0, in.available()); packer.unpack8Values(tempEncode, 0, decoded, 0); } else { ByteBuffer encoded = in.slice(bitWidth); packer.unpack8Values(encoded, encoded.position(), decoded, 0); } } catch (IOException e) { throw new ParquetDecodingException("Failed to read packed values", e); } decodedPosition = 0; } return decoded[decodedPosition]; }
public ByteBufferInputStream sliceStream(long length) throws EOFException { return ByteBufferInputStream.wrap(sliceBuffers(length)); }
/** * {@inheritDoc} * @see org.apache.parquet.column.values.ValuesReader#initFromPage(int, ByteBufferInputStream) */ @Override public void initFromPage(int valueCount, ByteBufferInputStream stream) throws IOException { LOG.debug("init from page at offset {} for length {}", stream.position(), stream.available()); this.in.initFromPage(valueCount, stream); }
/** * * @param descriptor descriptor for the chunk * @param buffers ByteBuffers that contain the chunk */ public Chunk(ChunkDescriptor descriptor, List<ByteBuffer> buffers) { this.descriptor = descriptor; this.stream = ByteBufferInputStream.wrap(buffers); }
@Override public void initFromPage(int valueCount, ByteBufferInputStream stream) throws IOException { int effectiveBitLength = valueCount * bitWidth; int length = BytesUtils.paddedByteCountFromBits(effectiveBitLength); // ceil LOG.debug("reading {} bytes for {} values of size {} bits.", length, valueCount, bitWidth); // work-around for null values. this will not happen for repetition or // definition levels (never null), but will happen when valueCount has not // been adjusted for null values in the data. length = Math.min(length, stream.available()); this.in = stream.sliceStream(length); this.decodedPosition = VALUES_AT_A_TIME - 1; }
public BytesInput readAsBytesInput(int size) throws IOException { int available = stream.available(); if (size > available) { // this is to workaround a bug where the compressedLength // of the chunk is missing the size of the header of the dictionary // to allow reading older files (using dictionary) we need this. // usually 13 to 19 bytes are missing int missingBytes = size - available; LOG.info("completed the column chunk with {} bytes", missingBytes); List<ByteBuffer> buffers = new ArrayList<>(); buffers.addAll(stream.sliceBuffers(available)); ByteBuffer lastBuffer = ByteBuffer.allocate(missingBytes); f.readFully(lastBuffer); buffers.add(lastBuffer); return BytesInput.from(buffers); } return super.readAsBytesInput(size); }
final ByteBufferInputStream in = ByteBufferInputStream.wrap(pageData.nioBuffer(0, pageData.capacity())); readPosInBytes = in.position(); repetitionLevels.readInteger(); definitionLevels = dlEncoding.getValuesReader(parentColumnReader.columnDescriptor, ValuesType.DEFINITION_LEVEL); definitionLevels.initFromPage(currentPageCount, in); readPosInBytes = in.position(); if (!valueEncoding.usesDictionary()) { valueReader = valueEncoding.getValuesReader(parentColumnReader.columnDescriptor, ValuesType.VALUES); ByteBuffer byteBuffer = pageData.nioBuffer(index, pageData.capacity() - index); dictionaryLengthDeterminingReader = new DictionaryValuesReader(dictionary); dictionaryLengthDeterminingReader.initFromPage(currentPageCount, ByteBufferInputStream.wrap(byteBuffer)); dictionaryValueReader = new DictionaryValuesReader(dictionary); dictionaryValueReader.initFromPage(currentPageCount, ByteBufferInputStream.wrap(byteBuffer)); parentColumnReader.usingDictionary = true; } else {
public ByteBufferInputStream remainingStream() { return ByteBufferInputStream.wrap(remainingBuffers()); } }
public int available() { return delegate.available(); }
protected PageHeader readPageHeader() throws IOException { PageHeader pageHeader; stream.mark(8192); // headers should not be larger than 8k try { pageHeader = Util.readPageHeader(stream); } catch (IOException e) { // this is to workaround a bug where the compressedLength // of the chunk is missing the size of the header of the dictionary // to allow reading older files (using dictionary) we need this. // usually 13 to 19 bytes are missing // if the last page is smaller than this, the page header itself is truncated in the buffer. stream.reset(); // resetting the buffer to the position before we got the error LOG.info("completing the column chunk to read the page header"); pageHeader = Util.readPageHeader(new SequenceInputStream(stream, f)); // trying again from the buffer + remainder of the stream. } return pageHeader; }
/** * {@inheritDoc} * @see org.apache.parquet.column.values.ValuesReader#initFromPage(int, ByteBufferInputStream) */ @Override public void initFromPage(int valueCount, ByteBufferInputStream stream) throws IOException { int effectiveBitLength = valueCount * bitsPerValue; int length = BytesUtils.paddedByteCountFromBits(effectiveBitLength); LOG.debug("reading {} bytes for {} values of size {} bits.", length, valueCount, bitsPerValue); this.in = stream.sliceStream(length); this.bitPackingReader = createBitPackingReader(bitsPerValue, this.in, valueCount); }
public List<ByteBuffer> sliceBuffers(long length) throws EOFException { return delegate.sliceBuffers(length); }