Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -121,12 +121,6 @@ public static Predicate buildPredicate(
return new TupleDomainParquetPredicate(parquetTupleDomain, columnReferences.build(), timeZone);
}

public static boolean predicateMatches(Predicate parquetPredicate, BlockMetaData block, ParquetDataSource dataSource, Map<List<String>, RichColumnDescriptor> descriptorsByPath, TupleDomain<ColumnDescriptor> parquetTupleDomain)
throws IOException
{
return predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain, Optional.empty());
}

public static boolean predicateMatches(Predicate parquetPredicate, BlockMetaData block, ParquetDataSource dataSource, Map<List<String>, RichColumnDescriptor> descriptorsByPath, TupleDomain<ColumnDescriptor> parquetTupleDomain, Optional<ColumnIndexStore> columnIndexStore)
throws IOException
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,20 +125,6 @@ public class ParquetReader
private final Map<ColumnPath, ColumnDescriptor> paths = new HashMap<>();
private final Map<String, Metric<?>> codecMetrics;

public ParquetReader(
Optional<String> fileCreatedBy,
MessageColumnIO messageColumnIO,
List<BlockMetaData> blocks,
List<Long> firstRowsOfBlocks,
ParquetDataSource dataSource,
DateTimeZone timeZone,
AggregatedMemoryContext memoryContext,
ParquetReaderOptions options)
throws IOException
{
this(fileCreatedBy, messageColumnIO, blocks, firstRowsOfBlocks, dataSource, timeZone, memoryContext, options, null, null);
}

public ParquetReader(
Optional<String> fileCreatedBy,
MessageColumnIO messageColumnIO,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,11 @@
import io.trino.parquet.ChunkReader;
import io.trino.parquet.DiskRange;
import io.trino.parquet.ParquetDataSource;
import io.trino.parquet.ParquetReaderOptions;
import io.trino.parquet.RichColumnDescriptor;
import io.trino.spi.predicate.Domain;
import io.trino.spi.predicate.TupleDomain;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.format.Util;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
Expand All @@ -35,12 +40,15 @@
import javax.annotation.Nullable;

import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.function.BiFunction;

import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static com.google.common.collect.Iterables.getOnlyElement;
import static java.util.Objects.requireNonNull;

Expand Down Expand Up @@ -184,4 +192,41 @@ private PrimitiveType getPrimitiveType()
return primitiveType;
}
}

public static Optional<ColumnIndexStore> getColumnIndexStore(
ParquetDataSource dataSource,
BlockMetaData blockMetadata,
Map<List<String>, RichColumnDescriptor> descriptorsByPath,
TupleDomain<ColumnDescriptor> parquetTupleDomain,
ParquetReaderOptions options)
{
if (!options.isUseColumnIndex() || parquetTupleDomain.isAll() || parquetTupleDomain.isNone()) {
return Optional.empty();
}

boolean hasColumnIndex = false;
for (ColumnChunkMetaData column : blockMetadata.getColumns()) {
if (column.getColumnIndexReference() != null && column.getOffsetIndexReference() != null) {
hasColumnIndex = true;
break;
}
}

if (!hasColumnIndex) {
return Optional.empty();
}

Set<ColumnPath> columnsReadPaths = new HashSet<>(descriptorsByPath.size());
for (List<String> path : descriptorsByPath.keySet()) {
columnsReadPaths.add(ColumnPath.get(path.toArray(new String[0])));
}

Map<ColumnDescriptor, Domain> parquetDomains = parquetTupleDomain.getDomains()
.orElseThrow(() -> new IllegalStateException("Predicate other than none should have domains"));
Set<ColumnPath> columnsFilteredPaths = parquetDomains.keySet().stream()
.map(column -> ColumnPath.get(column.getPath()))
.collect(toImmutableSet());

return Optional.of(new TrinoColumnIndexStore(dataSource, blockMetadata, columnsReadPaths, columnsFilteredPaths));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@
import static io.trino.plugin.hive.HiveErrorCode.HIVE_BAD_DATA;
import static io.trino.plugin.hive.HiveErrorCode.HIVE_CURSOR_ERROR;
import static java.lang.String.format;
import static java.util.Collections.nCopies;
import static java.util.Objects.requireNonNull;

public class ParquetPageSource
Expand All @@ -60,11 +59,6 @@ public class ParquetPageSource
private boolean closed;
private long completedPositions;

public ParquetPageSource(ParquetReader parquetReader, List<Type> types, List<Optional<Field>> fields)
{
this(parquetReader, types, nCopies(types.size(), false), fields);
}

/**
* @param types Column types
* @param rowIndexLocations Whether each column should be populated with the indices of its rows
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import io.trino.parquet.predicate.Predicate;
import io.trino.parquet.reader.MetadataReader;
import io.trino.parquet.reader.ParquetReader;
import io.trino.parquet.reader.TrinoColumnIndexStore;
import io.trino.plugin.hive.AcidInfo;
import io.trino.plugin.hive.FileFormatDataSourceStats;
import io.trino.plugin.hive.HdfsEnvironment;
Expand All @@ -50,8 +49,6 @@
import org.apache.hadoop.hdfs.BlockMissingException;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
import org.apache.parquet.hadoop.metadata.ColumnPath;
import org.apache.parquet.hadoop.metadata.FileMetaData;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore;
Expand All @@ -64,7 +61,6 @@

import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
Expand All @@ -75,14 +71,14 @@

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Strings.nullToEmpty;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext;
import static io.trino.parquet.ParquetTypeUtils.getColumnIO;
import static io.trino.parquet.ParquetTypeUtils.getDescriptors;
import static io.trino.parquet.ParquetTypeUtils.getParquetTypeByName;
import static io.trino.parquet.ParquetTypeUtils.lookupColumnByName;
import static io.trino.parquet.predicate.PredicateUtils.buildPredicate;
import static io.trino.parquet.predicate.PredicateUtils.predicateMatches;
import static io.trino.parquet.reader.TrinoColumnIndexStore.getColumnIndexStore;
import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR;
import static io.trino.plugin.hive.HiveErrorCode.HIVE_BAD_DATA;
import static io.trino.plugin.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT;
Expand Down Expand Up @@ -365,43 +361,6 @@ public static Optional<org.apache.parquet.schema.Type> getColumnType(HiveColumnH
return Optional.of(new GroupType(baseType.getRepetition(), baseType.getName(), ImmutableList.of(type)));
}

private static Optional<ColumnIndexStore> getColumnIndexStore(
ParquetDataSource dataSource,
BlockMetaData blockMetadata,
Map<List<String>, RichColumnDescriptor> descriptorsByPath,
TupleDomain<ColumnDescriptor> parquetTupleDomain,
ParquetReaderOptions options)
{
if (!options.isUseColumnIndex() || parquetTupleDomain.isAll() || parquetTupleDomain.isNone()) {
return Optional.empty();
}

boolean hasColumnIndex = false;
for (ColumnChunkMetaData column : blockMetadata.getColumns()) {
if (column.getColumnIndexReference() != null && column.getOffsetIndexReference() != null) {
hasColumnIndex = true;
break;
}
}

if (!hasColumnIndex) {
return Optional.empty();
}

Set<ColumnPath> columnsReadPaths = new HashSet<>(descriptorsByPath.size());
for (List<String> path : descriptorsByPath.keySet()) {
columnsReadPaths.add(ColumnPath.get(path.toArray(new String[0])));
}

Map<ColumnDescriptor, Domain> parquetDomains = parquetTupleDomain.getDomains()
.orElseThrow(() -> new IllegalStateException("Predicate other than none should have domains"));
Set<ColumnPath> columnsFilteredPaths = parquetDomains.keySet().stream()
.map(column -> ColumnPath.get(column.getPath()))
.collect(toImmutableSet());

return Optional.of(new TrinoColumnIndexStore(dataSource, blockMetadata, columnsReadPaths, columnsFilteredPaths));
}

public static TupleDomain<ColumnDescriptor> getParquetTupleDomain(
Map<List<String>, RichColumnDescriptor> descriptorsByPath,
TupleDomain<HiveColumnHandle> effectivePredicate,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,7 @@
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.FileMetaData;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore;
import org.apache.parquet.io.ColumnIO;
import org.apache.parquet.io.MessageColumnIO;
import org.apache.parquet.schema.MessageType;
Expand Down Expand Up @@ -150,6 +151,7 @@
import static io.trino.parquet.ParquetTypeUtils.getDescriptors;
import static io.trino.parquet.predicate.PredicateUtils.buildPredicate;
import static io.trino.parquet.predicate.PredicateUtils.predicateMatches;
import static io.trino.parquet.reader.TrinoColumnIndexStore.getColumnIndexStore;
import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_FILE_RECORD_COUNT;
import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_PARTITION_DATA;
import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_PARTITION_SPEC_ID;
Expand All @@ -170,6 +172,7 @@
import static io.trino.plugin.iceberg.IcebergSessionProperties.getParquetMaxReadBlockSize;
import static io.trino.plugin.iceberg.IcebergSessionProperties.isOrcBloomFiltersEnabled;
import static io.trino.plugin.iceberg.IcebergSessionProperties.isOrcNestedLazy;
import static io.trino.plugin.iceberg.IcebergSessionProperties.isParquetUseColumnIndex;
import static io.trino.plugin.iceberg.IcebergSessionProperties.isUseFileSizeFromMetadata;
import static io.trino.plugin.iceberg.IcebergSplitManager.ICEBERG_DOMAIN_COMPACTION_THRESHOLD;
import static io.trino.plugin.iceberg.IcebergUtil.deserializePartitionValue;
Expand Down Expand Up @@ -615,7 +618,8 @@ public ReaderPageSourceWithRowPositions createDataPageSource(
partitionData,
dataColumns,
parquetReaderOptions
.withMaxReadBlockSize(getParquetMaxReadBlockSize(session)),
.withMaxReadBlockSize(getParquetMaxReadBlockSize(session))
.withUseColumnIndex(isParquetUseColumnIndex(session)),
predicate,
fileFormatDataSourceStats,
nameMapping,
Expand Down Expand Up @@ -1062,12 +1066,15 @@ private static ReaderPageSourceWithRowPositions createParquetPageSource(
Optional<Long> endRowPosition = Optional.empty();
ImmutableList.Builder<Long> blockStarts = ImmutableList.builder();
List<BlockMetaData> blocks = new ArrayList<>();
ImmutableList.Builder<Optional<ColumnIndexStore>> columnIndexes = ImmutableList.builder();
for (BlockMetaData block : parquetMetadata.getBlocks()) {
long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset();
Optional<ColumnIndexStore> columnIndex = getColumnIndexStore(dataSource, block, descriptorsByPath, parquetTupleDomain, options);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Although we won't read the column index from file until later, it would be better to avoid creating columnIndex until it's needed (after the start <= firstDataPage && firstDataPage < start + length). We can make same change in ParquetPageSourceFactory as well.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will change the indentation of the block and make the diff harder to read, and is unrelated to the Iceberg change, so let's do that as a follow up.

if (start <= firstDataPage && firstDataPage < start + length &&
predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain)) {
predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain, columnIndex)) {
blocks.add(block);
blockStarts.add(nextStart);
columnIndexes.add(columnIndex);
if (startRowPosition.isEmpty()) {
startRowPosition = Optional.of(nextStart);
}
Expand All @@ -1085,7 +1092,9 @@ private static ReaderPageSourceWithRowPositions createParquetPageSource(
dataSource,
UTC,
memoryContext,
options);
options,
parquetPredicate,
columnIndexes.build());

ConstantPopulatingPageSource.Builder constantPopulatingPageSourceBuilder = ConstantPopulatingPageSource.builder();
int parquetSourceChannel = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ public final class IcebergSessionProperties
private static final String ORC_WRITER_MAX_STRIPE_ROWS = "orc_writer_max_stripe_rows";
private static final String ORC_WRITER_MAX_DICTIONARY_MEMORY = "orc_writer_max_dictionary_memory";
private static final String PARQUET_MAX_READ_BLOCK_SIZE = "parquet_max_read_block_size";
private static final String PARQUET_USE_COLUMN_INDEX = "parquet_use_column_index";
private static final String PARQUET_WRITER_BLOCK_SIZE = "parquet_writer_block_size";
private static final String PARQUET_WRITER_PAGE_SIZE = "parquet_writer_page_size";
private static final String PARQUET_WRITER_BATCH_SIZE = "parquet_writer_batch_size";
Expand Down Expand Up @@ -189,6 +190,11 @@ public IcebergSessionProperties(
"Parquet: Maximum size of a block to read",
parquetReaderConfig.getMaxReadBlockSize(),
false))
.add(booleanProperty(
PARQUET_USE_COLUMN_INDEX,
"Parquet: Use Parquet column index",
parquetReaderConfig.isUseColumnIndex(),
false))
.add(dataSizeProperty(
PARQUET_WRITER_BLOCK_SIZE,
"Parquet: Writer block size",
Expand Down Expand Up @@ -357,6 +363,11 @@ public static DataSize getParquetMaxReadBlockSize(ConnectorSession session)
return session.getProperty(PARQUET_MAX_READ_BLOCK_SIZE, DataSize.class);
}

public static boolean isParquetUseColumnIndex(ConnectorSession session)
{
return session.getProperty(PARQUET_USE_COLUMN_INDEX, Boolean.class);
}

public static DataSize getParquetWriterPageSize(ConnectorSession session)
{
return session.getProperty(PARQUET_WRITER_PAGE_SIZE, DataSize.class);
Expand Down