in exec/java-exec/src/main/java/org/apache/parquet/hadoop/ColumnChunkIncReadStore.java [147:275]
public DataPage readPage() {
PageHeader pageHeader = new PageHeader();
try {
if (lastPage != null) {
lastPage.release();
lastPage = null;
}
while(valueReadSoFar < metaData.getValueCount()) {
pageHeader = Util.readPageHeader(in);
int compPageSize = pageHeader.getCompressed_page_size();
int pageSize = pageHeader.getUncompressed_page_size();
DataPageHeaderInfoProvider pageHeaderInfo;
switch (pageHeader.type) {
case DICTIONARY_PAGE:
if (dictionaryPage == null) {
BytesInput pageBytes = decompressor.decompress(
BytesInput.from(in, compPageSize),
pageSize
);
dictionaryPage = new DictionaryPage(
pageBytes,
pageSize,
METADATA_CONVERTER.getEncoding(pageHeader.dictionary_page_header.encoding)
);
} else {
in.skip(compPageSize);
}
break;
case DATA_PAGE:
pageHeaderInfo = DataPageHeaderInfoProvider.builder(pageHeader);
valueReadSoFar += pageHeaderInfo.getNumValues();
ByteBuf buf = allocator.buffer(compPageSize);
lastPage = buf;
ByteBuffer pageBuf = buf.nioBuffer(0, compPageSize);
HadoopStreams.wrap(in).readFully(pageBuf);
pageBuf.flip();
BytesInput pageBytes = decompressor.decompress(BytesInput.from(pageBuf), pageSize);
Statistics stats = METADATA_CONVERTER.fromParquetStatistics(
null,
pageHeaderInfo.getStatistics(),
columnDescriptor.getPrimitiveType()
);
return new DataPageV1(
pageBytes,
pageHeaderInfo.getNumValues(),
pageSize,
stats,
METADATA_CONVERTER.getEncoding(pageHeaderInfo.getRepetitionLevelEncoding()),
METADATA_CONVERTER.getEncoding(pageHeaderInfo.getDefinitionLevelEncoding()),
METADATA_CONVERTER.getEncoding(pageHeaderInfo.getEncoding())
);
case DATA_PAGE_V2:
pageHeaderInfo = DataPageHeaderInfoProvider.builder(pageHeader);
int repLevelSize = pageHeader.data_page_header_v2.getRepetition_levels_byte_length();
int defLevelSize = pageHeader.data_page_header_v2.getDefinition_levels_byte_length();
valueReadSoFar += pageHeaderInfo.getNumValues();
buf = allocator.buffer(compPageSize);
lastPage = buf;
pageBuf = buf.nioBuffer(0, compPageSize);
HadoopStreams.wrap(in).readFully(pageBuf);
pageBuf.flip();
// Note that the repetition and definition levels are stored uncompressed in
// the v2 page format.
int pageBufOffset = 0;
ByteBuffer bb = (ByteBuffer) pageBuf.position(pageBufOffset);
BytesInput repLevelBytes = BytesInput.from(
(ByteBuffer) bb.slice().limit(repLevelSize)
);
pageBufOffset += repLevelSize;
bb = (ByteBuffer) pageBuf.position(pageBufOffset);
final BytesInput defLevelBytes = BytesInput.from(
(ByteBuffer) bb.slice().limit(defLevelSize)
);
pageBufOffset += defLevelSize;
// we've now reached the beginning of compressed column data
bb = (ByteBuffer) pageBuf.position(pageBufOffset);
final BytesInput colDataBytes = decompressor.decompress(
BytesInput.from((ByteBuffer) bb.slice()),
pageSize - repLevelSize - defLevelSize
);
stats = METADATA_CONVERTER.fromParquetStatistics(
null,
pageHeaderInfo.getStatistics(),
columnDescriptor.getPrimitiveType()
);
return new DataPageV2(
pageHeader.data_page_header_v2.getNum_rows(),
pageHeader.data_page_header_v2.getNum_nulls(),
pageHeaderInfo.getNumValues(),
repLevelBytes,
defLevelBytes,
METADATA_CONVERTER.getEncoding(pageHeaderInfo.getEncoding()),
colDataBytes,
pageSize,
stats,
pageHeader.data_page_header_v2.isIs_compressed()
);
default:
logger.warn("skipping page of type {} of size {}", pageHeader.getType(), compPageSize);
in.skip(compPageSize);
break;
}
}
in.close();
return null;
} catch (OutOfMemoryException e) {
throw e; // throw as it is
} catch (Exception e) {
throw new DrillRuntimeException("Error reading page." +
"\nFile path: " + path.toUri().getPath() +
"\nRow count: " + rowCount +
"\nColumn Chunk Metadata: " + metaData +
"\nPage Header: " + pageHeader +
"\nFile offset: " + fileOffset +
"\nSize: " + size +
"\nValue read so far: " + valueReadSoFar, e);
}
}