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 @@ -30,13 +30,13 @@ public static ParquetWriterOptions.Builder builder()
return new ParquetWriterOptions.Builder();
}

private final int maxRowGroupSize;
private final long maxRowGroupSize;
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.

do we want to allow setting row group size to >2GB?

private final int maxPageSize;
private final int batchSize;

private ParquetWriterOptions(DataSize maxBlockSize, DataSize maxPageSize, int batchSize)
{
this.maxRowGroupSize = toIntExact(requireNonNull(maxBlockSize, "maxBlockSize is null").toBytes());
this.maxRowGroupSize = requireNonNull(maxBlockSize, "maxBlockSize is null").toBytes();
this.maxPageSize = toIntExact(requireNonNull(maxPageSize, "maxPageSize is null").toBytes());
this.batchSize = batchSize;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -365,7 +365,7 @@ 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(
public static Optional<ColumnIndexStore> getColumnIndexStore(
ParquetDataSource dataSource,
BlockMetaData blockMetadata,
Map<List<String>, RichColumnDescriptor> descriptorsByPath,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,11 +13,9 @@
*/
package io.trino.plugin.hive;

import com.google.common.collect.ImmutableMap;
import io.trino.Session;
import io.trino.testing.AbstractTestQueryFramework;
import io.trino.testing.MaterializedResult;
import io.trino.testing.QueryRunner;
import org.intellij.lang.annotations.Language;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
Expand All @@ -27,49 +25,12 @@
import static java.lang.String.format;
import static org.assertj.core.api.Assertions.assertThat;

public class TestParquetPageSkipping
public abstract class BaseTestParquetPageSkipping
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.

Rename TestParquetPageSkipping to TestHiveTestParquetPageSkipping

this is not a "rename class" change.
it's more extraction of a test base class.

extends AbstractTestQueryFramework
{
@Override
protected QueryRunner createQueryRunner()
throws Exception
{
return HiveQueryRunner.builder()
.setHiveProperties(
ImmutableMap.of(
"parquet.use-column-index", "true",
// Small max-buffer-size allows testing mix of small and large ranges in HdfsParquetDataSource#planRead
"parquet.max-buffer-size", "400B"))
.build();
}

private void buildSortedTables(String tableName, String sortByColumnName, String sortByColumnType)
{
String createTableTemplate =
"CREATE TABLE %s ( " +
" orderkey bigint, " +
" custkey bigint, " +
" orderstatus varchar(1), " +
" totalprice double, " +
" orderdate date, " +
" orderpriority varchar(15), " +
" clerk varchar(15), " +
" shippriority integer, " +
" comment varchar(79), " +
" rvalues double array " +
") " +
"WITH ( " +
" format = 'PARQUET', " +
" bucketed_by = array['orderstatus'], " +
" bucket_count = 1, " +
" sorted_by = array['%s'] " +
")";
createTableTemplate = createTableTemplate.replaceFirst(sortByColumnName + "[ ]+([^,]*)", sortByColumnName + " " + sortByColumnType);

assertUpdate(format(
createTableTemplate,
tableName,
sortByColumnName));
assertUpdate(tableDefinitionForSortedTables(tableName, sortByColumnName, sortByColumnType));
String catalog = getSession().getCatalog().orElseThrow();
assertUpdate(
Session.builder(getSession())
Expand All @@ -80,8 +41,11 @@ private void buildSortedTables(String tableName, String sortByColumnName, String
15000);
}

protected abstract String tableDefinitionForSortedTables(String tableName, String sortByColumnName, String sortByColumnType);

@Test
public void testAndPredicates()
throws Exception
{
String tableName = "test_and_predicate_" + randomTableSuffix();
buildSortedTables(tableName, "totalprice", "double");
Expand Down Expand Up @@ -159,15 +123,15 @@ public void testPageSkipping(String sortByColumn, String sortByColumnType, Objec
assertUpdate("DROP TABLE " + tableName);
}

private int assertColumnIndexResults(String query)
protected int assertColumnIndexResults(String query)
{
MaterializedResult withColumnIndexing = computeActual(query);
MaterializedResult withoutColumnIndexing = computeActual(noParquetColumnIndexFiltering(getSession()), query);
assertEqualsIgnoreOrder(withColumnIndexing, withoutColumnIndexing);
return withoutColumnIndexing.getRowCount();
}

private void assertRowGroupPruning(@Language("SQL") String sql)
protected void assertRowGroupPruning(@Language("SQL") String sql)
{
assertQueryStats(
noParquetColumnIndexFiltering(getSession()),
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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 io.trino.plugin.hive;

import com.google.common.collect.ImmutableMap;
import io.trino.testing.QueryRunner;

import static java.lang.String.format;

public class TestHiveTestParquetPageSkipping
extends BaseTestParquetPageSkipping
{
@Override
protected QueryRunner createQueryRunner()
throws Exception
{
return HiveQueryRunner.builder()
.setHiveProperties(
ImmutableMap.of(
"parquet.use-column-index", "true",
// Small max-buffer-size allows testing mix of small and large ranges in HdfsParquetDataSource#planRead
"parquet.max-buffer-size", "400B"))
.build();
}

@Override
protected String tableDefinitionForSortedTables(String tableName, String sortByColumnName, String sortByColumnType)
{
String createTableTemplate =
"CREATE TABLE %s ( " +
" orderkey bigint, " +
" custkey bigint, " +
" orderstatus varchar(1), " +
" totalprice double, " +
" orderdate date, " +
" orderpriority varchar(15), " +
" clerk varchar(15), " +
" shippriority integer, " +
" comment varchar(79), " +
" rvalues double array " +
") " +
"WITH ( " +
" format = 'PARQUET', " +
" bucketed_by = array['orderstatus'], " +
" bucket_count = 1, " +
" sorted_by = array['%s'] " +
")";
createTableTemplate = createTableTemplate.replaceFirst(sortByColumnName + "[ ]+([^,]*)", sortByColumnName + " " + sortByColumnType);
return format(createTableTemplate, tableName, sortByColumnName);
}
}
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.plugin.hive.parquet.ParquetPageSourceFactory.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,16 +1066,19 @@ 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 is 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.

if (start <= firstDataPage && firstDataPage < start + length &&
predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain)) {
predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain, columnIndex)) {
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.

The version of predicateMatches without columnIndex in PredicateUtils should be removed now

blocks.add(block);
blockStarts.add(nextStart);
if (startRowPosition.isEmpty()) {
startRowPosition = Optional.of(nextStart);
}
endRowPosition = Optional.of(nextStart + block.getRowCount());
columnIndexes.add(columnIndex);
}
nextStart += block.getRowCount();
}
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
Loading