From 947c3f32cf2f4c92d55c1a6af6636b5aaa623aea Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 13 Jul 2018 12:02:41 -0700 Subject: [PATCH] Add Parquet realted stats recording --- .../presto/hive/HiveClientModule.java | 4 + .../hive/parquet/ParquetHiveRecordCursor.java | 7 +- .../hive/parquet/ParquetMetadataStats.java | 129 ++++++++++++++++++ .../parquet/ParquetPageSourceFactory.java | 29 +++- .../parquet/ParquetRecordCursorProvider.java | 19 ++- .../predicate/ParquetPredicateUtils.java | 8 +- .../parquet/reader/ParquetColumnChunk.java | 4 +- .../parquet/reader/ParquetMetadataReader.java | 8 +- .../hive/parquet/reader/ParquetReader.java | 9 +- 9 files changed, 197 insertions(+), 20 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetMetadataStats.java diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java index a9f7b8aa74c7..605c131c8c5f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientModule.java @@ -16,6 +16,7 @@ import com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore; import com.facebook.presto.hive.orc.DwrfPageSourceFactory; import com.facebook.presto.hive.orc.OrcPageSourceFactory; +import com.facebook.presto.hive.parquet.ParquetMetadataStats; import com.facebook.presto.hive.parquet.ParquetPageSourceFactory; import com.facebook.presto.hive.parquet.ParquetRecordCursorProvider; import com.facebook.presto.hive.rcfile.RcFilePageSourceFactory; @@ -102,6 +103,9 @@ public void configure(Binder binder) binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON); newExporter(binder).export(FileFormatDataSourceStats.class).as(generatedNameOf(FileFormatDataSourceStats.class, connectorId)); + binder.bind(ParquetMetadataStats.class).in(Scopes.SINGLETON); + newExporter(binder).export(ParquetMetadataStats.class).as(generatedNameOf(ParquetMetadataStats.class, connectorId)); + Multibinder pageSourceFactoryBinder = newSetBinder(binder, HivePageSourceFactory.class); pageSourceFactoryBinder.addBinding().to(OrcPageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(DwrfPageSourceFactory.class).in(Scopes.SINGLETON); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java index ce9b3f2bad52..4a2875e08331 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java @@ -119,6 +119,7 @@ public class ParquetHiveRecordCursor private boolean closed; private final FileFormatDataSourceStats stats; + private final ParquetMetadataStats metadataStats; public ParquetHiveRecordCursor( HdfsEnvironment hdfsEnvironment, @@ -134,13 +135,15 @@ public ParquetHiveRecordCursor( TypeManager typeManager, boolean predicatePushdownEnabled, TupleDomain effectivePredicate, - FileFormatDataSourceStats stats) + FileFormatDataSourceStats stats, + ParquetMetadataStats metadataStats) { requireNonNull(path, "path is null"); checkArgument(length >= 0, "length is negative"); requireNonNull(splitSchema, "splitSchema is null"); requireNonNull(columns, "columns is null"); this.stats = requireNonNull(stats, "stats is null"); + this.metadataStats = requireNonNull(metadataStats); this.totalBytes = length; @@ -353,7 +356,7 @@ private ParquetRecordReader createParquetRecordReader( Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); TupleDomain parquetTupleDomain = getParquetTupleDomain(descriptorsByPath, effectivePredicate); ParquetPredicate parquetPredicate = buildParquetPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath); - if (predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain)) { + if (predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain, metadataStats)) { offsets.add(block.getStartingPos()); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetMetadataStats.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetMetadataStats.java new file mode 100644 index 000000000000..ba0de676eb76 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetMetadataStats.java @@ -0,0 +1,129 @@ +/* + * Licensed 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 com.facebook.presto.hive.parquet; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.airlift.stats.DistributionStat; +import org.weakref.jmx.Managed; +import org.weakref.jmx.Nested; + +import javax.inject.Inject; + +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; + +public class ParquetMetadataStats +{ + private final DistributionStat metadataLength; + private final DistributionStat readSize; + private final Map dataReadSize; + private final Map dictionaryReadSize; + private final Map pageSize; + + private static ObjectMapper objectMapper = new ObjectMapper(); + + @Inject + public ParquetMetadataStats() + { + this.metadataLength = new DistributionStat(); + this.readSize = new DistributionStat(); + this.dataReadSize = new HashMap<>(); + this.dictionaryReadSize = new HashMap<>(); + this.pageSize = new HashMap<>(); + } + + @Managed + @Nested + public DistributionStat getMetadataLength() + { + return metadataLength; + } + + @Managed + @Nested + public DistributionStat getReadSize() + { + return readSize; + } + + @Managed + public String getDataReadSize() + { + return getSnapshot(dataReadSize); + } + + @Managed + public String getDictionaryReadSize() + { + return getSnapshot(dictionaryReadSize); + } + + @Managed + public String getPageSize() + { + return getSnapshot(pageSize); + } + + private static String getSnapshot(Map stats) + { + Map result = new LinkedHashMap<>(stats.size()); + for (Map.Entry entry : stats.entrySet()) { + result.put(entry.getKey(), entry.getValue().snapshot()); + } + return toJson(result); + } + + private static String toJson(Map snapshot) + { + try { + return objectMapper.writeValueAsString(snapshot); + } + catch (Exception ignore) { + return snapshot.toString(); + } + } + + public void addDataReadSize(String name, long size) + { + synchronized (this) { + addSize(dataReadSize, name, size); + readSize.add(size); + } + } + + public void addDictionaryReadSize(String name, long size) + { + synchronized (this) { + addSize(dictionaryReadSize, name, size); + readSize.add(size); + } + } + + public void addPageSize(String name, long size) + { + synchronized (this) { + addSize(pageSize, name, size); + } + } + + private static void addSize(Map stats, String name, long size) + { + if (!stats.containsKey(name)) { + stats.put(name, new DistributionStat()); + } + + stats.get(name).add(size); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java index ddac655c168e..8a701481244a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java @@ -83,19 +83,31 @@ public class ParquetPageSourceFactory private final boolean useParquetColumnNames; private final HdfsEnvironment hdfsEnvironment; private final FileFormatDataSourceStats stats; + private final ParquetMetadataStats metadataStats; @Inject + public ParquetPageSourceFactory(TypeManager typeManager, HiveClientConfig config, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats, ParquetMetadataStats metadataStats) + { + this(typeManager, requireNonNull(config, "hiveClientConfig is null").isUseParquetColumnNames(), hdfsEnvironment, stats, metadataStats); + } + public ParquetPageSourceFactory(TypeManager typeManager, HiveClientConfig config, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats) { - this(typeManager, requireNonNull(config, "hiveClientConfig is null").isUseParquetColumnNames(), hdfsEnvironment, stats); + this(typeManager, config, hdfsEnvironment, stats, new ParquetMetadataStats()); } public ParquetPageSourceFactory(TypeManager typeManager, boolean useParquetColumnNames, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats) + { + this(typeManager, useParquetColumnNames, hdfsEnvironment, stats, new ParquetMetadataStats()); + } + + public ParquetPageSourceFactory(TypeManager typeManager, boolean useParquetColumnNames, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats, ParquetMetadataStats metadataStats) { this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.useParquetColumnNames = useParquetColumnNames; this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.stats = requireNonNull(stats, "stats is null"); + this.metadataStats = requireNonNull(metadataStats); } @Override @@ -133,7 +145,9 @@ public Optional createPageSource( typeManager, isParquetPredicatePushdownEnabled(session), effectivePredicate, - stats)); + stats, + metadataStats, + session)); } public static ParquetPageSource createParquetPageSource( @@ -150,7 +164,9 @@ public static ParquetPageSource createParquetPageSource( TypeManager typeManager, boolean predicatePushdownEnabled, TupleDomain effectivePredicate, - FileFormatDataSourceStats stats) + FileFormatDataSourceStats stats, + ParquetMetadataStats metadataStats, + ConnectorSession session) { AggregatedMemoryContext systemMemoryContext = newSimpleAggregatedMemoryContext(); @@ -158,7 +174,7 @@ public static ParquetPageSource createParquetPageSource( try { FileSystem fileSystem = hdfsEnvironment.getFileSystem(user, path, configuration); FSDataInputStream inputStream = fileSystem.open(path); - ParquetMetadata parquetMetadata = ParquetMetadataReader.readFooter(inputStream, path, fileSize); + ParquetMetadata parquetMetadata = ParquetMetadataReader.readFooter(inputStream, path, fileSize, metadataStats); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); dataSource = buildHdfsParquetDataSource(inputStream, path, fileSize, stats); @@ -185,7 +201,7 @@ public static ParquetPageSource createParquetPageSource( ParquetPredicate parquetPredicate = buildParquetPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath); final ParquetDataSource finalDataSource = dataSource; blocks = blocks.stream() - .filter(block -> predicateMatches(parquetPredicate, block, finalDataSource, descriptorsByPath, parquetTupleDomain)) + .filter(block -> predicateMatches(parquetPredicate, block, finalDataSource, descriptorsByPath, parquetTupleDomain, metadataStats)) .collect(toList()); } MessageColumnIO messageColumnIO = getColumnIO(fileSchema, requestedSchema); @@ -193,7 +209,8 @@ public static ParquetPageSource createParquetPageSource( messageColumnIO, blocks, dataSource, - systemMemoryContext); + systemMemoryContext, + metadataStats); return new ParquetPageSource( parquetReader, diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java index 7b65cef10e53..6643ca352ef0 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java @@ -49,18 +49,30 @@ public class ParquetRecordCursorProvider private final boolean useParquetColumnNames; private final HdfsEnvironment hdfsEnvironment; private final FileFormatDataSourceStats stats; + private final ParquetMetadataStats metadataStats; - @Inject public ParquetRecordCursorProvider(HiveClientConfig hiveClientConfig, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats) { - this(requireNonNull(hiveClientConfig, "hiveClientConfig is null").isUseParquetColumnNames(), hdfsEnvironment, stats); + this(hiveClientConfig, hdfsEnvironment, stats, new ParquetMetadataStats()); } public ParquetRecordCursorProvider(boolean useParquetColumnNames, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats) + { + this(useParquetColumnNames, hdfsEnvironment, stats, new ParquetMetadataStats()); + } + + @Inject + public ParquetRecordCursorProvider(HiveClientConfig hiveClientConfig, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats, ParquetMetadataStats metadataStats) + { + this(requireNonNull(hiveClientConfig, "hiveClientConfig is null").isUseParquetColumnNames(), hdfsEnvironment, stats, metadataStats); + } + + public ParquetRecordCursorProvider(boolean useParquetColumnNames, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats, ParquetMetadataStats metadataStats) { this.useParquetColumnNames = useParquetColumnNames; this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.stats = requireNonNull(stats, "stats is null"); + this.metadataStats = requireNonNull(metadataStats); } @Override @@ -95,6 +107,7 @@ public Optional createRecordCursor( typeManager, isParquetPredicatePushdownEnabled(session), effectivePredicate, - stats)); + stats, + metadataStats)); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/predicate/ParquetPredicateUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/predicate/ParquetPredicateUtils.java index 1de48312d158..2a72592edf51 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/predicate/ParquetPredicateUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/predicate/ParquetPredicateUtils.java @@ -17,6 +17,7 @@ import com.facebook.presto.hive.parquet.ParquetDataSource; import com.facebook.presto.hive.parquet.ParquetDictionaryPage; import com.facebook.presto.hive.parquet.ParquetEncoding; +import com.facebook.presto.hive.parquet.ParquetMetadataStats; import com.facebook.presto.hive.parquet.RichColumnDescriptor; import com.facebook.presto.spi.predicate.Domain; import com.facebook.presto.spi.predicate.TupleDomain; @@ -110,14 +111,14 @@ public static ParquetPredicate buildParquetPredicate(MessageType requestedSchema return new TupleDomainParquetPredicate(parquetTupleDomain, columnReferences.build()); } - public static boolean predicateMatches(ParquetPredicate parquetPredicate, BlockMetaData block, ParquetDataSource dataSource, Map, RichColumnDescriptor> descriptorsByPath, TupleDomain parquetTupleDomain) + public static boolean predicateMatches(ParquetPredicate parquetPredicate, BlockMetaData block, ParquetDataSource dataSource, Map, RichColumnDescriptor> descriptorsByPath, TupleDomain parquetTupleDomain, ParquetMetadataStats metadataStats) { Map> columnStatistics = getStatistics(block, descriptorsByPath); if (!parquetPredicate.matches(block.getRowCount(), columnStatistics)) { return false; } - Map dictionaries = getDictionaries(block, dataSource, descriptorsByPath, parquetTupleDomain); + Map dictionaries = getDictionaries(block, dataSource, descriptorsByPath, parquetTupleDomain, metadataStats); return parquetPredicate.matches(dictionaries); } @@ -136,7 +137,7 @@ private static Map> getStatistics(BlockMetaData return statistics.build(); } - private static Map getDictionaries(BlockMetaData blockMetadata, ParquetDataSource dataSource, Map, RichColumnDescriptor> descriptorsByPath, TupleDomain parquetTupleDomain) + private static Map getDictionaries(BlockMetaData blockMetadata, ParquetDataSource dataSource, Map, RichColumnDescriptor> descriptorsByPath, TupleDomain parquetTupleDomain, ParquetMetadataStats metadataStats) { ImmutableMap.Builder dictionaries = ImmutableMap.builder(); for (ColumnChunkMetaData columnMetaData : blockMetadata.getColumns()) { @@ -146,6 +147,7 @@ private static Map getDictionarie int totalSize = toIntExact(columnMetaData.getTotalSize()); byte[] buffer = new byte[totalSize]; dataSource.readFully(columnMetaData.getStartingPos(), buffer); + metadataStats.addDictionaryReadSize(columnMetaData.getPath().toDotString(), totalSize); Optional dictionaryPage = readDictionaryPage(buffer, columnMetaData.getCodec()); dictionaries.put(descriptor, new ParquetDictionaryDescriptor(descriptor, dictionaryPage)); break; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetColumnChunk.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetColumnChunk.java index 188ae23d5259..3d811c6dacdf 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetColumnChunk.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetColumnChunk.java @@ -18,6 +18,7 @@ import com.facebook.presto.hive.parquet.ParquetDataPageV1; import com.facebook.presto.hive.parquet.ParquetDataPageV2; import com.facebook.presto.hive.parquet.ParquetDictionaryPage; +import com.facebook.presto.hive.parquet.ParquetMetadataStats; import io.airlift.slice.Slice; import parquet.column.Encoding; import parquet.format.DataPageHeader; @@ -60,7 +61,7 @@ protected PageHeader readPageHeader() return Util.readPageHeader(this); } - public ParquetPageReader readAllPages() + public ParquetPageReader readAllPages(ParquetMetadataStats metadataStats) throws IOException { List pages = new ArrayList<>(); @@ -70,6 +71,7 @@ public ParquetPageReader readAllPages() PageHeader pageHeader = readPageHeader(); int uncompressedPageSize = pageHeader.getUncompressed_page_size(); int compressedPageSize = pageHeader.getCompressed_page_size(); + metadataStats.addPageSize(descriptor.getColumnChunkMetaData().getPath().toDotString(), compressedPageSize); switch (pageHeader.type) { case DICTIONARY_PAGE: if (dictionaryPage != null) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java index dd370df26381..ed95dd7aa475 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive.parquet.reader; +import com.facebook.presto.hive.parquet.ParquetMetadataStats; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -62,15 +63,15 @@ public final class ParquetMetadataReader private ParquetMetadataReader() {} - public static ParquetMetadata readFooter(FileSystem fileSystem, Path file, long fileSize) + public static ParquetMetadata readFooter(FileSystem fileSystem, Path file, long fileSize, ParquetMetadataStats stats) throws IOException { try (FSDataInputStream inputStream = fileSystem.open(file)) { - return readFooter(inputStream, file, fileSize); + return readFooter(inputStream, file, fileSize, stats); } } - public static ParquetMetadata readFooter(FSDataInputStream inputStream, Path file, long fileSize) + public static ParquetMetadata readFooter(FSDataInputStream inputStream, Path file, long fileSize, ParquetMetadataStats stats) throws IOException { @@ -88,6 +89,7 @@ public static ParquetMetadata readFooter(FSDataInputStream inputStream, Path fil inputStream.seek(metadataLengthIndex); InputStream in = new BufferedInputStream(inputStream, PARQUET_METADATA_LENGTH + MAGIC.length); int metadataLength = readIntLittleEndian(in); + stats.getMetadataLength().add(metadataLength); byte[] magic = new byte[MAGIC.length]; validateParquet(in.read(magic) == magic.length, "No enough data for MAGIC"); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetReader.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetReader.java index aea5c90f2c5e..a9b01d4da7f2 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetReader.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetReader.java @@ -17,6 +17,7 @@ import com.facebook.presto.hive.parquet.GroupField; import com.facebook.presto.hive.parquet.ParquetCorruptionException; import com.facebook.presto.hive.parquet.ParquetDataSource; +import com.facebook.presto.hive.parquet.ParquetMetadataStats; import com.facebook.presto.hive.parquet.PrimitiveField; import com.facebook.presto.hive.parquet.RichColumnDescriptor; import com.facebook.presto.memory.context.AggregatedMemoryContext; @@ -72,13 +73,15 @@ public class ParquetReader private long nextRowInGroup; private int batchSize; private final ParquetPrimitiveColumnReader[] columnReaders; + private final ParquetMetadataStats metadataStats; private AggregatedMemoryContext currentRowGroupMemoryContext; public ParquetReader(MessageColumnIO messageColumnIO, List blocks, ParquetDataSource dataSource, - AggregatedMemoryContext systemMemoryContext) + AggregatedMemoryContext systemMemoryContext, + ParquetMetadataStats metadataStats) { this.blocks = blocks; this.dataSource = requireNonNull(dataSource, "dataSource is null"); @@ -86,6 +89,7 @@ public ParquetReader(MessageColumnIO messageColumnIO, this.currentRowGroupMemoryContext = systemMemoryContext.newAggregatedMemoryContext(); columns = messageColumnIO.getLeaves(); columnReaders = new ParquetPrimitiveColumnReader[columns.size()]; + this.metadataStats = requireNonNull(metadataStats); } @Override @@ -201,9 +205,10 @@ private ColumnChunk readPrimitive(PrimitiveField field) int totalSize = toIntExact(metadata.getTotalSize()); byte[] buffer = allocateBlock(totalSize); dataSource.readFully(startingPosition, buffer); + metadataStats.addDataReadSize(metadata.getPath().toDotString(), totalSize); ParquetColumnChunkDescriptor descriptor = new ParquetColumnChunkDescriptor(columnDescriptor, metadata, totalSize); ParquetColumnChunk columnChunk = new ParquetColumnChunk(descriptor, buffer, 0); - columnReader.setPageReader(columnChunk.readAllPages()); + columnReader.setPageReader(columnChunk.readAllPages(metadataStats)); } return columnReader.readPrimitive(field); }