private ColumnReaderImpl newMemColumnReader(ColumnDescriptor path, PageReader pageReader) { PrimitiveConverter converter = getPrimitiveConverter(path); return new ColumnReaderImpl(path, pageReader, converter, writerVersion); }
/** * creates a reader for triplets * * @param path * the descriptor for the corresponding column * @param pageReader * the underlying store to read from * @param converter * a converter that materializes the values in this column in the current record * @param writerVersion * writer version string from the Parquet file being read */ public ColumnReaderImpl(ColumnDescriptor path, PageReader pageReader, PrimitiveConverter converter, ParsedVersion writerVersion) { super(path, pageReader, converter, writerVersion); consume(); }
/** * {@inheritDoc} * @see org.apache.parquet.column.ColumnReader#consume() */ @Override public void consume() { checkRead(); valueRead = false; }
private void checkRead() { if (isPageFullyConsumed()) { if (isFullyConsumed()) { LOG.debug("end reached"); repetitionLevel = 0; // the next repetition level return; } readPage(); } readRepetitionAndDefinitionLevels(); }
private void readPageV2(DataPageV2 page) { this.repetitionLevelColumn = newRLEIterator(path.getMaxRepetitionLevel(), page.getRepetitionLevels()); this.definitionLevelColumn = newRLEIterator(path.getMaxDefinitionLevel(), page.getDefinitionLevels()); LOG.debug("page data size {} bytes and {} records", page.getData().size(), pageValueCount); try { initDataReader(page.getDataEncoding(), page.getData().toInputStream(), page.getValueCount()); } catch (IOException e) { throw new ParquetDecodingException("could not read page " + page + " in col " + path, e); } }
private void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) { ValuesReader previousReader = this.dataColumn; this.currentEncoding = dataEncoding; this.pageValueCount = valueCount; this.endOfPageValueCount = readValues + pageValueCount; if (dataEncoding.usesDictionary()) { if (dictionary == null) { throw new ParquetDecodingException( "could not read page in col " + path + " as the dictionary was missing for encoding " + dataEncoding); } this.dataColumn = dataEncoding.getDictionaryBasedValuesReader(path, VALUES, dictionary); } else { this.dataColumn = dataEncoding.getValuesReader(path, VALUES); } if (dataEncoding.usesDictionary() && converter.hasDictionarySupport()) { bindToDictionary(dictionary); } else { bind(path.getType()); } try { dataColumn.initFromPage(pageValueCount, in); } catch (IOException e) { throw new ParquetDecodingException("could not read page in col " + path, e); } if (CorruptDeltaByteArrays.requiresSequentialReads(writerVersion, dataEncoding) && previousReader != null && previousReader instanceof RequiresPreviousReader) { // previous reader can only be set if reading sequentially ((RequiresPreviousReader) dataColumn).setPreviousReader(previousReader); } }
private void readPageV1(DataPageV1 page) { ValuesReader rlReader = page.getRlEncoding().getValuesReader(path, REPETITION_LEVEL); ValuesReader dlReader = page.getDlEncoding().getValuesReader(path, DEFINITION_LEVEL); this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader); this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader); try { BytesInput bytes = page.getBytes(); LOG.debug("page size {} bytes and {} records", bytes.size(), pageValueCount); LOG.debug("reading repetition levels at 0"); ByteBufferInputStream in = bytes.toInputStream(); 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 " + path, e); } }
public ColumnReaderImpl newMemColumnReader(ColumnDescriptor path, PageReader pageReader) { PrimitiveConverter converter = getPrimitiveConverter(path); return new ColumnReaderImpl(path, pageReader, converter, writerVersion); }
throw new ParquetDecodingException("totalValueCount '" + totalValueCount + "' <= 0"); consume();
@Override public ColumnReader getColumnReader(ColumnDescriptor path) { PrimitiveConverter converter = getPrimitiveConverter(path); PageReader pageReader = pageReadStore.getPageReader(path); Optional<PrimitiveIterator.OfLong> rowIndexes = pageReadStore.getRowIndexes(); if (rowIndexes.isPresent()) { return new SynchronizingColumnReader(path, pageReader, converter, writerVersion, rowIndexes.get()); } else { return new ColumnReaderImpl(path, pageReader, converter, writerVersion); } }